distributed-1.20.2/000077500000000000000000000000001321233345200141145ustar00rootroot00000000000000distributed-1.20.2/.coveragerc000066400000000000000000000005731321233345200162420ustar00rootroot00000000000000[run] include = distributed/* omit = distributed/tests/test* distributed/hdfs.py distributed/cluster.py distributed/*/tests/test* distributed/compatibility.py distributed/cli/utils.py distributed/utils_test.py distributed/deploy/ssh.py distributed/_ipython_utils.py [report] show_missing = True [html] directory = coverage_html_report distributed-1.20.2/.gitignore000066400000000000000000000001341321233345200161020ustar00rootroot00000000000000*.pyc *.py~ build/ *.egg-info docs/build continuous_integration/hdfs-initialized .cache .#* distributed-1.20.2/.travis.yml000066400000000000000000000026101321233345200162240ustar00rootroot00000000000000language: python # sudo shouldn't be required, but currently tests fail when run in a container # on travis instead of a vm. See https://github.com/dask/distributed/pull/1563. sudo: required dist: trusty env: matrix: - PYTHON=2.7 TESTS=true PACKAGES="python-blosc futures faulthandler" - PYTHON=3.5.4 TESTS=true COVERAGE=true PACKAGES=python-blosc CRICK=true - PYTHON=3.6 TESTS=true matrix: fast_finish: true include: - os: linux # Using Travis-CI's python makes job faster by not downloading miniconda python: 3.6 env: LINT=true - os: osx env: PYTHON=3.6 RUNSLOW=false # Tornado dev builds disabled, as Bokeh and IPython choke on it #- env: PYTHON=3.6 TORNADO=dev #- env: PYTHON=2.7 TORNADO=dev RUNSLOW=false PACKAGES="futures faulthandler" # Together with fast_finish, allow build to be marked successful before the OS X job finishes allow_failures: - os: osx # This needs to be the exact same line as above env: PYTHON=3.6 RUNSLOW=false install: - if [[ $TESTS == true ]]; then source continuous_integration/travis/install.sh ; fi script: - if [[ $TESTS == true ]]; then source continuous_integration/travis/run_tests.sh ; fi - if [[ $LINT == true ]]; then pip install flake8 ; flake8 distributed ; fi after_success: - if [[ $COVERAGE == true ]]; then coverage report; pip install -q coveralls ; coveralls ; fi notifications: email: false distributed-1.20.2/AUTHORS.md000066400000000000000000000001501321233345200155570ustar00rootroot00000000000000[Matthew Rocklin](http://matthewrocklin.com) [Michael Broxton](http://graphics.stanford.edu/~broxton/) distributed-1.20.2/LICENSE.txt000066400000000000000000000027201321233345200157400ustar00rootroot00000000000000Copyright (c) 2015-2017, Anaconda, Inc. and contributors All rights reserved. Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. Neither the name of Anaconda nor the names of any contributors may be used to endorse or promote products derived from this software without specific prior written permission. THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. distributed-1.20.2/MANIFEST.in000066400000000000000000000007221321233345200156530ustar00rootroot00000000000000recursive-include distributed *.py recursive-include distributed *.js recursive-include distributed *.coffee recursive-include distributed *.html recursive-include docs *.rst include setup.py include README.rst include LICENSE.txt include MANIFEST.in include requirements.txt include distributed/config.yaml include distributed/tests/mytestegg-1.0.0-py3.4.egg include distributed/tests/*.pem prune docs/_build include versioneer.py include distributed/_version.py distributed-1.20.2/README.rst000066400000000000000000000002471321233345200156060ustar00rootroot00000000000000Distributed =========== A library for distributed computation. See documentation_ for more details. .. _documentation: https://distributed.readthedocs.io/en/latest distributed-1.20.2/appveyor.yml000066400000000000000000000024761321233345200165150ustar00rootroot00000000000000# Environment loosely based on https://github.com/conda/conda/blob/master/appveyor.yml environment: global: # SDK v7.0 MSVC Express 2008's SetEnv.cmd script will fail if the # /E:ON and /V:ON options are not enabled in the batch script intepreter # See: http://stackoverflow.com/a/13751649/163740 CMD_IN_ENV: "cmd /E:ON /V:ON /C .\\continuous_integration\\run_with_env.cmd" JUNIT_OUT: junit-results.xml matrix: # Since appveyor is quite slow, we only use a single configuration - PYTHON: "3.5" ARCH: "64" CONDA_ENV: testenv init: # Use AppVeyor's provided Miniconda: https://www.appveyor.com/docs/installed-software#python - if "%ARCH%" == "64" set MINICONDA=C:\Miniconda35-x64 - if "%ARCH%" == "32" set MINICONDA=C:\Miniconda35 - set PATH=%MINICONDA%;%MINICONDA%/Scripts;%MINICONDA%/Library/bin;%PATH% install: - continuous_integration\\setup_conda_environment.cmd build_script: - continuous_integration\\build.cmd test_script: # %CMD_IN_ENV% is needed for distutils/setuptools-based tests # on certain build configurations. - "%CMD_IN_ENV% continuous_integration\\run_tests.cmd" on_finish: - ps: $wc = New-Object 'System.Net.WebClient' - ps: $wc.UploadFile("https://ci.appveyor.com/api/testresults/junit/$($env:APPVEYOR_JOB_ID)", (Resolve-Path "$($env:JUNIT_OUT)")) distributed-1.20.2/conftest.py000066400000000000000000000011211321233345200163060ustar00rootroot00000000000000# https://pytest.org/latest/example/simple.html#control-skipping-of-tests-according-to-command-line-option import os import pytest # Uncomment to enable more logging and checks # (https://docs.python.org/3/library/asyncio-dev.html) # Note this makes things slower and might consume much memory. #os.environ["PYTHONASYNCIODEBUG"] = "1" try: import faulthandler except ImportError: pass else: faulthandler.enable() def pytest_addoption(parser): parser.addoption("--runslow", action="store_true", help="run slow tests") pytest_plugins = ['distributed.pytest_resourceleaks'] distributed-1.20.2/continuous_integration/000077500000000000000000000000001321233345200207255ustar00rootroot00000000000000distributed-1.20.2/continuous_integration/build.cmd000066400000000000000000000001331321233345200225060ustar00rootroot00000000000000call activate %CONDA_ENV% @echo on @rem Install Distributed %PIP_INSTALL% --no-deps -e . distributed-1.20.2/continuous_integration/run_tests.cmd000066400000000000000000000002721321233345200234410ustar00rootroot00000000000000call activate %CONDA_ENV% @echo on set PYTHONFAULTHANDLER=1 set PYTEST=py.test --tb=native --timeout=120 -r s %PYTEST% -v -m "not avoid_travis" --junit-xml="%JUNIT_OUT%" distributed distributed-1.20.2/continuous_integration/run_with_env.cmd000066400000000000000000000064721321233345200241320ustar00rootroot00000000000000:: From https://github.com/ogrisel/python-appveyor-demo :: :: To build extensions for 64 bit Python 3, we need to configure environment :: variables to use the MSVC 2010 C++ compilers from GRMSDKX_EN_DVD.iso of: :: MS Windows SDK for Windows 7 and .NET Framework 4 (SDK v7.1) :: :: To build extensions for 64 bit Python 2, we need to configure environment :: variables to use the MSVC 2008 C++ compilers from GRMSDKX_EN_DVD.iso of: :: MS Windows SDK for Windows 7 and .NET Framework 3.5 (SDK v7.0) :: :: 32 bit builds, and 64-bit builds for 3.5 and beyond, do not require specific :: environment configurations. :: :: Note: this script needs to be run with the /E:ON and /V:ON flags for the :: cmd interpreter, at least for (SDK v7.0) :: :: More details at: :: https://github.com/cython/cython/wiki/64BitCythonExtensionsOnWindows :: http://stackoverflow.com/a/13751649/163740 :: :: Author: Olivier Grisel :: License: CC0 1.0 Universal: http://creativecommons.org/publicdomain/zero/1.0/ :: :: Notes about batch files for Python people: :: :: Quotes in values are literally part of the values: :: SET FOO="bar" :: FOO is now five characters long: " b a r " :: If you don't want quotes, don't include them on the right-hand side. :: :: The CALL lines at the end of this file look redundant, but if you move them :: outside of the IF clauses, they do not run properly in the SET_SDK_64==Y :: case, I don't know why. @ECHO OFF SET COMMAND_TO_RUN=%* SET WIN_SDK_ROOT=C:\Program Files\Microsoft SDKs\Windows SET WIN_WDK=c:\Program Files (x86)\Windows Kits\10\Include\wdf :: Extract the major and minor versions, and allow for the minor version to be :: more than 9. This requires the version number to have two dots in it. SET MAJOR_PYTHON_VERSION=%PYTHON:~0,1% IF "%PYTHON:~3,1%" == "." ( SET MINOR_PYTHON_VERSION=%PYTHON:~2,1% ) ELSE ( SET MINOR_PYTHON_VERSION=%PYTHON:~2,2% ) :: Based on the Python version, determine what SDK version to use, and whether :: to set the SDK for 64-bit. IF %MAJOR_PYTHON_VERSION% == 2 ( SET WINDOWS_SDK_VERSION="v7.0" SET SET_SDK_64=Y ) ELSE ( IF %MAJOR_PYTHON_VERSION% == 3 ( SET WINDOWS_SDK_VERSION="v7.1" IF %MINOR_PYTHON_VERSION% LEQ 4 ( SET SET_SDK_64=Y ) ELSE ( SET SET_SDK_64=N IF EXIST "%WIN_WDK%" ( :: See: https://connect.microsoft.com/VisualStudio/feedback/details/1610302/ REN "%WIN_WDK%" 0wdf ) ) ) ELSE ( ECHO Unsupported Python version: "%MAJOR_PYTHON_VERSION%" EXIT 1 ) ) IF %ARCH% == 64 ( IF %SET_SDK_64% == Y ( ECHO Configuring Windows SDK %WINDOWS_SDK_VERSION% for Python %MAJOR_PYTHON_VERSION% on a 64 bit architecture SET DISTUTILS_USE_SDK=1 SET MSSdk=1 "%WIN_SDK_ROOT%\%WINDOWS_SDK_VERSION%\Setup\WindowsSdkVer.exe" -q -version:%WINDOWS_SDK_VERSION% "%WIN_SDK_ROOT%\%WINDOWS_SDK_VERSION%\Bin\SetEnv.cmd" /x64 /release ECHO Executing: %COMMAND_TO_RUN% call %COMMAND_TO_RUN% || EXIT 1 ) ELSE ( ECHO Using default MSVC build environment for 64 bit architecture ECHO Executing: %COMMAND_TO_RUN% call %COMMAND_TO_RUN% || EXIT 1 ) ) ELSE ( ECHO Using default MSVC build environment for 32 bit architecture ECHO Executing: %COMMAND_TO_RUN% call %COMMAND_TO_RUN% || EXIT 1 ) distributed-1.20.2/continuous_integration/setup_conda_environment.cmd000066400000000000000000000026041321233345200263440ustar00rootroot00000000000000@rem The cmd /C hack circumvents a regression where conda installs a conda.bat @rem script in non-root environments. set CONDA=cmd /C conda set CONDA_INSTALL=%CONDA% install -q -y set PIP_INSTALL=pip install -q @echo on @rem Deactivate any environment call deactivate @rem Update conda %CONDA% update -q -y conda @rem Display root environment (for debugging) %CONDA% list @rem Clean up any left-over from a previous build %CONDA% remove --all -q -y -n %CONDA_ENV% @rem Create test environment @rem (note: no cytoolz as it seems to prevent faulthandler tracebacks on crash) %CONDA% create -n %CONDA_ENV% -q -y ^ bokeh ^ click ^ cloudpickle ^ dask ^ dill ^ futures ^ lz4 ^ ipykernel ^ ipywidgets ^ joblib ^ jupyter_client ^ mock ^ msgpack-python ^ psutil ^ pytest=3.1 ^ python=%PYTHON% ^ requests ^ toolz ^ tblib ^ tornado=4.5 ^ zict ^ -c conda-forge call activate %CONDA_ENV% %CONDA% uninstall -q -y --force dask joblib zict %PIP_INSTALL% git+https://github.com/dask/dask --upgrade %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 @rem Display final environment (for reproducing) %CONDA% list %CONDA% list --explicit where python where pip pip list python -m site distributed-1.20.2/continuous_integration/travis/000077500000000000000000000000001321233345200222355ustar00rootroot00000000000000distributed-1.20.2/continuous_integration/travis/install.sh000066400000000000000000000041441321233345200242420ustar00rootroot00000000000000# # This file should be source'd, so as to update the caller's environment # (such as the PATH variable) # # Note we disable progress bars to make Travis log loading much faster # Install conda case "$(uname -s)" in 'Darwin') MINICONDA_FILENAME="Miniconda3-latest-MacOSX-x86_64.sh" ;; 'Linux') MINICONDA_FILENAME="Miniconda3-latest-Linux-x86_64.sh" ;; *) ;; esac wget https://repo.continuum.io/miniconda/$MINICONDA_FILENAME -O miniconda.sh bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" conda config --set always_yes yes --set changeps1 no conda update -q conda # Create conda environment conda create -q -n test-environment python=$PYTHON source activate test-environment # Install dependencies # (Tornado pinned to 4.5 until we fix our compatibility with Tornado 5.0) conda install -q -c conda-forge \ bokeh \ click \ coverage \ dask \ dill \ flake8 \ h5py \ ipykernel \ ipywidgets \ joblib \ jupyter_client \ lz4 \ mock \ netcdf4 \ paramiko \ psutil \ pytest=3.1 \ pytest-faulthandler \ pytest-timeout \ python=$PYTHON \ requests \ tblib \ toolz \ tornado=4.5 \ $PACKAGES pip install -q pytest-repeat pip install -q git+https://github.com/dask/dask.git --upgrade pip install -q git+https://github.com/joblib/joblib.git --upgrade pip install -q git+https://github.com/dask/s3fs.git --upgrade pip install -q git+https://github.com/dask/zict.git --upgrade pip install -q sortedcollections msgpack-python pip install -q keras --upgrade --no-deps if [[ $CRICK == true ]]; then conda install -q cython pip install -q git+https://github.com/jcrist/crick.git fi; # Install distributed pip install --no-deps -e . # Update Tornado to desired version if [[ $TORNADO == "dev" ]]; then pip install -U https://github.com/tornadoweb/tornado/archive/master.zip elif [[ ! -z $TORNADO ]]; then pip install -U tornado==$TORNADO fi # For debugging echo -e "--\n--Conda Environment\n--" conda list echo -e "--\n--Pip Environment\n--" pip list --format=columns distributed-1.20.2/continuous_integration/travis/run_tests.sh000066400000000000000000000011461321233345200246210ustar00rootroot00000000000000export PYTEST_OPTIONS="--verbose -r s --timeout-method=thread --timeout=300 --durations=20" if [[ $RUNSLOW != false ]]; then export PYTEST_OPTIONS="$PYTEST_OPTIONS --runslow" fi # On OS X builders, the default open files limit is too small (256) if [[ $TRAVIS_OS_NAME == osx ]]; then ulimit -n 8192 fi echo "--" echo "-- Soft limits" echo "--" ulimit -a -S echo "--" echo "-- Hard limits" echo "--" ulimit -a -H if [[ $COVERAGE == true ]]; then coverage run $(which py.test) distributed -m "not avoid_travis" $PYTEST_OPTIONS; else py.test -m "not avoid_travis" distributed $PYTEST_OPTIONS; fi; distributed-1.20.2/dev-requirements.txt000066400000000000000000000002771321233345200201620ustar00rootroot00000000000000joblib >= 0.10.2 mock >= 2.0.0 pandas >= 0.19.2 numpy >= 1.11.0 bokeh >= 0.12.3 requests >= 2.12.4 pyzmq >= 16.0.2 ipython >= 5.0.0 jupyter_client >= 4.4.0 ipykernel >= 4.5.2 pytest >= 3.0.5 distributed-1.20.2/distributed/000077500000000000000000000000001321233345200164365ustar00rootroot00000000000000distributed-1.20.2/distributed/__init__.py000066400000000000000000000015221321233345200205470ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from .config import config from .core import connect, rpc from .deploy import LocalCluster from .diagnostics import progress from .client import (Client, Executor, CompatibleExecutor, wait, as_completed, default_client, fire_and_forget, Future) from .lock import Lock from .nanny import Nanny from .queues import Queue from .scheduler import Scheduler from .threadpoolexecutor import rejoin from .utils import sync from .variable import Variable from .worker import Worker, get_worker, get_client, secede, Reschedule from .worker_client import local_client, worker_client from ._version import get_versions versions = get_versions() __version__ = versions['version'] __git_revision__ = versions['full-revisionid'] del get_versions, versions distributed-1.20.2/distributed/_concurrent_futures_thread.py000066400000000000000000000126071321233345200244430ustar00rootroot00000000000000# This was copied from CPython 3.6 # Copyright 2009 Brian Quinlan. All Rights Reserved. # Licensed to PSF under a Contributor Agreement. """Implements ThreadPoolExecutor.""" __author__ = 'Brian Quinlan (brian@sweetapp.com)' import atexit from concurrent.futures import _base import itertools try: import queue except ImportError: import Queue as queue import threading import weakref import os # Workers are created as daemon threads. This is done to allow the interpreter # to exit when there are still idle threads in a ThreadPoolExecutor's thread # pool (i.e. shutdown() was not called). However, allowing workers to die with # the interpreter has two undesirable properties: # - The workers would still be running during interpreter shutdown, # meaning that they would fail in unpredictable ways. # - The workers could be killed while evaluating a work item, which could # be bad if the callable being evaluated has external side-effects e.g. # writing to a file. # # To work around this problem, an exit handler is installed which tells the # workers to exit when their work queues are empty and then waits until the # threads finish. _threads_queues = weakref.WeakKeyDictionary() _shutdown = False def _python_exit(): global _shutdown _shutdown = True items = list(_threads_queues.items()) for t, q in items: q.put(None) for t, q in items: t.join() atexit.register(_python_exit) class _WorkItem(object): def __init__(self, future, fn, args, kwargs): self.future = future self.fn = fn self.args = args self.kwargs = kwargs def run(self): if not self.future.set_running_or_notify_cancel(): return try: result = self.fn(*self.args, **self.kwargs) except BaseException as e: self.future.set_exception(e) else: self.future.set_result(result) def _worker(executor_reference, work_queue): try: while True: work_item = work_queue.get(block=True) if work_item is not None: work_item.run() # Delete references to object. See issue16284 del work_item continue executor = executor_reference() # Exit if: # - The interpreter is shutting down OR # - The executor that owns the worker has been collected OR # - The executor that owns the worker has been shutdown. if _shutdown or executor is None or executor._shutdown: # Notice other workers work_queue.put(None) return del executor except BaseException: _base.LOGGER.critical('Exception in worker', exc_info=True) class ThreadPoolExecutor(_base.Executor): # Used to assign unique thread names when thread_name_prefix is not supplied. _counter = itertools.count() def __init__(self, max_workers=None, thread_name_prefix=''): """Initializes a new ThreadPoolExecutor instance. Args: max_workers: The maximum number of threads that can be used to execute the given calls. thread_name_prefix: An optional name prefix to give our threads. """ if max_workers is None: # Use this number because ThreadPoolExecutor is often # used to overlap I/O instead of CPU work. max_workers = (os.cpu_count() or 1) * 5 if max_workers <= 0: raise ValueError("max_workers must be greater than 0") self._max_workers = max_workers self._work_queue = queue.Queue() self._threads = set() self._shutdown = False self._shutdown_lock = threading.Lock() self._thread_name_prefix = (thread_name_prefix or ("ThreadPoolExecutor-%d" % next(self._counter))) def submit(self, fn, *args, **kwargs): with self._shutdown_lock: if self._shutdown: raise RuntimeError('cannot schedule new futures after shutdown') f = _base.Future() w = _WorkItem(f, fn, args, kwargs) self._work_queue.put(w) self._adjust_thread_count() return f submit.__doc__ = _base.Executor.submit.__doc__ def _adjust_thread_count(self): # When the executor gets lost, the weakref callback will wake up # the worker threads. def weakref_cb(_, q=self._work_queue): q.put(None) # TODO(bquinlan): Should avoid creating new threads if there are more # idle threads than items in the work queue. num_threads = len(self._threads) if num_threads < self._max_workers: thread_name = '%s_%d' % (self._thread_name_prefix or self, num_threads) t = threading.Thread(name=thread_name, target=_worker, args=(weakref.ref(self, weakref_cb), self._work_queue)) t.daemon = True t.start() self._threads.add(t) _threads_queues[t] = self._work_queue def shutdown(self, wait=True): with self._shutdown_lock: self._shutdown = True self._work_queue.put(None) if wait: for t in self._threads: t.join() shutdown.__doc__ = _base.Executor.shutdown.__doc__ distributed-1.20.2/distributed/_ipython_utils.py000066400000000000000000000173351321233345200220720ustar00rootroot00000000000000"""Utilities for integrating with IPython These functions should probably reside in Jupyter and IPython repositories, after which we can import them instead of having our own definitions. """ from __future__ import print_function import atexit import os try: import queue except ImportError: # Python 2 import Queue as queue from subprocess import Popen import sys from threading import Thread from uuid import uuid4 from tornado.gen import TimeoutError from tornado.ioloop import IOLoop from threading import Event from IPython import get_ipython from jupyter_client import BlockingKernelClient, write_connection_file from jupyter_core.paths import jupyter_runtime_dir OUTPUT_TIMEOUT = 10 def run_cell_remote(ip, kc, cell): """Run a cell on a KernelClient Any output from the cell will be redisplayed in the local session. """ msg_id = kc.execute(cell) in_kernel = getattr(ip, 'kernel', False) if in_kernel: socket = ip.display_pub.pub_socket session = ip.display_pub.session parent_header = ip.display_pub.parent_header while True: try: msg = kc.get_iopub_msg(timeout=OUTPUT_TIMEOUT) except queue.Empty: raise TimeoutError("Timeout waiting for IPython output") if msg['parent_header'].get('msg_id') != msg_id: continue msg_type = msg['header']['msg_type'] content = msg['content'] if msg_type == 'status': if content['execution_state'] == 'idle': # idle means output is done break elif msg_type == 'stream': stream = getattr(sys, content['name']) stream.write(content['text']) elif msg_type in ('display_data', 'execute_result', 'error'): if in_kernel: session.send(socket, msg_type, content, parent=parent_header) else: if msg_type == 'error': print('\n'.join(content['traceback']), file=sys.stderr) else: sys.stdout.write(content['data'].get('text/plain', '')) else: pass def register_worker_magic(connection_info, magic_name='worker'): """Register a %worker magic, given connection_info. Both a line and cell magic are registered, which run the given cell in a remote kernel. """ ip = get_ipython() info = dict(connection_info) # copy key = info.pop('key') kc = BlockingKernelClient(**connection_info) kc.session.key = key kc.start_channels() def remote(line, cell=None): """Run the current cell on a remote IPython kernel""" if cell is None: # both line and cell magic cell = line run_cell_remote(ip, kc, cell) remote.client = kc # preserve reference on kc, largely for mocking ip.register_magic_function(remote, magic_kind='line', magic_name=magic_name) ip.register_magic_function(remote, magic_kind='cell', magic_name=magic_name) def remote_magic(line, cell=None): """A magic for running code on a specified remote worker The connection_info dict of the worker will be looked up as the first positional arg to the magic. The rest of the line (or the entire cell for a %%cell magic) will be passed to the remote kernel. Usage: info = e.start_ipython(worker)[worker] %remote info print(worker.data) """ # get connection info from IPython's user namespace ip = get_ipython() split_line = line.split(None, 1) info_name = split_line[0] if info_name not in ip.user_ns: raise NameError(info_name) connection_info = dict(ip.user_ns[info_name]) if not cell: # line magic, use the rest of the line if len(split_line) == 1: raise ValueError("I need some code to run!") cell = split_line[1] # turn info dict to hashable str for use as lookup key in _clients cache key = ','.join(map(str, sorted(connection_info.items()))) session_key = connection_info.pop('key') if key in remote_magic._clients: kc = remote_magic._clients[key] else: kc = BlockingKernelClient(**connection_info) kc.session.key = session_key kc.start_channels() kc.wait_for_ready(timeout=10) remote_magic._clients[key] = kc # actually run the code run_cell_remote(ip, kc, cell) # cache clients for re-use in remote magic remote_magic._clients = {} def register_remote_magic(magic_name='remote'): """Define the parameterized %remote magic See remote_magic above for details. """ ip = get_ipython() if ip is None: return # do nothing if IPython's not running ip.register_magic_function(remote_magic, magic_kind='line', magic_name=magic_name) ip.register_magic_function(remote_magic, magic_kind='cell', magic_name=magic_name) def connect_qtconsole(connection_info, name=None, extra_args=None): """Open a QtConsole connected to a worker who has the given future - identify worker with who_has - start IPython kernel on the worker - start qtconsole connected to the kernel """ runtime_dir = jupyter_runtime_dir() if name is None: name = uuid4().hex path = os.path.join(runtime_dir, name + '.json') write_connection_file(path, **connection_info) cmd = ['jupyter', 'qtconsole', '--existing', path] if extra_args: cmd.extend(extra_args) Popen(cmd) @atexit.register def _cleanup_connection_file(): """Cleanup our connection file when we exit.""" try: os.remove(path) except OSError: pass def start_ipython(ip=None, ns=None, log=None): """Start an IPython kernel in a thread Parameters ---------- ip: str The IP address to listen on (likely the parent object's ip). ns: dict Any names that should be injected into the IPython namespace. log: logger instance Hook up IPython's logging to an existing logger instead of the default. """ from IPython import get_ipython if get_ipython() is not None: raise RuntimeError("Cannot start IPython, it's already running.") from zmq.eventloop.ioloop import ZMQIOLoop from ipykernel.kernelapp import IPKernelApp # save the global IOLoop instance # since IPython relies on it, but we are going to put it in a thread. save_inst = IOLoop.instance() IOLoop.clear_instance() zmq_loop = ZMQIOLoop() zmq_loop.install() # start IPython, disabling its signal handlers that won't work due to running in a thread: app = IPKernelApp.instance(log=log) # Don't connect to the history database app.config.HistoryManager.hist_file = ':memory:' # listen on all interfaces, so remote clients can connect: if ip: app.ip = ip # disable some signal handling, logging def noop(): return None app.init_signal = noop app.log_connection_info = noop # start IPython in a thread # initialization happens in the thread to avoid threading problems # with the sqlite history evt = Event() def _start(): app.initialize([]) app.kernel.pre_handler_hook = noop app.kernel.post_handler_hook = noop app.kernel.start() app.kernel.loop = IOLoop.instance() # save self in the IPython namespace as 'worker' # inject things into the IPython namespace if ns: app.kernel.shell.user_ns.update(ns) evt.set() zmq_loop.start() zmq_loop_thread = Thread(target=_start) zmq_loop_thread.daemon = True zmq_loop_thread.start() assert evt.wait(timeout=5), "IPython didn't start in a reasonable amount of time." # put the global IOLoop instance back: IOLoop.clear_instance() save_inst.install() return app distributed-1.20.2/distributed/_version.py000066400000000000000000000440341321233345200206410ustar00rootroot00000000000000 # This file helps to compute a version number in source trees obtained from # git-archive tarball (such as those provided by githubs download-from-tag # feature). Distribution tarballs (built by setup.py sdist) and build # directories (produced by setup.py build) will contain a much shorter file # that just contains the computed version number. # This file is released into the public domain. Generated by # versioneer-0.17 (https://github.com/warner/python-versioneer) """Git implementation of _version.py.""" import errno import os import re import subprocess import sys def get_keywords(): """Get the keywords needed to look up the version information.""" # these strings will be replaced by git during git-archive. # setup.py/versioneer.py will grep for the variable names, so they must # each be defined on a line of their own. _version.py will just call # get_keywords(). git_refnames = "$Format:%d$" git_full = "$Format:%H$" git_date = "$Format:%ci$" keywords = {"refnames": git_refnames, "full": git_full, "date": git_date} return keywords class VersioneerConfig: """Container for Versioneer configuration parameters.""" def get_config(): """Create, populate and return the VersioneerConfig() object.""" # these strings are filled in when 'setup.py versioneer' creates # _version.py cfg = VersioneerConfig() cfg.VCS = "git" cfg.style = "pep440" cfg.tag_prefix = "" cfg.parentdir_prefix = "distributed-" cfg.versionfile_source = "distributed/_version.py" cfg.verbose = False return cfg class NotThisMethod(Exception): """Exception raised if a method is not valid for the current scenario.""" LONG_VERSION_PY = {} HANDLERS = {} def register_vcs_handler(vcs, method): # decorator """Decorator to mark a method as the handler for a particular VCS.""" def decorate(f): """Store f in HANDLERS[vcs][method].""" if vcs not in HANDLERS: HANDLERS[vcs] = {} HANDLERS[vcs][method] = f return f return decorate def run_command(commands, args, cwd=None, verbose=False, hide_stderr=False, env=None): """Call the given command(s).""" assert isinstance(commands, list) p = None for c in commands: try: dispcmd = str([c] + args) # remember shell=False, so use git.cmd on windows, not just git p = subprocess.Popen([c] + args, cwd=cwd, env=env, stdout=subprocess.PIPE, stderr=(subprocess.PIPE if hide_stderr else None)) break except EnvironmentError: e = sys.exc_info()[1] if e.errno == errno.ENOENT: continue if verbose: print("unable to run %s" % dispcmd) print(e) return None, None else: if verbose: print("unable to find command, tried %s" % (commands,)) return None, None stdout = p.communicate()[0].strip() if sys.version_info[0] >= 3: stdout = stdout.decode() if p.returncode != 0: if verbose: print("unable to run %s (error)" % dispcmd) print("stdout was %s" % stdout) return None, p.returncode return stdout, p.returncode def versions_from_parentdir(parentdir_prefix, root, verbose): """Try to determine the version from the parent directory name. Source tarballs conventionally unpack into a directory that includes both the project name and a version string. We will also support searching up two directory levels for an appropriately named parent directory """ rootdirs = [] for i in range(3): dirname = os.path.basename(root) if dirname.startswith(parentdir_prefix): return {"version": dirname[len(parentdir_prefix):], "full-revisionid": None, "dirty": False, "error": None, "date": None} else: rootdirs.append(root) root = os.path.dirname(root) # up a level if verbose: print("Tried directories %s but none started with prefix %s" % (str(rootdirs), parentdir_prefix)) raise NotThisMethod("rootdir doesn't start with parentdir_prefix") @register_vcs_handler("git", "get_keywords") def git_get_keywords(versionfile_abs): """Extract version information from the given file.""" # the code embedded in _version.py can just fetch the value of these # keywords. When used from setup.py, we don't want to import _version.py, # so we do it with a regexp instead. This function is not used from # _version.py. keywords = {} try: f = open(versionfile_abs, "r") for line in f.readlines(): if line.strip().startswith("git_refnames ="): mo = re.search(r'=\s*"(.*)"', line) if mo: keywords["refnames"] = mo.group(1) if line.strip().startswith("git_full ="): mo = re.search(r'=\s*"(.*)"', line) if mo: keywords["full"] = mo.group(1) if line.strip().startswith("git_date ="): mo = re.search(r'=\s*"(.*)"', line) if mo: keywords["date"] = mo.group(1) f.close() except EnvironmentError: pass return keywords @register_vcs_handler("git", "keywords") def git_versions_from_keywords(keywords, tag_prefix, verbose): """Get version information from git keywords.""" if not keywords: raise NotThisMethod("no keywords at all, weird") date = keywords.get("date") if date is not None: # git-2.2.0 added "%cI", which expands to an ISO-8601 -compliant # datestamp. However we prefer "%ci" (which expands to an "ISO-8601 # -like" string, which we must then edit to make compliant), because # it's been around since git-1.5.3, and it's too difficult to # discover which version we're using, or to work around using an # older one. date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) refnames = keywords["refnames"].strip() if refnames.startswith("$Format"): if verbose: print("keywords are unexpanded, not using") raise NotThisMethod("unexpanded keywords, not a git-archive tarball") refs = set([r.strip() for r in refnames.strip("()").split(",")]) # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of # just "foo-1.0". If we see a "tag: " prefix, prefer those. TAG = "tag: " tags = set([r[len(TAG):] for r in refs if r.startswith(TAG)]) if not tags: # Either we're using git < 1.8.3, or there really are no tags. We use # a heuristic: assume all version tags have a digit. The old git %d # expansion behaves like git log --decorate=short and strips out the # refs/heads/ and refs/tags/ prefixes that would let us distinguish # between branches and tags. By ignoring refnames without digits, we # filter out many common branch names like "release" and # "stabilization", as well as "HEAD" and "master". tags = set([r for r in refs if re.search(r'\d', r)]) if verbose: print("discarding '%s', no digits" % ",".join(refs - tags)) if verbose: print("likely tags: %s" % ",".join(sorted(tags))) for ref in sorted(tags): # sorting will prefer e.g. "2.0" over "2.0rc1" if ref.startswith(tag_prefix): r = ref[len(tag_prefix):] if verbose: print("picking %s" % r) return {"version": r, "full-revisionid": keywords["full"].strip(), "dirty": False, "error": None, "date": date} # no suitable tags, so version is "0+unknown", but full hex is still there if verbose: print("no suitable tags, using unknown + full revision id") return {"version": "0+unknown", "full-revisionid": keywords["full"].strip(), "dirty": False, "error": "no suitable tags", "date": None} @register_vcs_handler("git", "pieces_from_vcs") def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): """Get version from 'git describe' in the root of the source tree. This only gets called if the git-archive 'subst' keywords were *not* expanded, and _version.py hasn't already been rewritten with a short version string, meaning we're inside a checked out source tree. """ GITS = ["git"] if sys.platform == "win32": GITS = ["git.cmd", "git.exe"] out, rc = run_command(GITS, ["rev-parse", "--git-dir"], cwd=root, hide_stderr=True) if rc != 0: if verbose: print("Directory %s not under git control" % root) raise NotThisMethod("'git rev-parse --git-dir' returned error") # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] # if there isn't one, this yields HEX[-dirty] (no NUM) describe_out, rc = run_command(GITS, ["describe", "--tags", "--dirty", "--always", "--long", "--match", "%s*" % tag_prefix], cwd=root) # --long was added in git-1.5.5 if describe_out is None: raise NotThisMethod("'git describe' failed") describe_out = describe_out.strip() full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) if full_out is None: raise NotThisMethod("'git rev-parse' failed") full_out = full_out.strip() pieces = {} pieces["long"] = full_out pieces["short"] = full_out[:7] # maybe improved later pieces["error"] = None # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] # TAG might have hyphens. git_describe = describe_out # look for -dirty suffix dirty = git_describe.endswith("-dirty") pieces["dirty"] = dirty if dirty: git_describe = git_describe[:git_describe.rindex("-dirty")] # now we have TAG-NUM-gHEX or HEX if "-" in git_describe: # TAG-NUM-gHEX mo = re.search(r'^(.+)-(\d+)-g([0-9a-f]+)$', git_describe) if not mo: # unparseable. Maybe git-describe is misbehaving? pieces["error"] = ("unable to parse git-describe output: '%s'" % describe_out) return pieces # tag full_tag = mo.group(1) if not full_tag.startswith(tag_prefix): if verbose: fmt = "tag '%s' doesn't start with prefix '%s'" print(fmt % (full_tag, tag_prefix)) pieces["error"] = ("tag '%s' doesn't start with prefix '%s'" % (full_tag, tag_prefix)) return pieces pieces["closest-tag"] = full_tag[len(tag_prefix):] # distance: number of commits since tag pieces["distance"] = int(mo.group(2)) # commit: short hex revision ID pieces["short"] = mo.group(3) else: # HEX: no tags pieces["closest-tag"] = None count_out, rc = run_command(GITS, ["rev-list", "HEAD", "--count"], cwd=root) pieces["distance"] = int(count_out) # total number of commits # commit date: see ISO-8601 comment in git_versions_from_keywords() date = run_command(GITS, ["show", "-s", "--format=%ci", "HEAD"], cwd=root)[0].strip() pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) return pieces def plus_or_dot(pieces): """Return a + if we don't already have one, else return a .""" if "+" in pieces.get("closest-tag", ""): return "." return "+" def render_pep440(pieces): """Build up version string, with post-release "local version identifier". Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty Exceptions: 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"] or pieces["dirty"]: rendered += plus_or_dot(pieces) rendered += "%d.g%s" % (pieces["distance"], pieces["short"]) if pieces["dirty"]: rendered += ".dirty" else: # exception #1 rendered = "0+untagged.%d.g%s" % (pieces["distance"], pieces["short"]) if pieces["dirty"]: rendered += ".dirty" return rendered def render_pep440_pre(pieces): """TAG[.post.devDISTANCE] -- No -dirty. Exceptions: 1: no tags. 0.post.devDISTANCE """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"]: rendered += ".post.dev%d" % pieces["distance"] else: # exception #1 rendered = "0.post.dev%d" % pieces["distance"] return rendered def render_pep440_post(pieces): """TAG[.postDISTANCE[.dev0]+gHEX] . The ".dev0" means dirty. Note that .dev0 sorts backwards (a dirty tree will appear "older" than the corresponding clean one), but you shouldn't be releasing software with -dirty anyways. Exceptions: 1: no tags. 0.postDISTANCE[.dev0] """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"] or pieces["dirty"]: rendered += ".post%d" % pieces["distance"] if pieces["dirty"]: rendered += ".dev0" rendered += plus_or_dot(pieces) rendered += "g%s" % pieces["short"] else: # exception #1 rendered = "0.post%d" % pieces["distance"] if pieces["dirty"]: rendered += ".dev0" rendered += "+g%s" % pieces["short"] return rendered def render_pep440_old(pieces): """TAG[.postDISTANCE[.dev0]] . The ".dev0" means dirty. Eexceptions: 1: no tags. 0.postDISTANCE[.dev0] """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"] or pieces["dirty"]: rendered += ".post%d" % pieces["distance"] if pieces["dirty"]: rendered += ".dev0" else: # exception #1 rendered = "0.post%d" % pieces["distance"] if pieces["dirty"]: rendered += ".dev0" return rendered def render_git_describe(pieces): """TAG[-DISTANCE-gHEX][-dirty]. Like 'git describe --tags --dirty --always'. Exceptions: 1: no tags. HEX[-dirty] (note: no 'g' prefix) """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"]: rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) else: # exception #1 rendered = pieces["short"] if pieces["dirty"]: rendered += "-dirty" return rendered def render_git_describe_long(pieces): """TAG-DISTANCE-gHEX[-dirty]. Like 'git describe --tags --dirty --always -long'. The distance/hash is unconditional. Exceptions: 1: no tags. HEX[-dirty] (note: no 'g' prefix) """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) else: # exception #1 rendered = pieces["short"] if pieces["dirty"]: rendered += "-dirty" return rendered def render(pieces, style): """Render the given version pieces into the requested style.""" if pieces["error"]: return {"version": "unknown", "full-revisionid": pieces.get("long"), "dirty": None, "error": pieces["error"], "date": None} if not style or style == "default": style = "pep440" # the default if style == "pep440": rendered = render_pep440(pieces) elif style == "pep440-pre": rendered = render_pep440_pre(pieces) elif style == "pep440-post": rendered = render_pep440_post(pieces) elif style == "pep440-old": rendered = render_pep440_old(pieces) elif style == "git-describe": rendered = render_git_describe(pieces) elif style == "git-describe-long": rendered = render_git_describe_long(pieces) else: raise ValueError("unknown style '%s'" % style) return {"version": rendered, "full-revisionid": pieces["long"], "dirty": pieces["dirty"], "error": None, "date": pieces.get("date")} def get_versions(): """Get version information or return default if unable to do so.""" # I am in _version.py, which lives at ROOT/VERSIONFILE_SOURCE. If we have # __file__, we can work backwards from there to the root. Some # py2exe/bbfreeze/non-CPython implementations don't do __file__, in which # case we can only use expanded keywords. cfg = get_config() verbose = cfg.verbose try: return git_versions_from_keywords(get_keywords(), cfg.tag_prefix, verbose) except NotThisMethod: pass try: root = os.path.realpath(__file__) # versionfile_source is the relative path from the top of the source # tree (where the .git directory might live) to this file. Invert # this to find the root from __file__. for i in cfg.versionfile_source.split('/'): root = os.path.dirname(root) except NameError: return {"version": "0+unknown", "full-revisionid": None, "dirty": None, "error": "unable to find root of source tree", "date": None} try: pieces = git_pieces_from_vcs(cfg.tag_prefix, root, verbose) return render(pieces, cfg.style) except NotThisMethod: pass try: if cfg.parentdir_prefix: return versions_from_parentdir(cfg.parentdir_prefix, root, verbose) except NotThisMethod: pass return {"version": "0+unknown", "full-revisionid": None, "dirty": None, "error": "unable to compute version", "date": None} distributed-1.20.2/distributed/asyncio.py000066400000000000000000000072601321233345200204620ustar00rootroot00000000000000"""Experimental interface for asyncio, may disappear without warning""" # flake8: noqa import asyncio from functools import wraps from toolz import merge from tornado.platform.asyncio import BaseAsyncIOLoop from tornado.platform.asyncio import to_asyncio_future from . import client from .client import Client, Future from .variable import Variable from .utils import ignoring def to_asyncio(fn, **default_kwargs): """Converts Tornado gen.coroutines and futures to asyncio ones""" @wraps(fn) def convert(*args, **kwargs): if default_kwargs: kwargs = merge(default_kwargs, kwargs) return to_asyncio_future(fn(*args, **kwargs)) return convert class AioClient(Client): """ Connect to and drive computation on a distributed Dask cluster This class provides an asyncio compatible async/await interface for dask.distributed. 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. AioClient is an **experimental** interface for distributed and may disappear without warning! Parameters ---------- address: string, or Cluster This can be the address of a ``Scheduler`` server like a string ``'127.0.0.1:8786'`` or a cluster object like ``LocalCluster()`` Examples -------- Provide cluster's scheduler address on initialization:: client = AioClient('127.0.0.1:8786') Start the client:: async def start_the_client(): client = await AioClient() # Use the client.... await client.close() An ``async with`` statement is a more convenient way to start and shut down the client:: async def start_the_client(): async with AioClient() as client: # Use the client within this block. pass Use the ``submit`` method to send individual computations to the cluster, and await the returned future to retrieve the result:: async def add_two_numbers(): async with AioClient() as client: a = client.submit(add, 1, 2) result = await a Continue using submit or map on results to build up larger computations, and gather results with the ``gather`` method:: async def gather_some_results(): async with AioClient() as client: a = client.submit(add, 1, 2) b = client.submit(add, 10, 20) c = client.submit(add, a, b) result = await client.gather([c]) See Also -------- distributed.client.Client: Blocking Client distributed.scheduler.Scheduler: Internal scheduler """ def __init__(self, *args, **kwargs): loop = asyncio.get_event_loop() ioloop = BaseAsyncIOLoop(loop) super().__init__(*args, loop=ioloop, asynchronous=True, **kwargs) def __enter__(self): raise RuntimeError("Use AioClient in an 'async with' block, not 'with'") async def __aenter__(self): await to_asyncio_future(self._started) return self async def __aexit__(self, type, value, traceback): await to_asyncio_future(self._close()) def __await__(self): return to_asyncio_future(self._started).__await__() get = to_asyncio(Client.get, sync=False) sync = to_asyncio(Client.sync) close = to_asyncio(Client.close) shutdown = to_asyncio(Client.shutdown) class as_completed(client.as_completed): __anext__ = to_asyncio(client.as_completed.__anext__) wait = to_asyncio(client._wait) distributed-1.20.2/distributed/batched.py000066400000000000000000000101501321233345200203770ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import deque import logging from tornado import gen, locks from tornado.ioloop import IOLoop from .core import CommClosedError logger = logging.getLogger(__name__) class BatchedSend(object): """ Batch messages in batches on a stream This takes an IOStream and an interval (in ms) and ensures that we send no more than one message every interval milliseconds. We send lists of messages. Batching several messages at once helps performance when sending a myriad of tiny messages. Example ------- >>> stream = yield connect(ip, port) >>> bstream = BatchedSend(interval=10) # 10 ms >>> bstream.start(stream) >>> bstream.send('Hello,') >>> bstream.send('world!') On the other side, the recipient will get a message like the following:: ['Hello,', 'world!'] """ # XXX why doesn't BatchedSend follow either the IOStream or Comm API? def __init__(self, interval, loop=None): # XXX is the loop arg useful? self.loop = loop or IOLoop.current() self.interval = interval / 1000. self.waker = locks.Event() self.stopped = locks.Event() self.please_stop = False self.buffer = [] self.comm = None self.message_count = 0 self.batch_count = 0 self.byte_count = 0 self.next_deadline = None self.recent_message_log = deque(maxlen=100) def start(self, comm): self.comm = comm self.loop.add_callback(self._background_send) def __repr__(self): return '' % len(self.buffer) @gen.coroutine def _background_send(self): while not self.please_stop: try: yield self.waker.wait(self.next_deadline) self.waker.clear() except gen.TimeoutError: pass if not self.buffer: # Nothing to send self.next_deadline = None continue if (self.next_deadline is not None and self.loop.time() < self.next_deadline): # Send interval not expired yet continue payload, self.buffer = self.buffer, [] self.batch_count += 1 self.next_deadline = self.loop.time() + self.interval try: nbytes = yield self.comm.write(payload) if nbytes < 1e6: self.recent_message_log.append(payload) else: self.recent_message_log.append('large-message') self.byte_count += nbytes except CommClosedError as e: logger.info("Batched Comm Closed: %s", e) break except Exception: logger.exception("Error in batched write") break finally: payload = None # lose ref self.stopped.set() def send(self, msg): """ Schedule a message for sending to the other side This completes quickly and synchronously """ if self.comm is not None and self.comm.closed(): raise CommClosedError self.message_count += 1 self.buffer.append(msg) # Avoid spurious wakeups if possible if self.next_deadline is None: self.waker.set() @gen.coroutine def close(self): """ Flush existing messages and then close comm """ if self.comm is None: return self.please_stop = True self.waker.set() yield self.stopped.wait() if not self.comm.closed(): try: if self.buffer: self.buffer, payload = [], self.buffer yield self.comm.write(payload) except CommClosedError: pass yield self.comm.close() def abort(self): if self.comm is None: return self.please_stop = True self.buffer = [] self.waker.set() if not self.comm.closed(): self.comm.abort() distributed-1.20.2/distributed/bokeh/000077500000000000000000000000001321233345200175265ustar00rootroot00000000000000distributed-1.20.2/distributed/bokeh/__init__.py000066400000000000000000000022501321233345200216360ustar00rootroot00000000000000from 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()]} } distributed-1.20.2/distributed/bokeh/components.py000066400000000000000000000535471321233345200223030ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from bisect import bisect from operator import add from time import time import weakref from bokeh.layouts import row, column from bokeh.models import ( ColumnDataSource, Plot, DataRange1d, LinearAxis, HoverTool, BoxZoomTool, ResetTool, PanTool, WheelZoomTool, Range1d, Quad, Text, value, TapTool, OpenURL, Button, Select) from bokeh.palettes import Spectral9 from bokeh.plotting import figure from tornado import gen from ..config import config from ..diagnostics.progress_stream import progress_quads, nbytes_bar from .. import profile from ..utils import log_errors if config.get('bokeh-export-tool', False): from .export_tool import ExportTool else: ExportTool = None profile_interval = config.get('profile-interval', 10) / 1000 class DashboardComponent(object): """ Base class for Dask.distributed UI dashboard components. This class must have two attributes, ``root`` and ``source``, and one method ``update``: * source: a Bokeh ColumnDataSource * root: a Bokeh Model * update: a method that consumes the messages dictionary found in distributed.bokeh.messages """ def __init__(self): self.source = None self.root = None def update(self, messages): """ Reads from bokeh.distributed.messages and updates self.source """ class TaskStream(DashboardComponent): """ Task Stream The start and stop time of tasks as they occur on each core of the cluster. """ def __init__(self, n_rectangles=1000, clear_interval=20000, **kwargs): """ kwargs are applied to the bokeh.models.plots.Plot constructor """ self.n_rectangles = n_rectangles self.clear_interval = clear_interval self.last = 0 self.source = ColumnDataSource(data=dict( start=[time() - clear_interval], duration=[0.1], key=['start'], name=['start'], color=['white'], duration_text=['100 ms'], worker=['foo'], y=[0], worker_thread=[1], alpha=[0.0]) ) x_range = DataRange1d(range_padding=0) y_range = DataRange1d(range_padding=0) self.root = figure( title="Task Stream", id='bk-task-stream-plot', x_range=x_range, y_range=y_range, toolbar_location="above", x_axis_type='datetime', min_border_right=35, tools='', **kwargs) self.root.yaxis.axis_label = 'Worker Core' rect = self.root.rect(source=self.source, x="start", y="y", width="duration", height=0.4, fill_color="color", line_color="color", line_alpha=0.6, fill_alpha="alpha", line_width=3) rect.nonselection_glyph = None self.root.yaxis.major_label_text_alpha = 0 self.root.yaxis.minor_tick_line_alpha = 0 self.root.xgrid.visible = False hover = HoverTool( point_policy="follow_mouse", tooltips="""
@name:  @duration_text
""" ) tap = TapTool(callback=OpenURL(url='/profile?key=@name')) self.root.add_tools( hover, tap, BoxZoomTool(), ResetTool(reset_size=False), PanTool(dimensions="width"), WheelZoomTool(dimensions="width") ) if ExportTool: export = ExportTool() export.register_plot(self.root) self.root.add_tools(export) # Required for update callback self.task_stream_index = [0] def update(self, messages): with log_errors(): index = messages['task-events']['index'] rectangles = messages['task-events']['rectangles'] if not index or index[-1] == self.task_stream_index[0]: return ind = bisect(index, self.task_stream_index[0]) rectangles = {k: [v[i] for i in range(ind, len(index))] for k, v in rectangles.items()} self.task_stream_index[0] = index[-1] # If there has been a significant delay then clear old rectangles if rectangles['start']: m = min(map(add, rectangles['start'], rectangles['duration'])) if m > self.last: self.last, last = m, self.last if m > last + self.clear_interval: self.source.data.update(rectangles) return self.source.stream(rectangles, self.n_rectangles) class TaskProgress(DashboardComponent): """ Progress bars per task type """ def __init__(self, **kwargs): data = progress_quads(dict(all={}, memory={}, erred={}, released={})) self.source = ColumnDataSource(data=data) x_range = DataRange1d() y_range = Range1d(-8, 0) self.root = Plot( id='bk-task-progress-plot', x_range=x_range, y_range=y_range, toolbar_location=None, **kwargs ) self.root.add_glyph( self.source, Quad(top='top', bottom='bottom', left='left', right='right', fill_color="#aaaaaa", line_color="#aaaaaa", fill_alpha=0.2) ) self.root.add_glyph( self.source, Quad(top='top', bottom='bottom', left='left', right='released-loc', fill_color="color", line_color="color", fill_alpha=0.6) ) self.root.add_glyph( self.source, Quad(top='top', bottom='bottom', left='released-loc', right='memory-loc', fill_color="color", line_color="color", fill_alpha=1.0) ) self.root.add_glyph( self.source, Quad(top='top', bottom='bottom', left='erred-loc', right='erred-loc', fill_color='#000000', line_color='#000000', fill_alpha=0.3) ) self.root.add_glyph( self.source, Text(text='show-name', y='bottom', x='left', x_offset=5, text_font_size=value('10pt')) ) self.root.add_glyph( self.source, Text(text='done', y='bottom', x='right', x_offset=-5, text_align='right', text_font_size=value('10pt')) ) hover = HoverTool( point_policy="follow_mouse", tooltips="""
Name:  @name
All:  @all
Memory:  @memory
Erred:  @erred
""" ) self.root.add_tools(hover) def update(self, messages): with log_errors(): msg = messages['progress'] if not msg: return d = progress_quads(msg) self.source.data.update(d) if messages['tasks']['deque']: self.root.title.text = ("Progress -- total: %(total)s, " "in-memory: %(in-memory)s, processing: %(processing)s, " "waiting: %(waiting)s, failed: %(failed)s" % messages['tasks']['deque'][-1]) class MemoryUsage(DashboardComponent): """ The memory usage across the cluster, grouped by task type """ def __init__(self, **kwargs): self.source = ColumnDataSource(data=dict( name=[], left=[], right=[], center=[], color=[], percent=[], MB=[], text=[]) ) self.root = Plot( id='bk-nbytes-plot', x_range=DataRange1d(), y_range=DataRange1d(), toolbar_location=None, outline_line_color=None, **kwargs ) self.root.add_glyph( self.source, Quad(top=1, bottom=0, left='left', right='right', fill_color='color', fill_alpha=1) ) self.root.add_layout(LinearAxis(), 'left') self.root.add_layout(LinearAxis(), 'below') hover = HoverTool( point_policy="follow_mouse", tooltips="""
Name:  @name
Percent:  @percent
MB:  @MB
""" ) self.root.add_tools(hover) def update(self, messages): with log_errors(): msg = messages['progress'] if not msg: return nb = nbytes_bar(msg['nbytes']) self.source.data.update(nb) self.root.title.text = \ "Memory Use: %0.2f MB" % (sum(msg['nbytes'].values()) / 1e6) class Processing(DashboardComponent): """ Processing and distribution per core This shows how many tasks are actively running on each worker and how many tasks are enqueued for each worker and how many are in the common pool """ def __init__(self, **kwargs): data = self.processing_update({'processing': {}, 'ncores': {}}) self.source = ColumnDataSource(data) x_range = Range1d(-1, 1) fig = figure(title='Processing and Pending', tools='', x_range=x_range, id='bk-processing-stacks-plot', **kwargs) fig.quad(source=self.source, left=0, right='right', color=Spectral9[0], top='top', bottom='bottom') fig.xaxis.minor_tick_line_alpha = 0 fig.yaxis.visible = False fig.ygrid.visible = False hover = HoverTool() fig.add_tools(hover) hover = fig.select(HoverTool) hover.tooltips = """
Host:  @name
Processing:  @processing
""" hover.point_policy = 'follow_mouse' self.root = fig def update(self, messages): with log_errors(): msg = messages['processing'] if not msg.get('ncores'): return data = self.processing_update(msg) x_range = self.root.x_range max_right = max(data['right']) cores = max(data['ncores']) if x_range.end < max_right: x_range.end = max_right + 2 elif x_range.end > 2 * max_right + cores: # way out there, walk back x_range.end = x_range.end * 0.95 + max_right * 0.05 self.source.data.update(data) @staticmethod def processing_update(msg): with log_errors(): names = sorted(msg['processing']) names = sorted(names) processing = msg['processing'] processing = [processing[name] for name in names] ncores = msg['ncores'] ncores = [ncores[name] for name in names] n = len(names) d = {'name': list(names), 'processing': processing, 'right': list(processing), 'top': list(range(n, 0, -1)), 'bottom': list(range(n - 1, -1, -1)), 'ncores': ncores} d['alpha'] = [0.7] * n return d class ProfilePlot(DashboardComponent): """ Time plots of the current resource usage on the cluster This is two plots, one for CPU and Memory and another for Network I/O """ def __init__(self, **kwargs): state = profile.create() data = profile.plot_data(state, profile_interval) self.states = data.pop('states') self.source = ColumnDataSource(data=data) def cb(attr, old, new): with log_errors(): try: ind = new['1d']['indices'][0] except IndexError: return data = profile.plot_data(self.states[ind], profile_interval) del self.states[:] self.states.extend(data.pop('states')) self.source.data.update(data) self.source.selected = old self.source.on_change('selected', cb) self.root = figure(tools='tap', **kwargs) self.root.quad('left', 'right', 'top', 'bottom', color='color', line_color='black', line_width=2, source=self.source) hover = HoverTool( point_policy="follow_mouse", tooltips="""
Name:  @name
Filename:  @filename
Line number:  @line_number
Line:  @line
Time:  @time
""" ) self.root.add_tools(hover) self.root.xaxis.visible = False self.root.yaxis.visible = False self.root.grid.visible = False def update(self, state): with log_errors(): self.state = state data = profile.plot_data(self.state, profile_interval) self.states = data.pop('states') self.source.data.update(data) class ProfileTimePlot(DashboardComponent): """ Time plots of the current resource usage on the cluster This is two plots, one for CPU and Memory and another for Network I/O """ def __init__(self, server, doc=None, **kwargs): if doc is not None: self.doc = weakref.ref(doc) try: self.key = doc.session_context.request.arguments.get('key', None) except AttributeError: self.key = None if isinstance(self.key, list): self.key = self.key[0] if isinstance(self.key, bytes): self.key = self.key.decode() self.task_names = ['All', self.key] else: self.key = None self.task_names = ['All'] self.server = server self.start = None self.stop = None self.ts = {'count': [], 'time': []} self.state = profile.create() data = profile.plot_data(self.state, profile_interval) self.states = data.pop('states') self.source = ColumnDataSource(data=data) def cb(attr, old, new): with log_errors(): try: ind = new['1d']['indices'][0] except IndexError: return data = profile.plot_data(self.states[ind], profile_interval) del self.states[:] self.states.extend(data.pop('states')) self.source.data.update(data) self.source.selected = old self.source.on_change('selected', cb) self.profile_plot = figure(tools='tap', height=400, **kwargs) self.profile_plot.quad('left', 'right', 'top', 'bottom', color='color', line_color='black', source=self.source) hover = HoverTool( point_policy="follow_mouse", tooltips="""
Name:  @name
Filename:  @filename
Line number:  @line_number
Line:  @line
Time:  @time
""" ) self.profile_plot.add_tools(hover) self.profile_plot.xaxis.visible = False self.profile_plot.yaxis.visible = False self.profile_plot.grid.visible = False self.ts_source = ColumnDataSource({'time': [], 'count': []}) self.ts_plot = figure(title='Activity over time', height=100, x_axis_type='datetime', active_drag='xbox_select', tools='xpan,xwheel_zoom,xbox_select,reset', **kwargs) self.ts_plot.line('time', 'count', source=self.ts_source) self.ts_plot.circle('time', 'count', source=self.ts_source, color=None, selection_color='orange') self.ts_plot.yaxis.visible = False self.ts_plot.grid.visible = False def ts_change(attr, old, new): with log_errors(): selected = self.ts_source.selected['1d']['indices'] if selected: start = self.ts_source.data['time'][min(selected)] / 1000 stop = self.ts_source.data['time'][max(selected)] / 1000 self.start, self.stop = min(start, stop), max(start, stop) else: self.start = self.stop = None self.trigger_update(update_metadata=False) self.ts_source.on_change('selected', ts_change) self.reset_button = Button(label="Reset", button_type="success") self.reset_button.on_click(lambda: self.update(self.state) ) self.update_button = Button(label="Update", button_type="success") self.update_button.on_click(self.trigger_update) self.select = Select(value=self.task_names[-1], options=self.task_names) def select_cb(attr, old, new): if new == 'All': new = None self.key = new self.trigger_update(update_metadata=False) self.select.on_change('value', select_cb) self.root = column(row(self.select, self.reset_button, self.update_button, sizing_mode='scale_width'), self.profile_plot, self.ts_plot, **kwargs) def update(self, state, metadata=None): with log_errors(): self.state = state data = profile.plot_data(self.state, profile_interval) self.states = data.pop('states') self.source.data.update(data) if metadata is not None and metadata['counts']: self.task_names = ['All'] + sorted(metadata['keys']) self.select.options = self.task_names if self.key: ts = metadata['keys'][self.key] else: ts = metadata['counts'] times, counts = zip(*ts) self.ts = {'count': counts, 'time': [t * 1000 for t in times]} self.ts_source.data.update(self.ts) def trigger_update(self, update_metadata=True): @gen.coroutine def cb(): with log_errors(): prof = self.server.get_profile(key=self.key, start=self.start, stop=self.stop) if update_metadata: metadata = self.server.get_profile_metadata() else: metadata = None if isinstance(prof, gen.Future): prof, metadata = yield [prof, metadata] self.doc().add_next_tick_callback(lambda: self.update(prof, metadata)) self.server.loop.add_callback(cb) distributed-1.20.2/distributed/bokeh/core.py000066400000000000000000000032521321233345200210320ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from distutils.version import LooseVersion import bokeh from bokeh.server.server import Server if LooseVersion(bokeh.__version__) < LooseVersion('0.12.6'): raise ImportError("Dask needs bokeh >= 0.12.6") class BokehServer(object): server_kwargs = {} def listen(self, addr): if self.server: return if isinstance(addr, tuple): ip, port = addr else: port = addr ip = None for i in range(5): try: self.server = Server(self.apps, io_loop=self.loop, port=port, address=ip, check_unused_sessions_milliseconds=500, allow_websocket_origin=["*"], **self.server_kwargs) self.server.start() return except (SystemExit, EnvironmentError): port = 0 if i == 4: raise @property def port(self): return (self.server.port or list(self.server._http._sockets.values())[0].getsockname()[1]) def stop(self): for context in self.server._tornado._applications.values(): context.run_unload_hook() self.server._tornado._stats_job.stop() self.server._tornado._cleanup_job.stop() if self.server._tornado._ping_job is not None: self.server._tornado._ping_job.stop() # https://github.com/bokeh/bokeh/issues/5494 if LooseVersion(bokeh.__version__) >= '0.12.4': self.server.stop() distributed-1.20.2/distributed/bokeh/export_tool.coffee000066400000000000000000000021051321233345200232530ustar00rootroot00000000000000p = require "core/properties" ActionTool = require "models/tools/actions/action_tool" class ExportToolView extends ActionTool.View initialize: (options) -> super(options) @listenTo(@model, 'change:content', @export) do: () -> # This is just to trigger an event a python callback can respond to @model.event = @model.event + 1 export: () -> if @model.content? blob = new Blob([@model.content], {type: "text/plain"}) url = window.URL.createObjectURL(blob); a = document.createElement("a") a.id = "bk-export-tool-link" a.style = "display: none" a.href = url a.download = 'task-stream.html' document.body.appendChild(a) a.click() document.getElementById('bk-export-tool-link').remove() window.URL.revokeObjectURL(url); class ExportTool extends ActionTool.Model default_view: ExportToolView type: "ExportTool" tool_name: "Export" icon: "bk-tool-icon-save" @define { event: [ p.Int, 0 ] content: [ p.String ] } module.exports = Model: ExportTool View: ExportToolView distributed-1.20.2/distributed/bokeh/export_tool.py000066400000000000000000000014321321233345200224560ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import os from bokeh.core.properties import Int, String from bokeh.embed import file_html from bokeh.models import Tool from bokeh.resources import CDN from bokeh.util.compiler import JavaScript fn = __file__ fn = os.path.join(os.path.dirname(fn), 'export_tool.js') with open(fn) as f: JS_CODE = f.read() class ExportTool(Tool): __implementation__ = JavaScript(JS_CODE) event = Int(default=0) content = String() def register_plot(self, plot): def export_callback(attr, old, new): # really, export the doc as JSON self.content = None html = file_html(plot, CDN, "Task Stream") self.content = html self.on_change('event', export_callback) distributed-1.20.2/distributed/bokeh/scheduler.py000066400000000000000000001200651321233345200220620ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from functools import partial import logging import math from math import sqrt from numbers import Number from operator import add import os from bokeh.application import Application from bokeh.application.handlers.function import FunctionHandler from bokeh.layouts import column, row from bokeh.models import (ColumnDataSource, DataRange1d, HoverTool, ResetTool, PanTool, WheelZoomTool, TapTool, OpenURL, Range1d, Plot, Quad, value, LinearAxis, NumeralTickFormatter, BasicTicker, NumberFormatter, BoxSelectTool) from bokeh.models.widgets import DataTable, TableColumn from bokeh.plotting import figure from bokeh.palettes import Viridis11 from bokeh.io import curdoc from toolz import pipe try: import numpy as np except ImportError: np = False from . import components from .components import DashboardComponent, ProfileTimePlot from .core import BokehServer from .worker import SystemMonitor, counters_doc from .utils import transpose from ..metrics import time from ..utils import log_errors, format_bytes, format_time from ..diagnostics.progress_stream import color_of, progress_quads, nbytes_bar from ..diagnostics.progress import AllProgress from .task_stream import TaskStreamPlugin try: from cytoolz.curried import map, concat, groupby, valmap, first except ImportError: from toolz.curried import map, concat, groupby, valmap, first logger = logging.getLogger(__name__) PROFILING = False import jinja2 with open(os.path.join(os.path.dirname(__file__), 'template.html')) as f: template_source = f.read() template = jinja2.Template(template_source) template_variables = {'pages': ['status', 'workers', 'tasks', 'system', 'profile', 'counters']} def update(source, data): """ Update source with data This checks a few things first 1. If the data is the same, then don't update 2. If numpy is available and the data is numeric, then convert to numpy arrays 3. If profiling then perform the update in another callback """ if (not np or not any(isinstance(v, np.ndarray) for v in source.data.values())): if source.data == data: return if np and len(data[first(data)]) > 10: d = {} for k, v in data.items(): if type(v) is not np.ndarray and isinstance(v[0], Number): d[k] = np.array(v) else: d[k] = v else: d = data if PROFILING: curdoc().add_next_tick_callback(lambda: source.data.update(d)) else: source.data.update(d) class StateTable(DashboardComponent): """ Currently running tasks """ def __init__(self, scheduler): self.scheduler = scheduler names = ['Tasks', 'Stored', 'Processing', 'Waiting', 'No Worker', 'Erred', 'Released'] self.source = ColumnDataSource({name: [] for name in names}) columns = {name: TableColumn(field=name, title=name) for name in names} table = DataTable( source=self.source, columns=[columns[n] for n in names], height=70, ) self.root = table def update(self): with log_errors(): s = self.scheduler d = {'Tasks': [len(s.tasks)], 'Stored': [len(s.who_has)], 'Processing': ['%d / %d' % (len(s.rprocessing), s.total_ncores)], 'Waiting': [len(s.waiting)], 'No Worker': [len(s.unrunnable)], 'Erred': [len(s.exceptions)], 'Released': [len(s.released)]} update(self.source, d) class Occupancy(DashboardComponent): """ Occupancy (in time) per worker """ def __init__(self, scheduler, **kwargs): with log_errors(): self.scheduler = scheduler self.source = ColumnDataSource({'occupancy': [0, 0], 'worker': ['a', 'b'], 'x': [0.0, 0.1], 'y': [1, 2], 'ms': [1, 2], 'color': ['red', 'blue'], 'bokeh_address': ['', '']}) fig = figure(title='Occupancy', tools='', id='bk-occupancy-plot', x_axis_type='datetime', **kwargs) rect = fig.rect(source=self.source, x='x', width='ms', y='y', height=1, color='color') rect.nonselection_glyph = None fig.xaxis.minor_tick_line_alpha = 0 fig.yaxis.visible = False fig.ygrid.visible = False # fig.xaxis[0].formatter = NumeralTickFormatter(format='0.0s') fig.x_range.start = 0 tap = TapTool(callback=OpenURL(url='http://@bokeh_address/')) hover = HoverTool() hover.tooltips = "@worker : @occupancy s." hover.point_policy = 'follow_mouse' fig.add_tools(hover, tap) self.root = fig def update(self): with log_errors(): o = self.scheduler.occupancy workers = list(self.scheduler.workers) bokeh_addresses = [] for worker in workers: addr = self.scheduler.get_worker_service_addr(worker, 'bokeh') bokeh_addresses.append('%s:%d' % addr if addr is not None else '') y = list(range(len(workers))) occupancy = [o[w] for w in workers] ms = [occ * 1000 for occ in occupancy] x = [occ / 500 for occ in occupancy] total = sum(occupancy) color = [] for w in workers: if w in self.scheduler.idle: color.append('red') elif w in self.scheduler.saturated: color.append('green') else: color.append('blue') if total: self.root.title.text = ('Occupancy -- total time: %s wall time: %s' % (format_time(total), format_time(total / self.scheduler.total_ncores))) else: self.root.title.text = 'Occupancy' if occupancy: result = {'occupancy': occupancy, 'worker': workers, 'ms': ms, 'color': color, 'bokeh_address': bokeh_addresses, 'x': x, 'y': y} update(self.source, result) class ProcessingHistogram(DashboardComponent): """ How many tasks are on each worker """ def __init__(self, scheduler, **kwargs): with log_errors(): self.last = 0 self.scheduler = scheduler self.source = ColumnDataSource({'left': [1, 2], 'right': [10, 10], 'top': [0, 0]}) self.root = figure(title='Tasks Processing', id='bk-nprocessing-histogram-plot', **kwargs) self.root.xaxis.minor_tick_line_alpha = 0 self.root.ygrid.visible = False self.root.toolbar.logo = None self.root.toolbar_location = None self.root.quad(source=self.source, left='left', right='right', bottom=0, top='top', color='blue') def update(self): L = list(map(len, self.scheduler.processing.values())) counts, x = np.histogram(L, bins=40) self.source.data.update({'left': x[:-1], 'right': x[1:], 'top': counts}) class NBytesHistogram(DashboardComponent): """ How many tasks are on each worker """ def __init__(self, scheduler, **kwargs): with log_errors(): self.last = 0 self.scheduler = scheduler self.source = ColumnDataSource({'left': [1, 2], 'right': [10, 10], 'top': [0, 0]}) self.root = figure(title='NBytes Stored', id='bk-nbytes-histogram-plot', **kwargs) self.root.xaxis[0].formatter = NumeralTickFormatter(format='0.0 b') self.root.xaxis.major_label_orientation = -math.pi / 12 self.root.xaxis.minor_tick_line_alpha = 0 self.root.ygrid.visible = False self.root.toolbar.logo = None self.root.toolbar_location = None self.root.quad(source=self.source, left='left', right='right', bottom=0, top='top', color='blue') def update(self): nbytes = np.asarray(list(self.scheduler.worker_bytes.values())) counts, x = np.histogram(nbytes, bins=40) d = {'left': x[:-1], 'right': x[1:], 'top': counts} self.source.data.update(d) self.root.title.text = 'Bytes stored: ' + format_bytes(nbytes.sum()) class CurrentLoad(DashboardComponent): """ How many tasks are on each worker """ def __init__(self, scheduler, width=600, **kwargs): with log_errors(): self.last = 0 self.scheduler = scheduler self.source = ColumnDataSource({'nprocessing': [1, 2], 'nprocessing-half': [0.5, 1], 'nprocessing-color': ['red', 'blue'], 'nbytes': [1, 2], 'nbytes-half': [0.5, 1], 'nbytes_text': ['1B', '2B'], 'worker': ['a', 'b'], 'y': [1, 2], 'nbytes-color': ['blue', 'blue'], 'bokeh_address': ['', '']}) processing = figure(title='Tasks Processing', tools='', id='bk-nprocessing-plot', width=int(width / 2), **kwargs) rect = processing.rect(source=self.source, x='nprocessing-half', y='y', width='nprocessing', height=1, color='nprocessing-color') processing.x_range.start = 0 rect.nonselection_glyph = None nbytes = figure(title='Bytes stored', tools='', id='bk-nbytes-worker-plot', width=int(width / 2), **kwargs) rect = nbytes.rect(source=self.source, x='nbytes-half', y='y', width='nbytes', height=1, color='nbytes-color') rect.nonselection_glyph = None nbytes.axis[0].ticker = BasicTicker(mantissas=[1, 256, 512], base=1024) nbytes.xaxis[0].formatter = NumeralTickFormatter(format='0.0 b') nbytes.xaxis.major_label_orientation = -math.pi / 12 nbytes.x_range.start = 0 for fig in [processing, nbytes]: fig.xaxis.minor_tick_line_alpha = 0 fig.yaxis.visible = False fig.ygrid.visible = False tap = TapTool(callback=OpenURL(url='http://@bokeh_address/')) fig.add_tools(tap) fig.toolbar.logo = None fig.toolbar_location = None fig.yaxis.visible = False hover = HoverTool() hover.tooltips = "@worker : @nprocessing tasks" hover.point_policy = 'follow_mouse' processing.add_tools(hover) hover = HoverTool() hover.tooltips = "@worker : @nbytes_text bytes" hover.point_policy = 'follow_mouse' nbytes.add_tools(hover) self.processing_figure = processing self.nbytes_figure = nbytes processing.y_range = nbytes.y_range self.root = row(nbytes, processing, sizing_mode='scale_width') def update(self): with log_errors(): processing = valmap(len, self.scheduler.processing) workers = list(self.scheduler.workers) bokeh_addresses = [] for worker in workers: addr = self.scheduler.get_worker_service_addr(worker, 'bokeh') bokeh_addresses.append('%s:%d' % addr if addr is not None else '') y = list(range(len(workers))) nprocessing = [processing[w] for w in workers] processing_color = [] for w in workers: if w in self.scheduler.idle: processing_color.append('red') elif w in self.scheduler.saturated: processing_color.append('green') else: processing_color.append('blue') nbytes = [self.scheduler.worker_bytes[w] for w in workers] nbytes_text = [format_bytes(nb) for nb in nbytes] nbytes_color = [] max_limit = 0 for w, nb in zip(workers, nbytes): try: limit = self.scheduler.worker_info[w]['memory_limit'] except KeyError: limit = 16e9 if limit > max_limit: max_limit = limit if nb > limit: nbytes_color.append('red') elif nb > limit / 2: nbytes_color.append('orange') else: nbytes_color.append('blue') now = time() if any(nprocessing) or self.last + 1 < now: self.last = now result = {'nprocessing': nprocessing, 'nprocessing-half': [np / 2 for np in nprocessing], 'nprocessing-color': processing_color, 'nbytes': nbytes, 'nbytes-half': [nb / 2 for nb in nbytes], 'nbytes-color': nbytes_color, 'nbytes_text': nbytes_text, 'bokeh_address': bokeh_addresses, 'worker': workers, 'y': y} self.nbytes_figure.title.text = 'Bytes stored: ' + format_bytes(sum(nbytes)) update(self.source, result) class StealingTimeSeries(DashboardComponent): def __init__(self, scheduler, **kwargs): self.scheduler = scheduler self.source = ColumnDataSource({'time': [time(), time() + 1], 'idle': [0, 0.1], 'saturated': [0, 0.1]}) x_range = DataRange1d(follow='end', follow_interval=20000, range_padding=0) fig = figure(title="Idle and Saturated Workers Over Time", x_axis_type='datetime', y_range=[-0.1, len(scheduler.workers) + 0.1], height=150, tools='', x_range=x_range, **kwargs) fig.line(source=self.source, x='time', y='idle', color='red') fig.line(source=self.source, x='time', y='saturated', color='green') fig.yaxis.minor_tick_line_color = None fig.add_tools( ResetTool(reset_size=False), PanTool(dimensions="width"), WheelZoomTool(dimensions="width") ) self.root = fig def update(self): with log_errors(): result = {'time': [time() * 1000], 'idle': [len(self.scheduler.idle)], 'saturated': [len(self.scheduler.saturated)]} if PROFILING: curdoc().add_next_tick_callback(lambda: self.source.stream(result, 10000)) else: self.source.stream(result, 10000) class StealingEvents(DashboardComponent): def __init__(self, scheduler, **kwargs): self.scheduler = scheduler self.steal = scheduler.extensions['stealing'] self.last = 0 self.source = ColumnDataSource({'time': [time() - 20, time()], 'level': [0, 15], 'color': ['white', 'white'], 'duration': [0, 0], 'radius': [1, 1], 'cost_factor': [0, 10], 'count': [1, 1]}) x_range = DataRange1d(follow='end', follow_interval=20000, range_padding=0) fig = figure(title="Stealing Events", x_axis_type='datetime', y_axis_type='log', height=250, tools='', x_range=x_range, **kwargs) fig.circle(source=self.source, x='time', y='cost_factor', color='color', size='radius', alpha=0.5) fig.yaxis.axis_label = "Cost Multiplier" hover = HoverTool() hover.tooltips = "Level: @level, Duration: @duration, Count: @count, Cost factor: @cost_factor" hover.point_policy = 'follow_mouse' fig.add_tools( hover, ResetTool(reset_size=False), PanTool(dimensions="width"), WheelZoomTool(dimensions="width") ) self.root = fig def convert(self, msgs): """ Convert a log message to a glyph """ total_duration = 0 for msg in msgs: time, level, key, duration, sat, occ_sat, idl, occ_idl = msg total_duration += duration try: color = Viridis11[level] except (KeyError, IndexError): color = 'black' radius = sqrt(min(total_duration, 10)) * 30 + 2 d = {'time': time * 1000, 'level': level, 'count': len(msgs), 'color': color, 'duration': total_duration, 'radius': radius, 'cost_factor': min(10, self.steal.cost_multipliers[level])} return d def update(self): with log_errors(): log = self.steal.log n = self.steal.count - self.last log = [log[-i] for i in range(1, n + 1) if isinstance(log[-i], list)] self.last = self.steal.count if log: new = pipe(log, map(groupby(1)), map(dict.values), concat, map(self.convert), list, transpose) if PROFILING: curdoc().add_next_tick_callback( lambda: self.source.stream(new, 10000)) else: self.source.stream(new, 10000) class Events(DashboardComponent): def __init__(self, scheduler, name, height=150, **kwargs): self.scheduler = scheduler self.action_ys = dict() self.last = 0 self.name = name self.source = ColumnDataSource({'time': [], 'action': [], 'hover': [], 'y': [], 'color': []}) x_range = DataRange1d(follow='end', follow_interval=200000) fig = figure(title=name, x_axis_type='datetime', height=height, tools='', x_range=x_range, **kwargs) fig.circle(source=self.source, x='time', y='y', color='color', size=50, alpha=0.5, legend='action') fig.yaxis.axis_label = "Action" fig.legend.location = 'top_left' hover = HoverTool() hover.tooltips = "@action
@hover" hover.point_policy = 'follow_mouse' fig.add_tools( hover, ResetTool(reset_size=False), PanTool(dimensions="width"), WheelZoomTool(dimensions="width") ) self.root = fig def update(self): with log_errors(): log = self.scheduler.events[self.name] n = self.scheduler.event_counts[self.name] - self.last if log: log = [log[-i] for i in range(1, n + 1)] self.last = self.scheduler.event_counts[self.name] if log: actions = [] times = [] hovers = [] ys = [] colors = [] for msg in log: times.append(msg['time'] * 1000) action = msg['action'] actions.append(action) try: ys.append(self.action_ys[action]) except KeyError: self.action_ys[action] = len(self.action_ys) ys.append(self.action_ys[action]) colors.append(color_of(action)) hovers.append('TODO') new = {'time': times, 'action': actions, 'hover': hovers, 'y': ys, 'color': colors} if PROFILING: curdoc().add_next_tick_callback(lambda: self.source.stream(new, 10000)) else: self.source.stream(new, 10000) class TaskStream(components.TaskStream): def __init__(self, scheduler, n_rectangles=1000, clear_interval=20000, **kwargs): self.scheduler = scheduler self.offset = 0 es = [p for p in self.scheduler.plugins if isinstance(p, TaskStreamPlugin)] if not es: self.plugin = TaskStreamPlugin(self.scheduler) else: self.plugin = es[0] self.index = max(0, self.plugin.index - n_rectangles) self.workers = dict() components.TaskStream.__init__(self, n_rectangles=n_rectangles, clear_interval=clear_interval, **kwargs) def update(self): if self.index == self.plugin.index: return with log_errors(): if self.index and self.source.data['start']: start = min(self.source.data['start']) duration = max(self.source.data['duration']) boundary = (self.offset + start - duration) / 1000 else: boundary = self.offset rectangles = self.plugin.rectangles(istart=self.index, workers=self.workers, start_boundary=boundary) n = len(rectangles['name']) self.index = self.plugin.index if not rectangles['start']: return # If there has been a significant delay then clear old rectangles first_end = min(map(add, rectangles['start'], rectangles['duration'])) if first_end > self.last: last = self.last self.last = first_end if first_end > last + self.clear_interval: self.offset = min(rectangles['start']) self.source.data.update({k: [] for k in rectangles}) rectangles['start'] = [x - self.offset for x in rectangles['start']] # Convert to numpy for serialization speed if n >= 10 and np: for k, v in rectangles.items(): if isinstance(v[0], Number): rectangles[k] = np.array(v) if PROFILING: curdoc().add_next_tick_callback(lambda: self.source.stream(rectangles, self.n_rectangles)) else: self.source.stream(rectangles, self.n_rectangles) class TaskProgress(DashboardComponent): """ Progress bars per task type """ def __init__(self, scheduler, **kwargs): self.scheduler = scheduler ps = [p for p in scheduler.plugins if isinstance(p, AllProgress)] if ps: self.plugin = ps[0] else: self.plugin = AllProgress(scheduler) data = progress_quads(dict(all={}, memory={}, erred={}, released={})) self.source = ColumnDataSource(data=data) x_range = DataRange1d(range_padding=0) y_range = Range1d(-8, 0) self.root = figure( id='bk-task-progress-plot', title='Progress', x_range=x_range, y_range=y_range, toolbar_location=None, **kwargs ) self.root.line( # just to define early ranges x=[0, 1], y=[-1, 0], line_color="#FFFFFF", alpha=0.0) self.root.quad( source=self.source, top='top', bottom='bottom', left='left', right='right', fill_color="#aaaaaa", line_color="#aaaaaa", fill_alpha=0.2 ) self.root.quad( source=self.source, top='top', bottom='bottom', left='left', right='released-loc', fill_color="color", line_color="color", fill_alpha=0.6 ) self.root.quad( source=self.source, top='top', bottom='bottom', left='released-loc', right='memory-loc', fill_color="color", line_color="color", fill_alpha=1.0 ) self.root.quad( source=self.source, top='top', bottom='bottom', left='released-loc', right='erred-loc', fill_color='black', line_color='#000000', fill_alpha=0.5 ) self.root.text( source=self.source, text='show-name', y='bottom', x='left', x_offset=5, text_font_size=value('10pt') ) self.root.text( source=self.source, text='done', y='bottom', x='right', x_offset=-5, text_align='right', text_font_size=value('10pt') ) self.root.ygrid.visible = False self.root.yaxis.minor_tick_line_alpha = 0 self.root.yaxis.visible = False self.root.xgrid.visible = False self.root.xaxis.minor_tick_line_alpha = 0 self.root.xaxis.visible = False hover = HoverTool( point_policy="follow_mouse", tooltips="""
Name:  @name
All:  @all
Memory:  @memory
Erred:  @erred
""" ) self.root.add_tools(hover) def update(self): with log_errors(): state = {'all': valmap(len, self.plugin.all), 'nbytes': self.plugin.nbytes} for k in ['memory', 'erred', 'released']: state[k] = valmap(len, self.plugin.state[k]) if not state['all'] and not len(self.source.data['all']): return d = progress_quads(state) update(self.source, d) totals = {k: sum(state[k].values()) for k in ['all', 'memory', 'erred', 'released']} totals['processing'] = totals['all'] - sum(v for k, v in totals.items() if k != 'all') self.root.title.text = ("Progress -- total: %(all)s, " "in-memory: %(memory)s, processing: %(processing)s, " "erred: %(erred)s" % totals) class MemoryUse(DashboardComponent): """ The memory usage across the cluster, grouped by task type """ def __init__(self, scheduler, **kwargs): self.scheduler = scheduler ps = [p for p in scheduler.plugins if isinstance(p, AllProgress)] if ps: self.plugin = ps[0] else: self.plugin = AllProgress(scheduler) self.source = ColumnDataSource(data=dict( name=[], left=[], right=[], center=[], color=[], percent=[], MB=[], text=[]) ) self.root = Plot( id='bk-nbytes-plot', x_range=DataRange1d(), y_range=DataRange1d(), toolbar_location=None, outline_line_color=None, **kwargs ) self.root.add_glyph( self.source, Quad(top=1, bottom=0, left='left', right='right', fill_color='color', fill_alpha=1) ) self.root.add_layout(LinearAxis(), 'left') self.root.add_layout(LinearAxis(), 'below') hover = HoverTool( point_policy="follow_mouse", tooltips="""
Name:  @name
Percent:  @percent
MB:  @MB
""" ) self.root.add_tools(hover) def update(self): with log_errors(): nb = nbytes_bar(self.plugin.nbytes) update(self.source, nb) self.root.title.text = \ "Memory Use: %0.2f MB" % (sum(self.plugin.nbytes.values()) / 1e6) class WorkerTable(DashboardComponent): """ Status of the current workers This is two plots, a text-based table for each host and a thin horizontal plot laying out hosts by their current memory use. """ def __init__(self, scheduler, width=800, **kwargs): self.scheduler = scheduler self.names = ['worker', 'ncores', 'cpu', 'memory', 'memory_limit', 'memory_percent', 'num_fds', 'read_bytes', 'write_bytes'] self.source = ColumnDataSource({k: [] for k in self.names}) columns = {name: TableColumn(field=name, title=name.replace('_percent', ' %')) for name in self.names} formatters = {'cpu': NumberFormatter(format='0.0 %'), 'memory_percent': NumberFormatter(format='0.0 %'), 'memory': NumberFormatter(format='0 b'), 'memory_limit': NumberFormatter(format='0 b'), 'read_bytes': NumberFormatter(format='0 b'), 'write_bytes': NumberFormatter(format='0 b'), 'num_fds': NumberFormatter(format='0'), 'ncores': NumberFormatter(format='0')} table = DataTable( source=self.source, columns=[columns[n] for n in self.names], row_headers=False, reorderable=True, sortable=True, width=width, ) for name in self.names: if name in formatters: table.columns[self.names.index(name)].formatter = formatters[name] hover = HoverTool( point_policy="follow_mouse", tooltips="""
@host: @memory_percent
""" ) mem_plot = figure(title='Memory Use (%)', toolbar_location=None, x_range=(0, 1), y_range=(-0.1, 0.1), height=80, width=width, tools='', **kwargs) mem_plot.circle(source=self.source, x='memory_percent', y=0, size=10, fill_alpha=0.5) mem_plot.ygrid.visible = False mem_plot.yaxis.minor_tick_line_alpha = 0 mem_plot.yaxis.visible = False mem_plot.add_tools(hover, BoxSelectTool()) hover = HoverTool( point_policy="follow_mouse", tooltips="""
@worker: @cpu
""" ) cpu_plot = figure(title='CPU Use (%)', toolbar_location=None, x_range=(0, 1), y_range=(-0.1, 0.1), height=80, width=width, tools='', **kwargs) cpu_plot.circle(source=self.source, x='cpu', y=0, size=10, fill_alpha=0.5) cpu_plot.ygrid.visible = False cpu_plot.yaxis.minor_tick_line_alpha = 0 cpu_plot.yaxis.visible = False cpu_plot.add_tools(hover, BoxSelectTool()) if 'sizing_mode' in kwargs: sizing_mode = {'sizing_mode': kwargs['sizing_mode']} else: sizing_mode = {} self.root = column(cpu_plot, mem_plot, table, id='bk-worker-table', **sizing_mode) def update(self): data = {name: [] for name in self.names} for worker, info in sorted(self.scheduler.worker_info.items()): for name in self.names: data[name].append(info.get(name, None)) data['worker'][-1] = worker data['memory_percent'][-1] = info['memory'] / info['memory_limit'] data['cpu'][-1] = info['cpu'] / 100.0 self.source.data.update(data) def systemmonitor_doc(scheduler, extra, doc): with log_errors(): table = StateTable(scheduler) sysmon = SystemMonitor(scheduler, sizing_mode='scale_width') doc.title = "Dask Scheduler Internal Monitor" doc.add_periodic_callback(table.update, 500) doc.add_periodic_callback(sysmon.update, 500) doc.add_root(column(table.root, sysmon.root, sizing_mode='scale_width')) doc.template = template doc.template_variables['active_page'] = 'system' doc.template_variables.update(extra) def stealing_doc(scheduler, extra, doc): with log_errors(): table = StateTable(scheduler) occupancy = Occupancy(scheduler, height=200, sizing_mode='scale_width') stealing_ts = StealingTimeSeries(scheduler, sizing_mode='scale_width') stealing_events = StealingEvents(scheduler, sizing_mode='scale_width') stealing_events.root.x_range = stealing_ts.root.x_range doc.title = "Dask Workers Monitor" doc.add_periodic_callback(table.update, 500) doc.add_periodic_callback(occupancy.update, 500) doc.add_periodic_callback(stealing_ts.update, 500) doc.add_periodic_callback(stealing_events.update, 500) doc.add_root(column(table.root, occupancy.root, stealing_ts.root, stealing_events.root, sizing_mode='scale_width')) doc.template = template doc.template_variables['active_page'] = 'stealing' doc.template_variables.update(extra) def events_doc(scheduler, extra, doc): with log_errors(): events = Events(scheduler, 'all', height=250) events.update() doc.add_periodic_callback(events.update, 500) doc.title = "Dask Scheduler Events" doc.add_root(column(events.root, sizing_mode='scale_width')) doc.template = template doc.template_variables['active_page'] = 'events' doc.template_variables.update(extra) def workers_doc(scheduler, extra, doc): with log_errors(): table = WorkerTable(scheduler) table.update() doc.add_periodic_callback(table.update, 500) doc.title = "Dask Workers" doc.add_root(table.root) doc.template = template doc.template_variables['active_page'] = 'workers' doc.template_variables.update(extra) def tasks_doc(scheduler, extra, doc): with log_errors(): ts = TaskStream(scheduler, n_rectangles=100000, clear_interval=60000, sizing_mode='stretch_both') ts.update() doc.add_periodic_callback(ts.update, 5000) doc.title = "Dask Task Stream" doc.add_root(ts.root) doc.template = template doc.template_variables['active_page'] = 'tasks' doc.template_variables.update(extra) def status_doc(scheduler, extra, doc): with log_errors(): task_stream = TaskStream(scheduler, n_rectangles=1000, clear_interval=10000, height=350) task_stream.update() doc.add_periodic_callback(task_stream.update, 100) task_progress = TaskProgress(scheduler, height=160) task_progress.update() doc.add_periodic_callback(task_progress.update, 100) if len(scheduler.workers) < 50: current_load = CurrentLoad(scheduler, height=160) current_load.update() doc.add_periodic_callback(current_load.update, 100) current_load_fig = current_load.root else: nbytes_hist = NBytesHistogram(scheduler, width=300, height=160) nbytes_hist.update() processing_hist = ProcessingHistogram(scheduler, width=300, height=160) processing_hist.update() doc.add_periodic_callback(nbytes_hist.update, 100) doc.add_periodic_callback(processing_hist.update, 100) current_load_fig = row(nbytes_hist.root, processing_hist.root, sizing_mode='scale_width') doc.title = "Dask Status" doc.add_root(column(current_load_fig, task_stream.root, task_progress.root, sizing_mode='scale_width')) doc.template = template doc.template_variables['active_page'] = 'status' doc.template_variables.update(extra) def profile_doc(scheduler, extra, doc): with log_errors(): doc.title = "Dask Profile" prof = ProfileTimePlot(scheduler, sizing_mode='scale_width', doc=doc) doc.add_root(prof.root) doc.template = template doc.template_variables['active_page'] = 'profile' doc.template_variables.update(extra) prof.trigger_update() class BokehScheduler(BokehServer): def __init__(self, scheduler, io_loop=None, prefix='', **kwargs): self.scheduler = scheduler self.server_kwargs = kwargs self.server_kwargs['prefix'] = prefix or None prefix = prefix or '' prefix = prefix.rstrip('/') if prefix and not prefix.startswith('/'): prefix = '/' + prefix extra = {'prefix': prefix} extra.update(template_variables) systemmonitor = Application(FunctionHandler(partial(systemmonitor_doc, scheduler, extra))) workers = Application(FunctionHandler(partial(workers_doc, scheduler, extra))) stealing = Application(FunctionHandler(partial(stealing_doc, scheduler, extra))) counters = Application(FunctionHandler(partial(counters_doc, scheduler, extra))) events = Application(FunctionHandler(partial(events_doc, scheduler, extra))) tasks = Application(FunctionHandler(partial(tasks_doc, scheduler, extra))) status = Application(FunctionHandler(partial(status_doc, scheduler, extra))) profile = Application(FunctionHandler(partial(profile_doc, scheduler, extra))) self.apps = { '/system': systemmonitor, '/stealing': stealing, '/workers': workers, '/events': events, '/counters': counters, '/tasks': tasks, '/status': status, '/profile': profile, } self.loop = io_loop or scheduler.loop self.server = None @property def my_server(self): return self.scheduler def listen(self, *args, **kwargs): super(BokehScheduler, self).listen(*args, **kwargs) from .scheduler_html import get_handlers self.server._tornado.add_handlers(r'.*', get_handlers(self.my_server)) distributed-1.20.2/distributed/bokeh/scheduler_html.py000066400000000000000000000111041321233345200230770ustar00rootroot00000000000000from 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 dirname = os.path.dirname(__file__) ns = {func.__name__: func for func in [format_bytes, format_time, datetime.fromtimestamp]} class RequestHandler(web.RequestHandler): def initialize(self, server=None): self.server = server def get_template_path(self): return os.path.join(dirname, 'templates') class Workers(RequestHandler): def get(self): with log_errors(): self.render('workers.html', title='Workers', **toolz.merge(self.server.__dict__, ns)) class Worker(RequestHandler): def get(self, worker): worker = escape.url_unescape(worker) with log_errors(): self.render('worker.html', title='Worker: ' + worker, Worker=worker, **toolz.merge(self.server.__dict__, ns)) class Task(RequestHandler): def get(self, task): task = escape.url_unescape(task) with log_errors(): self.render('task.html', title='Task: ' + task, Task=task, server=self.server, **toolz.merge(self.server.__dict__, ns)) class Logs(RequestHandler): def get(self): with log_errors(): logs = self.server.get_logs() self.render('logs.html', title="Logs", logs=logs) class WorkerLogs(RequestHandler): @gen.coroutine def get(self, worker): with log_errors(): worker = escape.url_unescape(worker) logs = yield self.server.get_worker_logs(workers=[worker]) logs = logs[worker] self.render('logs.html', title="Logs: " + worker, logs=logs) class WorkerCallStacks(RequestHandler): @gen.coroutine def get(self, worker): with log_errors(): worker = escape.url_unescape(worker) keys = self.server.processing[worker] call_stack = yield self.server.get_call_stack(keys=keys) self.render('call-stack.html', title="Call Stacks: " + worker, call_stack=call_stack) class TaskCallStack(RequestHandler): @gen.coroutine def get(self, key): with log_errors(): key = escape.url_unescape(key) call_stack = yield self.server.get_call_stack(keys=[key]) if not call_stack: self.write("

Task not actively running. " "It may be finished or not yet started

") else: self.render('call-stack.html', title="Call Stack: " + key, call_stack=call_stack) class CountsJSON(RequestHandler): def get(self): response = { 'bytes': sum(self.server.nbytes.values()), 'clients': len(self.server.wants_what), 'cores': sum(self.server.ncores.values()), 'erred': len(self.server.exceptions_blame), 'hosts': len(self.server.host_info), 'idle': len(self.server.idle), 'memory': len(self.server.who_has), 'processing': len(self.server.rprocessing), 'ready': len(self.server.ready), 'released': len(self.server.released), 'saturated': len(self.server.saturated), 'tasks': len(self.server.tasks), 'unrunnable': len(self.server.unrunnable), 'waiting': len(self.server.waiting), 'waiting_data': len(self.server.waiting_data), 'workers': len(self.server.workers), } self.write(response) class IdentityJSON(RequestHandler): def get(self): self.write(self.server.identity()) class IndexJSON(RequestHandler): def get(self): with log_errors(): r = [url for url, _ in routes if url.endswith('.json')] self.render('json-index.html', routes=r, title='Index of JSON routes') routes = [ (r'/info/workers.html', Workers), (r'/info/worker/(.*).html', Worker), (r'/info/task/(.*).html', Task), (r'/info/logs.html', Logs), (r'/info/call-stacks/(.*).html', WorkerCallStacks), (r'/info/call-stack/(.*).html', TaskCallStack), (r'/info/logs/(.*).html', WorkerLogs), (r'/json/counts.json', CountsJSON), (r'/json/identity.json', IdentityJSON), (r'/json/index.html', IndexJSON), ] def get_handlers(server): return [(url, cls, {'server': server}) for url, cls in routes] distributed-1.20.2/distributed/bokeh/task_stream.py000066400000000000000000000071141321233345200224200ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging from ..diagnostics.progress_stream import color_of from ..diagnostics.plugin import SchedulerPlugin from ..utils import key_split, format_time logger = logging.getLogger(__name__) class TaskStreamPlugin(SchedulerPlugin): def __init__(self, scheduler): self.buffer = [] self.scheduler = scheduler scheduler.add_plugin(self) self.index = 0 self.maxlen = 100000 def transition(self, key, start, finish, *args, **kwargs): if start == 'processing': if key not in self.scheduler.task_state: return kwargs['key'] = key if finish == 'memory' or finish == 'erred': self.buffer.append(kwargs) self.index += 1 if len(self.buffer) > self.maxlen: self.buffer = self.buffer[len(self.buffer):] def rectangles(self, istart, istop=None, workers=None, start_boundary=0): L_start = [] L_duration = [] L_duration_text = [] L_key = [] L_name = [] L_color = [] L_alpha = [] L_worker = [] L_worker_thread = [] L_y = [] diff = self.index - len(self.buffer) for msg in self.buffer[istart - diff: istop - diff if istop else istop]: key = msg['key'] name = key_split(key) startstops = msg.get('startstops', []) try: worker_thread = '%s-%d' % (msg['worker'], msg['thread']) except Exception: continue logger.warning("Message contained bad information: %s", msg, exc_info=True) worker_thread = '' if worker_thread not in workers: workers[worker_thread] = len(workers) / 2 for action, start, stop in startstops: if start < start_boundary: continue color = colors[action] if type(color) is not str: color = color(msg) L_start.append((start + stop) / 2 * 1000) L_duration.append(1000 * (stop - start)) L_duration_text.append(format_time(stop - start)) L_key.append(key) L_name.append(prefix[action] + name) L_color.append(color) L_alpha.append(alphas[action]) L_worker.append(msg['worker']) L_worker_thread.append(worker_thread) L_y.append(workers[worker_thread]) return {'start': L_start, 'duration': L_duration, 'duration_text': L_duration_text, 'key': L_key, 'name': L_name, 'color': L_color, 'alpha': L_alpha, 'worker': L_worker, 'worker_thread': L_worker_thread, 'y': L_y} def color_of_message(msg): if msg['status'] == 'OK': split = key_split(msg['key']) return color_of(split) else: return 'black' colors = {'transfer': 'red', 'disk-write': 'orange', 'disk-read': 'orange', 'deserialize': 'gray', 'compute': color_of_message} alphas = {'transfer': 0.4, 'compute': 1, 'deserialize': 0.4, 'disk-write': 0.4, 'disk-read': 0.4} prefix = {'transfer': 'transfer-', 'disk-write': 'disk-write-', 'disk-read': 'disk-read-', 'deserialize': 'deserialize-', 'compute': ''} distributed-1.20.2/distributed/bokeh/template.html000066400000000000000000000054251321233345200222350ustar00rootroot00000000000000 Dask - Status dashboard {{ bokeh_css }} {{ bokeh_js }}
{{ plot_div }}
{{ plot_script }} distributed-1.20.2/distributed/bokeh/templates/000077500000000000000000000000001321233345200215245ustar00rootroot00000000000000distributed-1.20.2/distributed/bokeh/templates/call-stack.html000066400000000000000000000006041321233345200244300ustar00rootroot00000000000000{% extends main.html %} {% block content %}

{{ title }}

{% for worker, stacks in call_stack.items() %}

Worker: {{worker}}

{% for task, stack in stacks.items() %}

Key: {{task}}

{% for line in stack %}

{{ line }}

{% end %} {% end %} {% end %} {% end %} distributed-1.20.2/distributed/bokeh/templates/json-index.html000066400000000000000000000004241321233345200244700ustar00rootroot00000000000000

Index of JSON routes

These are typically used by automated services to query the state of the scheduler

distributed-1.20.2/distributed/bokeh/templates/logs.html000066400000000000000000000001641321233345200233570ustar00rootroot00000000000000{% extends main.html %} {% block content %} {% for _, log in logs %}

{{ log }}

{% end %} {% end %} distributed-1.20.2/distributed/bokeh/templates/main.html000066400000000000000000000004311321233345200233340ustar00rootroot00000000000000 {{title}} {% block content %} {% end %} distributed-1.20.2/distributed/bokeh/templates/task.html000066400000000000000000000103351321233345200233560ustar00rootroot00000000000000{% extends main.html %} {% block content %}

Task: {{ Task }}

{% if Task in rprocessing %} {% end %} {% if Task in nbytes %} {% end %} {% if task_state[Task] == 'waiting' %} {% for dep in waiting[dep] %} {% end %} {% end %}
status {{ task_state[Task] }}
processing on {{ rprocessing.get(Task) }}
call stack Call Stack
bytes in memory {{ nbytes.get(Task) }}
waiting on {{dep}}

Dependencies

{% for dep in dependencies.get(Task, ()) %} {% end %}
Key State
{{dep}} {{ task_state[dep] }}

Dependents

{% for dep in dependents.get(Task, ()) %} {% end %}
Key State
{{dep}} {{ task_state[dep] }}
{% if task_state[Task] == 'memory' %}

Workers with data

{% set workers = who_has[Task] %} {% include "worker-table.html" %}
{% end %} {% if Task in who_wants %}

Clients with future

    {% for client in who_wants.get(Task, ()) %}
  • {{client}}
  • {% end %}
{% end %}

Transition Log

{% for key, start, finish, recommendations, time in server.story(Task) %} {% for key2, rec in recommendations.items() %} {% end %} {% end %}
Time Key Start Finish Recommended Key Recommended Action
{{ fromtimestamp(time) }} {{key}} {{ start }} {{ finish }}
{{key2}} {{ rec }}
{% end %} distributed-1.20.2/distributed/bokeh/templates/worker-table.html000066400000000000000000000023041321233345200250070ustar00rootroot00000000000000 {% for worker in workers %} {% if 'bokeh' in worker_info[worker]['services'] %} {% else %} {% end %} {% end %}
Worker Cores Memory Memory use Occupancy Processing In-memory Services Logs
{{worker}} {{ ncores[worker] }} {{ format_bytes(worker_info[worker]['memory_limit']) }} {{ format_time(occupancy[worker]) }} {{ len(processing[worker]) }} {{ len(has_what[worker]) }} bokeh logs
distributed-1.20.2/distributed/bokeh/templates/worker.html000066400000000000000000000030301321233345200237170ustar00rootroot00000000000000{% extends main.html %} {% block content %}

Worker: {{Worker}}

{% set workers = [Worker] %} {% include "worker-table.html" %}

In Memory

    {% for task in has_what[Worker] %}
  • {{task}}
  • {% end %}

Processing

Call Stacks
    {% for task in processing[Worker] %}
  • {{task}}
  • {% end %}
{% if worker in worker_resources %}

Resources

> {% for resource in worker_resources.get(Worker, []) %} {% end %}
Resource Consumed Total
{{ resource }} {{ used_resources[Worker][resource] }} {{ worker_resources[Worker][resource] }}
{% end %} {% end %} distributed-1.20.2/distributed/bokeh/templates/workers.html000066400000000000000000000003241321233345200241050ustar00rootroot00000000000000{% extends main.html %} {% block content %} Logs Bokeh {% include "worker-table.html" %} {% end %} distributed-1.20.2/distributed/bokeh/tests/000077500000000000000000000000001321233345200206705ustar00rootroot00000000000000distributed-1.20.2/distributed/bokeh/tests/test_components.py000066400000000000000000000027021321233345200244670ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import pytest pytest.importorskip('bokeh') 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 ( TaskStream, TaskProgress, MemoryUsage, Processing, ProfilePlot, ProfileTimePlot ) @pytest.mark.parametrize('Component', [TaskStream, TaskProgress, MemoryUsage, Processing]) def test_basic(Component): c = Component() assert isinstance(c.source, ColumnDataSource) assert isinstance(c.root, Model) c.update(messages) @gen_cluster(client=True) def test_profile_plot(c, s, a, b): p = ProfilePlot() assert len(p.source.data['left']) <= 1 yield c.map(slowinc, range(10), delay=0.05) p.update(a.profile_recent) assert len(p.source.data['left']) > 1 @gen_cluster(client=True) def test_profile_time_plot(c, s, a, b): from bokeh.io import curdoc sp = ProfileTimePlot(s, doc=curdoc()) sp.trigger_update() ap = ProfileTimePlot(a, doc=curdoc()) ap.trigger_update() assert len(sp.source.data['left']) <= 1 assert len(ap.source.data['left']) <= 1 yield c.map(slowinc, range(10), delay=0.05) ap.trigger_update() sp.trigger_update() yield gen.sleep(0.05) distributed-1.20.2/distributed/bokeh/tests/test_scheduler_bokeh.py000066400000000000000000000161751321233345200254410ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from time import sleep import pytest pytest.importorskip('bokeh') import sys from toolz import first from tornado import gen from tornado.httpclient import AsyncHTTPClient from distributed.client import wait from distributed.metrics import time from distributed.utils_test import gen_cluster, inc, dec, slowinc from distributed.bokeh.worker import Counters, BokehWorker from distributed.bokeh.scheduler import (BokehScheduler, StateTable, SystemMonitor, Occupancy, StealingTimeSeries, StealingEvents, Events, TaskStream, TaskProgress, MemoryUse, CurrentLoad, ProcessingHistogram, NBytesHistogram, WorkerTable) from distributed.bokeh import scheduler scheduler.PROFILING = False @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}}) def test_simple(c, s, a, b): assert isinstance(s.services['bokeh'], BokehScheduler) future = c.submit(sleep, 1) yield gen.sleep(0.1) http_client = AsyncHTTPClient() for suffix in ['system', 'counters', 'workers', 'status', 'tasks', 'stealing']: response = yield http_client.fetch('http://localhost:%d/%s' % (s.services['bokeh'].port, suffix)) assert 'bokeh' in response.body.decode().lower() @gen_cluster(client=True, worker_kwargs=dict(services={'bokeh': BokehWorker})) def test_basic(c, s, a, b): for component in [SystemMonitor, StateTable, Occupancy, StealingTimeSeries]: ss = component(s) ss.update() 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']) @gen_cluster(client=True) def test_counters(c, s, a, b): pytest.importorskip('crick') while 'tick-duration' not in s.digests: yield gen.sleep(0.01) ss = Counters(s) ss.update() yield gen.sleep(0.1) ss.update() start = time() while not len(ss.digest_sources['tick-duration'][0].data['x']): yield gen.sleep(1) assert time() < start + 5 @gen_cluster(client=True) def test_stealing_events(c, s, a, b): se = StealingEvents(s) futures = c.map(slowinc, range(100), delay=0.1, workers=a.address, allow_other_workers=True) while not b.task_state: # will steal soon yield gen.sleep(0.01) se.update() assert len(first(se.source.data.values())) @gen_cluster(client=True) def test_events(c, s, a, b): e = Events(s, 'all') futures = c.map(slowinc, range(100), delay=0.1, workers=a.address, allow_other_workers=True) while not b.task_state: yield gen.sleep(0.01) e.update() d = dict(e.source.data) assert sum(a == 'add-worker' for a in d['action']) == 2 @gen_cluster(client=True) def test_task_stream(c, s, a, b): ts = TaskStream(s) futures = c.map(slowinc, range(10), delay=0.001) yield wait(futures) ts.update() d = dict(ts.source.data) assert all(len(L) == 10 for L in d.values()) assert min(d['start']) == 0 # zero based ts.update() d = dict(ts.source.data) assert all(len(L) == 10 for L in d.values()) total = c.submit(sum, futures) yield wait(total) ts.update() d = dict(ts.source.data) assert len(set(map(len, d.values()))) == 1 @gen_cluster(client=True) def test_task_stream_n_rectangles(c, s, a, b): ts = TaskStream(s, n_rectangles=10) futures = c.map(slowinc, range(10), delay=0.001) yield wait(futures) ts.update() assert len(ts.source.data['start']) == 10 @gen_cluster(client=True) def test_task_stream_second_plugin(c, s, a, b): ts = TaskStream(s, n_rectangles=10, clear_interval=10) ts.update() futures = c.map(inc, range(10)) yield wait(futures) ts.update() ts2 = TaskStream(s, n_rectangles=5, clear_interval=10) ts2.update() @gen_cluster(client=True) def test_task_stream_clear_interval(c, s, a, b): ts = TaskStream(s, clear_interval=200) yield wait(c.map(inc, range(10))) ts.update() yield gen.sleep(0.010) yield wait(c.map(dec, range(10))) ts.update() assert len(set(map(len, ts.source.data.values()))) == 1 assert ts.source.data['name'].count('inc') == 10 assert ts.source.data['name'].count('dec') == 10 yield gen.sleep(0.300) yield wait(c.map(inc, range(10, 20))) ts.update() assert len(set(map(len, ts.source.data.values()))) == 1 assert ts.source.data['name'].count('inc') == 10 assert ts.source.data['name'].count('dec') == 0 @gen_cluster(client=True) def test_TaskProgress(c, s, a, b): tp = TaskProgress(s) futures = c.map(slowinc, range(10), delay=0.001) yield wait(futures) tp.update() d = dict(tp.source.data) assert all(len(L) == 1 for L in d.values()) assert d['name'] == ['slowinc'] futures2 = c.map(dec, range(5)) yield wait(futures2) tp.update() d = dict(tp.source.data) assert all(len(L) == 2 for L in d.values()) assert d['name'] == ['slowinc', 'dec'] del futures, futures2 while s.task_state: yield gen.sleep(0.01) tp.update() assert not tp.source.data['all'] @gen_cluster(client=True) def test_TaskProgress_empty(c, s, a, b): tp = TaskProgress(s) tp.update() futures = [c.submit(inc, i, key='f-' + 'a' * i) for i in range(20)] yield wait(futures) tp.update() del futures while s.tasks: yield gen.sleep(0.01) tp.update() assert not any(len(v) for v in tp.source.data.values()) @gen_cluster(client=True) def test_MemoryUse(c, s, a, b): mu = MemoryUse(s) futures = c.map(slowinc, range(10), delay=0.001) yield wait(futures) mu.update() d = dict(mu.source.data) assert all(len(L) == 1 for L in d.values()) assert d['name'] == ['slowinc'] @gen_cluster(client=True) def test_CurrentLoad(c, s, a, b): cl = CurrentLoad(s) futures = c.map(slowinc, range(10), delay=0.001) yield wait(futures) cl.update() d = dict(cl.source.data) assert all(len(L) == 2 for L in d.values()) assert all(d['nbytes']) @gen_cluster(client=True) def test_ProcessingHistogram(c, s, a, b): ph = ProcessingHistogram(s) ph.update() assert (ph.source.data['top'] != 0).sum() == 1 futures = c.map(slowinc, range(10), delay=0.050) yield gen.sleep(0.100) ph.update() assert ph.source.data['right'][-1] > 2 @gen_cluster(client=True) def test_NBytesHistogram(c, s, a, b): nh = NBytesHistogram(s) nh.update() assert (nh.source.data['top'] != 0).sum() == 1 futures = c.map(inc, range(10)) yield wait(futures) nh.update() assert nh.source.data['right'][-1] > 5 * 20 @gen_cluster(client=True) def test_WorkerTable(c, s, a, b): wt = WorkerTable(s) wt.update() assert all(wt.source.data.values()) assert all(len(v) == 2 for v in wt.source.data.values()) distributed-1.20.2/distributed/bokeh/tests/test_scheduler_bokeh_html.py000066400000000000000000000030131321233345200264500ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import json import xml.etree.ElementTree import pytest pytest.importorskip('bokeh') from tornado.escape import url_escape from tornado.httpclient import AsyncHTTPClient from distributed.utils_test import gen_cluster, inc, slowinc from distributed.bokeh.scheduler import BokehScheduler @gen_cluster(client=True, scheduler_kwargs={'services': {('bokeh', 0): BokehScheduler}}) def test_connect(c, s, a, b): future = c.submit(inc, 1) x = c.submit(slowinc, 1, delay=1) yield future http_client = AsyncHTTPClient() for suffix in ['info/workers.html', 'info/worker/' + url_escape(a.address) + '.html', 'info/task/' + url_escape(future.key) + '.html', 'info/logs.html', 'info/logs/' + url_escape(a.address) + '.html', 'info/call-stack/' + url_escape(x.key) + '.html', 'info/call-stacks/' + url_escape(a.address) + '.html', 'json/counts.json', 'json/identity.json', 'json/index.html', ]: response = yield http_client.fetch('http://localhost:%d/%s' % (s.services['bokeh'].port, suffix)) assert response.code == 200 body = response.body.decode() if suffix.endswith('.json'): json.loads(body) else: assert xml.etree.ElementTree.fromstring(body) is not None distributed-1.20.2/distributed/bokeh/tests/test_task_stream.py000066400000000000000000000022731321233345200246220ustar00rootroot00000000000000from __future__ import absolute_import, division, print_function import pytest pytest.importorskip('bokeh') from toolz import frequencies from distributed.utils_test import gen_cluster, div from distributed.client import wait from distributed.bokeh.task_stream import TaskStreamPlugin @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_TaskStreamPlugin(c, s, *workers): es = TaskStreamPlugin(s) assert not es.buffer futures = c.map(div, [1] * 10, range(10)) total = c.submit(sum, futures[1:]) yield wait(total) assert len(es.buffer) == 11 workers = dict() rects = es.rectangles(0, 10, workers) assert all(n == 'div' for n in rects['name']) assert all(d > 0 for d in rects['duration']) counts = frequencies(rects['color']) assert counts['black'] == 1 assert set(counts.values()) == {9, 1} assert len(set(rects['y'])) == 3 rects = es.rectangles(2, 5, workers) assert all(len(L) == 3 for L in rects.values()) starts = sorted(rects['start']) rects = es.rectangles(2, 5, workers=workers, start_boundary=(starts[0] + starts[1]) / 2000) assert set(rects['start']).issubset(set(starts[1:])) distributed-1.20.2/distributed/bokeh/tests/test_worker_bokeh.py000066400000000000000000000102351321233345200247630ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from operator import add, sub from time import sleep import pytest pytest.importorskip('bokeh') import sys from toolz import first from tornado import gen from tornado.httpclient import AsyncHTTPClient from distributed.client import wait from distributed.metrics import time from distributed.utils_test import gen_cluster, inc, dec from distributed.bokeh.worker import (BokehWorker, StateTable, CrossFilter, CommunicatingStream, ExecutingTimeSeries, CommunicatingTimeSeries, SystemMonitor, Counters) @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}}) def test_simple(c, s, a, b): assert s.worker_info[a.address]['services'] == {'bokeh': a.services['bokeh'].port} assert s.worker_info[b.address]['services'] == {'bokeh': b.services['bokeh'].port} future = c.submit(sleep, 1) yield gen.sleep(0.1) http_client = AsyncHTTPClient() for suffix in ['main', 'crossfilter', 'system']: response = yield http_client.fetch('http://localhost:%d/%s' % (a.services['bokeh'].port, suffix)) assert 'bokeh' in response.body.decode().lower() @gen_cluster(client=True, worker_kwargs={'services': {('bokeh', 0): (BokehWorker, {})}}) def test_services_kwargs(c, s, a, b): assert s.worker_info[a.address]['services'] == {'bokeh': a.services['bokeh'].port} assert isinstance(a.services['bokeh'], BokehWorker) @gen_cluster(client=True) def test_basic(c, s, a, b): for component in [StateTable, ExecutingTimeSeries, CommunicatingTimeSeries, CrossFilter, SystemMonitor]: aa = component(a) bb = component(b) xs = c.map(inc, range(10), workers=a.address) ys = c.map(dec, range(10), workers=b.address) def slowall(*args): sleep(1) x = c.submit(slowall, xs, ys, 1, workers=a.address) y = c.submit(slowall, xs, ys, 2, workers=b.address) yield gen.sleep(0.1) aa.update() bb.update() assert (len(first(aa.source.data.values())) and len(first(bb.source.data.values()))) @gen_cluster(client=True) def test_counters(c, s, a, b): pytest.importorskip('crick') while 'tick-duration' not in a.digests: yield gen.sleep(0.01) aa = Counters(a) aa.update() yield gen.sleep(0.1) aa.update() start = time() while not len(aa.digest_sources['tick-duration'][0].data['x']): yield gen.sleep(1) assert time() < start + 5 a.digests['foo'].add(1) a.digests['foo'].add(2) aa.add_digest_figure('foo') a.counters['bar'].add(1) a.counters['bar'].add(2) a.counters['bar'].add(2) aa.add_counter_figure('bar') for x in [aa.counter_sources.values(), aa.digest_sources.values()]: for y in x: for z in y.values(): assert len(set(map(len, z.data.values()))) == 1 @gen_cluster(client=True) def test_CommunicatingStream(c, s, a, b): aa = CommunicatingStream(a) bb = CommunicatingStream(b) xs = c.map(inc, range(10), workers=a.address) ys = c.map(dec, range(10), workers=b.address) adds = c.map(add, xs, ys, workers=a.address) subs = c.map(sub, xs, ys, workers=b.address) yield wait([adds, subs]) aa.update() bb.update() assert (len(first(aa.outgoing.data.values())) and len(first(bb.outgoing.data.values()))) assert (len(first(aa.incoming.data.values())) and len(first(bb.incoming.data.values()))) @pytest.mark.skipif(sys.version_info[0] == 2, reason='https://github.com/bokeh/bokeh/issues/5494') @gen_cluster(client=True) def test_port_overlap(c, s, a, b): # When the given port is unavailable, another one is chosen automatically sa = BokehWorker(a) sa.listen(57384) sb = BokehWorker(b) sb.listen(57384) assert sa.port assert sb.port assert sa.port != sb.port sa.stop() sb.stop() distributed-1.20.2/distributed/bokeh/utils.py000066400000000000000000000005611321233345200212420ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from toolz import partition def parse_args(args): options = dict(partition(2, args)) for k, v in options.items(): if v.isdigit(): options[k] = int(v) return options def transpose(lod): keys = list(lod[0].keys()) return {k: [d[k] for d in lod] for k in keys} distributed-1.20.2/distributed/bokeh/worker.py000066400000000000000000000575661321233345200214340ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from functools import partial import logging import math import os from bokeh.layouts import row, column, widgetbox from bokeh.models import (ColumnDataSource, DataRange1d, HoverTool, BoxZoomTool, ResetTool, PanTool, WheelZoomTool, NumeralTickFormatter, Select) from bokeh.models.widgets import DataTable, TableColumn from bokeh.plotting import figure from bokeh.palettes import RdBu from toolz import merge, partition_all from .components import DashboardComponent, ProfileTimePlot from .core import BokehServer from .utils import transpose from ..compatibility import WINDOWS from ..diagnostics.progress_stream import color_of from ..metrics import time from ..utils import log_errors, key_split, format_bytes, format_time logger = logging.getLogger(__name__) import jinja2 with open(os.path.join(os.path.dirname(__file__), 'template.html')) as f: template_source = f.read() template = jinja2.Template(template_source) template_variables = {'pages': ['main', 'system', 'profile', 'crossfilter', 'counters']} class StateTable(DashboardComponent): """ Currently running tasks """ def __init__(self, worker): self.worker = worker names = ['Stored', 'Executing', 'Ready', 'Waiting', 'Connections', 'Serving'] self.source = ColumnDataSource({name: [] for name in names}) columns = {name: TableColumn(field=name, title=name) for name in names} table = DataTable( source=self.source, columns=[columns[n] for n in names], height=70, ) self.root = table def update(self): with log_errors(): w = self.worker d = {'Stored': [len(w.data)], 'Executing': ['%d / %d' % (len(w.executing), w.ncores)], 'Ready': [len(w.ready)], 'Waiting': [len(w.waiting_for_data)], 'Connections': [len(w.in_flight_workers)], 'Serving': [len(w._comms)]} self.source.data.update(d) class CommunicatingStream(DashboardComponent): def __init__(self, worker, height=300, **kwargs): with log_errors(): self.worker = worker names = ['start', 'stop', 'middle', 'duration', 'who', 'y', 'hover', 'alpha', 'bandwidth', 'total'] self.incoming = ColumnDataSource({name: [] for name in names}) self.outgoing = ColumnDataSource({name: [] for name in names}) x_range = DataRange1d(range_padding=0) y_range = DataRange1d(range_padding=0) fig = figure(title='Peer Communications', x_axis_type='datetime', x_range=x_range, y_range=y_range, height=height, tools='', **kwargs) fig.rect(source=self.incoming, x='middle', y='y', width='duration', height=0.9, color='red', alpha='alpha') fig.rect(source=self.outgoing, x='middle', y='y', width='duration', height=0.9, color='blue', alpha='alpha') hover = HoverTool( point_policy="follow_mouse", tooltips="""@hover""" ) fig.add_tools( hover, ResetTool(reset_size=False), PanTool(dimensions="width"), WheelZoomTool(dimensions="width") ) self.root = fig self.last_incoming = 0 self.last_outgoing = 0 self.who = dict() def update(self): with log_errors(): outgoing = self.worker.outgoing_transfer_log n = self.worker.outgoing_count - self.last_outgoing outgoing = [outgoing[-i].copy() for i in range(1, n + 1)] self.last_outgoing = self.worker.outgoing_count incoming = self.worker.incoming_transfer_log n = self.worker.incoming_count - self.last_incoming incoming = [incoming[-i].copy() for i in range(1, n + 1)] self.last_incoming = self.worker.incoming_count for [msgs, source] in [[incoming, self.incoming], [outgoing, self.outgoing]]: for msg in msgs: if 'compressed' in msg: del msg['compressed'] del msg['keys'] bandwidth = msg['total'] / (msg['duration'] or 0.5) bw = max(min(bandwidth / 500e6, 1), 0.3) msg['alpha'] = bw try: msg['y'] = self.who[msg['who']] except KeyError: self.who[msg['who']] = len(self.who) msg['y'] = self.who[msg['who']] msg['hover'] = '%s / %s = %s/s' % ( format_bytes(msg['total']), format_time(msg['duration']), format_bytes(msg['total'] / msg['duration'])) for k in ['middle', 'duration', 'start', 'stop']: msg[k] = msg[k] * 1000 if msgs: msgs = transpose(msgs) if (len(source.data['stop']) and min(msgs['start']) > source.data['stop'][-1] + 10000): source.data.update(msgs) else: source.stream(msgs, rollover=10000) class CommunicatingTimeSeries(DashboardComponent): def __init__(self, worker, **kwargs): self.worker = worker self.source = ColumnDataSource({'x': [], 'in': [], 'out': []}) x_range = DataRange1d(follow='end', follow_interval=20000, range_padding=0) fig = figure(title="Communication History", x_axis_type='datetime', y_range=[-0.1, worker.total_connections + 0.5], height=150, tools='', x_range=x_range, **kwargs) fig.line(source=self.source, x='x', y='in', color='red') fig.line(source=self.source, x='x', y='out', color='blue') fig.add_tools( ResetTool(reset_size=False), PanTool(dimensions="width"), WheelZoomTool(dimensions="width") ) self.root = fig def update(self): with log_errors(): self.source.stream({'x': [time() * 1000], 'out': [len(self.worker._comms)], 'in': [len(self.worker.in_flight_workers)]}, 10000) class ExecutingTimeSeries(DashboardComponent): def __init__(self, worker, **kwargs): self.worker = worker self.source = ColumnDataSource({'x': [], 'y': []}) x_range = DataRange1d(follow='end', follow_interval=20000, range_padding=0) fig = figure(title="Executing History", x_axis_type='datetime', y_range=[-0.1, worker.ncores + 0.1], height=150, tools='', x_range=x_range, **kwargs) fig.line(source=self.source, x='x', y='y') fig.add_tools( ResetTool(reset_size=False), PanTool(dimensions="width"), WheelZoomTool(dimensions="width") ) self.root = fig def update(self): with log_errors(): self.source.stream({'x': [time() * 1000], 'y': [len(self.worker.executing)]}, 1000) class CrossFilter(DashboardComponent): def __init__(self, worker, **kwargs): with log_errors(): self.worker = worker quantities = ['nbytes', 'duration', 'bandwidth', 'count', 'start', 'stop'] colors = ['inout-color', 'type-color', 'key-color'] # self.source = ColumnDataSource({name: [] for name in names}) self.source = ColumnDataSource({ 'nbytes': [1, 2], 'duration': [0.01, 0.02], 'bandwidth': [0.01, 0.02], 'count': [1, 2], 'type': ['int', 'str'], 'inout-color': ['blue', 'red'], 'type-color': ['blue', 'red'], 'key': ['add', 'inc'], 'start': [1, 2], 'stop': [1, 2] }) self.x = Select(title='X-Axis', value='nbytes', options=quantities) self.x.on_change('value', self.update_figure) self.y = Select(title='Y-Axis', value='bandwidth', options=quantities) self.y.on_change('value', self.update_figure) self.size = Select(title='Size', value='None', options=['None'] + quantities) self.size.on_change('value', self.update_figure) self.color = Select(title='Color', value='inout-color', options=['black'] + colors) self.color.on_change('value', self.update_figure) if 'sizing_mode' in kwargs: kw = {'sizing_mode': kwargs['sizing_mode']} else: kw = {} self.control = widgetbox([self.x, self.y, self.size, self.color], width=200, **kw) self.last_outgoing = 0 self.last_incoming = 0 self.kwargs = kwargs self.layout = row(self.control, self.create_figure(**self.kwargs), **kw) self.root = self.layout def update(self): with log_errors(): outgoing = self.worker.outgoing_transfer_log n = self.worker.outgoing_count - self.last_outgoing n = min(n, 1000) outgoing = [outgoing[-i].copy() for i in range(1, n)] self.last_outgoing = self.worker.outgoing_count incoming = self.worker.incoming_transfer_log n = self.worker.incoming_count - self.last_incoming n = min(n, 1000) incoming = [incoming[-i].copy() for i in range(1, n)] self.last_incoming = self.worker.incoming_count out = [] for msg in incoming: if msg['keys']: d = self.process_msg(msg) d['inout-color'] = 'red' out.append(d) for msg in outgoing: if msg['keys']: d = self.process_msg(msg) d['inout-color'] = 'blue' out.append(d) if out: out = transpose(out) if (len(self.source.data['stop']) and min(out['start']) > self.source.data['stop'][-1] + 10): self.source.data.update(out) else: self.source.stream(out, rollover=1000) def create_figure(self, **kwargs): with log_errors(): fig = figure(title='', tools='', **kwargs) size = self.size.value if size == 'None': size = 1 fig.circle(source=self.source, x=self.x.value, y=self.y.value, color=self.color.value, size=10, alpha=0.5, hover_alpha=1) fig.xaxis.axis_label = self.x.value fig.yaxis.axis_label = self.y.value fig.add_tools( # self.hover, ResetTool(reset_size=False), PanTool(), WheelZoomTool(), BoxZoomTool(), ) return fig def update_figure(self, attr, old, new): with log_errors(): fig = self.create_figure(**self.kwargs) self.layout.children[1] = fig def process_msg(self, msg): try: 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) d = { 'nbytes': msg['total'], 'duration': msg['duration'], 'bandwidth': msg['bandwidth'], 'count': len(msg['keys']), 'type': typ, 'type-color': color_of(typ), 'key': keyname, 'key-color': color_of(keyname), 'start': msg['start'], 'stop': msg['stop'] } return d except Exception as e: logger.exception(e) raise class SystemMonitor(DashboardComponent): def __init__(self, worker, height=150, **kwargs): self.worker = worker names = worker.monitor.quantities self.last = 0 self.source = ColumnDataSource({name: [] for name in names}) self.source.data.update(self.get_data()) x_range = DataRange1d(follow='end', follow_interval=20000, range_padding=0) tools = 'reset,xpan,xwheel_zoom' self.cpu = figure(title="CPU", x_axis_type='datetime', height=height, tools=tools, x_range=x_range, **kwargs) self.cpu.line(source=self.source, x='time', y='cpu') self.cpu.yaxis.axis_label = 'Percentage' self.mem = figure(title="Memory", x_axis_type='datetime', height=height, tools=tools, x_range=x_range, **kwargs) self.mem.line(source=self.source, x='time', y='memory') self.mem.yaxis.axis_label = 'Bytes' self.bandwidth = figure(title='Bandwidth', x_axis_type='datetime', height=height, x_range=x_range, tools=tools, **kwargs) self.bandwidth.line(source=self.source, x='time', y='read_bytes', color='red') self.bandwidth.line(source=self.source, x='time', y='write_bytes', color='blue') self.bandwidth.yaxis.axis_label = 'Bytes / second' # self.cpu.yaxis[0].formatter = NumeralTickFormatter(format='0%') self.bandwidth.yaxis[0].formatter = NumeralTickFormatter(format='0.0b') self.mem.yaxis[0].formatter = NumeralTickFormatter(format='0.0b') plots = [self.cpu, self.mem, self.bandwidth] if not WINDOWS: self.num_fds = figure(title='Number of File Descriptors', x_axis_type='datetime', height=height, x_range=x_range, tools=tools, **kwargs) self.num_fds.line(source=self.source, x='time', y='num_fds') plots.append(self.num_fds) if 'sizing_mode' in kwargs: kw = {'sizing_mode': kwargs['sizing_mode']} else: kw = {} if not WINDOWS: self.num_fds.y_range.start = 0 self.mem.y_range.start = 0 self.cpu.y_range.start = 0 self.bandwidth.y_range.start = 0 self.root = column(*plots, **kw) self.worker.monitor.update() def get_data(self): d = self.worker.monitor.range_query(start=self.last) d['time'] = [x * 1000 for x in d['time']] self.last = self.worker.monitor.count return d def update(self): with log_errors(): self.source.stream(self.get_data(), 1000) class Counters(DashboardComponent): def __init__(self, server, sizing_mode='stretch_both', **kwargs): self.server = server self.counter_figures = {} self.counter_sources = {} self.digest_figures = {} self.digest_sources = {} self.sizing_mode = sizing_mode if self.server.digests: for name in self.server.digests: self.add_digest_figure(name) for name in self.server.counters: self.add_counter_figure(name) figures = merge(self.digest_figures, self.counter_figures) figures = [figures[k] for k in sorted(figures)] if len(figures) <= 5: self.root = column(figures, sizing_mode=sizing_mode) else: self.root = column(*[row(*pair, sizing_mode=sizing_mode) for pair in partition_all(2, figures)], sizing_mode=sizing_mode) def add_digest_figure(self, name): with log_errors(): n = len(self.server.digests[name].intervals) sources = {i: ColumnDataSource({'x': [], 'y': []}) for i in range(n)} kwargs = {} if name.endswith('duration'): kwargs['x_axis_type'] = 'datetime' fig = figure(title=name, tools='', height=150, sizing_mode=self.sizing_mode, **kwargs) fig.yaxis.visible = False fig.ygrid.visible = False if name.endswith('bandwidth') or name.endswith('bytes'): fig.xaxis[0].formatter = NumeralTickFormatter(format='0.0b') for i in range(n): alpha = 0.3 + 0.3 * (n - i) / n fig.line(source=sources[i], x='x', y='y', alpha=alpha, color=RdBu[max(n, 3)][-i]) fig.xaxis.major_label_orientation = math.pi / 12 fig.toolbar.logo = None self.digest_sources[name] = sources self.digest_figures[name] = fig return fig def add_counter_figure(self, name): with log_errors(): n = len(self.server.counters[name].intervals) sources = {i: ColumnDataSource({'x': [], 'y': [], 'y-center': [], 'counts': []}) for i in range(n)} fig = figure(title=name, tools='', height=150, sizing_mode=self.sizing_mode, x_range=sorted(map(str, self.server.counters[name].components[0]))) fig.ygrid.visible = False for i in range(n): width = 0.5 + 0.4 * i / n fig.rect(source=sources[i], x='x', y='y-center', width=width, height='y', alpha=0.3, color=RdBu[max(n, 3)][-i]) hover = HoverTool( point_policy="follow_mouse", tooltips="""@x : @counts""" ) fig.add_tools(hover) fig.xaxis.major_label_orientation = math.pi / 12 fig.toolbar.logo = None self.counter_sources[name] = sources self.counter_figures[name] = fig return fig def update(self): with log_errors(): for name, fig in self.digest_figures.items(): digest = self.server.digests[name] d = {} for i, d in enumerate(digest.components): if d.size(): ys, xs = d.histogram(100) xs = xs[1:] if name.endswith('duration'): xs *= 1000 self.digest_sources[name][i].data.update({'x': xs, 'y': ys}) fig.title.text = '%s: %d' % (name, digest.size()) for name, fig in self.counter_figures.items(): counter = self.server.counters[name] d = {} for i, d in enumerate(counter.components): if d: xs = sorted(d) factor = counter.intervals[0] / counter.intervals[i] counts = [d[x] for x in xs] ys = [factor * c for c in counts] y_centers = [y / 2 for y in ys] xs = list(map(str, xs)) d = {'x': xs, 'y': ys, 'y-center': y_centers, 'counts': counts} self.counter_sources[name][i].data.update(d) fig.title.text = '%s: %d' % (name, counter.size()) fig.x_range.factors = list(map(str, xs)) from bokeh.application.handlers.function import FunctionHandler from bokeh.application import Application def main_doc(worker, extra, doc): with log_errors(): statetable = StateTable(worker) executing_ts = ExecutingTimeSeries(worker, sizing_mode='scale_width') communicating_ts = CommunicatingTimeSeries(worker, sizing_mode='scale_width') communicating_stream = CommunicatingStream(worker, sizing_mode='scale_width') xr = executing_ts.root.x_range communicating_ts.root.x_range = xr communicating_stream.root.x_range = xr doc.title = "Dask Worker Internal Monitor" doc.add_periodic_callback(statetable.update, 200) doc.add_periodic_callback(executing_ts.update, 200) doc.add_periodic_callback(communicating_ts.update, 200) doc.add_periodic_callback(communicating_stream.update, 200) doc.add_root(column(statetable.root, executing_ts.root, communicating_ts.root, communicating_stream.root, sizing_mode='scale_width')) doc.template = template doc.template_variables['active_page'] = 'main' doc.template_variables.update(extra) def crossfilter_doc(worker, extra, doc): with log_errors(): statetable = StateTable(worker) crossfilter = CrossFilter(worker) doc.title = "Dask Worker Cross-filter" doc.add_periodic_callback(statetable.update, 500) doc.add_periodic_callback(crossfilter.update, 500) doc.add_root(column(statetable.root, crossfilter.root)) doc.template = template doc.template_variables['active_page'] = 'crossfilter' doc.template_variables.update(extra) def systemmonitor_doc(worker, extra, doc): with log_errors(): sysmon = SystemMonitor(worker, sizing_mode='scale_width') doc.title = "Dask Worker Monitor" doc.add_periodic_callback(sysmon.update, 500) doc.add_root(sysmon.root) doc.template = template doc.template_variables['active_page'] = 'system' doc.template_variables.update(extra) def counters_doc(server, extra, doc): with log_errors(): doc.title = "Dask Worker Counters" counter = Counters(server, sizing_mode='stretch_both') doc.add_periodic_callback(counter.update, 500) doc.add_root(counter.root) doc.template = template doc.template_variables['active_page'] = 'counters' doc.template_variables.update(extra) def profile_doc(server, extra, doc): with log_errors(): doc.title = "Dask Worker Profile" profile = ProfileTimePlot(server, sizing_mode='scale_width') profile.trigger_update() doc.add_root(profile.root) doc.template = template doc.template_variables['active_page'] = 'profile' doc.template_variables.update(extra) class BokehWorker(BokehServer): def __init__(self, worker, io_loop=None, prefix='', **kwargs): self.worker = worker self.server_kwargs = kwargs self.server_kwargs['prefix'] = prefix or None prefix = prefix or '' prefix = prefix.rstrip('/') if prefix and not prefix.startswith('/'): prefix = '/' + prefix extra = {'prefix': prefix} extra.update(template_variables) main = Application(FunctionHandler(partial(main_doc, worker, extra))) crossfilter = Application(FunctionHandler(partial(crossfilter_doc, worker, extra))) systemmonitor = Application(FunctionHandler(partial(systemmonitor_doc, worker, extra))) counters = Application(FunctionHandler(partial(counters_doc, worker, extra))) profile = Application(FunctionHandler(partial(profile_doc, worker, extra))) self.apps = {'/main': main, '/counters': counters, '/crossfilter': crossfilter, '/system': systemmonitor, '/profile': profile} self.loop = io_loop or worker.loop self.server = None @property def my_server(self): return self.worker distributed-1.20.2/distributed/cfexecutor.py000066400000000000000000000124611321233345200211630ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import concurrent.futures as cf import weakref import six from toolz import merge from tornado import gen from .metrics import time from .utils import sync @gen.coroutine def _cascade_future(future, cf_future): """ Coroutine that waits on Dask future, then transmits its outcome to cf_future. """ result = yield future._result(raiseit=False) status = future.status if status == 'finished': cf_future.set_result(result) elif status == 'cancelled': cf_future.cancel() # Necessary for wait() and as_completed() to wake up cf_future.set_running_or_notify_cancel() else: try: six.reraise(*result) except BaseException as exc: cf_future.set_exception(exc) @gen.coroutine def _wait_on_futures(futures): for fut in futures: try: yield fut except Exception: pass class ClientExecutor(cf.Executor): """ A concurrent.futures Executor that executes tasks on a dask.distributed Client. """ _allowed_kwargs = frozenset(['pure', 'workers', 'resources', 'allow_other_workers']) def __init__(self, client, **kwargs): sk = set(kwargs) if not sk <= self._allowed_kwargs: raise TypeError("unsupported arguments to ClientExecutor: %s" % sorted(sk - self._allowed_kwargs)) self._client = client self._futures = weakref.WeakSet() self._shutdown = False self._kwargs = kwargs def _wrap_future(self, future): """ Wrap a distributed Future in a concurrent.futures Future. """ cf_future = cf.Future() # Support cancelling task through .cancel() on c.f.Future def cf_callback(cf_future): if cf_future.cancelled() and future.status != 'cancelled': future.cancel() cf_future.add_done_callback(cf_callback) self._client.loop.add_callback(_cascade_future, future, cf_future) return cf_future def submit(self, fn, *args, **kwargs): """Submits a callable to be executed with the given arguments. Schedules the callable to be executed as ``fn(*args, **kwargs)`` and returns a Future instance representing the execution of the callable. Returns ------- A Future representing the given call. """ if self._shutdown: raise RuntimeError('cannot schedule new futures after shutdown') future = self._client.submit(fn, *args, **merge(self._kwargs, kwargs)) self._futures.add(future) return self._wrap_future(future) def map(self, fn, *iterables, **kwargs): """Returns an iterator equivalent to ``map(fn, *iterables)``. Parameters ---------- fn: A callable that will take as many arguments as there are passed iterables. iterables: One iterable for each parameter to *fn*. timeout: The maximum number of seconds to wait. If None, then there is no limit on the wait time. chunksize: ignored. Returns ------- An iterator equivalent to: ``map(fn, *iterables)`` but the calls may be evaluated out-of-order. Raises ------ TimeoutError: If the entire result iterator could not be generated before the given timeout. Exception: If ``fn(*args)`` raises for any values. """ timeout = kwargs.pop('timeout', None) if timeout is not None: end_time = timeout + time() if 'chunksize' in kwargs: del kwargs['chunksize'] if kwargs: raise TypeError("unexpected arguments to map(): %s" % sorted(kwargs)) fs = self._client.map(fn, *iterables, **self._kwargs) # Yield must be hidden in closure so that the tasks are submitted # before the first iterator value is required. def result_iterator(): try: for future in fs: self._futures.add(future) if timeout is not None: try: yield future.result(end_time - time()) except gen.TimeoutError: raise cf.TimeoutError else: yield future.result() finally: remaining = list(fs) for future in remaining: self._futures.add(future) self._client.cancel(remaining) return result_iterator() def shutdown(self, wait=True): """Clean-up the resources associated with the Executor. It is safe to call this method several times. Otherwise, no other methods can be called after this one. Parameters ---------- wait: If True then shutdown will not return until all running futures have finished executing. If False then all running futures are cancelled immediately. """ if not self._shutdown: self._shutdown = True fs = list(self._futures) if wait: sync(self._client.loop, _wait_on_futures, fs) else: self._client.cancel(fs) distributed-1.20.2/distributed/cli/000077500000000000000000000000001321233345200172055ustar00rootroot00000000000000distributed-1.20.2/distributed/cli/__init__.py000066400000000000000000000000001321233345200213040ustar00rootroot00000000000000distributed-1.20.2/distributed/cli/dask_mpi.py000066400000000000000000000070401321233345200213470ustar00rootroot00000000000000from functools import partial import click from mpi4py import MPI from tornado.ioloop import IOLoop from tornado import gen from distributed import Scheduler, Nanny, Worker from distributed.bokeh.worker import BokehWorker from distributed.cli.utils import check_python_3, uri_from_host_port from distributed.utils import get_ip_interface comm = MPI.COMM_WORLD rank = comm.Get_rank() loop = IOLoop() @click.command() @click.option('--scheduler-file', type=str, default='scheduler.json', help='Filename to JSON encoded scheduler information. ') @click.option('--interface', type=str, default=None, help="Network interface like 'eth0' or 'ib0'") @click.option('--nthreads', type=int, default=0, help="Number of threads per worker.") @click.option('--memory-limit', default='auto', help="Number of bytes before spilling data to disk. " "This can be an integer (nbytes) " "float (fraction of total memory) " "or 'auto'") @click.option('--local-directory', default='', type=str, help="Directory to place worker files") @click.option('--scheduler/--no-scheduler', default=True, help=("Whether or not to include a scheduler. " "Use --no-scheduler to increase an existing dask cluster")) @click.option('--nanny/--no-nanny', default=True, help="Start workers in nanny process for management") @click.option('--bokeh-port', type=int, default=8787, help="Bokeh port for visual diagnostics") @click.option('--bokeh-worker-port', type=int, default=8789, help="Worker's Bokeh port for visual diagnostics") @click.option('--bokeh-prefix', type=str, default=None, help="Prefix for the bokeh app") def main(scheduler_file, interface, nthreads, local_directory, memory_limit, scheduler, bokeh_port, bokeh_prefix, nanny, bokeh_worker_port): if interface: host = get_ip_interface(interface) else: host = None if rank == 0 and scheduler: try: from distributed.bokeh.scheduler import BokehScheduler except ImportError: services = {} else: services = {('bokeh', bokeh_port): partial(BokehScheduler, prefix=bokeh_prefix)} scheduler = Scheduler(scheduler_file=scheduler_file, loop=loop, services=services) addr = uri_from_host_port(host, None, 8786) scheduler.start(addr) try: loop.start() loop.close() finally: scheduler.stop() else: W = Nanny if nanny else Worker worker = W(scheduler_file=scheduler_file, loop=loop, name=rank if scheduler else None, ncores=nthreads, local_dir=local_directory, services={('bokeh', bokeh_worker_port): BokehWorker}, memory_limit=memory_limit) addr = uri_from_host_port(host, None, 0) @gen.coroutine def run(): yield worker._start(addr) while worker.status != 'closed': yield gen.sleep(0.2) try: loop.run_sync(run) loop.close() finally: pass @gen.coroutine def close(): yield worker._close(timeout=2) loop.run_sync(close) def go(): check_python_3() main() if __name__ == '__main__': go() distributed-1.20.2/distributed/cli/dask_remote.py000066400000000000000000000010401321233345200220470ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import click from distributed.cli.utils import check_python_3, install_signal_handlers from distributed.submit import _remote @click.command() @click.option('--host', type=str, default=None, help="IP or hostname of this server") @click.option('--port', type=int, default=8788, help="Remote Client Port") def main(host, port): _remote(host, port) def go(): install_signal_handlers() check_python_3() main() if __name__ == '__main__': go() distributed-1.20.2/distributed/cli/dask_scheduler.py000077500000000000000000000125431321233345200225470ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import atexit from functools import partial import logging import os import shutil import sys import tempfile import click from tornado.ioloop import IOLoop from distributed import Scheduler from distributed.security import Security from distributed.utils import get_ip_interface, ignoring from distributed.cli.utils import (check_python_3, install_signal_handlers, uri_from_host_port) from distributed.preloading import preload_modules from distributed.proctitle import (enable_proctitle_on_children, enable_proctitle_on_current) logger = logging.getLogger('distributed.scheduler') pem_file_option_type = click.Path(exists=True, resolve_path=True) @click.command() @click.option('--host', type=str, default='', help="URI, IP or hostname of this server") @click.option('--port', type=int, default=None, help="Serving port") @click.option('--interface', type=str, default=None, help="Preferred network interface like 'eth0' or 'ib0'") @click.option('--tls-ca-file', type=pem_file_option_type, default=None, help="CA cert(s) file for TLS (in PEM format)") @click.option('--tls-cert', type=pem_file_option_type, default=None, help="certificate file for TLS (in PEM format)") @click.option('--tls-key', type=pem_file_option_type, default=None, help="private key file for TLS (in PEM format)") # XXX default port (or URI) values should be centralized somewhere @click.option('--bokeh-port', type=int, default=8787, help="Bokeh port for visual diagnostics") @click.option('--bokeh/--no-bokeh', '_bokeh', default=True, show_default=True, required=False, help="Launch Bokeh Web UI") @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.") @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") @click.option('--pid-file', type=str, default='', help="File to write the process PID") @click.option('--scheduler-file', type=str, default='', help="File to write connection information. " "This may be a good way to share connection information if your " "cluster is on a shared network file system.") @click.option('--local-directory', default='', type=str, help="Directory to place scheduler files") @click.option('--preload', type=str, multiple=True, help='Module that should be loaded by each worker process like "foo.bar" or "/path/to/foo.py"') def main(host, port, bokeh_port, show, _bokeh, bokeh_whitelist, bokeh_prefix, use_xheaders, pid_file, scheduler_file, interface, local_directory, preload, tls_ca_file, tls_cert, tls_key): enable_proctitle_on_current() enable_proctitle_on_children() sec = Security(tls_ca_file=tls_ca_file, tls_scheduler_cert=tls_cert, tls_scheduler_key=tls_key, ) if pid_file: with open(pid_file, 'w') as f: f.write(str(os.getpid())) def del_pid_file(): if os.path.exists(pid_file): os.remove(pid_file) atexit.register(del_pid_file) local_directory_created = False if local_directory: if not os.path.exists(local_directory): os.mkdir(local_directory) local_directory_created = True else: local_directory = tempfile.mkdtemp(prefix='scheduler-') local_directory_created = True if local_directory not in sys.path: sys.path.insert(0, local_directory) if sys.platform.startswith('linux'): import resource # module fails importing on Windows soft, hard = resource.getrlimit(resource.RLIMIT_NOFILE) limit = max(soft, hard // 2) resource.setrlimit(resource.RLIMIT_NOFILE, (limit, hard)) if interface: if host: raise ValueError("Can not specify both interface and host") else: host = get_ip_interface(interface) addr = uri_from_host_port(host, port, 8786) loop = IOLoop.current() logger.info('-' * 47) services = {} if _bokeh: with ignoring(ImportError): from distributed.bokeh.scheduler import BokehScheduler services[('bokeh', bokeh_port)] = partial(BokehScheduler, prefix=bokeh_prefix) scheduler = Scheduler(loop=loop, services=services, scheduler_file=scheduler_file, security=sec) scheduler.start(addr) preload_modules(preload, parameter=scheduler, file_dir=local_directory) logger.info('Local Directory: %26s', local_directory) logger.info('-' * 47) install_signal_handlers(loop) try: loop.start() loop.close() finally: scheduler.stop() if local_directory_created: shutil.rmtree(local_directory) logger.info("End scheduler at %r", addr) def go(): check_python_3() main() if __name__ == '__main__': go() distributed-1.20.2/distributed/cli/dask_ssh.py000077500000000000000000000073771321233345200213770ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from distributed.deploy.ssh import SSHCluster import click from distributed.cli.utils import check_python_3 @click.command(help="""Launch a distributed cluster over SSH. A 'dask-scheduler' process will run on the first host specified in [HOSTNAMES] or in the hostfile (unless --scheduler is specified explicitly). One or more 'dask-worker' processes will be run each host in [HOSTNAMES] or in the hostfile. Use command line flags to adjust how many dask-worker process are run on each host (--nprocs) and how many cpus are used by each dask-worker process (--nthreads).""") @click.option('--scheduler', default=None, type=str, help="Specify scheduler node. Defaults to first address.") @click.option('--scheduler-port', default=8786, type=int, help="Specify scheduler port number. Defaults to port 8786.") @click.option('--nthreads', default=0, type=int, help=("Number of threads per worker process. " "Defaults to number of cores divided by the number of " "processes per host.")) @click.option('--nprocs', default=1, type=int, help="Number of worker processes per host. Defaults to one.") @click.argument('hostnames', nargs=-1, type=str) @click.option('--hostfile', default=None, type=click.Path(exists=True), help="Textfile with hostnames/IP addresses") @click.option('--ssh-username', default=None, type=str, help="Username to use when establishing SSH connections.") @click.option('--ssh-port', default=22, type=int, help="Port to use for SSH connections.") @click.option('--ssh-private-key', default=None, type=str, help="Private key file to use for SSH connections.") @click.option('--nohost', is_flag=True, help="Do not pass the hostname to the worker.") @click.option('--log-directory', default=None, type=click.Path(exists=True), help=("Directory to use on all cluster nodes for the output of " "dask-scheduler and dask-worker commands.")) @click.pass_context def main(ctx, scheduler, scheduler_port, hostnames, hostfile, nthreads, nprocs, ssh_username, ssh_port, ssh_private_key, nohost, log_directory): try: hostnames = list(hostnames) if hostfile: with open(hostfile) as f: hosts = f.read().split() hostnames.extend(hosts) if not scheduler: scheduler = hostnames[0] except IndexError: print(ctx.get_help()) exit(1) c = SSHCluster(scheduler, scheduler_port, hostnames, nthreads, nprocs, ssh_username, ssh_port, ssh_private_key, nohost, log_directory) import distributed print('\n---------------------------------------------------------------') print(' Dask.distributed v{version}\n'.format(version=distributed.__version__)) print('Worker nodes:'.format(n=len(hostnames))) for i, host in enumerate(hostnames): print(' {num}: {host}'.format(num=i, host=host)) print('\nscheduler node: {addr}:{port}'.format(addr=scheduler, port=scheduler_port)) print('---------------------------------------------------------------\n\n') # Monitor the output of remote processes. This blocks until the user issues a KeyboardInterrupt. c.monitor_remote_processes() # Close down the remote processes and exit. print("\n[ dask-ssh ]: Shutting down remote processes (this may take a moment).") c.shutdown() print("[ dask-ssh ]: Remote processes have been terminated. Exiting.") def go(): check_python_3() main() if __name__ == '__main__': go() distributed-1.20.2/distributed/cli/dask_submit.py000066400000000000000000000013551321233345200220700ustar00rootroot00000000000000import sys import click from tornado import gen from tornado.ioloop import IOLoop from distributed.cli.utils import check_python_3, install_signal_handlers from distributed.submit import _submit @click.command() @click.argument('remote_client_address', type=str, required=True) @click.argument('filepath', type=str, required=True) def main(remote_client_address, filepath): @gen.coroutine def f(): stdout, stderr = yield _submit(remote_client_address, filepath) if stdout: sys.stdout.write(str(stdout)) if stderr: sys.stderr.write(str(stderr)) IOLoop.instance().run_sync(f) def go(): install_signal_handlers() check_python_3() main() if __name__ == '__main__': go() distributed-1.20.2/distributed/cli/dask_worker.py000077500000000000000000000225111321233345200220760ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import atexit import logging import os from sys import exit import click from distributed import Nanny, Worker from distributed.utils import get_ip_interface from distributed.worker import _ncores from distributed.security import Security from distributed.cli.utils import (check_python_3, uri_from_host_port, install_signal_handlers) from distributed.comm import get_address_host_port from distributed.proctitle import (enable_proctitle_on_children, enable_proctitle_on_current) from toolz import valmap from tornado.ioloop import IOLoop, TimeoutError from tornado import gen logger = logging.getLogger('distributed.dask_worker') pem_file_option_type = click.Path(exists=True, resolve_path=True) @click.command() @click.argument('scheduler', type=str, required=False) @click.option('--tls-ca-file', type=pem_file_option_type, default=None, help="CA cert(s) file for TLS (in PEM format)") @click.option('--tls-cert', type=pem_file_option_type, default=None, help="certificate file for TLS (in PEM format)") @click.option('--tls-key', type=pem_file_option_type, default=None, help="private key file for TLS (in PEM format)") @click.option('--worker-port', type=int, default=0, help="Serving computation port, defaults to random") @click.option('--nanny-port', type=int, default=0, help="Serving nanny port, defaults to random") @click.option('--bokeh-port', type=int, default=8789, help="Bokeh port, defaults to 8789") @click.option('--bokeh/--no-bokeh', 'bokeh', default=True, show_default=True, required=False, help="Launch Bokeh Web UI") @click.option('--listen-address', type=str, default=None, help="The address to which the worker binds. " "Example: tcp://0.0.0.0:9000") @click.option('--contact-address', type=str, default=None, help="The address the worker advertises to the scheduler for " "communication with it and other workers. " "Example: tcp://127.0.0.1:9000") @click.option('--host', type=str, default=None, help="Serving host. Should be an ip address that is" " visible to the scheduler and other workers. " "See --listen-address and --contact-address if you " "need different listen and contact addresses. " "See --interface.") @click.option('--interface', type=str, default=None, help="Network interface like 'eth0' or 'ib0'") @click.option('--nthreads', type=int, default=0, help="Number of threads per process.") @click.option('--nprocs', type=int, default=1, help="Number of worker processes. Defaults to one.") @click.option('--name', type=str, default='', help="A unique name for this worker like 'worker-1'") @click.option('--memory-limit', default='auto', help="Bytes of memory that the worker can use. " "This can be an integer (bytes), " "float (fraction of total system memory), " "string (like 5GB or 5000M), " "'auto', or zero for no memory management") @click.option('--reconnect/--no-reconnect', default=True, help="Reconnect to scheduler if disconnected") @click.option('--nanny/--no-nanny', default=True, help="Start workers in nanny process for management") @click.option('--pid-file', type=str, default='', help="File to write the process PID") @click.option('--local-directory', default='', type=str, help="Directory to place worker files") @click.option('--resources', type=str, default='', help='Resources for task constraints like "GPU=2 MEM=10e9"') @click.option('--scheduler-file', type=str, default='', help='Filename to JSON encoded scheduler information. ' 'Use with dask-scheduler --scheduler-file') @click.option('--death-timeout', type=float, default=None, help="Seconds to wait for a scheduler before closing") @click.option('--bokeh-prefix', type=str, default=None, help="Prefix for the bokeh app") @click.option('--preload', type=str, multiple=True, help='Module that should be loaded by each worker process ' 'like "foo.bar" or "/path/to/foo.py"') def main(scheduler, host, worker_port, listen_address, contact_address, nanny_port, nthreads, nprocs, nanny, name, memory_limit, pid_file, reconnect, resources, bokeh, bokeh_port, local_directory, scheduler_file, interface, death_timeout, preload, bokeh_prefix, tls_ca_file, tls_cert, tls_key): enable_proctitle_on_current() enable_proctitle_on_children() sec = Security(tls_ca_file=tls_ca_file, tls_worker_cert=tls_cert, tls_worker_key=tls_key, ) if nprocs > 1 and worker_port != 0: logger.error("Failed to launch worker. You cannot use the --port argument when nprocs > 1.") exit(1) if nprocs > 1 and name: logger.error("Failed to launch worker. You cannot use the --name argument when nprocs > 1.") exit(1) if nprocs > 1 and not nanny: logger.error("Failed to launch worker. You cannot use the --no-nanny argument when nprocs > 1.") exit(1) if contact_address and not listen_address: logger.error("Failed to launch worker. " "Must specify --listen-address when --contact-address is given") exit(1) if nprocs > 1 and listen_address: logger.error("Failed to launch worker. " "You cannot specify --listen-address when nprocs > 1.") exit(1) if (worker_port or host) and listen_address: logger.error("Failed to launch worker. " "You cannot specify --listen-address when --worker-port or --host is given.") exit(1) try: if listen_address: (host, worker_port) = get_address_host_port(listen_address, strict=True) if contact_address: # we only need this to verify it is getting parsed (_, _) = get_address_host_port(contact_address, strict=True) else: # if contact address is not present we use the listen_address for contact contact_address = listen_address except ValueError as e: logger.error("Failed to launch worker. " + str(e)) exit(1) if nanny: port = nanny_port else: port = worker_port if not nthreads: nthreads = _ncores // nprocs if pid_file: with open(pid_file, 'w') as f: f.write(str(os.getpid())) def del_pid_file(): if os.path.exists(pid_file): os.remove(pid_file) atexit.register(del_pid_file) services = {} if bokeh: try: from distributed.bokeh.worker import BokehWorker except ImportError: pass else: if bokeh_prefix: result = (BokehWorker, {'prefix': bokeh_prefix}) else: result = BokehWorker services[('bokeh', bokeh_port)] = result if resources: resources = resources.replace(',', ' ').split() resources = dict(pair.split('=') for pair in resources) resources = valmap(float, resources) else: resources = None loop = IOLoop.current() if nanny: kwargs = {'worker_port': worker_port, 'listen_address': listen_address} t = Nanny else: kwargs = {} if nanny_port: kwargs['service_ports'] = {'nanny': nanny_port} t = Worker if not scheduler and not scheduler_file: raise ValueError("Need to provide scheduler address like\n" "dask-worker SCHEDULER_ADDRESS:8786") if interface: if host: raise ValueError("Can not specify both interface and host") else: host = get_ip_interface(interface) if host or port: addr = uri_from_host_port(host, port, 0) else: # Choose appropriate address for scheduler addr = None nannies = [t(scheduler, scheduler_file=scheduler_file, ncores=nthreads, services=services, name=name, loop=loop, resources=resources, memory_limit=memory_limit, reconnect=reconnect, local_dir=local_directory, death_timeout=death_timeout, preload=preload, security=sec, contact_address=contact_address, **kwargs) for i in range(nprocs)] @gen.coroutine def close_all(): # Unregister all workers from scheduler if nanny: yield [n._close(timeout=2) for n in nannies] def on_signal(signum): logger.info("Exiting on signal %d", signum) close_all() @gen.coroutine def run(): yield [n._start(addr) for n in nannies] while all(n.status != 'closed' for n in nannies): yield gen.sleep(0.2) install_signal_handlers(loop, cleanup=on_signal) try: loop.run_sync(run) except (KeyboardInterrupt, TimeoutError): pass finally: logger.info("End worker") def go(): check_python_3() main() if __name__ == '__main__': go() distributed-1.20.2/distributed/cli/tests/000077500000000000000000000000001321233345200203475ustar00rootroot00000000000000distributed-1.20.2/distributed/cli/tests/test_cli_utils.py000066400000000000000000000041151321233345200237500ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import pytest pytest.importorskip('requests') from distributed.cli.utils import uri_from_host_port from distributed.utils import get_ip external_ip = get_ip() def test_uri_from_host_port(): f = uri_from_host_port assert f('', 456, None) == 'tcp://:456' assert f('', 456, 123) == 'tcp://:456' assert f('', None, 123) == 'tcp://:123' assert f('', None, 0) == 'tcp://' assert f('', 0, 123) == 'tcp://' assert f('localhost', 456, None) == 'tcp://localhost:456' assert f('localhost', 456, 123) == 'tcp://localhost:456' assert f('localhost', None, 123) == 'tcp://localhost:123' assert f('localhost', None, 0) == 'tcp://localhost' assert f('192.168.1.2', 456, None) == 'tcp://192.168.1.2:456' assert f('192.168.1.2', 456, 123) == 'tcp://192.168.1.2:456' assert f('192.168.1.2', None, 123) == 'tcp://192.168.1.2:123' assert f('192.168.1.2', None, 0) == 'tcp://192.168.1.2' assert f('tcp://192.168.1.2', 456, None) == 'tcp://192.168.1.2:456' assert f('tcp://192.168.1.2', 456, 123) == 'tcp://192.168.1.2:456' assert f('tcp://192.168.1.2', None, 123) == 'tcp://192.168.1.2:123' assert f('tcp://192.168.1.2', None, 0) == 'tcp://192.168.1.2' assert f('tcp://192.168.1.2:456', None, None) == 'tcp://192.168.1.2:456' assert f('tcp://192.168.1.2:456', 0, 0) == 'tcp://192.168.1.2:456' assert f('tcp://192.168.1.2:456', 0, 123) == 'tcp://192.168.1.2:456' assert f('tcp://192.168.1.2:456', 456, 123) == 'tcp://192.168.1.2:456' with pytest.raises(ValueError): # Two incompatible port values f('tcp://192.168.1.2:456', 123, None) assert f('tls://192.168.1.2:456', None, None) == 'tls://192.168.1.2:456' assert f('tls://192.168.1.2:456', 0, 0) == 'tls://192.168.1.2:456' assert f('tls://192.168.1.2:456', 0, 123) == 'tls://192.168.1.2:456' assert f('tls://192.168.1.2:456', 456, 123) == 'tls://192.168.1.2:456' assert f('tcp://[::1]:456', None, None) == 'tcp://[::1]:456' assert f('tls://[::1]:456', None, None) == 'tls://[::1]:456' distributed-1.20.2/distributed/cli/tests/test_dask_mpi.py000066400000000000000000000046541321233345200235600ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import subprocess from time import sleep import pytest pytest.importorskip('mpi4py') import requests from distributed import Client from distributed.utils import tmpfile from distributed.metrics import time from distributed.utils_test import popen from distributed.utils_test import loop # flake8: noqa @pytest.mark.parametrize('nanny', ['--nanny', '--no-nanny']) def test_basic(loop, nanny): with tmpfile() as fn: with popen(['mpirun', '--np', '4', 'dask-mpi', '--scheduler-file', fn, nanny], stdin=subprocess.DEVNULL): with Client(scheduler_file=fn) as c: start = time() while len(c.scheduler_info()['workers']) != 3: assert time() < start + 10 sleep(0.2) assert c.submit(lambda x: x + 1, 10, workers=1).result() == 11 def test_no_scheduler(loop): with tmpfile() as fn: with popen(['mpirun', '--np', '2', 'dask-mpi', '--scheduler-file', fn], stdin=subprocess.DEVNULL): with Client(scheduler_file=fn) as c: start = time() while len(c.scheduler_info()['workers']) != 1: assert time() < start + 10 sleep(0.2) assert c.submit(lambda x: x + 1, 10).result() == 11 with popen(['mpirun', '--np', '1', 'dask-mpi', '--scheduler-file', fn, '--no-scheduler']): start = time() while len(c.scheduler_info()['workers']) != 2: assert time() < start + 10 sleep(0.2) def test_bokeh(loop): with tmpfile() as fn: with popen(['mpirun', '--np', '2', 'dask-mpi', '--scheduler-file', fn, '--bokeh-port', '59583', '--bokeh-worker-port', '59584'], stdin=subprocess.DEVNULL): for port in [59853, 59584]: start = time() while True: try: response = requests.get('http://localhost:%d/status/' % port) assert response.ok break except Exception: sleep(0.1) assert time() < start + 20 with pytest.raises(Exception): requests.get('http://localhost:59583/status/') distributed-1.20.2/distributed/cli/tests/test_dask_remote.py000066400000000000000000000004361321233345200242600ustar00rootroot00000000000000from click.testing import CliRunner from distributed.cli.dask_remote import main def test_dask_remote(): runner = CliRunner() result = runner.invoke(main, ['--help']) assert '--host TEXT IP or hostname of this server' in result.output assert result.exit_code == 0 distributed-1.20.2/distributed/cli/tests/test_dask_scheduler.py000066400000000000000000000230231321233345200247400ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import pytest pytest.importorskip('requests') import os import requests import socket import shutil import sys import tempfile from time import sleep from tornado import gen from distributed import Scheduler, Client from distributed.utils import get_ip, get_ip_interface, tmpfile from distributed.utils_test import (popen, assert_can_connect_from_everywhere_4_6, assert_can_connect_locally_4, ) from distributed.utils_test import loop # flake8: noqa from distributed.metrics import time def test_defaults(loop): with popen(['dask-scheduler', '--no-bokeh']) as proc: @gen.coroutine def f(): # Default behaviour is to listen on all addresses yield [ assert_can_connect_from_everywhere_4_6(8786, 2.0), # main port ] loop.run_sync(f) with Client('127.0.0.1:%d' % Scheduler.default_port, loop=loop) as c: pass with pytest.raises(Exception): requests.get('http://127.0.0.1:8787/status/') with pytest.raises(Exception): response = requests.get('http://127.0.0.1:9786/info.json') def test_hostport(loop): with popen(['dask-scheduler', '--no-bokeh', '--host', '127.0.0.1:8978']): @gen.coroutine def f(): yield [ # The scheduler's main port can't be contacted from the outside assert_can_connect_locally_4(8978, 2.0), ] loop.run_sync(f) with Client('127.0.0.1:8978', loop=loop) as c: assert len(c.ncores()) == 0 def test_no_bokeh(loop): pytest.importorskip('bokeh') with popen(['dask-scheduler', '--no-bokeh']) 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() with pytest.raises(Exception): requests.get('http://127.0.0.1:8787/status/') def test_bokeh(loop): pytest.importorskip('bokeh') with popen(['dask-scheduler']) as proc: with Client('127.0.0.1:%d' % Scheduler.default_port, loop=loop) as c: pass names = ['localhost', '127.0.0.1', get_ip()] if 'linux' in sys.platform: names.append(socket.gethostname()) start = time() while True: try: # All addresses should respond for name in names: uri = 'http://%s:8787/status/' % name response = requests.get(uri) assert response.ok break except Exception as f: print('got error on %r: %s' % (uri, f)) sleep(0.1) assert time() < start + 10 with pytest.raises(Exception): requests.get('http://127.0.0.1:8787/status/') def test_bokeh_non_standard_ports(loop): pytest.importorskip('bokeh') with popen(['dask-scheduler', '--port', '3448', '--bokeh-port', '4832']) as proc: with Client('127.0.0.1:3448', loop=loop) as c: pass start = time() while True: try: response = requests.get('http://localhost:4832/status/') assert response.ok break except Exception: sleep(0.1) assert time() < start + 20 with pytest.raises(Exception): requests.get('http://localhost:4832/status/') @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") def test_bokeh_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 Client('127.0.0.1:%d' % Scheduler.default_port, loop=loop) as c: pass start = time() while True: try: for name in ['127.0.0.2', '127.0.0.3']: response = requests.get('http://%s:8787/status/' % name) assert response.ok break except Exception as f: print(f) sleep(0.1) assert time() < start + 20 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 Client('127.0.0.1:%d' % Scheduler.default_port, loop=loop) as c: start = time() while len(c.ncores()) < 2: sleep(0.1) assert time() < start + 10 def test_interface(loop): psutil = pytest.importorskip('psutil') if_names = sorted(psutil.net_if_addrs()) for if_name in if_names: try: ipv4_addr = get_ip_interface(if_name) except ValueError: pass else: if ipv4_addr == '127.0.0.1': break else: pytest.skip("Could not find loopback interface. " "Available interfaces are: %s." % (if_names,)) with popen(['dask-scheduler', '--no-bokeh', '--interface', if_name]) as s: with popen(['dask-worker', '127.0.0.1:8786', '--no-bokeh', '--interface', if_name]) as a: with Client('tcp://127.0.0.1:%d' % Scheduler.default_port, loop=loop) as c: start = time() while not len(c.ncores()): sleep(0.1) assert time() - start < 5 info = c.scheduler_info() assert 'tcp://127.0.0.1' in info['address'] assert all('127.0.0.1' == d['host'] for d in info['workers'].values()) def test_pid_file(loop): def check_pidfile(proc, pidfile): start = time() while not os.path.exists(pidfile): sleep(0.01) assert time() < start + 5 text = False start = time() while not text: sleep(0.01) assert time() < start + 5 with open(pidfile) as f: text = f.read() pid = int(text) if sys.platform.startswith('win'): # On Windows, `dask-XXX` invokes the dask-XXX.exe # shim, but the PID is written out by the child Python process assert pid else: assert proc.pid == pid with tmpfile() as s: with popen(['dask-scheduler', '--pid-file', s, '--no-bokeh']) as sched: check_pidfile(sched, s) with tmpfile() as w: with popen(['dask-worker', '127.0.0.1:8786', '--pid-file', w, '--no-bokeh']) as worker: check_pidfile(worker, w) def test_scheduler_port_zero(loop): with tmpfile() as fn: with popen(['dask-scheduler', '--no-bokeh', '--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): pytest.importorskip('bokeh') with tmpfile() as fn: with popen(['dask-scheduler', '--bokeh-port', '0']) as proc: count = 0 while count < 1: line = proc.stderr.readline() if b'bokeh' in line.lower() or b'web' in line.lower(): sleep(0.01) count += 1 assert b':0' not in line PRELOAD_TEXT = """ _scheduler_info = {} def dask_setup(scheduler): _scheduler_info['address'] = scheduler.address def get_scheduler_address(): return _scheduler_info['address'] """ def test_preload_file(loop): def check_scheduler(): import scheduler_info return scheduler_info.get_scheduler_address() tmpdir = tempfile.mkdtemp() try: path = os.path.join(tmpdir, 'scheduler_info.py') with open(path, 'w') as f: f.write(PRELOAD_TEXT) with tmpfile() as fn: with popen(['dask-scheduler', '--scheduler-file', fn, '--preload', path]): with Client(scheduler_file=fn, loop=loop) as c: assert c.run_on_scheduler(check_scheduler) == \ c.scheduler.address finally: shutil.rmtree(tmpdir) def test_preload_module(loop): def check_scheduler(): import scheduler_info return scheduler_info.get_scheduler_address() tmpdir = tempfile.mkdtemp() try: path = os.path.join(tmpdir, 'scheduler_info.py') with open(path, 'w') as f: f.write(PRELOAD_TEXT) env = os.environ.copy() if 'PYTHONPATH' in env: env['PYTHONPATH'] = tmpdir + ':' + env['PYTHONPATH'] else: env['PYTHONPATH'] = tmpdir with tmpfile() as fn: with popen(['dask-scheduler', '--scheduler-file', fn, '--preload', 'scheduler_info'], env=env): with Client(scheduler_file=fn, loop=loop) as c: assert c.run_on_scheduler(check_scheduler) == \ c.scheduler.address finally: shutil.rmtree(tmpdir) distributed-1.20.2/distributed/cli/tests/test_dask_submit.py000066400000000000000000000004561321233345200242720ustar00rootroot00000000000000from click.testing import CliRunner from distributed.cli.dask_submit import main def test_submit_runs_as_a_cli(): runner = CliRunner() result = runner.invoke(main, ['--help']) assert result.exit_code == 0 assert 'Usage: main [OPTIONS] REMOTE_CLIENT_ADDRESS FILEPATH' in result.output distributed-1.20.2/distributed/cli/tests/test_dask_worker.py000066400000000000000000000154371321233345200243050ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import pytest pytest.importorskip('requests') import sys from time import sleep from toolz import first from distributed import Client from distributed.metrics import time from distributed.utils import sync, tmpfile from distributed.utils_test import (popen, slow, terminate_process, wait_for_port) from distributed.utils_test import loop # flake8: noqa def test_nanny_worker_ports(loop): with popen(['dask-scheduler', '--port', '9359', '--no-bokeh']) as sched: with popen(['dask-worker', '127.0.0.1:9359', '--host', '127.0.0.1', '--worker-port', '9684', '--nanny-port', '5273', '--no-bokeh']) as worker: with Client('127.0.0.1:9359', loop=loop) as c: start = time() while True: d = sync(c.loop, c.scheduler.identity) if d['workers']: break else: assert time() - start < 5 sleep(0.1) assert d['workers']['tcp://127.0.0.1:9684']['services']['nanny'] == 5273 def test_memory_limit(loop): with popen(['dask-scheduler', '--no-bokeh']) as sched: with popen(['dask-worker', '127.0.0.1:8786', '--memory-limit', '2e3MB', '--no-bokeh']) as worker: with Client('127.0.0.1:8786', loop=loop) as c: while not c.ncores(): sleep(0.1) info = c.scheduler_info() d = first(info['workers'].values()) assert isinstance(d['memory_limit'], int) assert d['memory_limit'] == 2e9 def test_no_nanny(loop): with popen(['dask-scheduler', '--no-bokeh']) as sched: with popen(['dask-worker', '127.0.0.1:8786', '--no-nanny', '--no-bokeh']) as worker: assert any(b'Registered' in worker.stderr.readline() for i in range(15)) @slow @pytest.mark.parametrize('nanny', ['--nanny', '--no-nanny']) def test_no_reconnect(nanny, loop): with popen(['dask-scheduler', '--no-bokeh']) as sched: wait_for_port(('127.0.0.1', 8786)) with popen(['dask-worker', 'tcp://127.0.0.1:8786', '--no-reconnect', nanny, '--no-bokeh']) as worker: sleep(2) terminate_process(sched) start = time() while worker.poll() is None: sleep(0.1) assert time() < start + 10 def test_resources(loop): with popen(['dask-scheduler', '--no-bokeh']) as sched: with popen(['dask-worker', 'tcp://127.0.0.1:8786', '--no-bokeh', '--resources', 'A=1 B=2,C=3']) as worker: with Client('127.0.0.1:8786', loop=loop) as c: while not c.scheduler_info()['workers']: sleep(0.1) info = c.scheduler_info() worker = list(info['workers'].values())[0] assert worker['resources'] == {'A': 1, 'B': 2, 'C': 3} @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-worker', '127.0.0.1:8786', nanny, '--no-bokeh', '--local-directory', fn]) as worker: with Client('127.0.0.1:8786', loop=loop, timeout=10) as c: start = time() while not c.scheduler_info()['workers']: sleep(0.1) assert time() < start + 8 info = c.scheduler_info() worker = list(info['workers'].values())[0] assert worker['local_directory'].startswith(fn) @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 Client(scheduler_file=fn, 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(['dask-worker', '127.0.0.1:8786', '--nprocs=2', '--no-nanny']) as worker: assert any(b'Failed to launch worker' in worker.stderr.readline() for i in range(15)) @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @pytest.mark.parametrize('nanny', ['--nanny', '--no-nanny']) @pytest.mark.parametrize('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-worker', '127.0.0.1:8786', nanny, '--no-bokeh', '--contact-address', 'tcp://127.0.0.2:39837', '--listen-address', listen_address]) as worker: with Client('127.0.0.1:8786') as client: while not client.ncores(): sleep(0.1) info = client.scheduler_info() assert 'tcp://127.0.0.2:39837' in info['workers'] # roundtrip works assert client.submit(lambda x: x + 1, 10).result() == 11 def func(dask_worker): return dask_worker.listener.listen_address assert client.run(func) == {'tcp://127.0.0.2:39837': listen_address} @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @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-worker', '127.0.0.1:8786', nanny, '--no-bokeh', '--host', host]) as worker: with Client('127.0.0.1:8786') as client: while not client.ncores(): sleep(0.1) info = client.scheduler_info() # roundtrip works assert client.submit(lambda x: x + 1, 10).result() == 11 def func(dask_worker): return dask_worker.listener.listen_address listen_addresses = client.run(func) assert all(host in v for v in listen_addresses.values()) distributed-1.20.2/distributed/cli/tests/test_tls_cli.py000066400000000000000000000043111321233345200234100ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from time import sleep from distributed import Client from distributed.utils_test import (popen, get_cert, new_config_file, tls_security, tls_only_config) from distributed.utils_test import loop # flake8: noqa from distributed.metrics import time ca_file = get_cert('tls-ca-cert.pem') cert = get_cert('tls-cert.pem') key = get_cert('tls-key.pem') keycert = get_cert('tls-key-cert.pem') tls_args = ['--tls-ca-file', ca_file, '--tls-cert', keycert] tls_args_2 = ['--tls-ca-file', ca_file, '--tls-cert', cert, '--tls-key', key] def wait_for_cores(c, ncores=1): start = time() while len(c.ncores()) < 1: sleep(0.1) assert time() < start + 10 def test_basic(loop): with popen(['dask-scheduler', '--no-bokeh', '--host', 'tls://'] + tls_args) as s: with popen(['dask-worker', '--no-bokeh', 'tls://127.0.0.1:8786'] + tls_args) as w: with Client('tls://127.0.0.1:8786', loop=loop, security=tls_security()) as c: wait_for_cores(c) def test_nanny(loop): with popen(['dask-scheduler', '--no-bokeh', '--host', 'tls://'] + tls_args) as s: with popen(['dask-worker', '--no-bokeh', '--nanny', 'tls://127.0.0.1:8786'] + tls_args) as w: with Client('tls://127.0.0.1:8786', loop=loop, security=tls_security()) as c: wait_for_cores(c) def test_separate_key_cert(loop): with popen(['dask-scheduler', '--no-bokeh', '--host', 'tls://'] + tls_args_2) as s: with popen(['dask-worker', '--no-bokeh', 'tls://127.0.0.1:8786'] + tls_args_2) as w: with Client('tls://127.0.0.1:8786', loop=loop, security=tls_security()) as c: wait_for_cores(c) 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 Client('tls://127.0.0.1:8786', loop=loop, security=tls_security()) as c: wait_for_cores(c) distributed-1.20.2/distributed/cli/utils.py000066400000000000000000000055221321233345200207230ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from tornado import gen from tornado.ioloop import IOLoop from distributed.comm import (parse_address, unparse_address, parse_host_port, unparse_host_port) py3_err_msg = """ Your terminal does not properly support unicode text required by command line utilities running Python 3. This is commonly solved by specifying encoding environment variables, though exact solutions may depend on your system: $ export LC_ALL=C.UTF-8 $ export LANG=C.UTF-8 For more information see: http://click.pocoo.org/5/python3/ """.strip() def check_python_3(): """Ensures that the environment is good for unicode on Python 3.""" try: from click import _unicodefun _unicodefun._verify_python3_env() except (TypeError, RuntimeError) as e: import sys import click click.echo(py3_err_msg, err=True) sys.exit(1) def install_signal_handlers(loop=None, cleanup=None): """ Install global signal handlers to halt the Tornado IOLoop in case of a SIGINT or SIGTERM. *cleanup* is an optional callback called, before the loop stops, with a single signal number argument. """ import signal loop = loop or IOLoop.current() old_handlers = {} def handle_signal(sig, frame): @gen.coroutine def cleanup_and_stop(): try: if cleanup is not None: yield cleanup(sig) finally: loop.stop() loop.add_callback_from_signal(cleanup_and_stop) # Restore old signal handler to allow for a quicker exit # if the user sends the signal again. signal.signal(sig, old_handlers[sig]) for sig in [signal.SIGINT, signal.SIGTERM]: old_handlers[sig] = signal.signal(sig, handle_signal) def uri_from_host_port(host_arg, port_arg, default_port): """ Process the *host* and *port* CLI options. Return a URI. """ # Much of distributed depends on a well-known IP being assigned to # each entity (Worker, Scheduler, etc.), so avoid "universal" addresses # like '' which would listen on all registered IPs and interfaces. scheme, loc = parse_address(host_arg or '') host, port = parse_host_port(loc, port_arg if port_arg is not None else default_port) if port is None and port_arg is None: port_arg = default_port if port and port_arg and port != port_arg: raise ValueError("port number given twice in options: " "host %r and port %r" % (host_arg, port_arg)) if port is None and port_arg is not None: port = port_arg # Note `port = 0` means "choose a random port" if port is None: port = default_port loc = unparse_host_port(host, port) addr = unparse_address(scheme, loc) return addr distributed-1.20.2/distributed/client.py000066400000000000000000003643521321233345200203030ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import atexit from collections import Iterator, Mapping, defaultdict from concurrent.futures import ThreadPoolExecutor from concurrent.futures._base import DoneAndNotDoneFutures, CancelledError from contextlib import contextmanager import copy from datetime import timedelta import errno from functools import partial from glob import glob import itertools import json import logging from numbers import Number, Integral import os import sys from time import sleep import uuid import threading import six import socket import warnings import weakref import dask from dask.base import tokenize, normalize_token, collections_to_dsk from dask.core import flatten, get_dependencies from dask.compatibility import apply, unicode from dask.context import _globals try: from cytoolz import first, groupby, merge, valmap, keymap except ImportError: from toolz import first, groupby, merge, valmap, keymap try: from dask.delayed import single_key except ImportError: single_key = first from tornado import gen from tornado.gen import TimeoutError from tornado.locks import Event, Condition from tornado.ioloop import IOLoop from tornado.queues import Queue from .batched import BatchedSend from .utils_comm import (WrappedKey, unpack_remotedata, pack_data, scatter_to_workers, gather_from_workers) from .cfexecutor import ClientExecutor from .compatibility import (Queue as pyQueue, Empty, isqueue, get_thread_identity, html_escape) from .config import config from .core import connect, rpc, clean_exception, CommClosedError from .metrics import time from .node import Node from .protocol import to_serialize from .protocol.pickle import dumps, loads from .publish import Datasets from .security import Security from .sizeof import sizeof from .threadpoolexecutor import rejoin from .worker import dumps_task, get_client, get_worker, secede from .utils import (All, sync, funcname, ignoring, queue_to_iterator, tokey, log_errors, str_graph, key_split, format_bytes, asciitable, thread_state, no_default, PeriodicCallback, LoopRunner) from .versions import get_versions logger = logging.getLogger(__name__) _global_client = [None] def _get_global_client(): wr = _global_client[0] return wr and wr() def _set_global_client(c): _global_client[0] = weakref.ref(c) if c is not None else None class Future(WrappedKey): """ A remotely running computation A Future is a local proxy to a result running on a remote worker. A user manages future objects in the local Python process to determine what happens in the larger cluster. Parameters ---------- key: str, or tuple Key of remote data to which this future refers client: Client Client that should own this future. Defaults to _get_global_client() inform: bool Do we inform the scheduler that we need an update on this future Examples -------- Futures typically emerge from Client computations >>> my_future = client.submit(add, 1, 2) # doctest: +SKIP We can track the progress and results of a future >>> my_future # doctest: +SKIP We can get the result or the exception and traceback from the future >>> my_future.result() # doctest: +SKIP See Also -------- Client: Creates futures """ _cb_executor = None _cb_executor_pid = None def __init__(self, key, client=None, inform=True, state=None): self.key = key self._cleared = False tkey = tokey(key) self.client = client or _get_global_client() self.client._inc_ref(tkey) self._generation = self.client.generation if tkey in self.client.futures: self._state = self.client.futures[tkey] else: self._state = self.client.futures[tkey] = FutureState() if inform: self.client._send_to_scheduler({'op': 'client-desires-keys', 'keys': [tokey(key)], 'client': self.client.id}) if state is not None: try: handler = self.client._state_handlers[state] except KeyError: pass else: handler(key=key) @property def executor(self): return self.client @property def status(self): return self._state.status def done(self): """ Is the computation complete? """ return self._state.done() def result(self, timeout=None): """ Wait until computation completes, gather result to local process. If *timeout* seconds are elapsed before returning, a TimeoutError is raised. """ if self.client.asynchronous: return self.client.sync(self._result, callback_timeout=timeout) # shorten error traceback result = self.client.sync(self._result, callback_timeout=timeout, raiseit=False) if self.status == 'error': six.reraise(*result) elif self.status == 'cancelled': raise result else: return result @gen.coroutine def _result(self, raiseit=True): yield self._state.wait() if self.status == 'error': exc = clean_exception(self._state.exception, self._state.traceback) if raiseit: six.reraise(*exc) else: raise gen.Return(exc) elif self.status == 'cancelled': exception = CancelledError(self.key) if raiseit: raise exception else: raise gen.Return(exception) else: result = yield self.client._gather([self]) raise gen.Return(result[0]) @gen.coroutine def _exception(self): yield self._state.wait() if self.status == 'error': raise gen.Return(self._state.exception) else: raise gen.Return(None) def exception(self, timeout=None, **kwargs): """ Return the exception of a failed task If *timeout* seconds are elapsed before returning, a TimeoutError is raised. See Also -------- Future.traceback """ return self.client.sync(self._exception, callback_timeout=timeout, **kwargs) def add_done_callback(self, fn): """ Call callback on future when callback has finished The callback ``fn`` should take the future as its only argument. This will be called regardless of if the future completes successfully, errs, or is cancelled The callback is executed in a separate thread. """ cls = Future if cls._cb_executor is None or cls._cb_executor_pid != os.getpid(): cls._cb_executor = ThreadPoolExecutor(1) cls._cb_executor_pid = os.getpid() def execute_callback(fut): try: fn(fut) except BaseException: logger.exception("Error in callback %s of %s:", fn, fut) self.client.loop.add_callback(done_callback, self, partial(cls._cb_executor.submit, execute_callback)) def cancel(self, **kwargs): """ Cancel request to run this future See Also -------- Client.cancel """ return self.client.cancel([self], **kwargs) def cancelled(self): """ Returns True if the future has been cancelled """ return self._state.status == 'cancelled' @gen.coroutine def _traceback(self): yield self._state.wait() if self.status == 'error': raise gen.Return(self._state.traceback) else: raise gen.Return(None) def traceback(self, timeout=None, **kwargs): """ Return the traceback of a failed task This returns a traceback object. You can inspect this object using the ``traceback`` module. Alternatively if you call ``future.result()`` this traceback will accompany the raised exception. If *timeout* seconds are elapsed before returning, a TimeoutError is raised. Examples -------- >>> import traceback # doctest: +SKIP >>> tb = future.traceback() # doctest: +SKIP >>> traceback.export_tb(tb) # doctest: +SKIP [...] See Also -------- Future.exception """ return self.client.sync(self._traceback, callback_timeout=timeout, **kwargs) @property def type(self): return self._state.type def release(self, _in_destructor=False): # NOTE: this method can be called from different threads # (see e.g. Client.get() or Future.__del__()) if not self._cleared and self.client.generation == self._generation: self._cleared = True self.client.loop.add_callback(self.client._dec_ref, tokey(self.key)) def __getstate__(self): return (self.key, self.client.scheduler.address) def __setstate__(self, state): key, address = state c = get_client(address) Future.__init__(self, key, c) c._send_to_scheduler({'op': 'update-graph', 'tasks': {}, 'keys': [tokey(self.key)], 'client': c.id}) def __del__(self): try: self.release() except RuntimeError: # closed event loop pass def __repr__(self): if self.type: try: typ = self.type.__name__ except AttributeError: typ = str(self.type) return '' % (self.status, typ, self.key) else: return '' % (self.status, self.key) def _repr_html_(self): text = 'Future: %s ' % html_escape(key_split(self.key)) text += ('status: ' '%(status)s, ') % { 'status': self.status, 'color': 'red' if self.status == 'error' else 'black'} if self.type: try: typ = self.type.__name__ except AttributeError: typ = str(self.type) text += 'type: %s, ' % typ text += 'key: %s' % html_escape(self.key) return text def __await__(self): return self.result().__await__() class FutureState(object): """A Future's internal state. This is shared between all Futures with the same key and client. """ __slots__ = ('_event', 'status', 'type', 'exception', 'traceback') def __init__(self): self._event = None self.status = 'pending' self.type = None def _get_event(self): # Can't create Event eagerly in constructor as it can fetch # its IOLoop from the wrong thread # (https://github.com/tornadoweb/tornado/issues/2189) event = self._event if event is None: event = self._event = Event() return event def cancel(self): self.status = 'cancelled' self._get_event().set() def finish(self, type=None): self.status = 'finished' self._get_event().set() if type is not None: self.type = type def lose(self): self.status = 'lost' self._get_event().clear() def set_error(self, exception, traceback): if isinstance(exception, bytes): try: exception = loads(exception) except TypeError: exception = Exception("Undeserializable exception", exception) if traceback: if isinstance(traceback, bytes): try: traceback = loads(traceback) except (TypeError, AttributeError): traceback = None else: traceback = None self.status = 'error' self.exception = exception self.traceback = traceback self._get_event().set() def done(self): return self._event is not None and self._event.is_set() def reset(self): self.status = 'pending' if self._event is not None: self._event.clear() @gen.coroutine def wait(self, timeout=None): yield self._get_event().wait(timeout) def __repr__(self): return '<%s: %s>' % (self.__class__.__name__, self.status) @gen.coroutine def done_callback(future, callback): """ Coroutine that waits on future, then calls callback """ while future.status == 'pending': yield future._state.wait() callback(future) @partial(normalize_token.register, Future) def normalize_future(f): return [f.key, type(f)] class AllExit(Exception): """Custom exception class to exit All(...) early. """ 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. Parameters ---------- address: string, or Cluster This can be the address of a ``Scheduler`` server like a string ``'127.0.0.1:8786'`` or a cluster object like ``LocalCluster()`` timeout: int Timeout duration for initial connection to the scheduler set_as_default: bool (True) Claim this scheduler as the global dask scheduler scheduler_file: string (optional) Path to a file with scheduler information if available security: (optional) Optional security information asynchronous: bool (False by default) Set to True if this client will be used within a Tornado event loop name: string (optional) Gives the client a name that will be included in logs generated on the scheduler for matters relating to this client Examples -------- Provide cluster's scheduler node address on initialization: >>> client = Client('127.0.0.1:8786') # doctest: +SKIP Use ``submit`` method to send individual computations to the cluster >>> a = client.submit(add, 1, 2) # doctest: +SKIP >>> b = client.submit(add, 10, 20) # doctest: +SKIP Continue using submit or map on results to build up larger computations >>> c = client.submit(add, a, b) # doctest: +SKIP Gather results with the ``gather`` method. >>> client.gather(c) # doctest: +SKIP 33 See Also -------- distributed.scheduler.Scheduler: Internal scheduler """ def __init__(self, address=None, loop=None, timeout=5, set_as_default=True, scheduler_file=None, security=None, asynchronous=False, name=None, **kwargs): self.futures = dict() self.refcount = defaultdict(lambda: 0) self.coroutines = [] self.id = type(self).__name__ + ('-' + name + '-' if name else '-') + str(uuid.uuid1(clock_seq=os.getpid())) self.generation = 0 self.status = 'newly-created' self._pending_msg_buffer = [] self.extensions = {} self.scheduler_file = scheduler_file self._startup_kwargs = kwargs self.scheduler = None self._scheduler_identity = {} self._lock = threading.Lock() self._refcount_lock = threading.Lock() self.datasets = Datasets(self) # Communication self.security = security or Security() self.scheduler_comm = None assert isinstance(self.security, Security) self.connection_args = self.security.get_connection_args('client') self._connecting_to_scheduler = False self._asynchronous = asynchronous self._loop_runner = LoopRunner(loop=loop, asynchronous=asynchronous) self.loop = self._loop_runner.loop self._periodic_callbacks = [] pc = PeriodicCallback(self._update_scheduler_info, 2000, io_loop=self.loop) self._periodic_callbacks.append(pc) if address is None and 'scheduler-address' in config: address = config['scheduler-address'] if address: logger.info("Config value `scheduler-address` found: %s", address) if hasattr(address, "scheduler_address"): # It's a LocalCluster or LocalCluster-compatible object self.cluster = address else: self.cluster = None self._start_arg = address if set_as_default: self._previous_get = _globals.get('get') dask.set_options(get=self.get) self._previous_shuffle = _globals.get('shuffle') dask.set_options(shuffle='tasks') self._handlers = { 'key-in-memory': self._handle_key_in_memory, 'lost-data': self._handle_lost_data, 'cancelled-key': self._handle_cancelled_key, 'task-erred': self._handle_task_erred, 'restart': self._handle_restart, 'error': self._handle_error } self._state_handlers = { 'memory': self._handle_key_in_memory, 'lost': self._handle_lost_data, 'erred': self._handle_task_erred } super(Client, self).__init__(connection_args=self.connection_args, io_loop=self.loop) self.start(timeout=timeout) from distributed.recreate_exceptions import ReplayExceptionClient ReplayExceptionClient(self) @classmethod def current(cls): """ Return global client if one exists, otherwise raise ValueError """ return default_client() @property def asynchronous(self): """ Are we running in the event loop? This is true if the user signaled that we might be when creating the client as in the following:: client = Client(asynchronous=True) However, we override this expectation if we can definitively tell that we are running from a thread that is not the event loop. This is common when calling get_client() from within a worker task. Even though the client was originally created in asynchronous mode we may find ourselves in contexts when it is better to operate synchronously. """ return self._asynchronous and self.loop is IOLoop.current() def sync(self, func, *args, **kwargs): asynchronous = kwargs.pop('asynchronous', None) if asynchronous or self.asynchronous or getattr(thread_state, 'asynchronous', False): 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 __repr__(self): # Note: avoid doing I/O here... info = self._scheduler_identity addr = info.get('address') if addr: workers = info.get('workers', {}) nworkers = len(workers) ncores = sum(w['ncores'] for w in workers.values()) return '<%s: scheduler=%r processes=%d cores=%d>' % ( self.__class__.__name__, addr, nworkers, ncores) elif self.scheduler is not None: return '<%s: scheduler=%r>' % ( self.__class__.__name__, self.scheduler.address) else: return '<%s: not connected>' % (self.__class__.__name__,) def _repr_html_(self): if self._loop_runner.is_started(): info = sync(self.loop, self.scheduler.identity) else: info = False if self.scheduler is not None: text = ("

Client

\n" "
    \n" "
  • Scheduler: %s\n") % self.scheduler.address else: text = ("

    Client

    \n" "
      \n" "
    • Scheduler: not connected\n") if info and 'bokeh' in info['services']: protocol, rest = self.scheduler.address.split('://') port = info['services']['bokeh'] if protocol == 'inproc': address = 'http://localhost:%d' % port else: host = rest.split(':')[0] address = 'http://%s:%d' % (host, port) text += "
    • Dashboard: %(web)s\n" % {'web': address} text += "
    \n" if info: workers = len(info['workers']) cores = sum(w['ncores'] for w in info['workers'].values()) memory = sum(w['memory_limit'] for w in info['workers'].values()) memory = format_bytes(memory) text2 = ("

    Cluster

    \n" "
      \n" "
    • Workers: %d
    • \n" "
    • Cores: %d
    • \n" "
    • Memory: %s
    • \n" "
    \n") % (workers, cores, memory) return ('\n' '\n' '\n' '\n' '\n
    \n%s\n%s
    ') % (text, text2) else: return text def start(self, **kwargs): """ Start scheduler running in separate thread """ if self.status != 'newly-created': return self._loop_runner.start() _set_global_client(self) self.status = 'connecting' if self.asynchronous: self._started = self._start(**kwargs) else: sync(self.loop, self._start, **kwargs) def __await__(self): return self._started.__await__() def _send_to_scheduler_safe(self, msg): if self.status in ('running', 'closing'): self.scheduler_comm.send(msg) elif self.status in ('connecting', 'newly-created'): self._pending_msg_buffer.append(msg) else: raise Exception("Tried sending message after closing. Status: %s\n" "Message: %s" % (self.status, msg)) def _send_to_scheduler(self, msg): if self.status in ('running', 'closing', 'connecting', 'newly-created'): self.loop.add_callback(self._send_to_scheduler_safe, msg) else: raise Exception("Tried sending message after closing. Status: %s\n" "Message: %s" % (self.status, msg)) @gen.coroutine def _start(self, timeout=5, **kwargs): address = self._start_arg if self.cluster is not None: # Ensure the cluster is started (no-op if already running) try: yield self.cluster._start() except AttributeError: # Some clusters don't have this method pass except Exception: logger.info("Tried to start cluster and received an error. " "Proceeding.", exc_info=True) address = self.cluster.scheduler_address elif self.scheduler_file is not None: while not os.path.exists(self.scheduler_file): yield gen.sleep(0.01) for i in range(10): try: with open(self.scheduler_file) as f: cfg = json.load(f) address = cfg['address'] break except (ValueError, KeyError): # JSON file not yet flushed yield gen.sleep(0.01) elif self._start_arg is None: from .deploy import LocalCluster try: self.cluster = LocalCluster(loop=self.loop, start=False, **self._startup_kwargs) yield self.cluster._start() except (OSError, socket.error) as e: if e.errno != errno.EADDRINUSE: raise # The default port was taken, use a random one self.cluster = LocalCluster(scheduler_port=0, loop=self.loop, start=False, **self._startup_kwargs) yield self.cluster._start() # Wait for all workers to be ready while (not self.cluster.workers or len(self.cluster.scheduler.ncores) < len(self.cluster.workers)): yield gen.sleep(0.01) address = self.cluster.scheduler_address self.scheduler = rpc(address, timeout=timeout, connection_args=self.connection_args) self.scheduler_comm = None yield self._ensure_connected(timeout=timeout) for pc in self._periodic_callbacks: pc.start() self.coroutines.append(self._handle_report()) raise gen.Return(self) @gen.coroutine def _reconnect(self, timeout=0.1): with log_errors(): assert self.scheduler_comm.comm.closed() self.status = 'connecting' self.scheduler_comm = None for st in self.futures.values(): st.cancel() self.futures.clear() while self.status == 'connecting': try: yield self._ensure_connected() break except EnvironmentError: yield gen.sleep(timeout) @gen.coroutine def _ensure_connected(self, timeout=None): if (self.scheduler_comm and not self.scheduler_comm.closed() or self._connecting_to_scheduler or self.scheduler is None): return self._connecting_to_scheduler = True try: comm = yield connect(self.scheduler.address, timeout=timeout, connection_args=self.connection_args) yield self._update_scheduler_info() yield comm.write({'op': 'register-client', 'client': self.id, 'reply': False}) finally: self._connecting_to_scheduler = False msg = yield comm.read() assert len(msg) == 1 assert msg[0]['op'] == 'stream-start' bcomm = BatchedSend(interval=10, loop=self.loop) bcomm.start(comm) self.scheduler_comm = bcomm _set_global_client(self) self.status = 'running' for msg in self._pending_msg_buffer: self._send_to_scheduler(msg) del self._pending_msg_buffer[:] logger.debug("Started scheduling coroutines. Synchronized") @gen.coroutine def _update_scheduler_info(self): self._scheduler_identity = yield self.scheduler.identity() def __enter__(self): if not self._loop_runner.is_started(): self.start() return self @gen.coroutine def __aenter__(self): yield self._started raise gen.Return(self) @gen.coroutine def __aexit__(self, typ, value, traceback): yield self._close() def __exit__(self, type, value, traceback): self.close() def __del__(self): self.close() def _inc_ref(self, key): with self._refcount_lock: self.refcount[key] += 1 def _dec_ref(self, key): with self._refcount_lock: self.refcount[key] -= 1 if self.refcount[key] == 0: del self.refcount[key] self._release_key(key) def _release_key(self, key): """ Release key from distributed memory """ logger.debug("Release key %s", key) st = self.futures.pop(key, None) if st is not None: st.cancel() if self.status != 'closed': self._send_to_scheduler({'op': 'client-releases-keys', 'keys': [key], 'client': self.id}) @gen.coroutine def _handle_report(self): """ Listen to scheduler """ with log_errors(): try: while True: if self.scheduler_comm is None: break try: msgs = yield self.scheduler_comm.comm.read() except CommClosedError: if self.status == 'running': logger.warning("Client report stream closed to scheduler") logger.info("Reconnecting...") self.status = 'connecting' yield self._reconnect() continue else: break if not isinstance(msgs, list): msgs = [msgs] breakout = False for msg in msgs: logger.debug("Client receives message %s", msg) if 'status' in msg and 'error' in msg['status']: six.reraise(*clean_exception(**msg)) op = msg.pop('op') if op == 'close' or op == 'stream-closed': breakout = True break try: handler = self._handlers[op] handler(**msg) except Exception as e: logger.exception(e) if breakout: break except CancelledError: pass def _handle_key_in_memory(self, key=None, type=None, workers=None): state = self.futures.get(key) if state is not None: if type and not state.type: # Type exists and not yet set try: type = loads(type) except Exception: type = None # Here, `type` may be a str if actual type failed # serializing in Worker else: type = None state.finish(type) def _handle_lost_data(self, key=None): state = self.futures.get(key) if state is not None: state.lose() def _handle_cancelled_key(self, key=None): state = self.futures.get(key) if state is not None: state.cancel() def _handle_task_erred(self, key=None, exception=None, traceback=None): state = self.futures.get(key) if state is not None: state.set_error(exception, traceback) def _handle_restart(self): logger.info("Receive restart signal from scheduler") for state in self.futures.values(): state.cancel() self.futures.clear() with ignoring(AttributeError): self._restart_event.set() def _handle_error(self, exception=None): logger.warning("Scheduler exception:") logger.exception(exception) @gen.coroutine def _close(self, fast=False): """ Send close signal and wait until scheduler completes """ self.status = 'closing' with log_errors(): for pc in self._periodic_callbacks: pc.stop() self._scheduler_identity = {} with ignoring(AttributeError): dask.set_options(get=self._previous_get) with ignoring(AttributeError): dask.set_options(shuffle=self._previous_shuffle) if self.get == _globals.get('get'): del _globals['get'] if self.status == 'closed': raise gen.Return() if self.scheduler_comm and self.scheduler_comm.comm and not self.scheduler_comm.comm.closed(): for key in list(self.futures): self._release_key(key=key) if self.status == 'running': self._send_to_scheduler({'op': 'close-stream'}) if self.scheduler_comm: yield self.scheduler_comm.close() if self._start_arg is None: with ignoring(AttributeError): yield self.cluster._close() self.status = 'closed' if _get_global_client() is self: _set_global_client(None) coroutines = set(self.coroutines) for f in self.coroutines: # cancel() works on asyncio futures (Tornado 5) # but is a no-op on Tornado futures f.cancel() if f.cancelled(): coroutines.remove(f) del self.coroutines[:] if not fast: with ignoring(TimeoutError): yield gen.with_timeout(timedelta(seconds=2), list(coroutines)) with ignoring(AttributeError): self.scheduler.close_rpc() self.scheduler = None self.status = 'closed' _shutdown = _close def close(self, timeout=10): """ Close this client Clients will also close automatically when your Python session ends If you started a client without arguments like ``Client()`` then this will also close the local cluster that was started at the same time. See Also -------- Client.restart """ # XXX handling of self.status here is not thread-safe if self.status == 'closed': return self.status = 'closing' if self.asynchronous: future = self._close() if timeout: future = gen.with_timeout(timedelta(seconds=timeout), future) return future if self._start_arg is None: with ignoring(AttributeError): self.cluster.close() sync(self.loop, self._close, fast=True) assert self.status == 'closed' self._loop_runner.stop() with ignoring(AttributeError): dask.set_options(get=self._previous_get) with ignoring(AttributeError): dask.set_options(shuffle=self._previous_shuffle) if self.get == _globals.get('get'): del _globals['get'] shutdown = close def get_executor(self, **kwargs): """ Return a concurrent.futures Executor for submitting tasks on this Client. Parameters ---------- **kwargs: Any submit()- or map()- compatible arguments, such as `workers` or `resources`. Returns ------- An Executor object that's fully compatible with the concurrent.futures API. """ return ClientExecutor(self, **kwargs) def submit(self, func, *args, **kwargs): """ Submit a function application to the scheduler Parameters ---------- func: callable *args: **kwargs: pure: bool (defaults to True) Whether or not the function is pure. Set ``pure=False`` for impure functions like ``np.random.random``. workers: set, iterable of sets A set of worker hostnames on which computations may be performed. Leave empty to default to all workers (common case) key: str Unique identifier for the task. Defaults to function-name and hash allow_other_workers: bool (defaults to False) Used with `workers`. Inidicates whether or not the computations may be performed on workers that are not in the `workers` set(s). retries: int (default to 0) Number of allowed automatic retries if the task fails Examples -------- >>> c = client.submit(add, a, b) # doctest: +SKIP Returns ------- Future See Also -------- Client.map: Submit on many arguments at once """ if not callable(func): raise TypeError("First input to submit must be a callable function") key = kwargs.pop('key', None) pure = kwargs.pop('pure', True) workers = kwargs.pop('workers', None) resources = kwargs.pop('resources', None) retries = kwargs.pop('retries', None) allow_other_workers = kwargs.pop('allow_other_workers', False) if allow_other_workers not in (True, False, None): raise TypeError("allow_other_workers= must be True or False") if key is None: if pure: key = funcname(func) + '-' + tokenize(func, kwargs, *args) else: key = funcname(func) + '-' + str(uuid.uuid4()) skey = tokey(key) with self._lock: if skey in self.futures: return Future(key, self, inform=False) if allow_other_workers and workers is None: raise ValueError("Only use allow_other_workers= if using workers=") if isinstance(workers, six.string_types + (Number,)): workers = [workers] if workers is not None: restrictions = {skey: workers} loose_restrictions = [skey] if allow_other_workers else [] else: restrictions = {} loose_restrictions = [] if kwargs: dsk = {skey: (apply, func, list(args), kwargs)} else: dsk = {skey: (func,) + tuple(args)} futures = self._graph_to_futures(dsk, [skey], restrictions, loose_restrictions, priority={skey: 0}, resources={skey: resources} if resources else None, retries={skey: retries} if retries else None) logger.debug("Submit %s(...), %s", funcname(func), key) 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 Arguments can be normal objects or Futures Parameters ---------- func: callable iterables: Iterables, Iterators, or Queues key: str, list Prefix for task names if string. Explicit names if list. pure: bool (defaults to True) Whether or not the function is pure. Set ``pure=False`` for impure functions like ``np.random.random``. workers: set, iterable of sets A set of worker hostnames on which computations may be performed. Leave empty to default to all workers (common case) retries: int (default to 0) Number of allowed automatic retries if a task fails Examples -------- >>> L = client.map(func, sequence) # doctest: +SKIP Returns ------- List, iterator, or Queue of futures, depending on the type of the inputs. See also -------- Client.submit: Submit a single function """ if not callable(func): raise TypeError("First input to map must be a callable function") 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) 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) pure = kwargs.pop('pure', True) workers = kwargs.pop('workers', None) retries = kwargs.pop('retries', None) resources = kwargs.pop('resources', None) allow_other_workers = kwargs.pop('allow_other_workers', False) if allow_other_workers and workers is None: raise ValueError("Only use allow_other_workers= if using workers=") iterables = list(zip(*zip(*iterables))) if isinstance(key, list): keys = key else: if pure: keys = [key + '-' + tokenize(func, kwargs, *args) for args in zip(*iterables)] else: uid = str(uuid.uuid4()) keys = [key + '-' + uid + '-' + str(i) for i in range(min(map(len, iterables)))] if iterables else [] if not kwargs: dsk = {key: (func,) + args for key, args in zip(keys, zip(*iterables))} else: dsk = {key: (apply, func, (tuple, list(args)), kwargs) for key, args in zip(keys, zip(*iterables))} if isinstance(workers, six.string_types + (Number,)): workers = [workers] if isinstance(workers, (list, set)): if workers and isinstance(first(workers), (list, set)): if len(workers) != len(keys): raise ValueError("You only provided %d worker restrictions" " for a sequence of length %d" % (len(workers), len(keys))) restrictions = dict(zip(keys, workers)) else: restrictions = {k: workers for k in keys} elif workers is None: restrictions = {} else: raise TypeError("Workers must be a list or set of workers or None") if allow_other_workers not in (True, False, None): raise TypeError("allow_other_workers= must be True or False") if allow_other_workers is True: loose_restrictions = set(keys) else: loose_restrictions = set() if retries: retries = {k: retries for k in keys} else: retries = None priority = dict(zip(keys, range(len(keys)))) if resources: resources = {k: resources for k in keys} else: resources = None futures = self._graph_to_futures(dsk, keys, restrictions, loose_restrictions, priority=priority, resources=resources, retries=retries) logger.debug("map(%s, ...)", funcname(func)) return [futures[tokey(k)] for k in keys] @gen.coroutine def _gather(self, futures, errors='raise', direct=None, local_worker=None): futures2, keys = unpack_remotedata(futures, byte_keys=True) keys = [tokey(key) for key in keys] bad_data = dict() if direct is None: try: w = get_worker() except Exception: direct = False else: if w.scheduler.address == self.scheduler.address: direct = True @gen.coroutine def wait(k): """ Want to stop the All(...) early if we find an error """ st = self.futures[k] yield st.wait() if st.status != 'finished': raise AllExit() while True: logger.debug("Waiting on futures to clear before gather") with ignoring(AllExit): yield All([wait(key) for key in keys if key in self.futures]) failed = ('error', 'cancelled') exceptions = set() bad_keys = set() for key in keys: if (key not in self.futures or self.futures[key].status in failed): exceptions.add(key) if errors == 'raise': try: st = self.futures[key] exception = st.exception traceback = st.traceback except (AttributeError, KeyError): six.reraise(CancelledError, CancelledError(key), None) else: six.reraise(type(exception), exception, traceback) if errors == 'skip': bad_keys.add(key) bad_data[key] = None else: raise ValueError("Bad value, `errors=%s`" % errors) keys = [k for k in keys if k not in bad_keys] data = {} if local_worker: # look inside local worker data.update({k: local_worker.data[k] for k in keys if k in local_worker.data}) keys = [k for k in keys if k not in data] if direct or local_worker: # gather directly from workers who_has = yield self.scheduler.who_has(keys=keys) data2, missing_keys, missing_workers = yield gather_from_workers( who_has, rpc=self.rpc, close=False) response = {'status': 'OK', 'data': data2} if missing_keys: keys2 = [key for key in keys if key not in data2] response = yield self.scheduler.gather(keys=keys2) if response['status'] == 'OK': response['data'].update(data2) else: # ask scheduler to gather data for us response = yield self.scheduler.gather(keys=keys) if response['status'] == 'error': logger.warning("Couldn't gather keys %s", response['keys']) for key in response['keys']: self._send_to_scheduler({'op': 'report-key', 'key': key}) for key in response['keys']: self.futures[key].reset() else: break if bad_data and errors == 'skip' and isinstance(futures2, list): futures2 = [f for f in futures2 if f not in bad_data] data.update(response['data']) result = pack_data(futures2, merge(data, bad_data)) raise gen.Return(result) 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): """ Gather futures from distributed memory Accepts a future, nested container of futures, iterator, or queue. The return type will match the input type. Parameters ---------- futures: Collection of futures This can be a possibly nested collection of Future objects. Collections can be lists, sets, iterators, queues 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 maxsize: int If the input is a queue then this produces an output queue with a maximum size. Returns ------- results: a collection of the same type as the input, but now with gathered results rather than futures Examples -------- >>> from operator import add # doctest: +SKIP >>> c = Client('127.0.0.1:8787') # doctest: +SKIP >>> x = c.submit(add, 1, 2) # doctest: +SKIP >>> c.gather(x) # doctest: +SKIP 3 >>> 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}) t.daemon = True t.start() return qout elif isinstance(futures, Iterator): return (self.gather(f, errors=errors, direct=direct) for f in futures) 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._gather, futures, errors=errors, direct=direct, local_worker=local_worker, asynchronous=asynchronous) @gen.coroutine def _scatter(self, data, workers=None, broadcast=False, direct=None, local_worker=None, timeout=3, hash=True): if isinstance(workers, six.string_types + (Number,)): workers = [workers] if isinstance(data, dict) and not all(isinstance(k, (bytes, unicode)) for k in data): d = yield self._scatter(keymap(tokey, data), workers, broadcast) raise gen.Return({k: d[tokey(k)] for k in data}) if isinstance(data, type(range(0))): data = list(data) input_type = type(data) names = False unpack = False if isinstance(data, Iterator): data = list(data) if isinstance(data, (set, frozenset)): data = list(data) if not isinstance(data, (dict, list, tuple, set, frozenset)): unpack = True data = [data] if isinstance(data, (list, tuple)): if hash: names = [type(x).__name__ + '-' + tokenize(x) for x in data] else: names = [type(x).__name__ + '-' + uuid.uuid4().hex for x in data] data = dict(zip(names, data)) assert isinstance(data, dict) types = valmap(type, data) if direct is None: try: w = get_worker() except Exception: direct = False else: if w.scheduler.address == self.scheduler.address: direct = True if local_worker: # running within task local_worker.update_data(data=data, report=False) yield self.scheduler.update_data( who_has={key: [local_worker.address] for key in data}, nbytes=valmap(sizeof, data), client=self.id) else: data2 = valmap(to_serialize, data) if direct: ncores = None start = time() while not ncores: if ncores is not None: yield gen.sleep(0.1) if time() > start + timeout: raise gen.TimeoutError("No valid workers found") ncores = yield self.scheduler.ncores(workers=workers) if not ncores: raise ValueError("No valid workers") _, who_has, nbytes = yield scatter_to_workers(ncores, data2, report=False, rpc=self.rpc) yield self.scheduler.update_data(who_has=who_has, nbytes=nbytes, client=self.id) else: yield self.scheduler.scatter(data=data2, workers=workers, client=self.id, broadcast=broadcast) out = {k: Future(k, self, inform=False) for k in data} for key, typ in types.items(): self.futures[key].finish(type=typ) if direct and broadcast: n = None if broadcast is True else broadcast yield self._replicate(list(out.values()), workers=workers, n=n) if issubclass(input_type, (list, tuple, set, frozenset)): out = input_type(out[k] for k in names) if unpack: assert len(out) == 1 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, workers=None, broadcast=False, direct=None, hash=True, maxsize=0, timeout=3, asynchronous=None): """ Scatter data into distributed memory This moves data from the local client process into the workers of the distributed scheduler. Note that it is often better to submit jobs to your workers to have them load the data rather than loading data locally and then scattering it out to them. Parameters ---------- data: list, iterator, dict, Queue, or object Data to scatter out to workers. Output type matches input type. workers: list of tuples (optional) Optionally constrain locations of data. Specify workers as hostname/port pairs, e.g. ``('127.0.0.1', 8787)``. broadcast: bool (defaults to False) Whether to send each data element to all workers. By default we round-robin based on number of cores. direct: bool (defaults to automatically check) Send data directly to workers, bypassing the central scheduler This avoids burdening the scheduler but assumes that the client is able to talk directly with the workers. 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 Returns ------- List, dict, iterator, or queue of futures matching the type of input. Examples -------- >>> c = Client('127.0.0.1:8787') # doctest: +SKIP >>> c.scatter(1) # doctest: +SKIP >>> c.scatter([1, 2, 3]) # doctest: +SKIP [, , ] >>> c.scatter({'x': 1, 'y': 2, 'z': 3}) # doctest: +SKIP {'x': , 'y': , 'z': } Constrain location of data to subset of workers >>> 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 See Also -------- Client.gather: Gather data back to local process """ 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}) t.daemon = True t.start() if isqueue(data): return qout else: return queue_to_iterator(qout) 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) @gen.coroutine def _cancel(self, futures, force=False): keys = list({tokey(f.key) for f in futures_of(futures)}) yield self.scheduler.cancel(keys=keys, client=self.id, force=force) for k in keys: st = self.futures.pop(k, None) if st is not None: st.cancel() def cancel(self, futures, asynchronous=None, force=False): """ Cancel running futures This stops future tasks from being scheduled if they have not yet run and deletes them if they have already run. After calling, this result and all dependent results will no longer be accessible Parameters ---------- futures: list of Futures force: boolean (False) Cancel this future even if other clients desire it """ return self.sync(self._cancel, futures, asynchronous=asynchronous, force=force) @gen.coroutine def _publish_dataset(self, **kwargs): with log_errors(): coroutines = [] for name, data in kwargs.items(): keys = [tokey(f.key) for f in futures_of(data)] coroutines.append(self.scheduler.publish_put(keys=keys, name=tokey(name), data=dumps(data), client=self.id)) yield coroutines def publish_dataset(self, **kwargs): """ Publish named datasets to scheduler This stores a named reference to a dask collection or list of futures on the scheduler. These references are available to other Clients which can download the collection or futures with ``get_dataset``. Datasets are not immediately computed. You may wish to call ``Client.persist`` prior to publishing a dataset. Parameters ---------- kwargs: dict named collections to publish on the scheduler Examples -------- Publishing client: >>> df = dd.read_csv('s3://...') # doctest: +SKIP >>> df = c.persist(df) # doctest: +SKIP >>> c.publish_dataset(my_dataset=df) # doctest: +SKIP Receiving client: >>> c.list_datasets() # doctest: +SKIP ['my_dataset'] >>> df2 = c.get_dataset('my_dataset') # doctest: +SKIP Returns ------- None See Also -------- Client.list_datasets Client.get_dataset Client.unpublish_dataset Client.persist """ return self.sync(self._publish_dataset, **kwargs) def unpublish_dataset(self, name, **kwargs): """ Remove named datasets from scheduler Examples -------- >>> c.list_datasets() # doctest: +SKIP ['my_dataset'] >>> c.unpublish_datasets('my_dataset') # doctest: +SKIP >>> c.list_datasets() # doctest: +SKIP [] See Also -------- Client.publish_dataset """ return self.sync(self.scheduler.publish_delete, name=name, **kwargs) def list_datasets(self, **kwargs): """ List named datasets available on the scheduler See Also -------- Client.publish_dataset Client.get_dataset """ return self.sync(self.scheduler.publish_list, **kwargs) @gen.coroutine def _get_dataset(self, name): out = yield self.scheduler.publish_get(name=name, client=self.id) with temp_default_client(self): data = loads(out['data']) raise gen.Return(data) def get_dataset(self, name, **kwargs): """ Get named dataset from the scheduler See Also -------- Client.publish_dataset Client.list_datasets """ return self.sync(self._get_dataset, tokey(name), **kwargs) @gen.coroutine def _run_on_scheduler(self, function, *args, **kwargs): response = yield self.scheduler.run_function(function=dumps(function), args=dumps(args), kwargs=dumps(kwargs)) if response['status'] == 'error': six.reraise(*clean_exception(**response)) else: raise gen.Return(response['result']) def run_on_scheduler(self, function, *args, **kwargs): """ Run a function on the scheduler process This is typically used for live debugging. The function should take a keyword argument ``dask_scheduler=``, which will be given the scheduler object itself. Examples -------- >>> def get_number_of_tasks(dask_scheduler=None): ... return len(dask_scheduler.task_state) >>> client.run_on_scheduler(get_number_of_tasks) # doctest: +SKIP 100 See Also -------- Client.run: Run a function on all workers Client.start_ipython_scheduler: Start an IPython session on scheduler """ return self.sync(self._run_on_scheduler, function, *args, **kwargs) @gen.coroutine def _run(self, function, *args, **kwargs): nanny = kwargs.pop('nanny', False) workers = kwargs.pop('workers', None) responses = yield self.scheduler.broadcast(msg=dict(op='run', function=dumps(function), args=dumps(args), kwargs=dumps(kwargs)), workers=workers, nanny=nanny) 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 run(self, function, *args, **kwargs): """ Run a function on all workers outside of task scheduling system This calls a function on all currently known workers immediately, blocks until those results come back, and returns the results asynchronously as a dictionary keyed by worker address. This method if generally used for side effects, such and collecting diagnostic information or installing libraries. If your function takes an input argument named ``dask_worker`` then that variable will be populated with the worker itself. Parameters ---------- function: callable *args: arguments for remote function **kwargs: keyword arguments for remote function workers: list Workers on which to run the function. Defaults to all known workers. Examples -------- >>> c.run(os.getpid) # doctest: +SKIP {'192.168.0.100:9000': 1234, '192.168.0.101:9000': 4321, '192.168.0.102:9000': 5555} Restrict computation to particular workers with the ``workers=`` keyword argument. >>> c.run(os.getpid, workers=['192.168.0.100:9000', ... '192.168.0.101:9000']) # doctest: +SKIP {'192.168.0.100:9000': 1234, '192.168.0.101:9000': 4321} >>> def get_status(dask_worker): ... return dask_worker.status >>> c.run(get_hostname) # doctest: +SKIP {'192.168.0.100:9000': 'running', '192.168.0.101:9000': 'running} """ return self.sync(self._run, function, *args, **kwargs) @gen.coroutine def _run_coroutine(self, function, *args, **kwargs): workers = kwargs.pop('workers', None) wait = kwargs.pop('wait', True) responses = yield self.scheduler.broadcast(msg=dict(op='run_coroutine', function=dumps(function), args=dumps(args), kwargs=dumps(kwargs), wait=wait), workers=workers) if not wait: raise gen.Return(None) else: 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 run_coroutine(self, function, *args, **kwargs): """ Spawn a coroutine on all workers. This spaws a coroutine on all currently known workers and then waits for the coroutine on each worker. The coroutines' results are returned as a dictionary keyed by worker address. Parameters ---------- function: a coroutine function (typically a function wrapped in gen.coroutine or a Python 3.5+ async function) *args: arguments for remote function **kwargs: keyword arguments for remote function wait: boolean (default True) Whether to wait for coroutines to end. workers: list Workers on which to run the function. Defaults to all known workers. """ return self.sync(self._run_coroutine, function, *args, **kwargs) def _graph_to_futures(self, dsk, keys, restrictions=None, loose_restrictions=None, priority=None, resources=None, retries=None): with self._lock: keyset = set(keys) flatkeys = list(map(tokey, keys)) futures = {key: Future(key, self, inform=False) for key in keyset} values = {k for k, v in dsk.items() if isinstance(v, Future) and k not in keyset} if values: dsk = dask.optimize.inline(dsk, keys=values) d = {k: unpack_remotedata(v) for k, v in dsk.items()} extra_keys = set.union(*[v[1] for v in d.values()]) if d else set() dsk2 = str_graph({k: v[0] for k, v in d.items()}, extra_keys) dsk3 = {k: v for k, v in dsk2.items() if k is not v} if restrictions: restrictions = keymap(tokey, restrictions) restrictions = valmap(list, restrictions) if loose_restrictions is not None: loose_restrictions = list(map(tokey, loose_restrictions)) dependencies = {tokey(k): set(map(tokey, v[1])) for k, v in d.items()} for s in dependencies.values(): for v in s: if v not in self.futures: raise CancelledError(v) for k, v in dsk3.items(): dependencies[k] |= get_dependencies(dsk3, task=v) if priority is None: dependencies2 = {key: {dep for dep in deps if dep in dependencies} for key, deps in dependencies.items()} priority = dask.order.order(dsk3, dependencies2) self._send_to_scheduler({'op': 'update-graph', 'tasks': valmap(dumps_task, dsk3), 'dependencies': valmap(list, dependencies), 'keys': list(flatkeys), 'restrictions': restrictions or {}, 'loose_restrictions': loose_restrictions, 'priority': priority, 'resources': resources, 'submitting_task': getattr(thread_state, 'key', None), 'retries': retries, }) return futures def get(self, dsk, keys, restrictions=None, loose_restrictions=None, resources=None, sync=True, asynchronous=None, **kwargs): """ Compute dask graph Parameters ---------- dsk: dict keys: object, or nested lists of objects restrictions: dict (optional) A mapping of {key: {set of worker hostnames}} that restricts where jobs can take place sync: bool (optional) Returns Futures if False or concrete values if True (default). Examples -------- >>> from operator import add # doctest: +SKIP >>> c = Client('127.0.0.1:8787') # doctest: +SKIP >>> c.get({'x': (add, 1, 2)}, 'x') # doctest: +SKIP 3 See Also -------- Client.compute: Compute asynchronous collections """ futures = self._graph_to_futures(dsk, set(flatten([keys])), restrictions, loose_restrictions, resources=resources) packed = pack_data(keys, futures) if sync: if getattr(thread_state, 'key', False): secede() try: results = self.gather(packed, asynchronous=asynchronous) finally: for f in futures.values(): f.release() if getattr(thread_state, 'key', False): rejoin() return results return packed def _optimize_insert_futures(self, dsk, keys): """ Replace known keys in dask graph with Futures When given a Dask graph that might have overlapping keys with our known results we replace the values of that graph with futures. This can be used as an optimization to avoid recomputation. This returns the same graph if unchanged but a new graph if any changes were necessary. """ changed = False for key in list(dsk): if tokey(key) in self.futures: if not changed: changed = True dsk = dict(dsk) dsk[key] = Future(key, self, inform=False) if changed: dsk, _ = dask.optimize.cull(dsk, keys) return dsk def normalize_collection(self, collection): """ Replace collection's tasks by already existing futures if they exist This normalizes the tasks within a collections task graph against the known futures within the scheduler. It returns a copy of the collection with a task graph that includes the overlapping futures. Examples -------- >>> len(x.__dask_graph__()) # x is a dask collection with 100 tasks # doctest: +SKIP 100 >>> set(client.futures).intersection(x.__dask_graph__()) # some overlap exists # doctest: +SKIP 10 >>> x = client.normalize_collection(x) # doctest: +SKIP >>> len(x.__dask_graph__()) # smaller computational graph # doctest: +SKIP 20 See Also -------- Client.persist: trigger computation of collection's tasks """ with self._lock: dsk = self._optimize_insert_futures( collection.__dask_graph__(), collection.__dask_keys__()) if dsk is collection.__dask_graph__(): return collection else: return redict_collection(collection, dsk) def compute(self, collections, sync=False, optimize_graph=True, workers=None, allow_other_workers=False, resources=None, retries=0, **kwargs): """ Compute dask collections on cluster Parameters ---------- collections: iterable of dask objects or single dask object Collections like dask.array or dataframe or dask.value objects sync: bool (optional) Returns Futures if False (default) or concrete values if True optimize_graph: bool Whether or not to optimize the underlying graphs workers: str, list, dict Which workers can run which parts of the computation If a string a list then the output collections will run on the listed workers, but other sub-computations can run anywhere If a dict then keys should be (tuples of) collections and values should be addresses or lists. allow_other_workers: bool, list If True then all restrictions in workers= are considered loose If a list then only the keys for the listed collections are loose retries: int (default to 0) Number of allowed automatic retries if computing a result fails **kwargs: Options to pass to the graph optimize calls Returns ------- List of Futures if input is a sequence, or a single future otherwise Examples -------- >>> from dask import delayed >>> from operator import add >>> x = delayed(add)(1, 2) >>> y = delayed(add)(x, x) >>> xx, yy = client.compute([x, y]) # doctest: +SKIP >>> xx # doctest: +SKIP >>> xx.result() # doctest: +SKIP 3 >>> yy.result() # doctest: +SKIP 6 Also support single arguments >>> xx = client.compute(x) # doctest: +SKIP See Also -------- Client.get: Normal synchronous dask.get function """ if isinstance(collections, (list, tuple, set, frozenset)): singleton = False else: collections = [collections] singleton = True traverse = kwargs.pop('traverse', True) if traverse: collections = tuple(dask.delayed(a) if isinstance(a, (list, set, tuple, dict, Iterator)) else a for a in collections) variables = [a for a in collections if dask.is_dask_collection(a)] dsk = self.collections_to_dsk(variables, optimize_graph, **kwargs) names = ['finalize-%s' % tokenize(v) for v in variables] dsk2 = {} for i, (name, v) in enumerate(zip(names, variables)): func, extra_args = v.__dask_postcompute__() keys = v.__dask_keys__() if func is single_key and len(keys) == 1 and not extra_args: names[i] = keys[0] else: dsk2[name] = (func, keys) + extra_args restrictions, loose_restrictions = self.get_restrictions(collections, workers, allow_other_workers) if resources: resources = self._expand_resources(resources, all_keys=itertools.chain(dsk, dsk2)) if retries: retries = self._expand_retries(retries, all_keys=itertools.chain(dsk, dsk2)) else: retries = None futures_dict = self._graph_to_futures(merge(dsk2, dsk), names, restrictions, loose_restrictions, resources=resources, retries=retries) i = 0 futures = [] for arg in collections: if dask.is_dask_collection(arg): futures.append(futures_dict[names[i]]) i += 1 else: futures.append(arg) if sync: result = self.gather(futures) else: result = futures if singleton: return first(result) else: return result def persist(self, collections, optimize_graph=True, workers=None, allow_other_workers=None, resources=None, retries=None, **kwargs): """ Persist dask collections on cluster Starts computation of the collection on the cluster in the background. Provides a new dask collection that is semantically identical to the previous one, but now based off of futures currently in execution. Parameters ---------- collections: sequence or single dask object Collections like dask.array or dataframe or dask.value objects optimize_graph: bool Whether or not to optimize the underlying graphs workers: str, list, dict Which workers can run which parts of the computation If a string a list then the output collections will run on the listed workers, but other sub-computations can run anywhere If a dict then keys should be (tuples of) collections and values should be addresses or lists. allow_other_workers: bool, list If True then all restrictions in workers= are considered loose If a list then only the keys for the listed collections are loose retries: int (default to 0) Number of allowed automatic retries if computing a result fails kwargs: Options to pass to the graph optimize calls Returns ------- List of collections, or single collection, depending on type of input. Examples -------- >>> xx = client.persist(x) # doctest: +SKIP >>> xx, yy = client.persist([x, y]) # doctest: +SKIP See Also -------- Client.compute """ if isinstance(collections, (tuple, list, set, frozenset)): singleton = False else: singleton = True collections = [collections] assert all(map(dask.is_dask_collection, collections)) dsk = self.collections_to_dsk(collections, optimize_graph, **kwargs) names = {k for c in collections for k in flatten(c.__dask_keys__())} restrictions, loose_restrictions = self.get_restrictions(collections, workers, allow_other_workers) if resources: resources = self._expand_resources(resources, all_keys=itertools.chain(dsk, names)) if retries: retries = self._expand_retries(retries, all_keys=itertools.chain(dsk, names)) else: retries = None futures = self._graph_to_futures(dsk, names, restrictions, loose_restrictions, resources=resources, retries=retries) postpersists = [c.__dask_postpersist__() for c in collections] result = [func({k: futures[k] for k in flatten(c.__dask_keys__())}, *args) for (func, args), c in zip(postpersists, collections)] if singleton: return first(result) else: return result @gen.coroutine def _upload_environment(self, zipfile): name = os.path.split(zipfile)[1] yield self._upload_large_file(zipfile, name) def unzip(dask_worker=None): from distributed.utils import log_errors import zipfile import shutil with log_errors(): a = os.path.join(dask_worker.worker_dir, name) b = os.path.join(dask_worker.local_dir, name) c = os.path.dirname(b) shutil.move(a, b) with zipfile.ZipFile(b) as f: f.extractall(path=c) for fn in glob(os.path.join(c, name[:-4], 'bin', '*')): st = os.stat(fn) os.chmod(fn, st.st_mode | 64) # chmod u+x fn assert os.path.exists(os.path.join(c, name[:-4])) return c yield self._run(unzip, nanny=True) raise gen.Return(name[:-4]) def upload_environment(self, name, zipfile): return self.sync(self._upload_environment, name, zipfile) @gen.coroutine def _restart(self, timeout=5): self._send_to_scheduler({'op': 'restart', 'timeout': timeout}) self._restart_event = Event() try: yield self._restart_event.wait(self.loop.time() + timeout) except gen.TimeoutError: logger.error("Restart timed out after %f seconds", timeout) pass self.generation += 1 with self._refcount_lock: self.refcount.clear() raise gen.Return(self) def restart(self, **kwargs): """ Restart the distributed network This kills all active work, deletes all data on the network, and restarts the worker processes. """ return self.sync(self._restart, **kwargs) @gen.coroutine def _upload_file(self, filename, raise_on_error=True): with open(filename, 'rb') as f: data = f.read() _, fn = os.path.split(filename) d = yield self.scheduler.broadcast(msg={'op': 'upload_file', 'filename': fn, 'data': to_serialize(data)}) if any(v['status'] == 'error' for v in d.values()): exceptions = [loads(v['exception']) for v in d.values() if v['status'] == 'error'] if raise_on_error: raise exceptions[0] else: raise gen.Return(exceptions[0]) assert all(len(data) == v['nbytes'] for v in d.values()) @gen.coroutine def _upload_large_file(self, local_filename, remote_filename=None): if remote_filename is None: remote_filename = os.path.split(local_filename)[1] with open(local_filename, 'rb') as f: data = f.read() [future] = yield self._scatter([data]) key = future.key yield self._replicate(future) def dump_to_file(dask_worker=None): if not os.path.isabs(remote_filename): fn = os.path.join(dask_worker.local_dir, remote_filename) else: fn = remote_filename with open(fn, 'wb') as f: f.write(dask_worker.data[key]) return len(dask_worker.data[key]) response = yield self._run(dump_to_file) assert all(len(data) == v for v in response.values()) def upload_file(self, filename, **kwargs): """ Upload local package to workers This sends a local file up to all worker nodes. This file is placed into a temporary directory on Python's system path so any .py, .pyc, .egg or .zip files will be importable. Parameters ---------- filename: string Filename of .py, .pyc, .egg or .zip file to send to workers Examples -------- >>> client.upload_file('mylibrary.egg') # doctest: +SKIP >>> from mylibrary import myfunc # doctest: +SKIP >>> L = c.map(myfunc, seq) # doctest: +SKIP """ result = self.sync(self._upload_file, filename, raise_on_error=self.asynchronous, **kwargs) if isinstance(result, Exception): raise result else: return result @gen.coroutine def _rebalance(self, futures=None, workers=None): yield _wait(futures) keys = list({tokey(f.key) for f in self.futures_of(futures)}) result = yield self.scheduler.rebalance(keys=keys, workers=workers) assert result['status'] == 'OK' def rebalance(self, futures=None, workers=None, **kwargs): """ Rebalance data within network Move data between workers to roughly balance memory burden. This either affects a subset of the keys/workers or the entire network, depending on keyword arguments. This operation is generally not well tested against normal operation of the scheduler. It it not recommended to use it while waiting on computations. Parameters ---------- futures: list, optional A list of futures to balance, defaults all data workers: list, optional A list of workers on which to balance, defaults to all workers """ return self.sync(self._rebalance, futures, workers, **kwargs) @gen.coroutine def _replicate(self, futures, n=None, workers=None, branching_factor=2): futures = self.futures_of(futures) yield _wait(futures) keys = {tokey(f.key) for f in futures} yield self.scheduler.replicate(keys=list(keys), n=n, workers=workers, branching_factor=branching_factor) def replicate(self, futures, n=None, workers=None, branching_factor=2, **kwargs): """ Set replication of futures within network Copy data onto many workers. This helps to broadcast frequently accessed data and it helps to improve resilience. This performs a tree copy of the data throughout the network individually on each piece of data. This operation blocks until complete. It does not guarantee replication of data to future workers. Parameters ---------- futures: list of futures Futures we wish to replicate n: int, optional Number of processes on the cluster on which to replicate the data. Defaults to all. workers: list of worker addresses Workers on which we want to restrict the replication. Defaults to all. branching_factor: int, optional The number of workers that can copy data in each generation Examples -------- >>> x = c.submit(func, *args) # doctest: +SKIP >>> c.replicate([x]) # send to all workers # doctest: +SKIP >>> c.replicate([x], n=3) # send to three workers # doctest: +SKIP >>> c.replicate([x], workers=['alice', 'bob']) # send to specific # doctest: +SKIP >>> c.replicate([x], n=1, workers=['alice', 'bob']) # send to one of specific workers # doctest: +SKIP >>> c.replicate([x], n=1) # reduce replications # doctest: +SKIP See also -------- Client.rebalance """ return self.sync(self._replicate, futures, n=n, workers=workers, branching_factor=branching_factor, **kwargs) def ncores(self, workers=None, **kwargs): """ The number of threads/cores available on each worker node Parameters ---------- workers: list (optional) A list of workers that we care about specifically. Leave empty to receive information about all workers. Examples -------- >>> c.ncores() # doctest: +SKIP {'192.168.1.141:46784': 8, '192.167.1.142:47548': 8, '192.167.1.143:47329': 8, '192.167.1.144:37297': 8} See Also -------- Client.who_has Client.has_what """ if (isinstance(workers, tuple) and all(isinstance(i, (str, tuple)) for i in workers)): workers = list(workers) if workers is not None and not isinstance(workers, (list, set)): workers = [workers] return self.sync(self.scheduler.ncores, workers=workers, **kwargs) def who_has(self, futures=None, **kwargs): """ The workers storing each future's data Parameters ---------- futures: list (optional) A list of futures, defaults to all data Examples -------- >>> x, y, z = c.map(inc, [1, 2, 3]) # doctest: +SKIP >>> wait([x, y, z]) # doctest: +SKIP >>> c.who_has() # doctest: +SKIP {'inc-1c8dd6be1c21646c71f76c16d09304ea': ['192.168.1.141:46784'], 'inc-1e297fc27658d7b67b3a758f16bcf47a': ['192.168.1.141:46784'], 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b': ['192.168.1.141:46784']} >>> c.who_has([x, y]) # doctest: +SKIP {'inc-1c8dd6be1c21646c71f76c16d09304ea': ['192.168.1.141:46784'], 'inc-1e297fc27658d7b67b3a758f16bcf47a': ['192.168.1.141:46784']} See Also -------- Client.has_what Client.ncores """ if futures is not None: futures = self.futures_of(futures) keys = list(map(tokey, {f.key for f in futures})) else: keys = None return self.sync(self.scheduler.who_has, keys=keys, **kwargs) def has_what(self, workers=None, **kwargs): """ Which keys are held by which workers Parameters ---------- workers: list (optional) A list of worker addresses, defaults to all Examples -------- >>> x, y, z = c.map(inc, [1, 2, 3]) # doctest: +SKIP >>> wait([x, y, z]) # doctest: +SKIP >>> c.has_what() # doctest: +SKIP {'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea', 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b', 'inc-1e297fc27658d7b67b3a758f16bcf47a']} See Also -------- Client.who_has Client.ncores """ if (isinstance(workers, tuple) and all(isinstance(i, (str, tuple)) for i in workers)): workers = list(workers) if workers is not None and not isinstance(workers, (list, set)): workers = [workers] return self.sync(self.scheduler.has_what, workers=workers, **kwargs) def stacks(self, workers=None): """ The task queues on each worker Parameters ---------- workers: list (optional) A list of worker addresses, defaults to all Examples -------- >>> x, y, z = c.map(inc, [1, 2, 3]) # doctest: +SKIP >>> c.stacks() # doctest: +SKIP {'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea', 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b', 'inc-1e297fc27658d7b67b3a758f16bcf47a']} See Also -------- Client.processing Client.who_has Client.has_what Client.ncores """ if (isinstance(workers, tuple) and all(isinstance(i, (str, tuple)) for i in workers)): workers = list(workers) if workers is not None and not isinstance(workers, (list, set)): workers = [workers] return sync(self.loop, self.scheduler.stacks, workers=workers) def processing(self, workers=None): """ The tasks currently running on each worker Parameters ---------- workers: list (optional) A list of worker addresses, defaults to all Examples -------- >>> x, y, z = c.map(inc, [1, 2, 3]) # doctest: +SKIP >>> c.processing() # doctest: +SKIP {'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea', 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b', 'inc-1e297fc27658d7b67b3a758f16bcf47a']} See Also -------- Client.stacks Client.who_has Client.has_what Client.ncores """ if (isinstance(workers, tuple) and all(isinstance(i, (str, tuple)) for i in workers)): workers = list(workers) if workers is not None and not isinstance(workers, (list, set)): workers = [workers] return valmap(set, sync(self.loop, self.scheduler.processing, workers=workers)) def nbytes(self, keys=None, summary=True, **kwargs): """ The bytes taken up by each key on the cluster This is as measured by ``sys.getsizeof`` which may not accurately reflect the true cost. Parameters ---------- keys: list (optional) A list of keys, defaults to all keys summary: boolean, (optional) Summarize keys into key types Examples -------- >>> x, y, z = c.map(inc, [1, 2, 3]) # doctest: +SKIP >>> c.nbytes(summary=False) # doctest: +SKIP {'inc-1c8dd6be1c21646c71f76c16d09304ea': 28, 'inc-1e297fc27658d7b67b3a758f16bcf47a': 28, 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b': 28} >>> c.nbytes(summary=True) # doctest: +SKIP {'inc': 84} See Also -------- Client.who_has """ return self.sync(self.scheduler.nbytes, keys=keys, summary=summary, **kwargs) def call_stack(self, futures=None, keys=None): """ The actively running call stack of all relevant keys You can specify data of interest either by providing futures or collections in the ``futures=`` keyword or a list of explicit keys in the ``keys=`` keyword. If neither are provided then all call stacks will be returned. Parameters ---------- futures: list (optional) List of futures, defaults to all data keys: list (optional) List of key names, defaults to all data Examples -------- >>> df = dd.read_parquet(...).persist() # doctest: +SKIP >>> client.call_stack(df) # call on collections >>> client.call_stack() # Or call with no arguments for all activity # doctest: +SKIP """ keys = keys or [] if futures is not None: futures = self.futures_of(futures) keys += list(map(tokey, {f.key for f in futures})) return self.sync(self.scheduler.call_stack, keys=keys or None) def profile(self, key=None, start=None, stop=None, workers=None, merge_workers=True): """ Collect statistical profiling information about recent work Parameters ---------- key: str Key prefix to select, this is typically a function name like 'inc' Leave as None to collect all data start: time stop: time workers: list List of workers to restrict profile information Examples -------- >>> client.profile() # call on collections """ if isinstance(workers, six.string_types + (Number,)): workers = [workers] return self.sync(self.scheduler.profile, key=key, workers=workers, merge_workers=merge_workers, start=start, stop=stop) def scheduler_info(self, **kwargs): """ Basic information about the workers in the cluster Examples -------- >>> c.scheduler_info() # doctest: +SKIP {'id': '2de2b6da-69ee-11e6-ab6a-e82aea155996', 'services': {}, 'type': 'Scheduler', 'workers': {'127.0.0.1:40575': {'active': 0, 'last-seen': 1472038237.4845693, 'name': '127.0.0.1:40575', 'services': {}, 'stored': 0, 'time-delay': 0.0061032772064208984}}} """ self.sync(self._update_scheduler_info) return self._scheduler_identity def get_metadata(self, keys, default=no_default): """ Get arbitrary metadata from scheduler See set_metadata for the full docstring with examples Parameter --------- keys: key or list Key to access. If a list then gets within a nested collection default: optional If the key does not exist then return this value instead. If not provided then this raises a KeyError if the key is not present See also -------- Client.set_metadata """ if not isinstance(keys, list): keys = [keys] return self.sync(self.scheduler.get_metadata, keys=keys, default=default) def get_scheduler_logs(self, n=None): """ Get logs from scheduler Parameters ---------- n: int Number of logs to retrive. Maxes out at 10000 by default, confiruable in config.yaml::log-length Returns ------- Logs in reversed order (newest first) """ return self.sync(self.scheduler.logs, n=n) def get_worker_logs(self, n=None, workers=None): """ Get logs from workers Parameters ---------- 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. 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) def set_metadata(self, key, value): """ Set arbitrary metadata in the scheduler This allows you to store small amounts of data on the central scheduler process for administrative purposes. Data should be msgpack serializable (ints, strings, lists, dicts) If the key corresponds to a task then that key will be cleaned up when the task is forgotten by the scheduler. If the key is a list then it will be assumed that you want to index into a nested dictionary structure using those keys. For example if you call the following:: >>> client.set_metadata(['a', 'b', 'c'], 123) Then this is the same as setting >>> scheduler.task_metadata['a']['b']['c'] = 123 The lower level dictionaries will be created on demand. Examples -------- >>> client.set_metadata('x', 123) # doctest: +SKIP >>> client.get_metadata('x') # doctest: +SKIP 123 >>> client.set_metadata(['x', 'y'], 123) # doctest: +SKIP >>> client.get_metadata('x') # doctest: +SKIP {'y': 123} >>> client.set_metadata(['x', 'w', 'z'], 456) # doctest: +SKIP >>> client.get_metadata('x') # doctest: +SKIP {'y': 123, 'w': {'z': 456}} >>> client.get_metadata(['x', 'w']) # doctest: +SKIP {'z': 456} See Also -------- get_metadata """ if not isinstance(key, list): key = [key] return self.sync(self.scheduler.set_metadata, keys=key, value=value) def get_versions(self, check=False): """ Return version info for the scheduler, all workers and myself Parameters ---------- check : boolean, default False raise ValueError if all required & optional packages do not match Examples -------- >>> c.get_versions() # doctest: +SKIP """ client = get_versions() try: scheduler = sync(self.loop, self.scheduler.versions) except KeyError: scheduler = None workers = sync(self.loop, self._run, get_versions) result = {'scheduler': scheduler, 'workers': workers, 'client': client} if check: # we care about the required & optional packages matching def to_packages(d): return dict(sum(d['packages'].values(), [])) client_versions = to_packages(result['client']) versions = [('scheduler', to_packages(result['scheduler']))] versions.extend((w, to_packages(d)) for w, d in sorted(workers.items())) mismatched = defaultdict(list) for name, vers in versions: for pkg, cv in client_versions.items(): v = vers.get(pkg, 'MISSING') if cv != v: mismatched[pkg].append((name, v)) if mismatched: errs = [] for pkg, versions in sorted(mismatched.items()): rows = [('client', client_versions[pkg])] rows.extend(versions) errs.append("%s\n%s" % (pkg, asciitable(['', 'version'], rows))) raise ValueError("Mismatched versions found\n" "\n" "%s" % ('\n\n'.join(errs))) return result def futures_of(self, futures): return futures_of(futures, client=self) def start_ipython(self, *args, **kwargs): raise Exception("Method moved to start_ipython_workers") @gen.coroutine def _start_ipython_workers(self, workers): if workers is None: workers = yield self.scheduler.ncores() responses = yield self.scheduler.broadcast( msg=dict(op='start_ipython'), workers=workers, ) raise gen.Return((workers, responses)) def start_ipython_workers(self, workers=None, magic_names=False, qtconsole=False, qtconsole_args=None): """ Start IPython kernels on workers Parameters ---------- workers: list (optional) A list of worker addresses, defaults to all magic_names: str or list(str) (optional) If defined, register IPython magics with these names for executing code on the workers. If string has asterix then expand asterix into 0, 1, ..., n for n workers qtconsole: bool (optional) If True, launch a Jupyter QtConsole connected to the worker(s). qtconsole_args: list(str) (optional) Additional arguments to pass to the qtconsole on startup. Examples -------- >>> info = c.start_ipython_workers() # doctest: +SKIP >>> %remote info['192.168.1.101:5752'] worker.data # doctest: +SKIP {'x': 1, 'y': 100} >>> c.start_ipython_workers('192.168.1.101:5752', magic_names='w') # doctest: +SKIP >>> %w worker.data # doctest: +SKIP {'x': 1, 'y': 100} >>> c.start_ipython_workers('192.168.1.101:5752', qtconsole=True) # doctest: +SKIP Add asterix * in magic names to add one magic per worker >>> c.start_ipython_workers(magic_names='w_*') # doctest: +SKIP >>> %w_0 worker.data # doctest: +SKIP {'x': 1, 'y': 100} >>> %w_1 worker.data # doctest: +SKIP {'z': 5} Returns ------- iter_connection_info: list List of connection_info dicts containing info necessary to connect Jupyter clients to the workers. See Also -------- Client.start_ipython_scheduler: start ipython on the scheduler """ if isinstance(workers, six.string_types + (Number,)): workers = [workers] (workers, info_dict) = sync(self.loop, self._start_ipython_workers, workers) if magic_names and isinstance(magic_names, six.string_types): if '*' in magic_names: magic_names = [magic_names.replace('*', str(i)) for i in range(len(workers))] else: magic_names = [magic_names] if 'IPython' in sys.modules: from ._ipython_utils import register_remote_magic register_remote_magic() if magic_names: from ._ipython_utils import register_worker_magic for worker, magic_name in zip(workers, magic_names): connection_info = info_dict[worker] register_worker_magic(connection_info, magic_name) if qtconsole: from ._ipython_utils import connect_qtconsole for worker, connection_info in info_dict.items(): name = 'dask-' + worker.replace(':', '-').replace('/', '-') connect_qtconsole(connection_info, name=name, extra_args=qtconsole_args, ) return info_dict def start_ipython_scheduler(self, magic_name='scheduler_if_ipython', qtconsole=False, qtconsole_args=None): """ Start IPython kernel on the scheduler Parameters ---------- magic_name: str or None (optional) If defined, register IPython magic with this name for executing code on the scheduler. If not defined, register %scheduler magic if IPython is running. qtconsole: bool (optional) If True, launch a Jupyter QtConsole connected to the worker(s). qtconsole_args: list(str) (optional) Additional arguments to pass to the qtconsole on startup. Examples -------- >>> c.start_ipython_scheduler() # doctest: +SKIP >>> %scheduler scheduler.processing # doctest: +SKIP {'127.0.0.1:3595': {'inc-1', 'inc-2'}, '127.0.0.1:53589': {'inc-2', 'add-5'}} >>> c.start_ipython_scheduler(qtconsole=True) # doctest: +SKIP Returns ------- connection_info: dict connection_info dict containing info necessary to connect Jupyter clients to the scheduler. See Also -------- Client.start_ipython_workers: Start IPython on the workers """ info = sync(self.loop, self.scheduler.start_ipython) if magic_name == 'scheduler_if_ipython': # default to %scheduler if in IPython, no magic otherwise in_ipython = False if 'IPython' in sys.modules: from IPython import get_ipython in_ipython = bool(get_ipython()) if in_ipython: magic_name = 'scheduler' else: magic_name = None if magic_name: from ._ipython_utils import register_worker_magic register_worker_magic(info, magic_name) if qtconsole: from ._ipython_utils import connect_qtconsole connect_qtconsole(info, name='dask-scheduler', extra_args=qtconsole_args,) return info @classmethod def _expand_key(cls, k): """ Expand a user-provided task key specification, e.g. in a resources or retries dictionary. """ if not isinstance(k, tuple): k = (k,) for kk in k: if dask.is_dask_collection(kk): for kkk in kk.__dask_keys__(): yield tokey(kkk) else: yield tokey(kk) @classmethod def _expand_retries(cls, retries, all_keys): """ Expand the user-provided "retries" specification to a {task key: Integral} dictionary. """ if retries and isinstance(retries, dict): return {name: value for key, value in retries.items() for name in cls._expand_key(key)} elif isinstance(retries, Integral): # Each task unit may potentially fail, allow retrying all of them return {name: retries for name in all_keys} else: raise TypeError("`retries` should be an integer or dict, got %r" % (type(retries,))) def _expand_resources(cls, resources, all_keys): """ Expand the user-provided "resources" specification to a {task key: {resource name: Number}} dictionary. """ # Resources can either be a single dict such as {'GPU': 2}, # indicating a requirement for all keys, or a nested dict # such as {'x': {'GPU': 1}, 'y': {'SSD': 4}} indicating # per-key requirements if not isinstance(resources, dict): raise TypeError("`retries` should be a dict, got %r" % (type(retries,))) per_key_reqs = {} global_reqs = {} all_keys = list(all_keys) for k, v in resources.items(): if isinstance(v, dict): # It's a per-key requirement per_key_reqs.update((kk, v) for kk in cls._expand_key(k)) else: # It's a global requirement global_reqs.update((kk, {k: v}) for kk in all_keys) if global_reqs and per_key_reqs: raise ValueError("cannot have both per-key and all-key requirements " "in resources dict %r" % (resources,)) return global_reqs or per_key_reqs @classmethod def get_restrictions(cls, collections, workers, allow_other_workers): """ Get restrictions from inputs to compute/persist """ if isinstance(workers, (str, tuple, list)): workers = {tuple(collections): workers} if isinstance(workers, dict): restrictions = {} for colls, ws in workers.items(): if isinstance(ws, str): ws = [ws] if dask.is_dask_collection(colls): keys = flatten(colls.__dask_keys__()) else: keys = list({k for c in flatten(colls) for k in flatten(c.__dask_keys__())}) restrictions.update({k: ws for k in keys}) else: restrictions = {} if allow_other_workers is True: loose_restrictions = list(restrictions) elif allow_other_workers: loose_restrictions = list({k for c in flatten(allow_other_workers) for k in c.__dask_keys__()}) else: loose_restrictions = [] return restrictions, loose_restrictions @staticmethod def collections_to_dsk(collections, *args, **kwargs): return collections_to_dsk(collections, *args, **kwargs) class Executor(Client): """ Deprecated: see Client """ def __init__(self, *args, **kwargs): warnings.warn("Executor has been renamed to Client") super(Executor, self).__init__(*args, **kwargs) def CompatibleExecutor(*args, **kwargs): raise Exception("This has been moved to the Client.get_executor() method") @gen.coroutine def _wait(fs, timeout=None, return_when='ALL_COMPLETED'): if timeout is not None and not isinstance(timeout, Number): raise TypeError("timeout= keyword received a non-numeric value.\n" "Beware that wait expects a list of values\n" " Bad: wait(x, y, z)\n" " Good: wait([x, y, z])") fs = futures_of(fs) if return_when == 'ALL_COMPLETED': future = All({f._state.wait() for f in fs}) if timeout is not None: future = gen.with_timeout(timedelta(seconds=timeout), future) yield future done, not_done = set(fs), set() cancelled = [f.key for f in done if f.status == 'cancelled'] if cancelled: raise CancelledError(cancelled) else: raise NotImplementedError("Only return_when='ALL_COMPLETED' supported") raise gen.Return(DoneAndNotDoneFutures(done, not_done)) ALL_COMPLETED = 'ALL_COMPLETED' def wait(fs, timeout=None, return_when='ALL_COMPLETED'): """ Wait until all futures are complete Parameters ---------- fs: list of futures timeout: number, optional Time in seconds after which to raise a gen.TimeoutError Returns ------- Named tuple of completed, not completed """ client = default_client() result = client.sync(_wait, fs, timeout=timeout, return_when=return_when) return result @gen.coroutine def _as_completed(fs, queue): fs = futures_of(fs) groups = groupby(lambda f: f.key, fs) firsts = [v[0] for v in groups.values()] wait_iterator = gen.WaitIterator(*[f._state.wait() for f in firsts]) while not wait_iterator.done(): yield wait_iterator.next() # TODO: handle case of restarted futures future = firsts[wait_iterator.current_index] for f in groups[future.key]: queue.put_nowait(f) @gen.coroutine def _first_completed(futures): """ Return a single completed future See Also: _as_completed """ q = Queue() yield _as_completed(futures, q) result = yield q.get() raise gen.Return(result) class as_completed(object): """ Return futures in the order in which they complete This returns an iterator that yields the input future objects in the order in which they complete. Calling ``next`` on the iterator will block until the next future completes, irrespective of order. Additionally, you can also add more futures to this object during computation with the ``.add`` method Examples -------- >>> x, y, z = client.map(inc, [1, 2, 3]) # doctest: +SKIP >>> for future in as_completed([x, y, z]): # doctest: +SKIP ... print(future.result()) # doctest: +SKIP 3 2 4 Add more futures during computation >>> x, y, z = client.map(inc, [1, 2, 3]) # doctest: +SKIP >>> ac = as_completed([x, y, z]) # doctest: +SKIP >>> for future in ac: # doctest: +SKIP ... print(future.result()) # doctest: +SKIP ... if random.random() < 0.5: # doctest: +SKIP ... ac.add(c.submit(double, future)) # doctest: +SKIP 4 2 8 3 6 12 24 Optionally wait until the result has been gathered as well >>> ac = as_completed([x, y, z], with_results=True) # doctest: +SKIP >>> for future, result in ac: # doctest: +SKIP ... print(result) # doctest: +SKIP 2 4 3 """ def __init__(self, futures=None, loop=None, with_results=False): if futures is None: futures = [] self.futures = defaultdict(lambda: 0) self.queue = pyQueue() self.lock = threading.Lock() self.loop = loop or default_client().loop self.condition = Condition() self.thread_condition = threading.Condition() self.with_results = with_results if futures: self.update(futures) def _notify(self): self.condition.notify() with self.thread_condition: self.thread_condition.notify() @gen.coroutine def track_future(self, future): try: yield _wait(future) except CancelledError: del self.futures[future] if not self.futures: self._notify() return if self.with_results: result = yield future._result() with self.lock: self.futures[future] -= 1 if not self.futures[future]: del self.futures[future] if self.with_results: self.queue.put_nowait((future, result)) else: self.queue.put_nowait(future) self._notify() def update(self, futures): """ Add multiple futures to the collection. The added futures will emit from the iterator once they finish""" with self.lock: for f in futures: if not isinstance(f, Future): raise TypeError("Input must be a future, got %s" % f) self.futures[f] += 1 self.loop.add_callback(self.track_future, f) def add(self, future): """ Add a future to the collection This future will emit from the iterator once it finishes """ self.update((future,)) def is_empty(self): """Return True if there no waiting futures, False otherwise""" with self.lock: return not self.futures and self.queue.empty() def __iter__(self): return self def __aiter__(self): return self def __next__(self): while self.queue.empty(): if self.is_empty(): raise StopIteration() with self.thread_condition: self.thread_condition.wait(timeout=0.100) return self.queue.get() @gen.coroutine def __anext__(self): if not self.futures and self.queue.empty(): raise StopAsyncIteration # flake8: noqa while self.queue.empty(): if not self.futures: raise StopAsyncIteration # flake8: noqa yield self.condition.wait() raise gen.Return(self.queue.get()) next = __next__ def next_batch(self, block=True): """ Get next batch of futures from as_completed iterator Parameters ---------- block: bool, optional If True then wait until we have some result, otherwise return immediately, even with an empty list. Defaults to True. Examples -------- >>> ac = as_completed(futures) # doctest: +SKIP >>> client.gather(ac.next_batch()) # doctest: +SKIP [4, 1, 3] >>> client.gather(ac.next_batch(block=False)) # doctest: +SKIP [] Returns ------- List of futures or (future, result) tuples """ if block: batch = [next(self)] else: batch = [] while not self.queue.empty(): batch.append(self.queue.get()) return batch def batches(self): """ Yield all finished futures at once rather than one-by-one This returns an iterator of lists of futures or lists of (future, result) tuples rather than individual futures or individual (future, result) tuples. It will yield these as soon as possible without waiting. Examples -------- >>> for batch in as_completed(futures).batches(): # doctest: +SKIP ... results = client.gather(batch) ... print(results) [4, 2] [1, 3, 7] [5] [6] """ while True: try: yield self.next_batch(block=True) except StopIteration: return def AsCompleted(*args, **kwargs): raise Exception("This has moved to as_completed") def default_client(c=None): """ Return a client if one has started """ c = c or _get_global_client() if c: return c else: raise ValueError("No clients found\n" "Start an client and point it to the scheduler address\n" " from distributed import Client\n" " client = Client('ip-addr-of-scheduler:8786')\n") def ensure_default_get(client): if _globals['get'] != client.get: print("Setting global dask scheduler to use distributed") dask.set_options(get=client.get) def redict_collection(c, dsk): from dask.delayed import Delayed if isinstance(c, Delayed): return Delayed(c.key, [dsk]) else: cc = copy.copy(c) cc.dask = dsk return cc def futures_of(o, client=None): """ Future objects in a collection """ stack = [o] futures = set() while stack: x = stack.pop() if type(x) in (tuple, set, list): stack.extend(x) if type(x) is dict: stack.extend(x.values()) if isinstance(x, Future): futures.add(x) if dask.is_dask_collection(x): stack.extend(x.__dask_graph__().values()) if client is not None: bad = {f for f in futures if f.cancelled()} if bad: raise CancelledError(bad) return list(futures) def fire_and_forget(obj): """ Run tasks at least once, even if we release the futures Under normal operation Dask will not run any tasks for which there is not an active future (this avoids unnecessary work in many situations). However sometimes you want to just fire off a task, not track its future, and expect it to finish eventually. You can use this function on a future or collection of futures to ask Dask to complete the task even if no active client is tracking it. The results will not be kept in memory after the task completes (unless there is an active future) so this is only useful for tasks that depend on side effects. Parameters ---------- obj: Future, list, dict, dask collection The futures that you want to run at least once Examples -------- >>> fire_and_forget(client.submit(func, *args)) # doctest: +SKIP """ futures = futures_of(obj) for future in futures: future.client._send_to_scheduler({'op': 'client-desires-keys', 'keys': [tokey(future.key)], 'client': 'fire-and-forget'}) @contextmanager def temp_default_client(c): """ Set the default client for the duration of the context Parameters ---------- c : Client This is what default_client() will return within the with-block. """ old_exec = default_client() _set_global_client(c) try: yield finally: _set_global_client(old_exec) def _close_global_client(): """ Force close of global client. This cleans up when a client wasn't close explicitly, e.g. interactive sessions. """ c = _get_global_client() if c is not None: c.close(timeout=2) atexit.register(_close_global_client) distributed-1.20.2/distributed/comm/000077500000000000000000000000001321233345200173715ustar00rootroot00000000000000distributed-1.20.2/distributed/comm/__init__.py000066400000000000000000000010411321233345200214760ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from .addressing import (parse_address, unparse_address, normalize_address, parse_host_port, unparse_host_port, resolve_address, get_address_host_port, get_address_host, get_local_address_for, ) from .core import connect, listen, Comm, CommClosedError def _register_transports(): from . import inproc from . import tcp _register_transports() distributed-1.20.2/distributed/comm/addressing.py000066400000000000000000000111701321233345200220660ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import six from ..config import config from . import registry DEFAULT_SCHEME = config.get('default-scheme', 'tcp') def parse_address(addr, strict=False): """ Split address into its scheme and scheme-dependent location string. >>> parse_address('tcp://127.0.0.1') ('tcp', '127.0.0.1') If strict is set to true the address must have a scheme. """ if not isinstance(addr, six.string_types): raise TypeError("expected str, got %r" % addr.__class__.__name__) scheme, sep, loc = addr.rpartition('://') if strict and not sep: msg = ("Invalid url scheme. " "Must include protocol like tcp://localhost:8000. " "Got %s" % addr) raise ValueError(msg) if not sep: scheme = DEFAULT_SCHEME return scheme, loc def unparse_address(scheme, loc): """ Undo parse_address(). >>> unparse_address('tcp', '127.0.0.1') 'tcp://127.0.0.1' """ return '%s://%s' % (scheme, loc) def normalize_address(addr): """ Canonicalize address, adding a default scheme if necessary. >>> normalize_address('tls://[::1]') 'tls://[::1]' >>> normalize_address('[::1]') 'tcp://[::1]' """ return unparse_address(*parse_address(addr)) def parse_host_port(address, default_port=None): """ Parse an endpoint address given in the form "host:port". """ if isinstance(address, tuple): return address def _fail(): raise ValueError("invalid address %r" % (address,)) def _default(): if default_port is None: raise ValueError("missing port number in address %r" % (address,)) return default_port if address.startswith('['): # IPv6 notation: '[addr]:port' or '[addr]'. # The address may contain multiple colons. host, sep, tail = address[1:].partition(']') if not sep: _fail() if not tail: port = _default() else: if not tail.startswith(':'): _fail() port = tail[1:] else: # Generic notation: 'addr:port' or 'addr'. host, sep, port = address.partition(':') if not sep: port = _default() elif ':' in host: _fail() return host, int(port) def unparse_host_port(host, port=None): """ Undo parse_host_port(). """ if ':' in host and not host.startswith('['): host = '[%s]' % host if port: return '%s:%s' % (host, port) else: return host def get_address_host_port(addr, strict=False): """ Get a (host, port) tuple out of the given address. For definition of strict check parse_address ValueError is raised if the address scheme doesn't allow extracting the requested information. >>> get_address_host_port('tcp://1.2.3.4:80') ('1.2.3.4', 80) """ scheme, loc = parse_address(addr, strict=strict) backend = registry.get_backend(scheme) try: return backend.get_address_host_port(loc) except NotImplementedError: raise ValueError("don't know how to extract host and port " "for address %r" % (addr,)) def get_address_host(addr): """ Return a hostname / IP address identifying the machine this address is located on. In contrast to get_address_host_port(), this function should always succeed for well-formed addresses. >>> get_address_host('tcp://1.2.3.4:80') '1.2.3.4' """ scheme, loc = parse_address(addr) backend = registry.get_backend(scheme) return backend.get_address_host(loc) def get_local_address_for(addr): """ Get a local listening address suitable for reaching *addr*. For instance, trying to reach an external TCP address will return a local TCP address that's routable to that external address. >>> get_local_address_for('tcp://8.8.8.8:1234') 'tcp://192.168.1.68' >>> get_local_address_for('tcp://127.0.0.1:1234') 'tcp://127.0.0.1' """ scheme, loc = parse_address(addr) backend = registry.get_backend(scheme) return unparse_address(scheme, backend.get_local_address_for(loc)) def resolve_address(addr): """ Apply scheme-specific address resolution to *addr*, replacing all symbolic references with concrete location specifiers. In practice, this can mean hostnames are resolved to IP addresses. >>> resolve_address('tcp://localhost:8786') 'tcp://127.0.0.1:8786' """ scheme, loc = parse_address(addr) backend = registry.get_backend(scheme) return unparse_address(scheme, backend.resolve_address(loc)) distributed-1.20.2/distributed/comm/core.py000066400000000000000000000137161321233345200207030ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from abc import ABCMeta, abstractmethod, abstractproperty from datetime import timedelta import logging from six import with_metaclass from tornado import gen from ..config import config from ..metrics import time from . import registry from .addressing import parse_address logger = logging.getLogger(__name__) class CommClosedError(IOError): pass class Comm(with_metaclass(ABCMeta)): """ A message-oriented communication object, representing an established communication channel. There should be only one reader and one writer at a time: to manage current communications, even with a single peer, you must create distinct ``Comm`` objects. Messages are arbitrary Python objects. Concrete implementations of this class can implement different serialization mechanisms depending on the underlying transport's characteristics. """ # XXX add set_close_callback()? @abstractmethod def read(self): """ Read and return a message (a Python object). This method is a coroutine. """ @abstractmethod def write(self, msg): """ Write a message (a Python object). This method is a coroutine. """ @abstractmethod def close(self): """ Close the communication cleanly. This will attempt to flush outgoing buffers before actually closing the underlying transport. This method is a coroutine. """ @abstractmethod def abort(self): """ Close the communication immediately and abruptly. Useful in destructors or generators' ``finally`` blocks. """ @abstractmethod def closed(self): """ Return whether the stream is closed. """ @abstractproperty def local_address(self): """ The local address. For logging and debugging purposes only. """ @abstractproperty def peer_address(self): """ The peer's address. For logging and debugging purposes only. """ @property def extra_info(self): """ Return backend-specific information about the communication, as a dict. Typically, this is information which is initialized when the communication is established and doesn't vary afterwards. """ return {} def __repr__(self): clsname = self.__class__.__name__ if self.closed(): return "" % (clsname,) else: return ("<%s local=%s remote=%s>" % (clsname, self.local_address, self.peer_address)) class Listener(with_metaclass(ABCMeta)): @abstractmethod def start(self): """ Start listening for incoming connections. """ @abstractmethod def stop(self): """ Stop listening. This does not shutdown already established communications, but prevents accepting new ones. """ @abstractproperty def listen_address(self): """ The listening address as a URI string. """ @abstractproperty def contact_address(self): """ An address this listener can be contacted on. This can be different from `listen_address` if the latter is some wildcard address such as 'tcp://0.0.0.0:123'. """ def __enter__(self): self.start() return self def __exit__(self, *exc): self.stop() class Connector(with_metaclass(ABCMeta)): @abstractmethod def connect(self, address, deserialize=True): """ Connect to the given address and return a Comm object. This function is a coroutine. It may raise EnvironmentError if the other endpoint is unreachable or unavailable. It may raise ValueError if the address is malformed. """ @gen.coroutine def connect(addr, timeout=None, deserialize=True, connection_args=None): """ Connect to the given address (a URI such as ``tcp://127.0.0.1:1234``) and yield a ``Comm`` object. If the connection attempt fails, it is retried until the *timeout* is expired. """ if timeout is None: timeout = float(config.get('connect-timeout', 3)) # default 3 s. scheme, loc = parse_address(addr) backend = registry.get_backend(scheme) connector = backend.get_connector() start = time() deadline = start + timeout error = None def _raise(error): error = error or "connect() didn't finish in time" msg = ("Timed out trying to connect to %r after %s s: %s" % (addr, timeout, error)) raise IOError(msg) while True: try: future = connector.connect(loc, deserialize=deserialize, **(connection_args or {})) comm = yield gen.with_timeout(timedelta(seconds=deadline - time()), future, quiet_exceptions=EnvironmentError) except EnvironmentError as e: error = str(e) if time() < deadline: yield gen.sleep(0.01) logger.debug("sleeping on connect") else: _raise(error) except gen.TimeoutError: _raise(error) else: break raise gen.Return(comm) def listen(addr, handle_comm, deserialize=True, connection_args=None): """ Create a listener object with the given parameters. When its ``start()`` method is called, the listener will listen on the given address (a URI such as ``tcp://0.0.0.0``) and call *handle_comm* with a ``Comm`` object for each incoming connection. *handle_comm* can be a regular function or a coroutine. """ scheme, loc = parse_address(addr) backend = registry.get_backend(scheme) return backend.get_listener(loc, handle_comm, deserialize, **(connection_args or {})) distributed-1.20.2/distributed/comm/inproc.py000066400000000000000000000227361321233345200212470ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import deque, namedtuple import itertools import logging import os import threading import weakref from tornado import gen, locks from tornado.concurrent import Future from tornado.ioloop import IOLoop from ..compatibility import finalize from ..protocol import nested_deserialize from ..utils import get_ip from .registry import Backend, backends from .core import Comm, Connector, Listener, CommClosedError logger = logging.getLogger(__name__) ConnectionRequest = namedtuple('ConnectionRequest', ('c2s_q', 's2c_q', 'c_loop', 'c_addr', 'conn_event')) class Manager(object): """ An object coordinating listeners and their addresses. """ def __init__(self): self.listeners = weakref.WeakValueDictionary() self.addr_suffixes = itertools.count(1) self.ip = get_ip() self.lock = threading.Lock() def add_listener(self, addr, listener): with self.lock: if addr in self.listeners: raise RuntimeError("already listening on %r" % (addr,)) self.listeners[addr] = listener def remove_listener(self, addr): with self.lock: try: del self.listeners[addr] except KeyError: pass def get_listener_for(self, addr): with self.lock: self.validate_address(addr) return self.listeners.get(addr) def new_address(self): return "%s/%d/%s" % (self.ip, os.getpid(), next(self.addr_suffixes)) def validate_address(self, addr): """ Validate the address' IP and pid. """ ip, pid, suffix = addr.split('/') if ip != self.ip or int(pid) != os.getpid(): raise ValueError("inproc address %r does not match host (%r) or pid (%r)" % (addr, self.ip, os.getpid())) global_manager = Manager() def new_address(): """ Generate a new address. """ return 'inproc://' + global_manager.new_address() class QueueEmpty(Exception): pass class Queue(object): """ A single-reader, single-writer, non-threadsafe, peekable queue. """ def __init__(self): self._q = deque() self._read_future = None def get_nowait(self): q = self._q if not q: raise QueueEmpty return q.popleft() def get(self): assert not self._read_future, "Only one reader allowed" fut = Future() q = self._q if q: fut.set_result(q.popleft()) else: self._read_future = fut return fut def put_nowait(self, value): q = self._q fut = self._read_future if fut is not None: assert len(q) == 0 self._read_future = None fut.set_result(value) else: q.append(value) put = put_nowait _omitted = object() def peek(self, default=_omitted): """ Get the next object in the queue without removing it from the queue. """ q = self._q if q: return q[0] elif default is not self._omitted: return default else: raise QueueEmpty _EOF = object() class InProc(Comm): """ An established communication based on a pair of in-process queues. Reminder: a Comm must always be used from a single thread. Its peer Comm can be running in any thread. """ _initialized = False def __init__(self, local_addr, peer_addr, read_q, write_q, write_loop, deserialize=True): self._local_addr = local_addr self._peer_addr = peer_addr self.deserialize = deserialize self._read_q = read_q self._write_q = write_q self._write_loop = write_loop self._closed = False self._finalizer = finalize(self, self._get_finalizer()) self._finalizer.atexit = False self._initialized = True def _get_finalizer(self): def finalize(write_q=self._write_q, write_loop=self._write_loop, r=repr(self)): logger.warning("Closing dangling queue in %s" % (r,)) write_loop.add_callback(write_q.put_nowait, _EOF) return finalize @property def local_address(self): return self._local_addr @property def peer_address(self): return self._peer_addr @gen.coroutine def read(self): if self._closed: raise CommClosedError msg = yield self._read_q.get() if msg is _EOF: self._closed = True self._finalizer.detach() raise CommClosedError if self.deserialize: msg = nested_deserialize(msg) raise gen.Return(msg) @gen.coroutine def write(self, msg): if self.closed(): raise CommClosedError # Ensure we feed the queue in the same thread it is read from. self._write_loop.add_callback(self._write_q.put_nowait, msg) raise gen.Return(1) @gen.coroutine def close(self): self.abort() def abort(self): if not self.closed(): # Putting EOF is cheap enough that we do it on abort() too self._write_loop.add_callback(self._write_q.put_nowait, _EOF) self._read_q.put_nowait(_EOF) self._write_q = self._read_q = None self._closed = True self._finalizer.detach() def closed(self): """ Whether this comm is closed. An InProc comm is closed if: 1) close() or abort() was called on this comm 2) close() or abort() was called on the other end and the read queue is empty """ if self._closed: return True # NOTE: repr() is called by finalize() during __init__()... if self._initialized and self._read_q.peek(None) is _EOF: self._closed = True self._finalizer.detach() return True else: return False class InProcListener(Listener): prefix = 'inproc' def __init__(self, address, comm_handler, deserialize=True): self.manager = global_manager self.address = address or self.manager.new_address() self.comm_handler = comm_handler self.deserialize = deserialize self.listen_q = Queue() @gen.coroutine def _listen(self): while True: conn_req = yield self.listen_q.get() if conn_req is None: break comm = InProc(local_addr='inproc://' + self.address, peer_addr='inproc://' + conn_req.c_addr, read_q=conn_req.c2s_q, write_q=conn_req.s2c_q, write_loop=conn_req.c_loop, deserialize=self.deserialize) # Notify connector conn_req.c_loop.add_callback(conn_req.conn_event.set) self.comm_handler(comm) def connect_threadsafe(self, conn_req): self.loop.add_callback(self.listen_q.put_nowait, conn_req) def start(self): self.loop = IOLoop.current() self.loop.add_callback(self._listen) self.manager.add_listener(self.address, self) def stop(self): self.listen_q.put_nowait(None) self.manager.remove_listener(self.address) @property def listen_address(self): return 'inproc://' + self.address @property def contact_address(self): return 'inproc://' + self.address class InProcConnector(Connector): def __init__(self, manager): self.manager = manager @gen.coroutine def connect(self, address, deserialize=True, **connection_args): listener = self.manager.get_listener_for(address) if listener is None: raise IOError("no endpoint for inproc address %r" % (address,)) conn_req = ConnectionRequest(c2s_q=Queue(), s2c_q=Queue(), c_loop=IOLoop.current(), c_addr=self.manager.new_address(), conn_event=locks.Event(), ) listener.connect_threadsafe(conn_req) # Wait for connection acknowledgement # (do not pretend we're connected if the other comm never gets # created, for example if the listener was stopped in the meantime) yield conn_req.conn_event.wait() comm = InProc(local_addr='inproc://' + conn_req.c_addr, peer_addr='inproc://' + address, read_q=conn_req.s2c_q, write_q=conn_req.c2s_q, write_loop=listener.loop, deserialize=deserialize) raise gen.Return(comm) class InProcBackend(Backend): manager = global_manager # I/O def get_connector(self): return InProcConnector(self.manager) def get_listener(self, loc, handle_comm, deserialize, **connection_args): return InProcListener(loc, handle_comm, deserialize) # Address handling def get_address_host(self, loc): self.manager.validate_address(loc) return self.manager.ip def resolve_address(self, loc): return loc def get_local_address_for(self, loc): self.manager.validate_address(loc) return self.manager.new_address() backends['inproc'] = InProcBackend() distributed-1.20.2/distributed/comm/registry.py000066400000000000000000000034511321233345200216160ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from abc import ABCMeta, abstractmethod from six import with_metaclass class Backend(with_metaclass(ABCMeta)): """ A communication backend, selected by a given URI scheme (e.g. 'tcp'). """ # I/O @abstractmethod def get_connector(self): """ Get a connector object usable for connecting to addresses. """ @abstractmethod def get_listener(self, loc, handle_comm, deserialize, **connection_args): """ Get a listener object for the scheme-less address *loc*. """ # Address handling @abstractmethod def get_address_host(self, loc): """ Get a host name (normally an IP address) identifying the host the address is located on. *loc* is a scheme-less address. """ @abstractmethod def resolve_address(self, loc): """ Resolve the address into a canonical form. *loc* is a scheme-less address. Simple implementations may return *loc* unchanged. """ def get_address_host_port(self, loc): """ Get the (host, port) tuple of the scheme-less address *loc*. This should only be implemented by IP-based transports. """ raise NotImplementedError @abstractmethod def get_local_address_for(self, loc): """ Get the local listening address suitable for reaching *loc*. """ # The {scheme: Backend} mapping backends = {} def get_backend(scheme): """ Get the Backend instance for the given *scheme*. """ backend = backends.get(scheme) if backend is None: raise ValueError("unknown address scheme %r (known schemes: %s)" % (scheme, sorted(backends))) return backend distributed-1.20.2/distributed/comm/tcp.py000066400000000000000000000375211321233345200205410ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import errno import logging import socket import struct import sys try: import ssl except ImportError: ssl = None import tornado from tornado import gen, netutil from tornado.iostream import StreamClosedError from tornado.tcpclient import TCPClient from tornado.tcpserver import TCPServer from .. import config from ..compatibility import finalize from ..utils import (ensure_bytes, ensure_ip, get_ip, get_ipv6, nbytes) from .registry import Backend, backends from .addressing import parse_host_port, unparse_host_port from .core import Comm, Connector, Listener, CommClosedError from .utils import (to_frames, from_frames, get_tcp_server_address, ensure_concrete_host) logger = logging.getLogger(__name__) def get_total_physical_memory(): try: import psutil return psutil.virtual_memory().total / 2 except ImportError: return 2e9 MAX_BUFFER_SIZE = get_total_physical_memory() DEFAULT_BACKLOG = 2048 def set_tcp_timeout(stream): """ Set kernel-level TCP timeout on the stream. """ if stream.closed(): return timeout = int(config.get('tcp-timeout', 30)) sock = stream.socket # Default (unsettable) value on Windows # https://msdn.microsoft.com/en-us/library/windows/desktop/dd877220(v=vs.85).aspx nprobes = 10 assert timeout >= nprobes + 1, "Timeout too low" idle = max(2, timeout // 4) interval = max(1, (timeout - idle) // nprobes) idle = timeout - interval * nprobes assert idle > 0 try: if sys.platform.startswith("win"): logger.debug("Setting TCP keepalive: idle=%d, interval=%d", idle, interval) sock.ioctl(socket.SIO_KEEPALIVE_VALS, (1, idle * 1000, interval * 1000)) else: sock.setsockopt(socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1) try: TCP_KEEPIDLE = socket.TCP_KEEPIDLE TCP_KEEPINTVL = socket.TCP_KEEPINTVL TCP_KEEPCNT = socket.TCP_KEEPCNT except AttributeError: if sys.platform == "darwin": TCP_KEEPIDLE = 0x10 # (named "TCP_KEEPALIVE" in C) TCP_KEEPINTVL = 0x101 TCP_KEEPCNT = 0x102 else: TCP_KEEPIDLE = None if TCP_KEEPIDLE is not None: logger.debug("Setting TCP keepalive: nprobes=%d, idle=%d, interval=%d", nprobes, idle, interval) sock.setsockopt(socket.SOL_TCP, TCP_KEEPCNT, nprobes) sock.setsockopt(socket.SOL_TCP, TCP_KEEPIDLE, idle) sock.setsockopt(socket.SOL_TCP, TCP_KEEPINTVL, interval) if sys.platform.startswith("linux"): logger.debug("Setting TCP user timeout: %d ms", timeout * 1000) TCP_USER_TIMEOUT = 18 # since Linux 2.6.37 sock.setsockopt(socket.SOL_TCP, TCP_USER_TIMEOUT, timeout * 1000) except EnvironmentError as e: logger.warning("Could not set timeout on TCP stream: %s", e) def get_stream_address(stream): """ Get a stream's local address. """ if stream.closed(): return "" try: return unparse_host_port(*stream.socket.getsockname()[:2]) except EnvironmentError: # Probably EBADF return "" def convert_stream_closed_error(obj, exc): """ Re-raise StreamClosedError as CommClosedError. """ if exc.real_error is not None: # The stream was closed because of an underlying OS error exc = exc.real_error raise CommClosedError("in %s: %s: %s" % (obj, exc.__class__.__name__, exc)) else: raise CommClosedError("in %s: %s" % (obj, exc)) class TCP(Comm): """ An established communication based on an underlying Tornado IOStream. """ _iostream_allows_memoryview = tornado.version_info >= (4, 5) def __init__(self, stream, local_addr, peer_addr, deserialize=True): self._local_addr = local_addr self._peer_addr = peer_addr self.stream = stream self.deserialize = deserialize self._finalizer = finalize(self, self._get_finalizer()) self._finalizer.atexit = False self._extra = {} stream.set_nodelay(True) set_tcp_timeout(stream) self._read_extra() def _read_extra(self): pass def _get_finalizer(self): def finalize(stream=self.stream, r=repr(self)): if not stream.closed(): logger.warning("Closing dangling stream in %s" % (r,)) stream.close() return finalize @property def local_address(self): return self._local_addr @property def peer_address(self): return self._peer_addr @gen.coroutine def read(self): stream = self.stream if stream is None: raise CommClosedError try: n_frames = yield stream.read_bytes(8) n_frames = struct.unpack('Q', n_frames)[0] lengths = yield stream.read_bytes(8 * n_frames) lengths = struct.unpack('Q' * n_frames, lengths) frames = [] for length in lengths: if length: frame = yield stream.read_bytes(length) else: frame = b'' frames.append(frame) except StreamClosedError as e: self.stream = None convert_stream_closed_error(self, e) try: msg = yield from_frames(frames, deserialize=self.deserialize) except EOFError: # Frames possibly garbled or truncated by communication error self.abort() raise CommClosedError("aborted stream on truncated data") raise gen.Return(msg) @gen.coroutine def write(self, msg): stream = self.stream bytes_since_last_yield = 0 if stream is None: raise CommClosedError frames = yield to_frames(msg) try: lengths = ([struct.pack('Q', len(frames))] + [struct.pack('Q', nbytes(frame)) for frame in frames]) stream.write(b''.join(lengths)) for frame in frames: # Can't wait for the write() Future as it may be lost # ("If write is called again before that Future has resolved, # the previous future will be orphaned and will never resolve") if not self._iostream_allows_memoryview: frame = ensure_bytes(frame) future = stream.write(frame) bytes_since_last_yield += nbytes(frame) if bytes_since_last_yield > 32e6: yield future bytes_since_last_yield = 0 except StreamClosedError as e: stream = None convert_stream_closed_error(self, e) except TypeError as e: if stream._write_buffer is None: logger.info("tried to write message %s on closed stream", msg) else: raise raise gen.Return(sum(map(nbytes, frames))) @gen.coroutine def close(self): stream, self.stream = self.stream, None if stream is not None and not stream.closed(): try: # Flush the stream's write buffer by waiting for a last write. if stream.writing(): yield stream.write(b'') stream.socket.shutdown(socket.SHUT_RDWR) except EnvironmentError: pass finally: self._finalizer.detach() stream.close() def abort(self): stream, self.stream = self.stream, None if stream is not None and not stream.closed(): self._finalizer.detach() stream.close() def closed(self): return self.stream is None or self.stream.closed() @property def extra_info(self): return self._extra class TLS(TCP): """ A TLS-specific version of TCP. """ def _read_extra(self): TCP._read_extra(self) sock = self.stream.socket if sock is not None: self._extra.update(peercert=sock.getpeercert(), cipher=sock.cipher()) cipher, proto, bits = self._extra['cipher'] logger.debug("TLS connection with %r: protocol=%s, cipher=%s, bits=%d", self._peer_addr, proto, cipher, bits) def _expect_tls_context(connection_args): ctx = connection_args.get('ssl_context') if not isinstance(ctx, ssl.SSLContext): raise TypeError("TLS expects a `ssl_context` argument of type " "ssl.SSLContext (perhaps check your TLS configuration?)") return ctx class RequireEncryptionMixin(object): def _check_encryption(self, address, connection_args): if not self.encrypted and connection_args.get('require_encryption'): # XXX Should we have a dedicated SecurityError class? raise RuntimeError("encryption required by Dask configuration, " "refusing communication from/to %r" % (self.prefix + address,)) class BaseTCPConnector(Connector, RequireEncryptionMixin): @gen.coroutine def connect(self, address, deserialize=True, **connection_args): self._check_encryption(address, connection_args) ip, port = parse_host_port(address) kwargs = self._get_connect_args(**connection_args) client = TCPClient() try: stream = yield client.connect(ip, port, max_buffer_size=MAX_BUFFER_SIZE, **kwargs) except StreamClosedError as e: # The socket connect() call failed convert_stream_closed_error(self, e) local_address = self.prefix + get_stream_address(stream) raise gen.Return(self.comm_class(stream, local_address, self.prefix + address, deserialize)) class TCPConnector(BaseTCPConnector): prefix = 'tcp://' comm_class = TCP encrypted = False def _get_connect_args(self, **connection_args): return {} class TLSConnector(BaseTCPConnector): prefix = 'tls://' comm_class = TLS encrypted = True def _get_connect_args(self, **connection_args): ctx = _expect_tls_context(connection_args) return {'ssl_options': ctx} class BaseTCPListener(Listener, RequireEncryptionMixin): def __init__(self, address, comm_handler, deserialize=True, default_port=0, **connection_args): self._check_encryption(address, connection_args) self.ip, self.port = parse_host_port(address, default_port) self.comm_handler = comm_handler self.deserialize = deserialize self.server_args = self._get_server_args(**connection_args) self.tcp_server = None self.bound_address = None def start(self): self.tcp_server = TCPServer(max_buffer_size=MAX_BUFFER_SIZE, **self.server_args) self.tcp_server.handle_stream = self._handle_stream backlog = int(config.get('socket-backlog', DEFAULT_BACKLOG)) for i in range(5): try: # When shuffling data between workers, there can # really be O(cluster size) connection requests # on a single worker socket, make sure the backlog # is large enough not to lose any. sockets = netutil.bind_sockets(self.port, address=self.ip, backlog=backlog) except EnvironmentError as e: # EADDRINUSE can happen sporadically when trying to bind # to an ephemeral port if self.port != 0 or e.errno != errno.EADDRINUSE: raise exc = e else: self.tcp_server.add_sockets(sockets) break else: raise exc def stop(self): tcp_server, self.tcp_server = self.tcp_server, None if tcp_server is not None: tcp_server.stop() def _check_started(self): if self.tcp_server is None: raise ValueError("invalid operation on non-started TCPListener") @gen.coroutine def _handle_stream(self, stream, address): address = self.prefix + unparse_host_port(*address[:2]) stream = yield self._prepare_stream(stream, address) if stream is None: # Preparation failed return logger.debug("Incoming connection from %r to %r", address, self.contact_address) local_address = self.prefix + get_stream_address(stream) comm = self.comm_class(stream, local_address, address, self.deserialize) self.comm_handler(comm) def get_host_port(self): """ The listening address as a (host, port) tuple. """ self._check_started() if self.bound_address is None: self.bound_address = get_tcp_server_address(self.tcp_server) # IPv6 getsockname() can return more a 4-len tuple return self.bound_address[:2] @property def listen_address(self): """ The listening address as a string. """ return self.prefix + unparse_host_port(*self.get_host_port()) @property def contact_address(self): """ The contact address as a string. """ host, port = self.get_host_port() host = ensure_concrete_host(host) return self.prefix + unparse_host_port(host, port) class TCPListener(BaseTCPListener): prefix = 'tcp://' comm_class = TCP encrypted = False def _get_server_args(self, **connection_args): return {} @gen.coroutine def _prepare_stream(self, stream, address): raise gen.Return(stream) class TLSListener(BaseTCPListener): prefix = 'tls://' comm_class = TLS encrypted = True def _get_server_args(self, **connection_args): ctx = _expect_tls_context(connection_args) return {'ssl_options': ctx} @gen.coroutine def _prepare_stream(self, stream, address): try: yield stream.wait_for_handshake() except EnvironmentError as e: # The handshake went wrong, log and ignore logger.warning("Listener on %r: TLS handshake failed with remote %r: %s", self.listen_address, address, getattr(e, "real_error", None) or e) else: raise gen.Return(stream) class BaseTCPBackend(Backend): # I/O def get_connector(self): return self._connector_class() def get_listener(self, loc, handle_comm, deserialize, **connection_args): return self._listener_class(loc, handle_comm, deserialize, **connection_args) # Address handling 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) class TCPBackend(BaseTCPBackend): _connector_class = TCPConnector _listener_class = TCPListener class TLSBackend(BaseTCPBackend): _connector_class = TLSConnector _listener_class = TLSListener backends['tcp'] = TCPBackend() backends['tls'] = TLSBackend() distributed-1.20.2/distributed/comm/tests/000077500000000000000000000000001321233345200205335ustar00rootroot00000000000000distributed-1.20.2/distributed/comm/tests/test_comms.py000066400000000000000000000743701321233345200232750ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from functools import partial import os import sys import threading import pytest from tornado import gen, ioloop, locks, queues from tornado.concurrent import Future from distributed.metrics import time from distributed.utils import get_ip, get_ipv6 from distributed.utils_test import (gen_test, requires_ipv6, has_ipv6, get_cert, get_server_ssl_context, get_client_ssl_context) from distributed.utils_test import loop # flake8: noqa from distributed.protocol import (to_serialize, Serialized, serialize, deserialize) from distributed.comm import (tcp, inproc, connect, listen, CommClosedError, parse_address, parse_host_port, unparse_host_port, resolve_address, get_address_host, get_local_address_for) EXTERNAL_IP4 = get_ip() if has_ipv6(): EXTERNAL_IP6 = get_ipv6() ca_file = get_cert('tls-ca-cert.pem') # The Subject field of our test certs cert_subject = ( (('countryName', 'XY'),), (('localityName', 'Dask-distributed'),), (('organizationName', 'Dask'),), (('commonName', 'localhost'),) ) def check_tls_extra(info): assert isinstance(info, dict) assert info['peercert']['subject'] == cert_subject assert 'cipher' in info cipher_name, proto_name, secret_bits = info['cipher'] # Most likely assert 'AES' in cipher_name assert 'TLS' in proto_name assert secret_bits >= 128 tls_kwargs = dict(listen_args={'ssl_context': get_server_ssl_context()}, connect_args={'ssl_context': get_client_ssl_context()}) @gen.coroutine def get_comm_pair(listen_addr, listen_args=None, connect_args=None, **kwargs): q = queues.Queue() def handle_comm(comm): q.put(comm) listener = listen(listen_addr, handle_comm, connection_args=listen_args, **kwargs) listener.start() comm = yield connect(listener.contact_address, connection_args=connect_args, **kwargs) serv_comm = yield q.get() raise gen.Return((comm, serv_comm)) def get_tcp_comm_pair(**kwargs): return get_comm_pair('tcp://', **kwargs) def get_tls_comm_pair(**kwargs): kwargs.update(tls_kwargs) return get_comm_pair('tls://', **kwargs) def get_inproc_comm_pair(**kwargs): return get_comm_pair('inproc://', **kwargs) @gen.coroutine def debug_loop(): """ Debug helper """ while True: loop = ioloop.IOLoop.current() print('.', loop, loop._handlers) yield gen.sleep(0.50) # # Test utility functions # def test_parse_host_port(): f = parse_host_port assert f('localhost:123') == ('localhost', 123) assert f('127.0.0.1:456') == ('127.0.0.1', 456) assert f('localhost:123', 80) == ('localhost', 123) assert f('localhost', 80) == ('localhost', 80) with pytest.raises(ValueError): f('localhost') assert f('[::1]:123') == ('::1', 123) assert f('[fe80::1]:123', 80) == ('fe80::1', 123) assert f('[::1]', 80) == ('::1', 80) with pytest.raises(ValueError): f('[::1]') with pytest.raises(ValueError): f('::1:123') with pytest.raises(ValueError): f('::1') def test_unparse_host_port(): f = unparse_host_port assert f('localhost', 123) == 'localhost:123' assert f('127.0.0.1', 123) == '127.0.0.1:123' assert f('::1', 123) == '[::1]:123' 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:*' assert f('::1') == '[::1]' assert f('[::1]') == '[::1]' assert f('::1', '*') == '[::1]:*' def test_get_address_host(): f = get_address_host assert f('tcp://127.0.0.1:123') == '127.0.0.1' assert f('inproc://%s/%d/123' % (get_ip(), os.getpid())) == get_ip() def test_resolve_address(): f = resolve_address assert f('tcp://127.0.0.1:123') == 'tcp://127.0.0.1:123' assert f('127.0.0.2:789') == 'tcp://127.0.0.2:789' assert f('tcp://0.0.0.0:456') == 'tcp://0.0.0.0:456' assert f('tcp://0.0.0.0:456') == 'tcp://0.0.0.0:456' if has_ipv6(): assert f('tcp://[::1]:123') == 'tcp://[::1]:123' assert f('tls://[::1]:123') == 'tls://[::1]:123' # OS X returns '::0.0.0.2' as canonical representation assert f('[::2]:789') in ('tcp://[::2]:789', 'tcp://[::0.0.0.2]:789') assert f('tcp://[::]:123') == 'tcp://[::]:123' assert f('localhost:123') == 'tcp://127.0.0.1:123' assert f('tcp://localhost:456') == 'tcp://127.0.0.1:456' assert f('tls://localhost:456') == 'tls://127.0.0.1:456' def test_get_local_address_for(): f = get_local_address_for assert f('tcp://127.0.0.1:80') == 'tcp://127.0.0.1' assert f('tcp://8.8.8.8:4444') == 'tcp://' + get_ip() if has_ipv6(): assert f('tcp://[::1]:123') == 'tcp://[::1]' inproc_arg = 'inproc://%s/%d/444' % (get_ip(), os.getpid()) inproc_res = f(inproc_arg) assert inproc_res.startswith('inproc://') assert inproc_res != inproc_arg # # Test concrete transport APIs # @gen_test() def test_tcp_specific(): """ Test concrete TCP API. """ @gen.coroutine def handle_comm(comm): assert comm.peer_address.startswith('tcp://' + host) assert comm.extra_info == {} msg = yield comm.read() msg['op'] = 'pong' yield comm.write(msg) yield comm.close() listener = tcp.TCPListener('localhost', handle_comm) listener.start() host, port = listener.get_host_port() assert host in ('localhost', '127.0.0.1', '::1') assert port > 0 connector = tcp.TCPConnector() l = [] @gen.coroutine def client_communicate(key, delay=0): addr = '%s:%d' % (host, port) comm = yield connector.connect(addr) assert comm.peer_address == 'tcp://' + addr assert comm.extra_info == {} yield comm.write({'op': 'ping', 'data': key}) if delay: yield gen.sleep(delay) msg = yield comm.read() assert msg == {'op': 'pong', 'data': key} l.append(key) yield comm.close() yield client_communicate(key=1234) # Many clients at once N = 100 futures = [client_communicate(key=i, delay=0.05) for i in range(N)] yield futures assert set(l) == {1234} | set(range(N)) @gen_test() def test_tls_specific(): """ Test concrete TLS API. """ @gen.coroutine def handle_comm(comm): assert comm.peer_address.startswith('tls://' + host) check_tls_extra(comm.extra_info) msg = yield comm.read() msg['op'] = 'pong' yield comm.write(msg) yield comm.close() server_ctx = get_server_ssl_context() client_ctx = get_client_ssl_context() listener = tcp.TLSListener('localhost', handle_comm, ssl_context=server_ctx) listener.start() host, port = listener.get_host_port() assert host in ('localhost', '127.0.0.1', '::1') assert port > 0 connector = tcp.TLSConnector() l = [] @gen.coroutine def client_communicate(key, delay=0): addr = '%s:%d' % (host, port) comm = yield connector.connect(addr, ssl_context=client_ctx) assert comm.peer_address == 'tls://' + addr check_tls_extra(comm.extra_info) yield comm.write({'op': 'ping', 'data': key}) if delay: yield gen.sleep(delay) msg = yield comm.read() assert msg == {'op': 'pong', 'data': key} l.append(key) yield comm.close() yield client_communicate(key=1234) # Many clients at once N = 100 futures = [client_communicate(key=i, delay=0.05) for i in range(N)] yield futures assert set(l) == {1234} | set(range(N)) @gen.coroutine def check_inproc_specific(run_client): """ Test concrete InProc API. """ listener_addr = inproc.global_manager.new_address() addr_head = listener_addr.rpartition('/')[0] client_addresses = set() N_MSGS = 3 @gen.coroutine def handle_comm(comm): assert comm.peer_address.startswith('inproc://' + addr_head) client_addresses.add(comm.peer_address) for i in range(N_MSGS): msg = yield comm.read() msg['op'] = 'pong' yield comm.write(msg) yield comm.close() listener = inproc.InProcListener(listener_addr, handle_comm) listener.start() assert listener.listen_address == listener.contact_address == 'inproc://' + listener_addr connector = inproc.InProcConnector(inproc.global_manager) l = [] @gen.coroutine def client_communicate(key, delay=0): comm = yield connector.connect(listener_addr) assert comm.peer_address == 'inproc://' + listener_addr for i in range(N_MSGS): yield comm.write({'op': 'ping', 'data': key}) if delay: yield gen.sleep(delay) msg = yield comm.read() assert msg == {'op': 'pong', 'data': key} l.append(key) with pytest.raises(CommClosedError): yield comm.read() yield comm.close() client_communicate = partial(run_client, client_communicate) yield client_communicate(key=1234) # Many clients at once N = 20 futures = [client_communicate(key=i, delay=0.001) for i in range(N)] yield futures assert set(l) == {1234} | set(range(N)) assert len(client_addresses) == N + 1 assert listener.contact_address not in client_addresses def run_coro(func, *args, **kwargs): return func(*args, **kwargs) def run_coro_in_thread(func, *args, **kwargs): fut = Future() main_loop = ioloop.IOLoop.current() def run(): thread_loop = ioloop.IOLoop() # need fresh IO loop for run_sync() try: res = thread_loop.run_sync(partial(func, *args, **kwargs), timeout=10) except Exception: main_loop.add_callback(fut.set_exc_info, sys.exc_info()) else: main_loop.add_callback(fut.set_result, res) finally: thread_loop.close() t = threading.Thread(target=run) t.start() return fut @gen_test() def test_inproc_specific_same_thread(): yield check_inproc_specific(run_coro) @gen_test() def test_inproc_specific_different_threads(): yield check_inproc_specific(run_coro_in_thread) # # Test communications through the abstract API # @gen.coroutine def check_client_server(addr, check_listen_addr=None, check_contact_addr=None, listen_args=None, connect_args=None): """ Abstract client / server test. """ @gen.coroutine def handle_comm(comm): scheme, loc = parse_address(comm.peer_address) assert scheme == bound_scheme msg = yield comm.read() assert msg['op'] == 'ping' msg['op'] = 'pong' yield comm.write(msg) msg = yield comm.read() assert msg['op'] == 'foobar' yield comm.close() # Arbitrary connection args should be ignored listen_args = listen_args or {'xxx': 'bar'} connect_args = connect_args or {'xxx': 'foo'} listener = listen(addr, handle_comm, connection_args=listen_args) listener.start() # 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 == parse_address(addr)[0] if check_listen_addr is not None: check_listen_addr(bound_loc) contact_addr = listener.contact_address contact_scheme, contact_loc = parse_address(contact_addr) assert contact_scheme == bound_scheme if check_contact_addr is not None: check_contact_addr(contact_loc) else: assert contact_addr == bound_addr # Check client <-> server comms l = [] @gen.coroutine def client_communicate(key, delay=0): comm = yield connect(listener.contact_address, connection_args=connect_args) assert comm.peer_address == listener.contact_address yield comm.write({'op': 'ping', 'data': key}) yield comm.write({'op': 'foobar'}) if delay: yield gen.sleep(delay) msg = yield comm.read() assert msg == {'op': 'pong', 'data': key} l.append(key) yield comm.close() yield client_communicate(key=1234) # Many clients at once futures = [client_communicate(key=i, delay=0.05) for i in range(20)] yield futures assert set(l) == {1234} | set(range(20)) listener.stop() def tcp_eq(expected_host, expected_port=None): def checker(loc): host, port = parse_host_port(loc) assert host == expected_host if expected_port is not None: assert port == expected_port else: assert 1023 < port < 65536 return checker tls_eq = tcp_eq def inproc_check(): expected_ip = get_ip() expected_pid = os.getpid() def checker(loc): ip, pid, suffix = loc.split('/') assert ip == expected_ip assert int(pid) == expected_pid return checker @gen_test() def test_default_client_server_ipv4(): # Default scheme is (currently) TCP yield check_client_server('127.0.0.1', tcp_eq('127.0.0.1')) yield check_client_server('127.0.0.1:3201', tcp_eq('127.0.0.1', 3201)) yield check_client_server('0.0.0.0', tcp_eq('0.0.0.0'), tcp_eq(EXTERNAL_IP4)) yield check_client_server('0.0.0.0:3202', tcp_eq('0.0.0.0', 3202), tcp_eq(EXTERNAL_IP4, 3202)) # IPv4 is preferred for the bound address yield check_client_server('', tcp_eq('0.0.0.0'), tcp_eq(EXTERNAL_IP4)) yield check_client_server(':3203', tcp_eq('0.0.0.0', 3203), tcp_eq(EXTERNAL_IP4, 3203)) @requires_ipv6 @gen_test() def test_default_client_server_ipv6(): yield check_client_server('[::1]', tcp_eq('::1')) yield check_client_server('[::1]:3211', tcp_eq('::1', 3211)) yield check_client_server('[::]', tcp_eq('::'), tcp_eq(EXTERNAL_IP6)) yield check_client_server('[::]:3212', tcp_eq('::', 3212), tcp_eq(EXTERNAL_IP6, 3212)) @gen_test() def test_tcp_client_server_ipv4(): yield check_client_server('tcp://127.0.0.1', tcp_eq('127.0.0.1')) yield check_client_server('tcp://127.0.0.1:3221', tcp_eq('127.0.0.1', 3221)) yield check_client_server('tcp://0.0.0.0', tcp_eq('0.0.0.0'), tcp_eq(EXTERNAL_IP4)) yield check_client_server('tcp://0.0.0.0:3222', tcp_eq('0.0.0.0', 3222), tcp_eq(EXTERNAL_IP4, 3222)) yield check_client_server('tcp://', tcp_eq('0.0.0.0'), tcp_eq(EXTERNAL_IP4)) yield check_client_server('tcp://:3223', tcp_eq('0.0.0.0', 3223), tcp_eq(EXTERNAL_IP4, 3223)) @requires_ipv6 @gen_test() def test_tcp_client_server_ipv6(): yield check_client_server('tcp://[::1]', tcp_eq('::1')) yield check_client_server('tcp://[::1]:3231', tcp_eq('::1', 3231)) yield check_client_server('tcp://[::]', tcp_eq('::'), tcp_eq(EXTERNAL_IP6)) yield check_client_server('tcp://[::]:3232', tcp_eq('::', 3232), tcp_eq(EXTERNAL_IP6, 3232)) @gen_test() def test_tls_client_server_ipv4(): yield check_client_server('tls://127.0.0.1', tls_eq('127.0.0.1'), **tls_kwargs) yield check_client_server('tls://127.0.0.1:3221', tls_eq('127.0.0.1', 3221), **tls_kwargs) yield check_client_server('tls://', tls_eq('0.0.0.0'), tls_eq(EXTERNAL_IP4), **tls_kwargs) @requires_ipv6 @gen_test() def test_tls_client_server_ipv6(): yield check_client_server('tls://[::1]', tls_eq('::1'), **tls_kwargs) @gen_test() def test_inproc_client_server(): yield check_client_server('inproc://', inproc_check()) yield check_client_server(inproc.new_address(), inproc_check()) # # TLS certificate handling # @gen_test() def test_tls_reject_certificate(): cli_ctx = get_client_ssl_context() serv_ctx = get_server_ssl_context() # These certs are not signed by our test CA bad_cert_key = ('tls-self-signed-cert.pem', 'tls-self-signed-key.pem') bad_cli_ctx = get_client_ssl_context(*bad_cert_key) bad_serv_ctx = get_server_ssl_context(*bad_cert_key) @gen.coroutine def handle_comm(comm): scheme, loc = parse_address(comm.peer_address) assert scheme == 'tls' yield comm.close() # Listener refuses a connector not signed by the CA listener = listen('tls://', handle_comm, connection_args={'ssl_context': serv_ctx}) listener.start() with pytest.raises(EnvironmentError) as excinfo: yield connect(listener.contact_address, timeout=0.5, connection_args={'ssl_context': bad_cli_ctx}) # The wrong error is reported on Python 2, see https://github.com/tornadoweb/tornado/pull/2028 if sys.version_info >= (3,) and os.name != 'nt': try: # See https://serverfault.com/questions/793260/what-does-tlsv1-alert-unknown-ca-mean assert "unknown ca" in str(excinfo.value) except AssertionError: if os.name == 'nt': assert "An existing connection was forcibly closed" in str(excinfo.value) else: raise # Sanity check comm = yield connect(listener.contact_address, timeout=0.5, connection_args={'ssl_context': cli_ctx}) yield comm.close() # Connector refuses a listener not signed by the CA listener = listen('tls://', handle_comm, connection_args={'ssl_context': bad_serv_ctx}) listener.start() with pytest.raises(EnvironmentError) as excinfo: yield connect(listener.contact_address, timeout=0.5, connection_args={'ssl_context': cli_ctx}) # The wrong error is reported on Python 2, see https://github.com/tornadoweb/tornado/pull/2028 if sys.version_info >= (3,): assert "certificate verify failed" in str(excinfo.value) # # Test communication closing # @gen.coroutine def check_comm_closed_implicit(addr, delay=None, listen_args=None, connect_args=None): @gen.coroutine def handle_comm(comm): yield comm.close() listener = listen(addr, handle_comm, connection_args=listen_args) listener.start() contact_addr = listener.contact_address comm = yield connect(contact_addr, connection_args=connect_args) with pytest.raises(CommClosedError): yield comm.write({}) comm = yield connect(contact_addr, connection_args=connect_args) with pytest.raises(CommClosedError): yield comm.read() @gen_test() def test_tcp_comm_closed_implicit(): yield check_comm_closed_implicit('tcp://127.0.0.1') @gen_test() def test_tls_comm_closed_implicit(): yield check_comm_closed_implicit('tls://127.0.0.1', **tls_kwargs) @gen_test() def test_inproc_comm_closed_implicit(): yield check_comm_closed_implicit(inproc.new_address()) @gen.coroutine def check_comm_closed_explicit(addr, listen_args=None, connect_args=None): a, b = yield get_comm_pair(addr, listen_args=listen_args, connect_args=connect_args) a_read = a.read() b_read = b.read() yield a.close() # In-flight reads should abort with CommClosedError with pytest.raises(CommClosedError): yield a_read with pytest.raises(CommClosedError): yield b_read # New reads as well with pytest.raises(CommClosedError): yield a.read() with pytest.raises(CommClosedError): yield b.read() # And writes with pytest.raises(CommClosedError): yield a.write({}) with pytest.raises(CommClosedError): yield b.write({}) yield b.close() @gen_test() def test_tcp_comm_closed_explicit(): yield check_comm_closed_explicit('tcp://127.0.0.1') @gen_test() def test_tls_comm_closed_explicit(): yield check_comm_closed_explicit('tls://127.0.0.1', **tls_kwargs) @gen_test() def test_inproc_comm_closed_explicit(): yield check_comm_closed_explicit(inproc.new_address()) @gen_test() def test_inproc_comm_closed_explicit_2(): listener_errors = [] @gen.coroutine def handle_comm(comm): # Wait try: yield comm.read() except CommClosedError: assert comm.closed() listener_errors.append(True) else: comm.close() listener = listen('inproc://', handle_comm) listener.start() contact_addr = listener.contact_address comm = yield connect(contact_addr) comm.close() assert comm.closed() yield gen.sleep(0.01) assert len(listener_errors) == 1 with pytest.raises(CommClosedError): yield comm.read() with pytest.raises(CommClosedError): yield comm.write("foo") comm = yield connect(contact_addr) comm.write("foo") with pytest.raises(CommClosedError): yield comm.read() with pytest.raises(CommClosedError): yield comm.write("foo") assert comm.closed() comm = yield connect(contact_addr) comm.write("foo") yield gen.sleep(0.01) assert comm.closed() comm.close() comm.close() # # Various stress tests # @gen.coroutine def check_connect_timeout(addr): t1 = time() with pytest.raises(IOError): yield connect(addr, timeout=0.15) dt = time() - t1 assert 0.5 >= dt >= 0.1 @gen_test() def test_tcp_connect_timeout(): yield check_connect_timeout('tcp://127.0.0.1:44444') @gen_test() def test_inproc_connect_timeout(): yield check_connect_timeout(inproc.new_address()) def check_many_listeners(addr): @gen.coroutine def handle_comm(comm): pass listeners = [] N = 100 for i in range(N): listener = listen(addr, handle_comm) listener.start() listeners.append(listener) assert len(set(l.listen_address for l in listeners)) == N assert len(set(l.contact_address for l in listeners)) == N for listener in listeners: listener.stop() @gen_test() def test_tcp_many_listeners(): check_many_listeners('tcp://127.0.0.1') check_many_listeners('tcp://0.0.0.0') check_many_listeners('tcp://') @gen_test() def test_inproc_many_listeners(): check_many_listeners('inproc://') # # Test deserialization # @gen.coroutine def check_listener_deserialize(addr, deserialize, in_value, check_out): q = queues.Queue() @gen.coroutine def handle_comm(comm): msg = yield comm.read() q.put_nowait(msg) yield comm.close() with listen(addr, handle_comm, deserialize=deserialize) as listener: comm = yield connect(listener.contact_address) yield comm.write(in_value) out_value = yield q.get() check_out(out_value) yield comm.close() @gen.coroutine def check_connector_deserialize(addr, deserialize, in_value, check_out): done = locks.Event() @gen.coroutine def handle_comm(comm): yield comm.write(in_value) yield done.wait() yield comm.close() with listen(addr, handle_comm) as listener: comm = yield connect(listener.contact_address, deserialize=deserialize) out_value = yield comm.read() done.set() yield comm.close() check_out(out_value) @gen.coroutine def check_deserialize(addr): """ Check the "deserialize" flag on connect() and listen(). """ # Test with Serialize and Serialized objects msg = {'op': 'update', 'x': b'abc', 'to_ser': [to_serialize(123)], 'ser': Serialized(*serialize(456)), } msg_orig = msg.copy() def check_out_false(out_value): # Check output with deserialize=False out_value = out_value.copy() # in case transport passed the object as-is to_ser = out_value.pop('to_ser') ser = out_value.pop('ser') expected_msg = msg_orig.copy() del expected_msg['ser'] del expected_msg['to_ser'] assert out_value == expected_msg assert isinstance(ser, Serialized) assert deserialize(ser.header, ser.frames) == 456 assert isinstance(to_ser, list) to_ser, = to_ser # The to_serialize() value could have been actually serialized # or not (it's a transport-specific optimization) if isinstance(to_ser, Serialized): assert deserialize(to_ser.header, to_ser.frames) == 123 else: assert to_ser == to_serialize(123) def check_out_true(out_value): # Check output with deserialize=True expected_msg = msg.copy() expected_msg['ser'] = 456 expected_msg['to_ser'] = [123] assert out_value == expected_msg yield check_listener_deserialize(addr, False, msg, check_out_false) yield check_connector_deserialize(addr, False, msg, check_out_false) yield check_listener_deserialize(addr, True, msg, check_out_true) yield check_connector_deserialize(addr, True, msg, check_out_true) # Test with long bytestrings, large enough to be transferred # as a separate payload _uncompressible = os.urandom(1024 ** 2) * 4 # end size: 8 MB msg = {'op': 'update', 'x': _uncompressible, 'to_ser': [to_serialize(_uncompressible)], 'ser': Serialized(*serialize(_uncompressible)), } msg_orig = msg.copy() def check_out(deserialize_flag, out_value): # Check output with deserialize=False assert sorted(out_value) == sorted(msg_orig) out_value = out_value.copy() # in case transport passed the object as-is to_ser = out_value.pop('to_ser') ser = out_value.pop('ser') expected_msg = msg_orig.copy() del expected_msg['ser'] del expected_msg['to_ser'] assert out_value == expected_msg if deserialize_flag: assert isinstance(ser, (bytes, bytearray)) assert bytes(ser) == _uncompressible else: assert isinstance(ser, Serialized) assert deserialize(ser.header, ser.frames) == _uncompressible assert isinstance(to_ser, list) to_ser, = to_ser # The to_serialize() value could have been actually serialized # or not (it's a transport-specific optimization) if isinstance(to_ser, Serialized): assert deserialize(to_ser.header, to_ser.frames) == _uncompressible else: assert to_ser == to_serialize(_uncompressible) yield check_listener_deserialize(addr, False, msg, partial(check_out, False)) yield check_connector_deserialize(addr, False, msg, partial(check_out, False)) yield check_listener_deserialize(addr, True, msg, partial(check_out, True)) yield check_connector_deserialize(addr, True, msg, partial(check_out, True)) @gen_test() def test_tcp_deserialize(): yield check_deserialize('tcp://') @gen_test() def test_inproc_deserialize(): yield check_deserialize('inproc://') @gen.coroutine def check_deserialize_roundtrip(addr): """ Sanity check round-tripping with "deserialize" on and off. """ # Test with long bytestrings, large enough to be transferred # as a separate payload _uncompressible = os.urandom(1024 ** 2) * 4 # end size: 4 MB msg = {'op': 'update', 'x': _uncompressible, 'to_ser': [to_serialize(_uncompressible)], 'ser': Serialized(*serialize(_uncompressible)), } for deserialize in (True, False): a, b = yield get_comm_pair(addr, deserialize=deserialize) yield a.write(msg) got = yield b.read() yield b.write(got) got = yield a.read() assert sorted(got) == sorted(msg) for k in ('op', 'x'): assert got[k] == msg[k] if deserialize: assert isinstance(got['to_ser'][0], (bytes, bytearray)) assert isinstance(got['ser'], (bytes, bytearray)) else: assert isinstance(got['to_ser'][0], (to_serialize, Serialized)) assert isinstance(got['ser'], Serialized) @gen_test() def test_inproc_deserialize_roundtrip(): yield check_deserialize_roundtrip('inproc://') @gen_test() def test_tcp_deserialize_roundtrip(): yield check_deserialize_roundtrip('tcp://') def _raise_eoferror(): raise EOFError class _EOFRaising(object): def __reduce__(self): return _raise_eoferror, () @gen.coroutine def check_deserialize_eoferror(addr): """ EOFError when deserializing should close the comm. """ @gen.coroutine def handle_comm(comm): yield comm.write({'data': to_serialize(_EOFRaising())}) with pytest.raises(CommClosedError): yield comm.read() with listen(addr, handle_comm) as listener: comm = yield connect(listener.contact_address, deserialize=deserialize) with pytest.raises(CommClosedError): yield comm.read() @gen_test() def test_tcp_deserialize_eoferror(): yield check_deserialize_eoferror('tcp://') # # Test various properties # @gen.coroutine def check_repr(a, b): assert 'closed' not in repr(a) assert 'closed' not in repr(b) yield a.close() assert 'closed' in repr(a) yield b.close() assert 'closed' in repr(b) @gen_test() def test_tcp_repr(): a, b = yield get_tcp_comm_pair() assert a.local_address in repr(b) assert b.local_address in repr(a) yield check_repr(a, b) @gen_test() def test_tls_repr(): a, b = yield get_tls_comm_pair() assert a.local_address in repr(b) assert b.local_address in repr(a) yield check_repr(a, b) @gen_test() def test_inproc_repr(): a, b = yield get_inproc_comm_pair() assert a.local_address in repr(b) assert b.local_address in repr(a) yield check_repr(a, b) @gen.coroutine def check_addresses(a, b): assert a.peer_address == b.local_address assert a.local_address == b.peer_address a.abort() b.abort() @gen_test() def test_tcp_adresses(): a, b = yield get_tcp_comm_pair() yield check_addresses(a, b) @gen_test() def test_tls_adresses(): a, b = yield get_tls_comm_pair() yield check_addresses(a, b) @gen_test() def test_inproc_adresses(): a, b = yield get_inproc_comm_pair() yield check_addresses(a, b) distributed-1.20.2/distributed/comm/utils.py000066400000000000000000000060001321233345200210770ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from concurrent.futures import ThreadPoolExecutor import logging import socket from tornado import gen from .. import protocol from ..compatibility import finalize from ..sizeof import sizeof from ..utils import get_ip, get_ipv6, mp_context, nbytes logger = logging.getLogger(__name__) # Offload (de)serializing large frames to improve event loop responsiveness. # We use at most 4 threads to allow for parallel processing of large messages. FRAME_OFFLOAD_THRESHOLD = 10 * 1024 ** 2 # 10 MB _offload_executor = ThreadPoolExecutor(max_workers=min(4, mp_context.cpu_count())) finalize(_offload_executor, _offload_executor.shutdown) def offload(fn, *args, **kwargs): return _offload_executor.submit(fn, *args, **kwargs) @gen.coroutine def to_frames(msg): """ Serialize a message into a list of Distributed protocol frames. """ def _to_frames(): try: return list(protocol.dumps(msg)) except Exception as e: logger.info("Unserializable Message: %s", msg) logger.exception(e) raise if sizeof(msg) > FRAME_OFFLOAD_THRESHOLD: res = yield offload(_to_frames) else: res = _to_frames() raise gen.Return(res) @gen.coroutine def from_frames(frames, deserialize=True): """ Unserialize a list of Distributed protocol frames. """ size = sum(map(nbytes, frames)) def _from_frames(): try: return protocol.loads(frames, deserialize=deserialize) except EOFError: if size > 1000: datastr = "[too large to display]" else: datastr = frames # Aid diagnosing logger.error("truncated data stream (%d bytes): %s", size, datastr) raise if deserialize and size > FRAME_OFFLOAD_THRESHOLD: res = yield offload(_from_frames) else: res = _from_frames() raise gen.Return(res) def get_tcp_server_address(tcp_server): """ Get the bound address of a started Tornado TCPServer. """ sockets = list(tcp_server._sockets.values()) if not sockets: raise RuntimeError("TCP Server %r not started yet?" % (tcp_server,)) def _look_for_family(fam): for sock in sockets: if sock.family == fam: return sock return None # If listening on both IPv4 and IPv6, prefer IPv4 as defective IPv6 # is common (e.g. Travis-CI). sock = _look_for_family(socket.AF_INET) if sock is None: sock = _look_for_family(socket.AF_INET6) if sock is None: raise RuntimeError("No Internet socket found on TCPServer??") return sock.getsockname() def ensure_concrete_host(host): """ Ensure the given host string (or IP) denotes a concrete host, not a wildcard listening address. """ if host in ('0.0.0.0', ''): return get_ip() elif host == '::': return get_ipv6() else: return host distributed-1.20.2/distributed/compatibility.py000066400000000000000000000172311321233345200216650ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging import sys if sys.version_info[0] == 2: from Queue import Queue, Empty from io import BytesIO from thread import get_ident as get_thread_identity from inspect import getargspec from cgi import escape as html_escape reload = reload # flake8: noqa unicode = unicode # flake8: noqa PY2 = True PY3 = False ConnectionRefusedError = OSError FileExistsError = OSError import gzip def gzip_decompress(b): f = gzip.GzipFile(fileobj=BytesIO(b)) result = f.read() f.close() return result def gzip_compress(b): bio = BytesIO() f = gzip.GzipFile(fileobj=bio, mode='w') f.write(b) f.close() bio.seek(0) result = bio.read() return result def isqueue(o): return (hasattr(o, 'queue') and hasattr(o, '__module__') and o.__module__ == 'Queue') def invalidate_caches(): pass def cache_from_source(path): import os name, ext = os.path.splitext(path) return name + '.pyc' logging_names = logging._levelNames if sys.version_info[0] == 3: from queue import Queue, Empty # flake8: noqa from importlib import reload from threading import get_ident as get_thread_identity from importlib import invalidate_caches from importlib.util import cache_from_source from inspect import getfullargspec as getargspec from html import escape as html_escape PY2 = False PY3 = True unicode = str from gzip import decompress as gzip_decompress from gzip import compress as gzip_compress ConnectionRefusedError = ConnectionRefusedError FileExistsError = FileExistsError def isqueue(o): return isinstance(o, Queue) logging_names = logging._levelToName.copy() logging_names.update(logging._nameToLevel) WINDOWS = sys.platform.startswith('win') try: from json.decoder import JSONDecodeError except (ImportError, AttributeError): JSONDecodeError = ValueError try: from functools import singledispatch except ImportError: from singledispatch import singledispatch try: from weakref import finalize except ImportError: # Backported from Python 3.6 import itertools from weakref import ref class finalize(object): """Class for finalization of weakrefable objects finalize(obj, func, *args, **kwargs) returns a callable finalizer object which will be called when obj is garbage collected. The first time the finalizer is called it evaluates func(*arg, **kwargs) and returns the result. After this the finalizer is dead, and calling it just returns None. When the program exits any remaining finalizers for which the atexit attribute is true will be run in reverse order of creation. By default atexit is true. """ # Finalizer objects don't have any state of their own. They are # just used as keys to lookup _Info objects in the registry. This # ensures that they cannot be part of a ref-cycle. __slots__ = () _registry = {} _shutdown = False _index_iter = itertools.count() _dirty = False _registered_with_atexit = False class _Info: __slots__ = ("weakref", "func", "args", "kwargs", "atexit", "index") def __init__(self, obj, func, *args, **kwargs): if not self._registered_with_atexit: # We may register the exit function more than once because # of a thread race, but that is harmless import atexit atexit.register(self._exitfunc) finalize._registered_with_atexit = True info = self._Info() info.weakref = ref(obj, self) info.func = func info.args = args info.kwargs = kwargs or None info.atexit = True info.index = next(self._index_iter) self._registry[self] = info finalize._dirty = True def __call__(self, _=None): """If alive then mark as dead and return func(*args, **kwargs); otherwise return None""" info = self._registry.pop(self, None) if info and not self._shutdown: return info.func(*info.args, **(info.kwargs or {})) def detach(self): """If alive then mark as dead and return (obj, func, args, kwargs); otherwise return None""" info = self._registry.get(self) obj = info and info.weakref() if obj is not None and self._registry.pop(self, None): return (obj, info.func, info.args, info.kwargs or {}) def peek(self): """If alive then return (obj, func, args, kwargs); otherwise return None""" info = self._registry.get(self) obj = info and info.weakref() if obj is not None: return (obj, info.func, info.args, info.kwargs or {}) @property def alive(self): """Whether finalizer is alive""" return self in self._registry @property def atexit(self): """Whether finalizer should be called at exit""" info = self._registry.get(self) return bool(info) and info.atexit @atexit.setter def atexit(self, value): info = self._registry.get(self) if info: info.atexit = bool(value) def __repr__(self): info = self._registry.get(self) obj = info and info.weakref() if obj is None: return '<%s object at %#x; dead>' % (type(self).__name__, id(self)) else: return '<%s object at %#x; for %r at %#x>' % \ (type(self).__name__, id(self), type(obj).__name__, id(obj)) @classmethod def _select_for_exit(cls): # Return live finalizers marked for exit, oldest first L = [(f, i) for (f, i) in cls._registry.items() if i.atexit] L.sort(key=lambda item: item[1].index) return [f for (f, i) in L] @classmethod def _exitfunc(cls): # At shutdown invoke finalizers for which atexit is true. # This is called once all other non-daemonic threads have been # joined. reenable_gc = False try: if cls._registry: import gc if gc.isenabled(): reenable_gc = True gc.disable() pending = None while True: if pending is None or finalize._dirty: pending = cls._select_for_exit() finalize._dirty = False if not pending: break f = pending.pop() try: # gc is disabled, so (assuming no daemonic # threads) the following is the only line in # this function which might trigger creation # of a new finalizer f() except Exception: sys.excepthook(*sys.exc_info()) assert f not in cls._registry finally: # prevent any more finalizers from executing during shutdown finalize._shutdown = True if reenable_gc: gc.enable() distributed-1.20.2/distributed/config.py000066400000000000000000000115601321233345200202600ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from contextlib import contextmanager import logging import logging.config import os import sys import warnings from .compatibility import FileExistsError, logging_names logger = logging.getLogger(__name__) config = {} def ensure_config_file(source, destination): if not os.path.exists(destination): import shutil if not os.path.exists(os.path.dirname(destination)): try: os.mkdir(os.path.dirname(destination)) except FileExistsError: pass # Atomically create destination. Parallel testing discovered # a race condition where a process can be busy creating the # destination while another process reads an empty config file. tmp = '%s.tmp.%d' % (destination, os.getpid()) shutil.copy(source, tmp) try: os.rename(tmp, destination) except OSError: os.remove(tmp) def determine_config_file(): path = os.environ.get('DASK_CONFIG') if path: if (os.path.exists(path) and (os.path.isfile(path) or os.path.islink(path))): return path warnings.warn("DASK_CONFIG set to '%s' but file does not exist " "or is not a regular file" % (path,), UserWarning) dirname = os.path.dirname(__file__) default_path = os.path.join(dirname, 'config.yaml') path = os.path.join(os.path.expanduser('~'), '.dask', 'config.yaml') try: ensure_config_file(default_path, path) except EnvironmentError as e: warnings.warn("Could not write default config file to '%s'. " "Received error %s" % (path, e), UserWarning) return path if os.path.exists(path) else default_path def load_config_file(config, path): with open(path) as f: text = f.read() config.update(yaml.load(text)) def load_env_vars(config): for name, value in os.environ.items(): if name.startswith('DASK_'): varname = name[5:].lower().replace('_', '-') config[varname] = value def _initialize_logging_old_style(config): """ Initialize logging using the "old-style" configuration scheme, e.g.: { 'logging': { 'distributed': 'info', 'tornado': 'critical', 'tornado.application': 'error', } } """ loggers = config.get('logging', {}) loggers.setdefault('distributed', 'info') # We could remove those lines and let the default config.yaml handle it loggers.setdefault('tornado', 'critical') loggers.setdefault('tornado.application', 'error') handler = logging.StreamHandler(sys.stderr) handler.setFormatter(logging.Formatter(log_format)) for name, level in loggers.items(): if isinstance(level, str): level = logging_names[level.upper()] logger = logging.getLogger(name) logger.setLevel(level) logger.handlers[:] = [] logger.addHandler(handler) logger.propagate = False def _initialize_logging_new_style(config): """ Initialize logging using logging's "Configuration dictionary schema". (ref.: https://docs.python.org/2/library/logging.config.html#logging-config-dictschema) """ logging.config.dictConfig(config['logging']) def _initialize_logging_file_config(config): """ Initialize logging using logging's "Configuration file format". (ref.: https://docs.python.org/2/library/logging.config.html#configuration-file-format) """ logging.config.fileConfig(config['logging-file-config'], disable_existing_loggers=False) def initialize_logging(config): if 'logging-file-config' in config: if 'logging' in config: raise RuntimeError("Config options 'logging-file-config' and 'logging' are mutually exclusive.") _initialize_logging_file_config(config) else: log_config = config.get('logging', {}) if 'version' in log_config: # logging module mandates version to be an int log_config['version'] = int(log_config['version']) _initialize_logging_new_style(config) else: _initialize_logging_old_style(config) @contextmanager def set_config(**kwargs): old = {} for key in kwargs: if key in config: old[key] = config[key] for key, value in kwargs.items(): config[key] = value try: yield finally: for key in kwargs: if key in old: config[key] = old[key] else: del config[key] try: import yaml except ImportError: pass else: path = determine_config_file() load_config_file(config, path) load_env_vars(config) log_format = config.get('log-format', '%(name)s - %(levelname)s - %(message)s') initialize_logging(config) distributed-1.20.2/distributed/config.yaml000066400000000000000000000040521321233345200205700ustar00rootroot00000000000000logging: distributed: info distributed.client: warning bokeh: critical # http://stackoverflow.com/questions/21234772/python-tornado-disable-logging-to-stderr tornado: critical tornado.application: error compression: auto # Scheduler options bandwidth: 100000000 # 100 MB/s estimated worker-worker bandwidth allowed-failures: 3 # number of retries before a task is considered bad pdb-on-err: False # enter debug mode on scheduling error transition-log-length: 100000 work-stealing: True # workers should steal tasks from each other # Worker options multiprocessing-method: forkserver use-file-locking: True # Communication options connect-timeout: 3 # seconds delay before connecting fails tcp-timeout: 30 # seconds delay before calling an unresponsive connection dead default-scheme: tcp require-encryption: False # whether to require encryption on non-local comms socket-backlog: 2048 #tls: #ca-file: xxx.pem #scheduler: #key: xxx.pem #cert: xxx.pem #worker: #key: xxx.pem #cert: xxx.pem #client: #key: xxx.pem #cert: xxx.pem #ciphers: #ECDHE-ECDSA-AES128-GCM-SHA256 # Bokeh web dashboard bokeh-export-tool: False tick-time: 20 # milliseconds between event loop health checks tick-maximum-delay: 1000 # milliseconds allowed before triggering a warning profile-interval: 10 # milliseconds in between statistical profiling queries profile-cycle-interval: 1000 # milliseconds between starting new profile # Fractions of worker memory at which we take action to avoid memory blowup # Set any of the lower three values to False to turn off the behavior entirely worker-memory-target: 0.60 # target fraction to stay below worker-memory-spill: 0.70 # fraction at which we spill to disk worker-memory-pause: 0.80 # fraction at which we pause worker threads worker-memory-terminate: 0.95 # fraction at which we terminate the worker log-length: 10000 # default length of logs to keep in memory log-format: '%(name)s - %(levelname)s - %(message)s' distributed-1.20.2/distributed/core.py000066400000000000000000000560051321233345200177460ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import defaultdict, deque from functools import partial import logging import six import traceback import uuid import weakref from six import string_types from toolz import assoc from tornado import gen from tornado.ioloop import IOLoop from tornado.locks import Event from .comm import (connect, listen, CommClosedError, normalize_address, unparse_host_port, get_address_host_port) from .config import config from .metrics import time from .system_monitor import SystemMonitor from .utils import (get_traceback, truncate_exception, ignoring, shutting_down, PeriodicCallback) from . import protocol class RPCClosed(IOError): pass logger = logging.getLogger(__name__) def get_total_physical_memory(): try: import psutil return psutil.virtual_memory().total / 2 except ImportError: return 2e9 MAX_BUFFER_SIZE = get_total_physical_memory() class Server(object): """ Distributed TCP Server Superclass for endpoints in a distributed cluster, such as Worker and Scheduler objects. **Handlers** Servers define operations with a ``handlers`` dict mapping operation names to functions. The first argument of a handler function will be a ``Comm`` for the communication established with the client. Other arguments will receive inputs from the keys of the incoming message which will always be a dictionary. >>> def pingpong(comm): ... return b'pong' >>> def add(comm, x, y): ... return x + y >>> handlers = {'ping': pingpong, 'add': add} >>> server = Server(handlers) # doctest: +SKIP >>> server.listen('tcp://0.0.0.0:8000') # doctest: +SKIP **Message Format** The server expects messages to be dictionaries with a special key, `'op'` that corresponds to the name of the operation, and other key-value pairs as required by the function. So in the example above the following would be good messages. * ``{'op': 'ping'}`` * ``{'op': 'add', 'x': 10, 'y': 20}`` """ default_ip = '' default_port = 0 def __init__(self, handlers, connection_limit=512, deserialize=True, io_loop=None): self.handlers = assoc(handlers, 'identity', self.identity) self.id = type(self).__name__ + '-' + str(uuid.uuid4()) self._address = None self._listen_address = None self._port = None self._comms = {} self.deserialize = deserialize self.monitor = SystemMonitor() self.counters = None self.digests = None self.events = None self.event_counts = None self._ongoing_coroutines = weakref.WeakSet() self.listener = None self.io_loop = io_loop or IOLoop.current() # Statistics counters for various events with ignoring(ImportError): from .counter import Digest self.digests = defaultdict(partial(Digest, loop=self.io_loop)) from .counter import Counter self.counters = defaultdict(partial(Counter, loop=self.io_loop)) self.events = defaultdict(lambda: deque(maxlen=10000)) self.event_counts = defaultdict(lambda: 0) self.periodic_callbacks = dict() pc = PeriodicCallback(self.monitor.update, 500) self.periodic_callbacks['monitor'] = pc self._last_tick = time() pc = PeriodicCallback(self._measure_tick, config.get('tick-time', 20)) self.periodic_callbacks['tick'] = pc self.__stopped = False def start_periodic_callbacks(self): """ Start Periodic Callbacks consistently This starts all PeriodicCallbacks stored in self.periodic_callbacks if they are not yet running. It does this safely on the IOLoop. """ def start_pcs(): for pc in self.periodic_callbacks.values(): if not pc.is_running(): pc.start() self.loop.add_callback(start_pcs) def stop(self): if not self.__stopped: self.__stopped = True if self.listener is not None: # Delay closing the server socket until the next IO loop tick. # Otherwise race conditions can appear if an event handler # for an accept() call is already scheduled by the IO loop, # raising EBADF. # The demonstrator for this is Worker.terminate(), which # closes the server socket in response to an incoming message. # See https://github.com/tornadoweb/tornado/issues/2069 self.io_loop.add_callback(self.listener.stop) def _measure_tick(self): now = time() diff = now - self._last_tick self._last_tick = now if diff > config.get('tick-maximum-delay', 1000) / 1000: logger.warning("Event loop was unresponsive for %.2fs. " "This is often caused by long-running GIL-holding " "functions or moving large chunks of data. " "This can cause timeouts and instability.", diff) if self.digests is not None: self.digests['tick-duration'].add(diff) def log_event(self, name, msg): msg['time'] = time() if isinstance(name, list): for n in name: self.events[n].append(msg) self.event_counts[n] += 1 else: self.events[name].append(msg) self.event_counts[name] += 1 @property def address(self): """ The address this Server can be contacted on. """ if not self._address: if self.listener is None: raise ValueError("cannot get address of non-running Server") self._address = self.listener.contact_address return self._address @property def listen_address(self): """ The address this Server is listening on. This may be a wildcard address such as `tcp://0.0.0.0:1234`. """ if not self._listen_address: if self.listener is None: raise ValueError("cannot get listen address of non-running Server") self._listen_address = self.listener.listen_address return self._listen_address @property def port(self): """ The port number this Server is listening on. This will raise ValueError if the Server is listening on a non-IP based protocol. """ if not self._port: _, self._port = get_address_host_port(self.address) return self._port def identity(self, comm): return {'type': type(self).__name__, 'id': self.id} def listen(self, port_or_addr=None, listen_args=None): if port_or_addr is None: port_or_addr = self.default_port if isinstance(port_or_addr, int): addr = unparse_host_port(self.default_ip, port_or_addr) elif isinstance(port_or_addr, tuple): addr = unparse_host_port(*port_or_addr) else: addr = port_or_addr assert isinstance(addr, string_types) self.listener = listen(addr, self.handle_comm, deserialize=self.deserialize, connection_args=listen_args) self.listener.start() @gen.coroutine def handle_comm(self, comm, shutting_down=shutting_down): """ Dispatch new communications to coroutine-handlers Handlers is a dictionary mapping operation names to functions or coroutines. {'get_data': get_data, 'ping': pingpong} Coroutines should expect a single Comm object. """ if self.__stopped: comm.abort() return address = comm.peer_address op = None logger.debug("Connection from %r to %s", address, type(self).__name__) self._comms[comm] = op try: while True: try: msg = yield comm.read() logger.debug("Message from %r: %s", address, msg) except EnvironmentError as e: if not shutting_down(): logger.debug("Lost connection to %r while reading message: %s." " Last operation: %s", address, e, op) break except Exception as e: logger.exception(e) yield comm.write(error_message(e, status='uncaught-error')) continue if not isinstance(msg, dict): raise TypeError("Bad message type. Expected dict, got\n " + str(msg)) op = msg.pop('op') if self.counters is not None: self.counters['op'].add(op) self._comms[comm] = op close_desired = msg.pop('close', False) reply = msg.pop('reply', True) if op == 'close': if reply: yield comm.write('OK') break try: handler = self.handlers[op] except KeyError: result = "No handler found: %s" % op logger.warning(result, exc_info=True) else: logger.debug("Calling into handler %s", handler.__name__) try: result = handler(comm, **msg) if type(result) is gen.Future: self._ongoing_coroutines.add(result) result = yield result except CommClosedError as e: logger.warning("Lost connection to %r: %s", address, e) break except Exception as e: logger.exception(e) result = error_message(e, status='uncaught-error') if reply and result != 'dont-reply': try: yield comm.write(result) except EnvironmentError as e: logger.debug("Lost connection to %r while sending result for op %r: %s", address, op, e) break msg = result = None if close_desired: yield comm.close() if comm.closed(): break finally: del self._comms[comm] if not shutting_down() and not comm.closed(): try: comm.abort() except Exception as e: logger.error("Failed while closing connection to %r: %s", address, e) @gen.coroutine def close(self): self.listener.stop() for comm in self._comms: comm.close() for cb in self._ongoing_coroutines: cb.cancel() for i in range(10): if all(cb.cancelled() for c in self._ongoing_coroutines): break else: yield gen.sleep(0.01) def pingpong(comm): return b'pong' @gen.coroutine def send_recv(comm, reply=True, deserialize=True, **kwargs): """ Send and recv with a Comm. Keyword arguments turn into the message response = yield send_recv(comm, op='ping', reply=True) """ msg = kwargs msg['reply'] = reply please_close = kwargs.get('close') force_close = False try: yield comm.write(msg) if reply: response = yield comm.read() else: response = None except EnvironmentError: # On communication errors, we should simply close the communication force_close = True raise finally: if please_close: yield comm.close() elif force_close: comm.abort() if isinstance(response, dict) and response.get('status') == 'uncaught-error': six.reraise(*clean_exception(**response)) raise gen.Return(response) def addr_from_args(addr=None, ip=None, port=None): if addr is None: addr = (ip, port) else: assert ip is None and port is None if isinstance(addr, tuple): addr = unparse_host_port(*addr) return normalize_address(addr) class rpc(object): """ Conveniently interact with a remote server >>> remote = rpc(address) # doctest: +SKIP >>> response = yield remote.add(x=10, y=20) # doctest: +SKIP One rpc object can be reused for several interactions. Additionally, this object creates and destroys many comms as necessary and so is safe to use in multiple overlapping communications. When done, close comms explicitly. >>> remote.close_comms() # doctest: +SKIP """ active = weakref.WeakSet() comms = () address = None def __init__(self, arg=None, comm=None, deserialize=True, timeout=None, connection_args=None): self.comms = {} self.address = coerce_to_address(arg) self.timeout = timeout self.status = 'running' self.deserialize = deserialize self.connection_args = connection_args rpc.active.add(self) @gen.coroutine def live_comm(self): """ Get an open communication Some comms to the ip/port target may be in current use by other coroutines. We track this with the `comms` dict :: {comm: True/False if open and ready for use} This function produces an open communication, either by taking one that we've already made or making a new one if they are all taken. This also removes comms that have been closed. When the caller is done with the stream they should set self.comms[comm] = True As is done in __getattr__ below. """ if self.status == 'closed': raise RPCClosed("RPC Closed") to_clear = set() open = False for comm, open in self.comms.items(): if comm.closed(): to_clear.add(comm) if open: break for s in to_clear: del self.comms[s] if not open or comm.closed(): comm = yield connect(self.address, self.timeout, deserialize=self.deserialize, connection_args=self.connection_args) self.comms[comm] = False # mark as taken raise gen.Return(comm) def close_comms(self): @gen.coroutine def _close_comm(comm): # Make sure we tell the peer to close try: yield comm.write({'op': 'close', 'reply': False}) yield comm.close() except EnvironmentError: comm.abort() for comm in list(self.comms): if comm and not comm.closed(): _close_comm(comm) self.comms.clear() def __getattr__(self, key): @gen.coroutine def send_recv_from_rpc(**kwargs): try: comm = yield self.live_comm() result = yield send_recv(comm=comm, op=key, **kwargs) except (RPCClosed, CommClosedError) as e: raise e.__class__("%s: while trying to call remote method %r" % (e, key,)) self.comms[comm] = True # mark as open raise gen.Return(result) return send_recv_from_rpc def close_rpc(self): if self.status != 'closed': rpc.active.discard(self) self.status = 'closed' self.close_comms() def __enter__(self): return self def __exit__(self, *args): self.close_rpc() def __del__(self): if self.status != 'closed': rpc.active.discard(self) self.status = 'closed' still_open = [comm for comm in self.comms if not comm.closed()] if still_open: logger.warning("rpc object %s deleted with %d open comms", self, len(still_open)) for comm in still_open: comm.abort() def __repr__(self): return "" % (self.address, len(self.comms)) class PooledRPCCall(object): """ The result of ConnectionPool()('host:port') See Also: ConnectionPool """ def __init__(self, addr, pool): self.addr = addr self.pool = pool def __getattr__(self, key): @gen.coroutine def send_recv_from_rpc(**kwargs): comm = yield self.pool.connect(self.addr) try: result = yield send_recv(comm=comm, op=key, **kwargs) finally: self.pool.reuse(self.addr, comm) raise gen.Return(result) return send_recv_from_rpc def close_rpc(self): pass # For compatibility with rpc() def __enter__(self): return self def __exit__(self, *args): pass def __repr__(self): return "" % (self.addr,) class ConnectionPool(object): """ A maximum sized pool of Comm objects. This provides a connect method that mirrors the normal distributed.connect method, but provides connection sharing and tracks connection limits. This object provides an ``rpc`` like interface:: >>> rpc = ConnectionPool(limit=512) >>> scheduler = rpc('127.0.0.1:8786') >>> workers = [rpc(address) for address ...] >>> info = yield scheduler.identity() It creates enough comms to satisfy concurrent connections to any particular address:: >>> a, b = yield [scheduler.who_has(), scheduler.has_what()] It reuses existing comms so that we don't have to continuously reconnect. It also maintains a comm limit to avoid "too many open file handle" issues. Whenever this maximum is reached we clear out all idling comms. If that doesn't do the trick then we wait until one of the occupied comms closes. Parameters ---------- limit: int The number of open comms to maintain at once deserialize: bool Whether or not to deserialize data by default or pass it through """ def __init__(self, limit=512, deserialize=True, connection_args=None): self.open = 0 # Total number of open comms self.active = 0 # Number of comms currently in use self.limit = limit # Max number of open comms # Invariant: len(available) == open - active self.available = defaultdict(set) # Invariant: len(occupied) == active self.occupied = defaultdict(set) self.deserialize = deserialize self.connection_args = connection_args self.event = Event() def __repr__(self): return "" % (self.open, self.active) def __call__(self, addr=None, ip=None, port=None): """ Cached rpc objects """ addr = addr_from_args(addr=addr, ip=ip, port=port) return PooledRPCCall(addr, self) @gen.coroutine def connect(self, addr, timeout=None): """ Get a Comm to the given address. For internal use. """ available = self.available[addr] occupied = self.occupied[addr] if available: comm = available.pop() if not comm.closed(): self.active += 1 occupied.add(comm) raise gen.Return(comm) else: self.open -= 1 while self.open >= self.limit: self.event.clear() self.collect() yield self.event.wait() self.open += 1 try: comm = yield connect(addr, timeout=timeout, deserialize=self.deserialize, connection_args=self.connection_args) except Exception: self.open -= 1 raise self.active += 1 occupied.add(comm) if self.open >= self.limit: self.event.clear() raise gen.Return(comm) def reuse(self, addr, comm): """ Reuse an open communication to the given address. For internal use. """ self.occupied[addr].remove(comm) self.active -= 1 if comm.closed(): self.open -= 1 if self.open < self.limit: self.event.set() else: self.available[addr].add(comm) def collect(self): """ Collect open but unused communications, to allow opening other ones. """ logger.info("Collecting unused comms. open: %d, active: %d", self.open, self.active) for addr, comms in self.available.items(): for comm in comms: comm.close() comms.clear() self.open = self.active if self.open < self.limit: self.event.set() def close(self): """ Close all communications abruptly. """ for comms in self.available.values(): for comm in comms: comm.abort() for comms in self.occupied.values(): for comm in comms: comm.abort() def coerce_to_address(o): if isinstance(o, (list, tuple)): o = unparse_host_port(*o) return normalize_address(o) def error_message(e, status='error'): """ Produce message to send back given an exception has occurred This does the following: 1. Gets the traceback 2. Truncates the exception and the traceback 3. Serializes the exception and traceback or 4. If they can't be serialized send string versions 5. Format a message and return See Also -------- clean_exception: deserialize and unpack message into exception/traceback six.reraise: raise exception/traceback """ tb = get_traceback() e2 = truncate_exception(e, 1000) try: e3 = protocol.pickle.dumps(e2) protocol.pickle.loads(e3) except Exception: e3 = Exception(str(e2)) e3 = protocol.pickle.dumps(e3) try: tb2 = protocol.pickle.dumps(tb) except Exception: tb2 = ''.join(traceback.format_tb(tb)) tb2 = protocol.pickle.dumps(tb2) if len(tb2) > 10000: tb2 = None return {'status': status, 'exception': e3, 'traceback': tb2} def clean_exception(exception, traceback, **kwargs): """ Reraise exception and traceback. Deserialize if necessary See Also -------- error_message: create and serialize errors into message """ if isinstance(exception, bytes): exception = protocol.pickle.loads(exception) if isinstance(traceback, bytes): traceback = protocol.pickle.loads(traceback) elif isinstance(traceback, string_types): traceback = None # happens if the traceback failed serializing return type(exception), exception, traceback distributed-1.20.2/distributed/counter.py000066400000000000000000000042041321233345200204670ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import defaultdict from tornado.ioloop import IOLoop from .utils import PeriodicCallback try: from crick import TDigest except ImportError: pass else: class Digest(object): def __init__(self, loop=None, intervals=(5, 60, 3600)): self.intervals = intervals self.components = [TDigest() for i in self.intervals] self.loop = loop or IOLoop.current() self._pc = PeriodicCallback(self.shift, self.intervals[0] * 1000) self.loop.add_callback(self._pc.start) def add(self, item): self.components[0].add(item) def update(self, seq): self.components[0].update(seq) def shift(self): for i in range(len(self.intervals) - 1): frac = 0.2 * self.intervals[0] / self.intervals[i] part = self.components[i].scale(frac) rest = self.components[i].scale(1 - frac) self.components[i + 1].merge(part) self.components[i] = rest def size(self): return sum(d.size() for d in self.components) class Counter(object): def __init__(self, loop=None, intervals=(5, 60, 3600)): self.intervals = intervals self.components = [defaultdict(lambda: 0) for i in self.intervals] self.loop = loop or IOLoop.current() self._pc = PeriodicCallback(self.shift, self.intervals[0] * 1000) self.loop.add_callback(self._pc.start) def add(self, item): self.components[0][item] += 1 def shift(self): for i in range(len(self.intervals) - 1): frac = 0.2 * self.intervals[0] / self.intervals[i] part = {k: v * frac for k, v in self.components[i].items()} rest = {k: v * (1 - frac) for k, v in self.components[i].items()} for k, v in part.items(): self.components[i + 1][k] += v d = defaultdict(lambda: 0) d.update(rest) self.components[i] = d def size(self): return sum(sum(d.values()) for d in self.components) distributed-1.20.2/distributed/deploy/000077500000000000000000000000001321233345200177325ustar00rootroot00000000000000distributed-1.20.2/distributed/deploy/__init__.py000066400000000000000000000003331321233345200220420ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from ..utils import ignoring from .local import LocalCluster from .adaptive import Adaptive with ignoring(ImportError): from .ssh import SSHCluster distributed-1.20.2/distributed/deploy/adaptive.py000066400000000000000000000147301321233345200221060ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging from tornado import gen from ..utils import log_errors, PeriodicCallback logger = logging.getLogger(__name__) class Adaptive(object): ''' Adaptively allocate workers based on scheduler load. A superclass. Contains logic to dynamically resize a Dask cluster based on current use. Parameters ---------- scheduler: distributed.Scheduler cluster: object Must have scale_up and scale_down methods/coroutines startup_cost : int, default 1 Factor representing how costly it is to start an additional worker. Affects quickly to adapt to high tasks per worker loads scale_factor : int, default 2 Factor to scale by when it's determined additional workers are needed Examples -------- >>> class MyCluster(object): ... def scale_up(self, n): ... """ Bring worker count up to n """ ... def scale_down(self, workers): ... """ Remove worker addresses from cluster """ Notes ----- Subclasses can override :meth:`Adaptive.should_scale_up` and :meth:`Adaptive.should_scale_down` to control when the cluster should be resized. The default implementation checks if there are too many tasks per worker or too little memory available (see :meth:`Adaptive.needs_cpu` and :meth:`Adaptive.needs_memory`). :meth:`Adaptive.get_scale_up_kwargs` method controls the arguments passed to the cluster's ``scale_up`` method. ''' def __init__(self, scheduler, cluster, interval=1000, startup_cost=1, scale_factor=2): self.scheduler = scheduler self.cluster = cluster self.startup_cost = startup_cost self.scale_factor = scale_factor self._adapt_callback = PeriodicCallback(self._adapt, interval) self.scheduler.loop.add_callback(self._adapt_callback.start) self._adapting = False 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``. """ total_occupancy = self.scheduler.total_occupancy total_cores = sum(self.scheduler.ncores.values()) if total_occupancy / (total_cores + 1e-9) > self.startup_cost * 2: logger.info("CPU limit exceeded [%d occupancy / %d cores]", total_occupancy, total_cores) return True else: 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 = {w: self.scheduler.worker_info[w]['memory_limit'] for w in self.scheduler.worker_info} worker_bytes = self.scheduler.worker_bytes limit = sum(limit_bytes.values()) total = sum(worker_bytes.values()) 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 See Also -------- needs_cpu needs_memory """ with log_errors(): if self.scheduler.unrunnable and not self.scheduler.ncores: return True needs_cpu = self.needs_cpu() needs_memory = self.needs_memory() if needs_cpu or needs_memory: return True return False def should_scale_down(self): """ Determine whether any workers should potentially be removed from the cluster. Returns ------- scale_down : bool Notes ----- ``Adaptive.should_scale_down`` always returns True, so we will always attempt to remove workers as determined by ``Scheduler.workers_to_close``. See Also -------- Scheduler.workers_to_close """ return len(self.scheduler.workers_to_close()) > 0 @gen.coroutine def _retire_workers(self): with log_errors(): workers = yield self.scheduler.retire_workers(remove=True, close_workers=True) if workers: logger.info("Retiring workers %s", workers) f = self.cluster.scale_down(workers) if gen.is_future(f): yield f def get_scale_up_kwargs(self): """ Get the arguments to be passed to ``self.cluster.scale_up``. Notes ----- By default the desired number of total workers is returned (``n``). Subclasses should ensure that the return dictionary includes a key- value pair for ``n``, either by implementing it or by calling the parent's ``get_scale_up_kwargs``. See Also -------- LocalCluster.scale_up """ instances = max(1, len(self.scheduler.ncores) * self.scale_factor) logger.info("Scaling up to %d workers", instances) return {'n': instances} @gen.coroutine def _adapt(self): if self._adapting: # Semaphore to avoid overlapping adapt calls return self._adapting = True try: should_scale_up = self.should_scale_up() should_scale_down = self.should_scale_down() if should_scale_up and should_scale_down: logger.info("Attempting to scale up and scale down simultaneously.") else: if should_scale_up: kwargs = self.get_scale_up_kwargs() f = self.cluster.scale_up(**kwargs) if gen.is_future(f): yield f if should_scale_down: yield self._retire_workers() finally: self._adapting = False def adapt(self): self.scheduler.loop.add_callback(self._adapt) distributed-1.20.2/distributed/deploy/local.py000066400000000000000000000232041321233345200213770ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import atexit import logging import math from time import sleep import weakref from tornado import gen from ..core import CommClosedError from ..utils import sync, ignoring, All, silence_logging, LoopRunner from ..nanny import Nanny from ..scheduler import Scheduler from ..worker import Worker, _ncores logger = logging.getLogger(__name__) class LocalCluster(object): """ Create local Scheduler and Workers This creates a "cluster" of a scheduler and workers running on the local machine. Parameters ---------- n_workers: int Number of workers to start processes: bool Whether to use processes (True) or threads (False). Defaults to True threads_per_worker: int Number of threads per each worker scheduler_port: int Port of the scheduler. 8786 by default, use 0 to choose a random port silence_logs: logging level Level of logs to print out to stdout. ``logging.CRITICAL`` by default. Use a falsey value like False or None for no change. ip: string IP address on which the scheduler will listen, defaults to only localhost kwargs: dict Extra worker arguments, will be passed to the Worker constructor. Examples -------- >>> c = LocalCluster() # Create a local cluster with as many workers as cores # doctest: +SKIP >>> c # doctest: +SKIP LocalCluster("127.0.0.1:8786", workers=8, ncores=8) >>> c = Client(c) # connect to local cluster # doctest: +SKIP Add a new worker to the cluster >>> w = c.start_worker(ncores=2) # doctest: +SKIP Shut down the extra worker >>> c.remove_worker(w) # doctest: +SKIP """ def __init__(self, n_workers=None, threads_per_worker=None, processes=True, loop=None, start=True, ip=None, scheduler_port=0, silence_logs=logging.CRITICAL, diagnostics_port=8787, services={}, worker_services={}, **worker_kwargs): self.status = None self.processes = processes self.silence_logs = silence_logs if silence_logs: silence_logging(level=silence_logs) if n_workers is None and threads_per_worker is None: if processes: n_workers = _ncores threads_per_worker = 1 else: n_workers = 1 threads_per_worker = _ncores if n_workers is None and threads_per_worker is not None: n_workers = max(1, _ncores // threads_per_worker) if n_workers and threads_per_worker is None: # Overcommit threads per worker, rather than undercommit threads_per_worker = max(1, int(math.ceil(_ncores / n_workers))) self._loop_runner = LoopRunner(loop=loop) self.loop = self._loop_runner.loop if start: self._loop_runner.start() if diagnostics_port is not None: try: from distributed.bokeh.scheduler import BokehScheduler from distributed.bokeh.worker import BokehWorker except ImportError: logger.debug("To start diagnostics web server please install Bokeh") else: services[('bokeh', diagnostics_port)] = BokehScheduler worker_services[('bokeh', 0)] = BokehWorker self.scheduler = Scheduler(loop=self.loop, services=services) self.scheduler_port = scheduler_port self.workers = [] self.n_workers = n_workers self.threads_per_worker = threads_per_worker self.worker_services = worker_services self.worker_kwargs = worker_kwargs if start: sync(self.loop, self._start, ip) 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)) ) @gen.coroutine def _start(self, ip=None): """ Start all cluster services. Wait on this if you passed `start=False` to the LocalCluster constructor. """ if self.status == 'running': return if ip is None and not self.scheduler_port and not self.processes: # Use inproc transport for optimization scheduler_address = 'inproc://' else: if ip is None: ip = '127.0.0.1' scheduler_address = (ip, self.scheduler_port) self.scheduler.start(scheduler_address) yield self._start_all_workers( self.n_workers, ncores=self.threads_per_worker, services=self.worker_services, **self.worker_kwargs) self.status = 'running' @gen.coroutine def _start_all_workers(self, n_workers, **kwargs): yield [self._start_worker(**kwargs) for i in range(n_workers)] @gen.coroutine def _start_worker(self, port=0, processes=None, death_timeout=60, **kwargs): if processes is not None: raise ValueError("overriding `processes` for individual workers " "in a LocalCluster is not supported anymore") if port: raise ValueError("overriding `port` for individual workers " "in a LocalCluster is not supported anymore") if self.processes: W = Nanny kwargs['quiet'] = True else: W = Worker w = W(self.scheduler.address, loop=self.loop, death_timeout=death_timeout, silence_logs=self.silence_logs, **kwargs) yield w._start() self.workers.append(w) while w.status != 'closed' and w.worker_address not in self.scheduler.worker_info: yield gen.sleep(0.01) if w.status == 'closed': self.workers.remove(w) raise gen.TimeoutError("Worker failed to start") raise gen.Return(w) def start_worker(self, ncores=0, **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 sync(self.loop, self._start_worker, ncores=ncores, **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 """ sync(self.loop, self._stop_worker, w) @gen.coroutine def _close(self): # Can be 'closing' as we're called by close() below if self.status == 'closed': return try: with ignoring(gen.TimeoutError, CommClosedError, OSError): yield All([w._close() for w in self.workers]) with ignoring(gen.TimeoutError, CommClosedError, OSError): yield self.scheduler.close(fast=True) del self.workers[:] finally: self.status = 'closed' def close(self, timeout=20): """ Close the cluster """ if self.status == 'closed': return try: self.scheduler.clear_task_state() for w in self.workers: self.loop.add_callback(self._stop_worker, w) for i in range(10): if not self.workers: break else: sleep(0.01) del self.workers[:] self._loop_runner.run_sync(self._close, callback_timeout=timeout) self._loop_runner.stop() finally: self.status = 'closed' @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. """ yield [self._start_worker(**kwargs) for i in range(n - len(self.workers))] @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. """ workers = set(workers) yield [self._stop_worker(w) for w in self.workers if w.worker_address in workers] while workers & set(self.workers): yield gen.sleep(0.01) def __del__(self): self.close() def __enter__(self): return self def __exit__(self, *args): self.close() @property def scheduler_address(self): try: return self.scheduler.address except ValueError: return '' clusters_to_close = weakref.WeakSet() @atexit.register def close_clusters(): for cluster in list(clusters_to_close): cluster.close(timeout=10) distributed-1.20.2/distributed/deploy/ssh.py000066400000000000000000000302631321233345200211050ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging import socket import os import sys import time import traceback try: from queue import Queue except ImportError: # Python 2.7 fix from Queue import Queue from threading import Thread from toolz import merge from tornado import gen logger = logging.getLogger(__name__) # These are handy for creating colorful terminal output to enhance readability # of the output generated by dask-ssh. class bcolors: HEADER = '\033[95m' OKBLUE = '\033[94m' OKGREEN = '\033[92m' WARNING = '\033[93m' FAIL = '\033[91m' ENDC = '\033[0m' BOLD = '\033[1m' UNDERLINE = '\033[4m' def async_ssh(cmd_dict): import paramiko from paramiko.buffered_pipe import PipeTimeout from paramiko.ssh_exception import (SSHException, PasswordRequiredException) ssh = paramiko.SSHClient() ssh.set_missing_host_key_policy(paramiko.AutoAddPolicy()) retries = 0 while True: # Be robust to transient SSH failures. try: # Set paramiko logging to WARN or higher to squelch INFO messages. logging.getLogger('paramiko').setLevel(logging.WARN) ssh.connect(hostname=cmd_dict['address'], username=cmd_dict['ssh_username'], port=cmd_dict['ssh_port'], key_filename=cmd_dict['ssh_private_key'], compress=True, timeout=20, banner_timeout=20) # Helps prevent timeouts when many concurrent ssh connections are opened. # Connection successful, break out of while loop break except (SSHException, PasswordRequiredException) as e: print('[ dask-ssh ] : ' + bcolors.FAIL + 'SSH connection error when connecting to {addr}:{port}' 'to run \'{cmd}\''.format(addr=cmd_dict['address'], port=cmd_dict['ssh_port'], cmd=cmd_dict['cmd']) + bcolors.ENDC) print(bcolors.FAIL + ' SSH reported this exception: ' + str(e) + bcolors.ENDC) # Print an exception traceback traceback.print_exc() # Transient SSH errors can occur when many SSH connections are # simultaneously opened to the same server. This makes a few # attempts to retry. retries += 1 if retries >= 3: print('[ dask-ssh ] : ' + bcolors.FAIL + 'SSH connection failed after 3 retries. Exiting.' + bcolors.ENDC) # Connection failed after multiple attempts. Terminate this thread. os._exit(1) # Wait a moment before retrying print(' ' + bcolors.FAIL + 'Retrying... (attempt {n}/{total})'.format(n=retries, total=3) + bcolors.ENDC) time.sleep(1) # Execute the command, and grab file handles for stdout and stderr. Note # that we run the command using the user's default shell, but force it to # run in an interactive login shell, which hopefully ensures that all of the # user's normal environment variables (via the dot files) have been loaded # before the command is run. This should help to ensure that important # aspects of the environment like PATH and PYTHONPATH are configured. print('[ {label} ] : {cmd}'.format(label=cmd_dict['label'], cmd=cmd_dict['cmd'])) stdin, stdout, stderr = ssh.exec_command('$SHELL -i -c \'' + cmd_dict['cmd'] + '\'', get_pty=True) # Set up channel timeout (which we rely on below to make readline() non-blocking) channel = stdout.channel channel.settimeout(0.1) def read_from_stdout(): """ Read stdout stream, time out if necessary. """ try: line = stdout.readline() while len(line) > 0: # Loops until a timeout exception occurs line = line.rstrip() logger.debug('stdout from ssh channel: %s', line) cmd_dict['output_queue'].put('[ {label} ] : {output}'.format(label=cmd_dict['label'], output=line)) line = stdout.readline() except (PipeTimeout, socket.timeout): pass def read_from_stderr(): """ Read stderr stream, time out if necessary. """ try: line = stderr.readline() while len(line) > 0: line = line.rstrip() logger.debug('stderr from ssh channel: %s', line) cmd_dict['output_queue'].put('[ {label} ] : '.format(label=cmd_dict['label']) + bcolors.FAIL + '{output}'.format(output=line) + bcolors.ENDC) line = stderr.readline() except (PipeTimeout, socket.timeout): pass def communicate(): """ Communicate a little bit, without blocking too long. Return True if the command ended. """ read_from_stdout() read_from_stderr() # Check to see if the process has exited. If it has, we let this thread # terminate. if channel.exit_status_ready(): exit_status = channel.recv_exit_status() cmd_dict['output_queue'].put('[ {label} ] : '.format(label=cmd_dict['label']) + bcolors.FAIL + "remote process exited with exit status " + str(exit_status) + bcolors.ENDC) return True # Wait for a message on the input_queue. Any message received signals this # thread to shut itself down. while cmd_dict['input_queue'].empty(): # Kill some time so that this thread does not hog the CPU. time.sleep(1.0) if communicate(): break # Ctrl-C the executing command and wait a bit for command to end cleanly start = time.time() while time.time() < start + 5.0: channel.send(b'\x03') # Ctrl-C if communicate(): break time.sleep(1.0) # Shutdown the channel, and close the SSH connection channel.close() ssh.close() def start_scheduler(logdir, addr, port, ssh_username, ssh_port, ssh_private_key): cmd = '{python} -m distributed.cli.dask_scheduler --port {port}'.format( python=sys.executable, port=port, logdir=logdir) # Optionally re-direct stdout and stderr to a logfile if logdir is not None: cmd = 'mkdir -p {logdir} && '.format(logdir=logdir) + cmd cmd += '&> {logdir}/dask_scheduler_{addr}:{port}.log'.format(addr=addr, port=port, logdir=logdir) # Format output labels we can prepend to each line of output, and create # a 'status' key to keep track of jobs that terminate prematurely. label = (bcolors.BOLD + 'scheduler {addr}:{port}'.format(addr=addr, port=port) + bcolors.ENDC) # Create a command dictionary, which contains everything we need to run and # interact with this command. input_queue = Queue() output_queue = Queue() cmd_dict = {'cmd': cmd, 'label': label, 'address': addr, 'port': port, 'input_queue': input_queue, 'output_queue': output_queue, 'ssh_username': ssh_username, 'ssh_port': ssh_port, 'ssh_private_key': ssh_private_key} # Start the thread thread = Thread(target=async_ssh, args=[cmd_dict]) thread.daemon = True thread.start() return merge(cmd_dict, {'thread': thread}) def start_worker(logdir, scheduler_addr, scheduler_port, worker_addr, nthreads, nprocs, ssh_username, ssh_port, ssh_private_key, nohost): cmd = ('{python} -m distributed.cli.dask_worker ' '{scheduler_addr}:{scheduler_port} ' '--nthreads {nthreads} --nprocs {nprocs}') if not nohost: cmd += ' --host {worker_addr}' cmd = cmd.format( python=sys.executable, scheduler_addr=scheduler_addr, scheduler_port=scheduler_port, worker_addr=worker_addr, nthreads=nthreads, nprocs=nprocs) # Optionally redirect stdout and stderr to a logfile if logdir is not None: cmd = 'mkdir -p {logdir} && '.format(logdir=logdir) + cmd cmd += '&> {logdir}/dask_scheduler_{addr}.log'.format( addr=worker_addr, logdir=logdir) label = 'worker {addr}'.format(addr=worker_addr) # Create a command dictionary, which contains everything we need to run and # interact with this command. input_queue = Queue() output_queue = Queue() cmd_dict = {'cmd': cmd, 'label': label, 'address': worker_addr, 'input_queue': input_queue, 'output_queue': output_queue, 'ssh_username': ssh_username, 'ssh_port': ssh_port, 'ssh_private_key': ssh_private_key} # Start the thread thread = Thread(target=async_ssh, args=[cmd_dict]) thread.daemon = True thread.start() return merge(cmd_dict, {'thread': thread}) class SSHCluster(object): def __init__(self, scheduler_addr, scheduler_port, worker_addrs, nthreads=0, nprocs=1, ssh_username=None, ssh_port=22, ssh_private_key=None, nohost=False, logdir=None): self.scheduler_addr = scheduler_addr self.scheduler_port = scheduler_port self.nthreads = nthreads self.nprocs = nprocs self.ssh_username = ssh_username self.ssh_port = ssh_port self.ssh_private_key = ssh_private_key self.nohost = nohost # Generate a universal timestamp to use for log files import datetime if logdir is not None: logdir = os.path.join(logdir, "dask-ssh_" + datetime.datetime.now().strftime("%Y-%m-%d_%H:%M:%S")) print(bcolors.WARNING + 'Output will be redirected to logfiles ' 'stored locally on individual worker nodes under "{logdir}".'.format(logdir=logdir) + bcolors.ENDC) self.logdir = logdir # Keep track of all running threads self.threads = [] # Start the scheduler node self.scheduler = start_scheduler(logdir, scheduler_addr, scheduler_port, ssh_username, ssh_port, ssh_private_key) # Start worker nodes self.workers = [] for i, addr in enumerate(worker_addrs): self.add_worker(addr) @gen.coroutine def _start(self): pass @property def scheduler_address(self): return '%s:%d' % (self.scheduler_addr, self.scheduler_port) def monitor_remote_processes(self): # Form a list containing all processes, since we treat them equally from here on out. all_processes = [self.scheduler] + self.workers try: while True: for process in all_processes: while not process['output_queue'].empty(): print(process['output_queue'].get()) # Kill some time and free up CPU before starting the next sweep # through the processes. time.sleep(0.1) # end while true except KeyboardInterrupt: pass # Return execution to the calling process def add_worker(self, address): self.workers.append(start_worker(self.logdir, self.scheduler_addr, self.scheduler_port, address, self.nthreads, self.nprocs, self.ssh_username, self.ssh_port, self.ssh_private_key, self.nohost)) def shutdown(self): all_processes = [self.scheduler] + self.workers for process in all_processes: process['input_queue'].put('shutdown') process['thread'].join() def __enter__(self): return self def __exit__(self, *args): self.shutdown() distributed-1.20.2/distributed/deploy/tests/000077500000000000000000000000001321233345200210745ustar00rootroot00000000000000distributed-1.20.2/distributed/deploy/tests/test_adaptive.py000066400000000000000000000067161321233345200243140ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from time import sleep from tornado import gen from tornado.ioloop import IOLoop from distributed import Client from distributed.deploy import Adaptive, LocalCluster from distributed.utils_test import gen_cluster, gen_test, slowinc from distributed.utils_test import loop, nodebug # flake8: noqa from distributed.metrics import time def test_get_scale_up_kwargs(loop): with LocalCluster(0, scheduler_port=0, silence_logs=False, diagnostics_port=None, loop=loop) as cluster: alc = Adaptive(cluster.scheduler, cluster, interval=100, scale_factor=3) assert alc.get_scale_up_kwargs() == {'n': 1} with Client(cluster, loop=loop) as c: future = c.submit(lambda x: x + 1, 1) assert future.result() == 2 assert c.ncores() assert alc.get_scale_up_kwargs() == {'n': 3} @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 4) def test_simultaneous_scale_up_and_down(c, s, *workers): class TestAdaptive(Adaptive): def get_scale_up_kwargs(self): assert False def _retire_workers(self): assert False class TestCluster(object): def scale_up(self, n, **kwargs): assert False def scale_down(self, workers): assert False cluster = TestCluster() s.task_duration['a'] = 4 s.task_duration['b'] = 4 s.task_duration['c'] = 1 future = c.map(slowinc, [1, 1, 1], key=['a-4', 'b-4', 'c-1']) while len(s.rprocessing) < 3: yield gen.sleep(0.001) ta = TestAdaptive(s, cluster, interval=100, scale_factor=2) yield gen.sleep(0.3) def test_adaptive_local_cluster(loop): with LocalCluster(0, scheduler_port=0, silence_logs=False, diagnostics_port=None, loop=loop) as cluster: alc = Adaptive(cluster.scheduler, cluster, interval=100) with Client(cluster, loop=loop) as c: assert not c.ncores() future = c.submit(lambda x: x + 1, 1) assert future.result() == 2 assert c.ncores() sleep(0.1) assert c.ncores() # still there after some time del future start = time() while cluster.scheduler.ncores: sleep(0.01) assert time() < start + 5 assert not c.ncores() @nodebug @gen_test(timeout=30) def test_adaptive_local_cluster_multi_workers(): loop = IOLoop.current() cluster = LocalCluster(0, scheduler_port=0, silence_logs=False, processes=False, diagnostics_port=None, loop=loop, start=False) cluster.scheduler.allowed_failures = 1000 alc = Adaptive(cluster.scheduler, cluster, interval=100) c = yield Client(cluster, asynchronous=True, loop=loop) futures = c.map(slowinc, range(100), delay=0.01) start = time() while not cluster.scheduler.worker_info: yield gen.sleep(0.01) assert time() < start + 15 yield c._gather(futures) del futures start = time() while cluster.workers: yield gen.sleep(0.01) assert time() < start + 5 assert not cluster.workers assert not cluster.scheduler.workers yield gen.sleep(0.2) assert not cluster.workers assert not cluster.scheduler.workers futures = c.map(slowinc, range(100), delay=0.01) yield c._gather(futures) yield c._close() yield cluster._close() distributed-1.20.2/distributed/deploy/tests/test_local.py000066400000000000000000000267471321233345200236170ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from functools import partial import subprocess import sys from time import sleep from threading import Lock import unittest from tornado.ioloop import IOLoop from tornado import gen import pytest from distributed import Client, Worker, Nanny from distributed.deploy.local import LocalCluster from distributed.metrics import time from distributed.utils_test import (inc, gen_test, assert_can_connect_locally_4, assert_can_connect_from_everywhere_4_6, captured_logger) from distributed.utils_test import loop # flake8: noqa from distributed.utils import sync from distributed.worker import TOTAL_MEMORY from distributed.deploy.utils_test import ClusterTest def test_simple(loop): with LocalCluster(4, scheduler_port=0, processes=False, silence_logs=False, diagnostics_port=None, loop=loop) as c: with Client(c.scheduler_address, loop=loop) 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) def test_close_twice(): cluster = LocalCluster() with Client(cluster.scheduler_address) as client: f = client.map(inc, range(100)) client.gather(f) with captured_logger('tornado.application') as log: cluster.close() cluster.close() sleep(0.5) log = log.getvalue() print(log) assert not log @pytest.mark.skipif('sys.version_info[0] == 2', reason='multi-loop') def test_procs(): with LocalCluster(2, scheduler_port=0, processes=False, threads_per_worker=3, diagnostics_port=None, silence_logs=False) as c: assert len(c.workers) == 2 assert all(isinstance(w, Worker) for w in c.workers) 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) repr(c) with LocalCluster(2, scheduler_port=0, processes=True, threads_per_worker=3, diagnostics_port=None, silence_logs=False) as c: assert len(c.workers) == 2 assert all(isinstance(w, Nanny) for w in c.workers) 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) repr(c) def test_move_unserializable_data(): """ Test that unserializable data is still fine to transfer over inproc transports. """ with LocalCluster(processes=False, silence_logs=False, diagnostics_port=None) as cluster: assert cluster.scheduler_address.startswith('inproc://') assert cluster.workers[0].address.startswith('inproc://') with Client(cluster) as client: lock = Lock() x = client.scatter(lock) y = client.submit(lambda x: x, x) assert y.result() is lock def test_transports(): """ Test the transport chosen by LocalCluster depending on arguments. """ with LocalCluster(1, processes=False, silence_logs=False, diagnostics_port=None) as c: assert c.scheduler_address.startswith('inproc://') assert c.workers[0].address.startswith('inproc://') with Client(c.scheduler.address) as e: assert e.submit(inc, 4).result() == 5 # Have nannies => need TCP with LocalCluster(1, processes=True, silence_logs=False, diagnostics_port=None) as c: assert c.scheduler_address.startswith('tcp://') assert c.workers[0].address.startswith('tcp://') with Client(c.scheduler.address) as e: assert e.submit(inc, 4).result() == 5 # Scheduler port specified => need TCP with LocalCluster(1, processes=False, scheduler_port=8786, silence_logs=False, diagnostics_port=None) as c: assert c.scheduler_address == 'tcp://127.0.0.1:8786' assert c.workers[0].address.startswith('tcp://') with Client(c.scheduler.address) as e: assert e.submit(inc, 4).result() == 5 @pytest.mark.skipif('sys.version_info[0] == 2', reason='') class LocalTest(ClusterTest, unittest.TestCase): Cluster = partial(LocalCluster, silence_logs=False, diagnostics_port=None) def test_Client_with_local(loop): with LocalCluster(1, scheduler_port=0, silence_logs=False, diagnostics_port=None, loop=loop) as c: with Client(c, loop=loop) as e: assert len(e.ncores()) == len(c.workers) assert c.scheduler_address in repr(c) def test_Client_solo(loop): with Client(loop=loop) as c: pass assert c.cluster.status == 'closed' def test_Client_kwargs(loop): with Client(loop=loop, processes=False, n_workers=2) as c: assert len(c.cluster.workers) == 2 assert all(isinstance(w, Worker) for w in c.cluster.workers) assert c.cluster.status == 'closed' def test_Client_twice(loop): with Client(loop=loop) as c: with Client(loop=loop) as f: assert c.cluster.scheduler.port != f.cluster.scheduler.port def test_defaults(): from distributed.worker import _ncores with LocalCluster(scheduler_port=0, silence_logs=False, diagnostics_port=None) as c: assert sum(w.ncores for w in c.workers) == _ncores assert all(isinstance(w, Nanny) for w in c.workers) assert all(w.ncores == 1 for w in c.workers) with LocalCluster(processes=False, scheduler_port=0, silence_logs=False, diagnostics_port=None) as c: assert sum(w.ncores for w in c.workers) == _ncores assert all(isinstance(w, Worker) for w in c.workers) assert len(c.workers) == 1 with LocalCluster(n_workers=2, scheduler_port=0, silence_logs=False, diagnostics_port=None) as c: if _ncores % 2 == 0: expected_total_threads = max(2, _ncores) 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 with LocalCluster(threads_per_worker=_ncores * 2, scheduler_port=0, silence_logs=False, diagnostics_port=None) as c: assert len(c.workers) == 1 with LocalCluster(n_workers=_ncores * 2, scheduler_port=0, silence_logs=False, diagnostics_port=None) as c: assert all(w.ncores == 1 for w in c.workers) with LocalCluster(threads_per_worker=2, n_workers=3, scheduler_port=0, silence_logs=False, diagnostics_port=None) as c: assert len(c.workers) == 3 assert all(w.ncores == 2 for w in c.workers) def test_worker_params(): with LocalCluster(n_workers=2, scheduler_port=0, silence_logs=False, diagnostics_port=None, memory_limit=500) as c: assert [w.memory_limit for w in c.workers] == [500] * 2 def test_cleanup(): c = LocalCluster(2, scheduler_port=0, silence_logs=False, diagnostics_port=None) port = c.scheduler.port c.close() c2 = LocalCluster(2, scheduler_port=port, silence_logs=False, diagnostics_port=None) c.close() def test_repeated(): with LocalCluster(scheduler_port=8448, silence_logs=False, diagnostics_port=None) as c: pass with LocalCluster(scheduler_port=8448, silence_logs=False, diagnostics_port=None) as c: pass def test_bokeh(loop): pytest.importorskip('bokeh') import requests with LocalCluster(scheduler_port=0, silence_logs=False, loop=loop, diagnostics_port=0) as c: bokeh_port = c.scheduler.services['bokeh'].port url = 'http://127.0.0.1:%d/status/' % bokeh_port start = time() while True: response = requests.get(url) if response.ok: break assert time() < start + 20 sleep(0.01) with pytest.raises(requests.RequestException): requests.get(url, timeout=0.2) def test_blocks_until_full(loop): with Client(loop=loop) as c: assert len(c.ncores()) > 0 @gen_test() def test_scale_up_and_down(): loop = IOLoop.current() cluster = LocalCluster(0, scheduler_port=0, processes=False, silence_logs=False, diagnostics_port=None, loop=loop, start=False) c = yield Client(cluster, loop=loop, asynchronous=True) assert not cluster.workers yield cluster.scale_up(2) assert len(cluster.workers) == 2 assert len(cluster.scheduler.ncores) == 2 addr = cluster.workers[0].address yield cluster.scale_down([addr]) assert len(cluster.workers) == 1 assert addr not in cluster.scheduler.ncores yield c._close() yield cluster._close() def test_silent_startup(): code = """if 1: from time import sleep from distributed import LocalCluster with LocalCluster(1, diagnostics_port=None, scheduler_port=0): sleep(1.5) """ out = subprocess.check_output([sys.executable, "-Wi", "-c", code], stderr=subprocess.STDOUT) out = out.decode() try: assert not out except AssertionError: print("=== Cluster stdout / stderr ===") print(out) raise def test_only_local_access(loop): with LocalCluster(scheduler_port=0, silence_logs=False, diagnostics_port=None, loop=loop) as c: sync(loop, assert_can_connect_locally_4, c.scheduler.port) def test_remote_access(loop): with LocalCluster(scheduler_port=0, silence_logs=False, diagnostics_port=None, ip='', loop=loop) as c: sync(loop, assert_can_connect_from_everywhere_4_6, c.scheduler.port) def test_memory(loop): with LocalCluster(scheduler_port=0, processes=False, silence_logs=False, diagnostics_port=None, loop=loop) as cluster: assert sum(w.memory_limit for w in cluster.workers) <= TOTAL_MEMORY def test_memory_nanny(loop): with LocalCluster(scheduler_port=0, processes=True, silence_logs=False, diagnostics_port=None, loop=loop) as cluster: with Client(cluster.scheduler_address, loop=loop) as c: info = c.scheduler_info() assert (sum(w['memory_limit'] for w in info['workers'].values()) <= TOTAL_MEMORY) def test_death_timeout_raises(loop): with pytest.raises(gen.TimeoutError): with LocalCluster(scheduler_port=0, silence_logs=False, death_timeout=1e-10, diagnostics_port=None, loop=loop) as cluster: pass @pytest.mark.parametrize('processes', [True, False]) def test_diagnostics_available_at_localhost(loop, processes): pytest.importorskip('bokeh') import requests import random for i in range(3): port = random.randint(10000, 60000) try: with LocalCluster(2, scheduler_port=0, processes=processes, silence_logs=False, diagnostics_port=port, loop=loop) as c: requests.get('http://localhost:%d/status/' % port, timeout=3) requests.get('http://127.0.0.1:%d/status/' % port, timeout=1) except OSError: pass else: break distributed-1.20.2/distributed/deploy/tests/test_ssh.py000066400000000000000000000015451321233345200233070ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from time import sleep import pytest pytest.importorskip('paramiko') from distributed import Client from distributed.deploy.ssh import SSHCluster from distributed.metrics import time from distributed.utils_test import loop # flake8: noqa @pytest.mark.avoid_travis def test_cluster(loop): with SSHCluster(scheduler_addr='127.0.0.1', scheduler_port=7437, worker_addrs=['127.0.0.1', '127.0.0.1']) as c: with Client(c, loop=loop) as e: start = time() while len(e.ncores()) != 2: sleep(0.01) assert time() < start + 5 c.add_worker('127.0.0.1') start = time() while len(e.ncores()) != 3: sleep(0.01) assert time() < start + 5 distributed-1.20.2/distributed/deploy/utils_test.py000066400000000000000000000020331321233345200225010ustar00rootroot00000000000000from ..client import Client class ClusterTest(object): Cluster = None def setUp(self): self.cluster = self.Cluster(2, scheduler_port=0) self.client = Client(self.cluster.scheduler_address) def tearDown(self): self.client.close() self.cluster.close() def test_cores(self): 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() as c: with Client(c) as e: assert e.ncores() def test_no_workers(self): with self.Cluster(0, scheduler_port=0): pass distributed-1.20.2/distributed/diagnostics/000077500000000000000000000000001321233345200207455ustar00rootroot00000000000000distributed-1.20.2/distributed/diagnostics/__init__.py000066400000000000000000000004711321233345200230600ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from ..utils import ignoring with ignoring(ImportError): from .progressbar import progress with ignoring(ImportError): from .resource_monitor import Occupancy with ignoring(ImportError): from .scheduler_widgets import scheduler_status distributed-1.20.2/distributed/diagnostics/eventstream.py000066400000000000000000000042121321233345200236530ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging from tornado import gen from .plugin import SchedulerPlugin from ..core import connect, coerce_to_address from ..worker import dumps_function logger = logging.getLogger(__name__) class EventStream(SchedulerPlugin): """ Maintain a copy of worker events """ def __init__(self, scheduler=None): self.buffer = [] if scheduler: scheduler.add_plugin(self) def transition(self, key, start, finish, *args, **kwargs): if start == 'processing': kwargs['key'] = key if finish == 'memory' or finish == 'erred': self.buffer.append(kwargs) def swap_buffer(scheduler, es): es.buffer, buffer = [], es.buffer return buffer def teardown(scheduler, es): scheduler.remove_plugin(es) @gen.coroutine def eventstream(address, interval): """ Open a TCP connection to scheduler, receive batched task messages The messages coming back are lists of dicts. Each dict is of the following form:: {'key': 'mykey', 'worker': 'host:port', 'status': status, 'compute_start': time(), 'compute_stop': time(), 'transfer_start': time(), 'transfer_stop': time(), 'disk_load_start': time(), 'disk_load_stop': time(), 'other': 'junk'} Where ``status`` is either 'OK', or 'error' Parameters ---------- address: address of scheduler interval: time between batches, in seconds Examples -------- >>> stream = yield eventstream('127.0.0.1:8786', 0.100) # doctest: +SKIP >>> print(yield read(stream)) # doctest: +SKIP [{'key': 'x', 'status': 'OK', 'worker': '192.168.0.1:54684', ...}, {'key': 'y', 'status': 'error', 'worker': '192.168.0.1:54684', ...}] """ address = coerce_to_address(address) comm = yield connect(address) yield comm.write({'op': 'feed', 'setup': dumps_function(EventStream), 'function': dumps_function(swap_buffer), 'interval': interval, 'teardown': dumps_function(teardown)}) raise gen.Return(comm) distributed-1.20.2/distributed/diagnostics/plugin.py000066400000000000000000000045751321233345200226300ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging logger = logging.getLogger(__name__) class SchedulerPlugin(object): """ Interface to extend the Scheduler The scheduler operates by triggering and responding to events like ``task_finished``, ``update_graph``, ``task_erred``, etc.. A plugin enables custom code to run at each of those same events. The scheduler will run the analogous methods on this class when each event is triggered. This runs user code within the scheduler thread that can perform arbitrary operations in synchrony with the scheduler itself. Plugins are often used for diagnostics and measurement, but have full access to the scheduler and could in principle affect core scheduling. To implement a plugin implement some of the methods of this class and add the plugin to the scheduler with ``Scheduler.add_plugin(myplugin)``. Examples -------- >>> class Counter(SchedulerPlugin): ... def __init__(self): ... self.counter = 0 ... ... def transition(self, key, start, finish, *args, **kwargs): ... if start == 'processing' and finish == 'memory': ... self.counter += 1 ... ... def restart(self, scheduler): ... self.counter = 0 >>> c = Counter() >>> scheduler.add_plugin(c) # doctest: +SKIP """ def update_graph(self, scheduler, dsk=None, keys=None, restrictions=None, **kwargs): """ Run when a new graph / tasks enter the scheduler """ def restart(self, scheduler, **kwargs): """ Run when the scheduler restarts itself """ def transition(self, key, start, finish, *args, **kwargs): """ Run whenever a task changes state Parameters ---------- key: string start: string Start state of the transition. One of released, waiting, processing, memory, error. finish: string Final state of the transition. *args, **kwargs: More options passed when transitioning This may include worker ID, compute time, etc. """ def add_worker(self, scheduler=None, worker=None, **kwargs): """ Run when a new worker enters the cluster """ def remove_worker(self, scheduler=None, worker=None, **kwargs): """ Run when a worker leaves the cluster""" distributed-1.20.2/distributed/diagnostics/progress.py000066400000000000000000000302071321233345200231650ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import defaultdict import logging from timeit import default_timer from toolz import groupby, valmap from tornado import gen from .plugin import SchedulerPlugin from ..utils import key_split, key_split_group, log_errors, tokey logger = logging.getLogger(__name__) def dependent_keys(keys, who_has, processing, dependencies, exceptions, complete=False): """ All keys that need to compute for these keys to finish """ out = set() errors = set() stack = list(keys) while stack: key = stack.pop() if key in out: continue if not complete and (who_has.get(key) or key in processing): continue if key in exceptions: errors.add(key) if not complete: continue out.add(key) stack.extend(dependencies.get(key, [])) return out, errors class Progress(SchedulerPlugin): """ Tracks progress of a set of keys or futures On creation we provide a set of keys or futures that interest us as well as a scheduler. We traverse through the scheduler's dependencies to find all relevant keys on which our keys depend. We then plug into the scheduler to learn when our keys become available in memory at which point we record their completion. State ----- keys: set Set of keys that are not yet computed all_keys: set Set of all keys that we track This class performs no visualization. However it is used by other classes, notably TextProgressBar and ProgressWidget, which do perform visualization. """ def __init__(self, keys, scheduler, minimum=0, dt=0.1, complete=False): self.keys = {k.key if hasattr(k, 'key') else k for k in keys} self.keys = {tokey(k) for k in self.keys} self.scheduler = scheduler self.complete = complete self._minimum = minimum self._dt = dt self.last_duration = 0 self._start_time = default_timer() self._running = False self.status = None @gen.coroutine def setup(self): keys = self.keys while not keys.issubset(self.scheduler.task_state): yield gen.sleep(0.05) self.keys = None self.scheduler.add_plugin(self) # subtle race condition here self.all_keys, errors = dependent_keys(keys, self.scheduler.who_has, self.scheduler.processing, self.scheduler.dependencies, self.scheduler.exceptions, complete=self.complete) if not self.complete: self.keys = self.all_keys.copy() else: self.keys, _ = dependent_keys(keys, self.scheduler.who_has, self.scheduler.processing, self.scheduler.dependencies, self.scheduler.exceptions, complete=False) self.all_keys.update(keys) self.keys |= errors & self.all_keys if not self.keys: self.stop(exception=None, key=None) logger.debug("Set up Progress keys") for k in errors: self.transition(k, None, 'erred', exception=True) def transition(self, key, start, finish, *args, **kwargs): if key in self.keys and start == 'processing' and finish == 'memory': logger.debug("Progress sees key %s", key) self.keys.remove(key) if not self.keys: self.stop() if key in self.all_keys and finish == 'erred': logger.debug("Progress sees task erred") self.stop(exception=kwargs['exception'], key=key) if key in self.keys and finish == 'forgotten': logger.debug("A task was cancelled (%s), stopping progress", key) self.stop(exception=True) def restart(self, scheduler): self.stop() def stop(self, exception=None, key=None): if self in self.scheduler.plugins: self.scheduler.plugins.remove(self) if exception: self.status = 'error' else: self.status = 'finished' logger.debug("Remove Progress plugin") class MultiProgress(Progress): """ Progress variant that keeps track of different groups of keys See Progress for most details. This only adds a function ``func=`` that splits keys. This defaults to ``key_split`` which aligns with naming conventions chosen in the dask project (tuples, hyphens, etc..) State ----- keys: dict Maps group name to set of not-yet-complete keys for that group all_keys: dict Maps group name to set of all keys for that group Examples -------- >>> split = lambda s: s.split('-')[0] >>> p = MultiProgress(['y-2'], func=split) # doctest: +SKIP >>> p.keys # doctest: +SKIP {'x': {'x-1', 'x-2', 'x-3'}, 'y': {'y-1', 'y-2'}} """ def __init__(self, keys, scheduler=None, func=key_split, minimum=0, dt=0.1, complete=False): self.func = func Progress.__init__(self, keys, scheduler, minimum=minimum, dt=dt, complete=complete) @gen.coroutine def setup(self): keys = self.keys while not keys.issubset(self.scheduler.tasks): yield gen.sleep(0.05) self.keys = None self.scheduler.add_plugin(self) # subtle race condition here self.all_keys, errors = dependent_keys(keys, self.scheduler.who_has, self.scheduler.processing, self.scheduler.dependencies, self.scheduler.exceptions, complete=self.complete) if not self.complete: self.keys = self.all_keys.copy() else: self.keys, _ = dependent_keys(keys, self.scheduler.who_has, self.scheduler.processing, self.scheduler.dependencies, self.scheduler.exceptions, complete=False) self.all_keys.update(keys) self.keys |= errors & self.all_keys if not self.keys: self.stop(exception=None, key=None) # Group keys by func name self.keys = valmap(set, groupby(self.func, self.keys)) self.all_keys = valmap(set, groupby(self.func, self.all_keys)) for k in self.all_keys: if k not in self.keys: self.keys[k] = set() for k in errors: self.transition(k, None, 'erred', exception=True) logger.debug("Set up Progress keys") def transition(self, key, start, finish, *args, **kwargs): if start == 'processing' and finish == 'memory': s = self.keys.get(self.func(key), None) if s and key in s: s.remove(key) if not self.keys or not any(self.keys.values()): self.stop() if finish == 'erred': logger.debug("Progress sees task erred") k = self.func(key) if (k in self.all_keys and key in self.all_keys[k]): self.stop(exception=kwargs.get('exception'), key=key) if finish == 'forgotten': k = self.func(key) if k in self.all_keys and key in self.all_keys[k]: logger.debug("A task was cancelled (%s), stopping progress", key) self.stop(exception=True) def format_time(t): """Format seconds into a human readable form. >>> format_time(10.4) '10.4s' >>> format_time(1000.4) '16min 40.4s' >>> format_time(100000.4) '27hr 46min 40.4s' """ m, s = divmod(t, 60) h, m = divmod(m, 60) if h: return '{0:2.0f}hr {1:2.0f}min {2:4.1f}s'.format(h, m, s) elif m: return '{0:2.0f}min {1:4.1f}s'.format(m, s) else: return '{0:4.1f}s'.format(s) class AllProgress(SchedulerPlugin): """ Keep track of all keys, grouped by key_split """ def __init__(self, scheduler): self.all = defaultdict(set) self.nbytes = defaultdict(lambda: 0) self.state = defaultdict(lambda: defaultdict(set)) self.scheduler = scheduler for key, state in self.scheduler.task_state.items(): k = key_split(key) self.all[k].add(key) self.state[state][k].add(key) if key in self.scheduler.nbytes: self.nbytes[k] += self.scheduler.nbytes[key] scheduler.add_plugin(self) def transition(self, key, start, finish, *args, **kwargs): k = key_split(key) self.all[k].add(key) try: self.state[start][k].remove(key) except KeyError: # TODO: remove me once we have a new or clean state pass if finish != 'forgotten': self.state[finish][k].add(key) else: self.all[k].remove(key) if not self.all[k]: del self.all[k] try: del self.nbytes[k] except KeyError: pass for v in self.state.values(): try: del v[k] except KeyError: pass if start == 'memory': self.nbytes[k] -= self.scheduler.nbytes.get(key, 0) if finish == 'memory': self.nbytes[k] += self.scheduler.nbytes.get(key, 0) def restart(self, scheduler): self.all.clear() self.state.clear() class GroupProgress(SchedulerPlugin): """ Keep track of all keys, grouped by key_split """ def __init__(self, scheduler): self.scheduler = scheduler self.keys = dict() self.groups = dict() self.nbytes = dict() self.durations = dict() self.dependencies = dict() self.dependents = dict() for key, state in self.scheduler.task_state.items(): k = key_split_group(key) if k not in self.groups: self.create(key, k) self.keys[k].add(key) self.groups[k][state] += 1 if state == 'memory' and key in self.scheduler.nbytes: self.nbytes[k] += self.scheduler.nbytes[key] scheduler.add_plugin(self) def create(self, key, k): with log_errors(pdb=True): g = {'memory': 0, 'erred': 0, 'waiting': 0, 'released': 0, 'processing': 0} self.keys[k] = set() self.groups[k] = g self.nbytes[k] = 0 self.durations[k] = 0 self.dependents[k] = {key_split_group(dep) for dep in self.scheduler.dependents[key]} self.dependencies[k] = set() for dep in self.scheduler.dependencies[key]: d = key_split_group(dep) self.dependents[d].add(k) self.dependencies[k].add(d) def transition(self, key, start, finish, *args, **kwargs): with log_errors(): k = key_split_group(key) if k not in self.groups: self.create(key, k) g = self.groups[k] if key not in self.keys[k]: self.keys[k].add(key) else: g[start] -= 1 if finish != 'forgotten': g[finish] += 1 else: self.keys[k].remove(key) if not self.keys[k]: del self.groups[k] del self.nbytes[k] for dep in self.dependencies.pop(k): self.dependents[key_split_group(dep)].remove(k) if start == 'memory': self.nbytes[k] -= self.scheduler.nbytes[key] if finish == 'memory': self.nbytes[k] += self.scheduler.nbytes[key] def restart(self, scheduler): self.keys.clear() self.groups.clear() self.nbytes.clear() self.durations.clear() self.dependencies.clear() self.dependents.clear() distributed-1.20.2/distributed/diagnostics/progress_stream.py000066400000000000000000000146771321233345200245550ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import itertools import logging import random from bokeh.palettes import viridis from toolz import valmap, merge, memoize from tornado import gen from .progress import AllProgress from ..core import connect, coerce_to_address from ..scheduler import Scheduler from ..utils import key_split from ..worker import dumps_function logger = logging.getLogger(__name__) task_stream_palette = list(viridis(25)) random.shuffle(task_stream_palette) def counts(scheduler, allprogress): return merge({'all': valmap(len, allprogress.all), 'nbytes': allprogress.nbytes}, {state: valmap(len, allprogress.state[state]) for state in ['memory', 'erred', 'released']}) counter = itertools.count() _incrementing_index_cache = dict() @memoize(cache=_incrementing_index_cache) def incrementing_index(o): return next(counter) def color_of(o, palette=task_stream_palette): return palette[incrementing_index(o) % len(palette)] @gen.coroutine def progress_stream(address, interval): """ Open a TCP connection to scheduler, receive progress messages The messages coming back are dicts containing counts of key groups:: {'inc': {'all': 5, 'memory': 2, 'erred': 0, 'released': 1}, 'dec': {'all': 1, 'memory': 0, 'erred': 0, 'released': 0}} Parameters ---------- address: address of scheduler interval: time between batches, in seconds Examples -------- >>> stream = yield eventstream('127.0.0.1:8786', 0.100) # doctest: +SKIP >>> print(yield read(stream)) # doctest: +SKIP """ address = coerce_to_address(address) comm = yield connect(address) yield comm.write({'op': 'feed', 'setup': dumps_function(AllProgress), 'function': dumps_function(counts), 'interval': interval, 'teardown': dumps_function(Scheduler.remove_plugin)}) raise gen.Return(comm) def nbytes_bar(nbytes): """ Convert nbytes message into rectangle placements >>> nbytes_bar({'inc': 1000, 'dec': 3000}) # doctest: +NORMALIZE_WHITESPACE {'names': ['dec', 'inc'], 'left': [0, 0.75], 'center': [0.375, 0.875], 'right': [0.75, 1.0]} """ total = sum(nbytes.values()) names = sorted(nbytes) d = {'name': [], 'text': [], 'left': [], 'right': [], 'center': [], 'color': [], 'percent': [], 'MB': []} if not total: return d right = 0 for name in names: left = right right = nbytes[name] / total + left center = (right + left) / 2 d['MB'].append(nbytes[name] / 1000000) d['percent'].append(round(nbytes[name] / total * 100, 2)) d['left'].append(left) d['right'].append(right) d['center'].append(center) d['color'].append(color_of(name)) d['name'].append(name) if right - left > 0.1: d['text'].append(name) else: d['text'].append('') return d def progress_quads(msg, nrows=8, ncols=3): """ >>> msg = {'all': {'inc': 5, 'dec': 1, 'add': 4}, ... 'memory': {'inc': 2, 'dec': 0, 'add': 1}, ... 'erred': {'inc': 0, 'dec': 1, 'add': 0}, ... 'released': {'inc': 1, 'dec': 0, 'add': 1}} >>> progress_quads(msg, nrows=2) # doctest: +SKIP {'name': ['inc', 'add', 'dec'], 'left': [0, 0, 1], 'right': [0.9, 0.9, 1.9], 'top': [0, -1, 0], 'bottom': [-.8, -1.8, -.8], 'released': [1, 1, 0], 'memory': [2, 1, 0], 'erred': [0, 0, 1], 'done': ['3 / 5', '2 / 4', '1 / 1'], 'released-loc': [.2/.9, .25 / 0.9, 1], 'memory-loc': [3 / 5 / .9, .5 / 0.9, 1], 'erred-loc': [3 / 5 / .9, .5 / 0.9, 1.9]} """ width = 0.9 names = sorted(msg['all'], key=msg['all'].get, reverse=True) names = names[:nrows * ncols] n = len(names) d = {k: [v.get(name, 0) for name in names] for k, v in msg.items()} d['name'] = names d['show-name'] = [name if len(name) <= 15 else name[:12] + '...' for name in names] d['left'] = [i // nrows for i in range(n)] d['right'] = [i // nrows + width for i in range(n)] d['top'] = [-(i % nrows) for i in range(n)] d['bottom'] = [-(i % nrows) - 0.8 for i in range(n)] d['color'] = [color_of(name) for name in names] d['released-loc'] = [] d['memory-loc'] = [] d['erred-loc'] = [] d['done'] = [] for r, m, e, a, l in zip(d['released'], d['memory'], d['erred'], d['all'], d['left']): rl = width * r / a + l ml = width * (r + m) / a + l el = width * (r + m + e) / a + l done = '%d / %d' % (r + m + e, a) d['released-loc'].append(rl) d['memory-loc'].append(ml) d['erred-loc'].append(el) d['done'].append(done) return d def color_of_message(msg): if msg['status'] == 'OK': split = key_split(msg['key']) return color_of(split) else: return 'black' colors = {'transfer': 'red', 'disk-write': 'orange', 'disk-read': 'orange', 'deserialize': 'gray', 'compute': color_of_message} alphas = {'transfer': 0.4, 'compute': 1, 'deserialize': 0.4, 'disk-write': 0.4, 'disk-read': 0.4} prefix = {'transfer': 'transfer-', 'disk-write': 'disk-write-', 'disk-read': 'disk-read-', 'deserialize': 'deserialize-', 'compute': ''} def task_stream_append(lists, msg, workers, palette=task_stream_palette): key = msg['key'] name = key_split(key) startstops = msg.get('startstops', []) for action, start, stop in startstops: color = colors[action] if type(color) is not str: color = color(msg) lists['start'].append((start + stop) / 2 * 1000) lists['duration'].append(1000 * (stop - start)) lists['key'].append(key) lists['name'].append(prefix[action] + name) lists['color'].append(color) lists['alpha'].append(alphas[action]) lists['worker'].append(msg['worker']) worker_thread = '%s-%d' % (msg['worker'], msg['thread']) lists['worker_thread'].append(worker_thread) if worker_thread not in workers: workers[worker_thread] = len(workers) / 2 lists['y'].append(workers[worker_thread]) return len(startstops) distributed-1.20.2/distributed/diagnostics/progressbar.py000066400000000000000000000300771321233345200236570ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging from timeit import default_timer import sys from toolz import valmap from tornado import gen from tornado.ioloop import IOLoop from .progress import format_time, Progress, MultiProgress from ..compatibility import html_escape from ..core import connect, coerce_to_address, CommClosedError from ..client import default_client, futures_of from ..protocol.pickle import dumps from ..utils import ignoring, key_split, is_kernel, LoopRunner logger = logging.getLogger(__name__) def get_scheduler(scheduler): if scheduler is None: return default_client().scheduler.address return coerce_to_address(scheduler) class ProgressBar(object): def __init__(self, keys, scheduler=None, interval=0.1, complete=True): self.scheduler = get_scheduler(scheduler) self.keys = {k.key if hasattr(k, 'key') else k for k in keys} self.interval = interval self.complete = complete self._start_time = default_timer() @property def elapsed(self): return default_timer() - self._start_time @gen.coroutine def listen(self): complete = self.complete keys = self.keys @gen.coroutine def setup(scheduler): p = Progress(keys, scheduler, complete=complete) yield p.setup() raise gen.Return(p) def function(scheduler, p): return {'all': len(p.all_keys), 'remaining': len(p.keys), 'status': p.status} self.comm = yield connect(self.scheduler) logger.debug("Progressbar Connected to scheduler") yield self.comm.write({'op': 'feed', 'setup': dumps(setup), 'function': dumps(function), 'interval': self.interval}) while True: try: response = yield self.comm.read() except CommClosedError: break self._last_response = response self.status = response['status'] self._draw_bar(**response) if response['status'] in ('error', 'finished'): yield self.comm.close() self._draw_stop(**response) break logger.debug("Progressbar disconnected from scheduler") def _draw_stop(self, **kwargs): pass def __del__(self): with ignoring(AttributeError): self.comm.abort() class TextProgressBar(ProgressBar): def __init__(self, keys, scheduler=None, interval=0.1, width=40, loop=None, complete=True, start=True): super(TextProgressBar, self).__init__(keys, scheduler, interval, complete) self.width = width self.loop = loop or IOLoop() if start: loop_runner = LoopRunner(self.loop) loop_runner.run_sync(self.listen) def _draw_bar(self, remaining, all, **kwargs): frac = (1 - remaining / all) if all else 1.0 bar = '#' * int(self.width * frac) percent = int(100 * frac) elapsed = format_time(self.elapsed) msg = '\r[{0:<{1}}] | {2}% Completed | {3}'.format(bar, self.width, percent, elapsed) with ignoring(ValueError): sys.stdout.write(msg) sys.stdout.flush() class ProgressWidget(ProgressBar): """ ProgressBar that uses an IPython ProgressBar widget for the notebook See Also -------- progress: User function TextProgressBar: Text version suitable for the console """ def __init__(self, keys, scheduler=None, interval=0.1, complete=False, loop=None): super(ProgressWidget, self).__init__(keys, scheduler, interval, complete) from ipywidgets import FloatProgress, HBox, VBox, HTML self.elapsed_time = HTML('') self.bar = FloatProgress(min=0, max=1, description='', height='10px') self.bar_text = HTML('', width="140px") self.bar_widget = HBox([self.bar_text, self.bar]) self.widget = VBox([self.elapsed_time, self.bar_widget]) def _ipython_display_(self, **kwargs): IOLoop.current().add_callback(self.listen) return self.widget._ipython_display_(**kwargs) def _draw_stop(self, remaining, status, **kwargs): if status == 'error': self.bar.bar_style = 'danger' self.elapsed_time.value = '
    Exception: ' + \ format_time(self.elapsed) + '
    ' elif not remaining: self.bar.bar_style = 'success' self.elapsed_time.value = '
    Finished: ' + \ format_time(self.elapsed) + '
    ' def _draw_bar(self, remaining, all, **kwargs): ndone = all - remaining self.elapsed_time.value = '
    Computing: ' + \ format_time(self.elapsed) + '
    ' self.bar.value = ndone / all if all else 1.0 self.bar_text.value = '
    %d / %d
    ' % (ndone, all) class MultiProgressBar(object): def __init__(self, keys, scheduler=None, func=key_split, interval=0.1, complete=False): self.scheduler = get_scheduler(scheduler) self.keys = {k.key if hasattr(k, 'key') else k for k in keys} self.func = func self.interval = interval self.complete = complete self._start_time = default_timer() @property def elapsed(self): return default_timer() - self._start_time @gen.coroutine def listen(self): complete = self.complete keys = self.keys func = self.func @gen.coroutine def setup(scheduler): p = MultiProgress(keys, scheduler, complete=complete, func=func) yield p.setup() raise gen.Return(p) def function(scheduler, p): return {'all': valmap(len, p.all_keys), 'remaining': valmap(len, p.keys), 'status': p.status} self.comm = yield connect(self.scheduler) logger.debug("Progressbar Connected to scheduler") yield self.comm.write({'op': 'feed', 'setup': dumps(setup), 'function': dumps(function), 'interval': self.interval}) while True: response = yield self.comm.read() self._last_response = response self.status = response['status'] self._draw_bar(**response) if response['status'] in ('error', 'finished'): yield self.comm.close() self._draw_stop(**response) break logger.debug("Progressbar disconnected from scheduler") def _draw_stop(self, **kwargs): pass def __del__(self): with ignoring(AttributeError): self.comm.abort() class MultiProgressWidget(MultiProgressBar): """ Multiple progress bar Widget suitable for the notebook Displays multiple progress bars for a computation, split on computation type. See Also -------- progress: User-level function <--- use this MultiProgress: Non-visualization component that contains most logic ProgressWidget: Single progress bar widget """ def __init__(self, keys, scheduler=None, minimum=0, interval=0.1, func=key_split, complete=False): super(MultiProgressWidget, self).__init__(keys, scheduler, func, interval, complete) from ipywidgets import VBox self.widget = VBox([]) def make_widget(self, all): from ipywidgets import FloatProgress, HBox, VBox, HTML self.elapsed_time = HTML('') self.bars = {key: FloatProgress(min=0, max=1, description='', height='10px') for key in all} self.bar_texts = {key: HTML('', width="140px") for key in all} self.bar_labels = {key: HTML('
    ' + html_escape(key.decode() if isinstance(key, bytes) else key) + '
    ') for key in all} def keyfunc(kv): """ Order keys by most numerous, then by string name """ return kv[::-1] key_order = [k for k, v in sorted(all.items(), key=keyfunc, reverse=True)] self.bar_widgets = VBox([HBox([self.bar_texts[key], self.bars[key], self.bar_labels[key]]) for key in key_order]) self.widget.children = (self.elapsed_time, self.bar_widgets) def _ipython_display_(self, **kwargs): IOLoop.current().add_callback(self.listen) return self.widget._ipython_display_(**kwargs) def _draw_stop(self, remaining, status, exception=None, key=None, **kwargs): for k, v in remaining.items(): if not v: self.bars[k].bar_style = 'success' else: self.bars[k].bar_style = 'danger' if status == 'error': # self.bars[self.func(key)].bar_style = 'danger' # TODO self.elapsed_time.value = '
    Exception: ' + \ format_time(self.elapsed) + '
    ' else: self.elapsed_time.value = '
    Finished: ' + \ format_time(self.elapsed) + '
    ' def _draw_bar(self, remaining, all, status, **kwargs): if self.keys and not self.widget.children: self.make_widget(all) for k, ntasks in all.items(): ndone = ntasks - remaining[k] self.elapsed_time.value = '
    Computing: ' + \ format_time(self.elapsed) + '
    ' self.bars[k].value = ndone / ntasks if ntasks else 1.0 self.bar_texts[k].value = '
    %d / %d
    ' % ( ndone, ntasks) def progress(*futures, **kwargs): """ Track progress of futures This operates differently in the notebook and the console * Notebook: This returns immediately, leaving an IPython widget on screen * Console: This blocks until the computation completes Parameters ---------- futures: Futures A list of futures or keys to track notebook: bool (optional) Running in the notebook or not (defaults to guess) multi: bool (optional) Track different functions independently (defaults to True) complete: bool (optional) Track all keys (True) or only keys that have not yet run (False) (defaults to True) Notes ----- In the notebook, the output of `progress` must be the last statement in the cell. Typically, this means calling `progress` at the end of a cell. Examples -------- >>> progress(futures) # doctest: +SKIP [########################################] | 100% Completed | 1.7s """ 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)): futures = [futures] if notebook is None: notebook = is_kernel() # often but not always correct assumption if notebook: if multi: bar = MultiProgressWidget(futures, complete=complete) else: bar = ProgressWidget(futures, complete=complete) return bar else: TextProgressBar(futures, complete=complete) distributed-1.20.2/distributed/diagnostics/tests/000077500000000000000000000000001321233345200221075ustar00rootroot00000000000000distributed-1.20.2/distributed/diagnostics/tests/test_eventstream.py000066400000000000000000000035541321233345200260640ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from copy import deepcopy import pytest from tornado import gen from distributed.client import wait from distributed.diagnostics.eventstream import EventStream, eventstream from distributed.metrics import time from distributed.utils_test import div, gen_cluster @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_eventstream(c, s, *workers): pytest.importorskip('bokeh') es = EventStream() s.add_plugin(es) assert es.buffer == [] futures = c.map(div, [1] * 10, range(10)) total = c.submit(sum, futures[1:]) yield wait(total) yield wait(futures) 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']) workers = dict() for msg in es.buffer: task_stream_append(lists, msg, workers) assert len([n for n in lists['name'] if n.startswith('transfer')]) == 2 for name, color in zip(lists['name'], lists['color']): if name == 'transfer': assert color == 'red' assert any(c == 'black' for c in lists['color']) @gen_cluster(client=True) def test_eventstream_remote(c, s, a, b): base_plugins = len(s.plugins) comm = yield eventstream(s.address, interval=0.010) start = time() while len(s.plugins) == base_plugins: yield gen.sleep(0.01) assert time() < start + 5 futures = c.map(div, [1] * 10, range(10)) start = time() total = [] while len(total) < 10: msgs = yield comm.read() assert isinstance(msgs, list) total.extend(msgs) assert time() < start + 5 yield comm.close() start = time() while len(s.plugins) > base_plugins: yield gen.sleep(0.01) assert time() < start + 5 distributed-1.20.2/distributed/diagnostics/tests/test_plugin.py000066400000000000000000000035221321233345200250200ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from distributed import Worker from distributed.utils_test import inc, gen_cluster from distributed.diagnostics.plugin import SchedulerPlugin @gen_cluster(client=True) def test_simple(c, s, a, b): class Counter(SchedulerPlugin): def start(self, scheduler): self.scheduler = scheduler scheduler.add_plugin(self) self.count = 0 def transition(self, key, start, finish, *args, **kwargs): if start == 'processing' and finish == 'memory': self.count += 1 counter = Counter() counter.start(s) assert counter in s.plugins assert counter.count == 0 x = c.submit(inc, 1) y = c.submit(inc, x) z = c.submit(inc, y) yield z assert counter.count == 3 s.remove_plugin(counter) assert counter not in s.plugins @gen_cluster(ncores=[], client=False) def test_add_remove_worker(s): events = [] class MyPlugin(SchedulerPlugin): def add_worker(self, worker, scheduler): assert scheduler is s events.append(('add_worker', worker)) def remove_worker(self, worker, scheduler): assert scheduler is s events.append(('remove_worker', worker)) plugin = MyPlugin() s.add_plugin(plugin) assert events == [] a = Worker(s.address) b = Worker(s.address) yield a._start() yield b._start() yield a._close() yield b._close() assert events == [('add_worker', a.address), ('add_worker', b.address), ('remove_worker', a.address), ('remove_worker', b.address), ] events[:] = [] s.remove_plugin(plugin) a = Worker(s.address) yield a._start() yield a._close() assert events == [] distributed-1.20.2/distributed/diagnostics/tests/test_progress.py000066400000000000000000000130651321233345200253710ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from operator import add import pytest from tornado import gen from dask.core import get_deps from distributed import Nanny from distributed.client import wait from distributed.metrics import time from distributed.utils_test import gen_cluster, inc, dec, div, nodebug from distributed.diagnostics.progress import (Progress, SchedulerPlugin, AllProgress, GroupProgress, MultiProgress, dependent_keys) def test_dependent_keys(): a, b, c, d, e, f, g = 'abcdefg' who_has = {a: [1], b: [1]} processing = {'alice': {c}} exceptions = {} dsk = {a: 1, b: 2, c: (add, a, b), d: (inc, a), e: (add, c, d), f: (inc, e)} dependencies, dependents = get_deps(dsk) assert dependent_keys(f, who_has, processing, dependencies, exceptions, complete=False)[0] == {f, e, c, d} assert dependent_keys(f, who_has, processing, dependencies, exceptions, complete=True)[0] == {a, b, c, d, e, f} def f(*args): pass def g(*args): pass def h(*args): pass @nodebug @gen_cluster(client=True) def test_many_Progress(c, s, a, b): x = c.submit(f, 1) y = c.submit(g, x) z = c.submit(h, y) bars = [Progress(keys=[z], scheduler=s) for i in range(10)] yield [bar.setup() for bar in bars] yield z start = time() while not all(b.status == 'finished' for b in bars): yield gen.sleep(0.1) assert time() < start + 2 @gen_cluster(client=True) def test_multiprogress(c, s, a, b): x1 = c.submit(f, 1) x2 = c.submit(f, x1) x3 = c.submit(f, x2) y1 = c.submit(g, x3) y2 = c.submit(g, y1) p = MultiProgress([y2], scheduler=s, complete=True) yield p.setup() assert p.all_keys == {'f': {f.key for f in [x1, x2, x3]}, 'g': {f.key for f in [y1, y2]}} yield x3 assert p.keys['f'] == set() yield y2 assert p.keys == {'f': set(), 'g': set()} assert p.status == 'finished' @gen_cluster(client=True) def test_robust_to_bad_plugin(c, s, a, b): class Bad(SchedulerPlugin): def transition(self, key, start, finish, **kwargs): raise Exception() bad = Bad() s.add_plugin(bad) x = c.submit(inc, 1) y = c.submit(inc, x) result = yield y assert result == 3 def check_bar_completed(capsys, width=40): out, err = capsys.readouterr() bar, percent, time = [i.strip() for i in out.split('\r')[-1].split('|')] assert bar == '[' + '#' * width + ']' assert percent == '100% Completed' @gen_cluster(client=True, Worker=Nanny, timeout=None) def test_AllProgress(c, s, a, b): x, y, z = c.map(inc, [1, 2, 3]) xx, yy, zz = c.map(dec, [x, y, z]) yield wait([x, y, z]) p = AllProgress(s) assert p.all['inc'] == {x.key, y.key, z.key} assert p.state['memory']['inc'] == {x.key, y.key, z.key} assert p.state['released'] == {} assert p.state['erred'] == {} assert 'inc' in p.nbytes assert isinstance(p.nbytes['inc'], int) assert p.nbytes['inc'] > 0 yield wait([xx, yy, zz]) assert p.all['dec'] == {xx.key, yy.key, zz.key} assert p.state['memory']['dec'] == {xx.key, yy.key, zz.key} assert p.state['released'] == {} assert p.state['erred'] == {} assert p.nbytes['inc'] == p.nbytes['dec'] t = c.submit(sum, [x, y, z]) yield t keys = {x.key, y.key, z.key} del x, y, z import gc gc.collect() while any(k in s.who_has for k in keys): yield gen.sleep(0.01) assert p.state['released']['inc'] == keys assert p.all['inc'] == keys assert p.all['dec'] == {xx.key, yy.key, zz.key} if 'inc' in p.nbytes: assert p.nbytes['inc'] == 0 xxx = c.submit(div, 1, 0) yield wait([xxx]) assert p.state['erred'] == {'div': {xxx.key}} tkey = t.key del xx, yy, zz, t import gc gc.collect() while tkey in s.task_state: yield gen.sleep(0.01) for coll in [p.all, p.nbytes] + list(p.state.values()): assert 'inc' not in coll assert 'dec' not in coll def f(x): return x for i in range(4): future = c.submit(f, i) import gc gc.collect() yield gen.sleep(1) yield wait([future]) assert p.state['memory'] == {'f': {future.key}} yield c._restart() for coll in [p.all] + list(p.state.values()): assert not coll x = c.submit(div, 1, 2) yield wait([x]) assert set(p.all) == {'div'} assert all(set(d) == {'div'} for d in p.state.values()) @gen_cluster(client=True, Worker=Nanny) def test_AllProgress_lost_key(c, s, a, b, timeout=None): p = AllProgress(s) futures = c.map(inc, range(5)) yield wait(futures) assert len(p.state['memory']['inc']) == 5 yield a._close() yield b._close() start = time() while len(p.state['memory']['inc']) > 0: yield gen.sleep(0.1) assert time() < start + 2 @gen_cluster(client=True) def test_GroupProgress(c, s, a, b): da = pytest.importorskip('dask.array') fp = GroupProgress(s) x = da.ones(100, chunks=10) y = x + 1 z = (x * y).sum().persist(optimize_graph=False) yield wait(z) assert 3 < len(fp.groups) < 10 for k, g in fp.groups.items(): assert fp.keys[k] assert len(fp.keys[k]) == sum(g.values()) assert all(v >= 0 for v in g.values()) assert fp.dependencies[y.name] == {x.name} assert fp.dependents[x.name] == {y.name, (x * y).name} del x, y, z while s.tasks: yield gen.sleep(0.01) assert not fp.groups distributed-1.20.2/distributed/diagnostics/tests/test_progress_stream.py000066400000000000000000000063441321233345200267460ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import pytest pytest.importorskip('bokeh') from dask import delayed from distributed.client import wait from distributed.diagnostics.progress_stream import (progress_quads, nbytes_bar, progress_stream, _incrementing_index_cache) from distributed.utils_test import div, gen_cluster, inc def test_progress_quads(): msg = {'all': {'inc': 5, 'dec': 1, 'add': 4}, 'memory': {'inc': 2, 'dec': 0, 'add': 1}, 'erred': {'inc': 0, 'dec': 1, 'add': 0}, 'released': {'inc': 1, 'dec': 0, 'add': 1}} _incrementing_index_cache.clear() d = progress_quads(msg, nrows=2) color = d.pop('color') assert len(set(color)) == 3 assert d == {'name': ['inc', 'add', 'dec'], 'show-name': ['inc', 'add', 'dec'], 'left': [0, 0, 1], 'right': [0.9, 0.9, 1.9], 'top': [0, -1, 0], 'bottom': [-.8, -1.8, -.8], 'all': [5, 4, 1], 'released': [1, 1, 0], 'memory': [2, 1, 0], 'erred': [0, 0, 1], 'done': ['3 / 5', '2 / 4', '1 / 1'], 'released-loc': [.9 * 1 / 5, .25 * 0.9, 1.0], 'memory-loc': [.9 * 3 / 5, .5 * 0.9, 1.0], 'erred-loc': [.9 * 3 / 5, .5 * 0.9, 1.9]} def test_progress_quads_too_many(): keys = ['x-%d' % i for i in range(1000)] msg = {'all': {k: 1 for k in keys}, 'memory': {k: 0 for k in keys}, 'erred': {k: 0 for k in keys}, 'released': {k: 0 for k in keys}} d = progress_quads(msg, nrows=6, ncols=3) assert len(d['name']) == 6 * 3 @gen_cluster(client=True) def test_progress_stream(c, s, a, b): futures = c.map(div, [1] * 10, range(10)) x = 1 for i in range(5): x = delayed(inc)(x) future = c.compute(x) yield wait(futures + [future]) comm = yield progress_stream(s.address, interval=0.010) msg = yield comm.read() nbytes = msg.pop('nbytes') assert msg == {'all': {'div': 10, 'inc': 5}, 'erred': {'div': 1}, 'memory': {'div': 9, 'inc': 1}, 'released': {'inc': 4}} assert set(nbytes) == set(msg['all']) assert all(v > 0 for v in nbytes.values()) assert progress_quads(msg) yield comm.close() def test_nbytes_bar(): nbytes = {'inc': 1000, 'dec': 3000} expected = {'name': ['dec', 'inc'], 'left': [0, 0.75], 'center': [0.375, 0.875], 'right': [0.75, 1.0], 'percent': [75, 25], 'MB': [0.003, 0.001], 'text': ['dec', 'inc']} result = nbytes_bar(nbytes) color = result.pop('color') assert len(set(color)) == 2 assert result == expected def test_progress_quads_many_functions(): funcnames = ['fn%d' % i for i in range(1000)] msg = {'all': {fn: 1 for fn in funcnames}, 'memory': {fn: 1 for fn in funcnames}, 'erred': {fn: 0 for fn in funcnames}, 'released': {fn: 0 for fn in funcnames}} d = progress_quads(msg, nrows=2) color = d.pop('color') assert len(set(color)) < 100 distributed-1.20.2/distributed/diagnostics/tests/test_progressbar.py000066400000000000000000000054411321233345200260550ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from time import sleep from tornado import gen from distributed import Client, Scheduler, Worker from distributed.diagnostics.progressbar import TextProgressBar, progress from distributed.metrics import time from distributed.utils_test import (cluster, inc, div, gen_cluster) from distributed.utils_test import loop # flake8: noqa def test_text_progressbar(capsys, loop): with cluster(nanny=True) as (s, [a, b]): with Client(s['address'], loop=loop) as c: futures = c.map(inc, range(10)) p = TextProgressBar(futures, interval=0.01, complete=True) c.gather(futures) start = time() while p.status != 'finished': sleep(0.01) assert time() - start < 5 check_bar_completed(capsys) assert p._last_response == {'all': 10, 'remaining': 0, 'status': 'finished'} assert p.comm.closed() @gen_cluster(client=True) def test_TextProgressBar_error(c, s, a, b): x = c.submit(div, 1, 0) progress = TextProgressBar([x.key], scheduler=(s.ip, s.port), start=False, interval=0.01) yield progress.listen() assert progress.status == 'error' assert progress.comm.closed() progress = TextProgressBar([x.key], scheduler=(s.ip, s.port), start=False, interval=0.01) yield progress.listen() assert progress.status == 'error' assert progress.comm.closed() def test_TextProgressBar_empty(loop, capsys): @gen.coroutine def f(): s = Scheduler(loop=loop) done = s.start(0) a = Worker(s.ip, s.port, loop=loop, ncores=1) b = Worker(s.ip, s.port, loop=loop, ncores=1) yield [a._start(0), b._start(0)] progress = TextProgressBar([], scheduler=(s.ip, s.port), start=False, interval=0.01) yield progress.listen() assert progress.status == 'finished' check_bar_completed(capsys) yield [a._close(), b._close()] s.close() yield done loop.run_sync(f) def check_bar_completed(capsys, width=40): out, err = capsys.readouterr() bar, percent, time = [i.strip() for i in out.split('\r')[-1].split('|')] assert bar == '[' + '#' * width + ']' assert percent == '100% Completed' def test_progress_function(loop, capsys): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: f = c.submit(lambda: 1) g = c.submit(lambda: 2) progress([[f], [[g]]], notebook=False) check_bar_completed(capsys) progress(f) check_bar_completed(capsys) distributed-1.20.2/distributed/diagnostics/tests/test_widgets.py000066400000000000000000000200441321233345200251660ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import pytest pytest.importorskip('ipywidgets') from ipykernel.comm import Comm from ipywidgets import Widget ################# # Utility stuff # ################# # Taken from ipywidgets/widgets/tests/test_interaction.py # https://github.com/ipython/ipywidgets # Licensed under Modified BSD. Copyright IPython Development Team. See: # https://github.com/ipython/ipywidgets/blob/master/COPYING.md class DummyComm(Comm): comm_id = 'a-b-c-d' def open(self, *args, **kwargs): pass def send(self, *args, **kwargs): pass def close(self, *args, **kwargs): pass _widget_attrs = {} displayed = [] undefined = object() def setup(): _widget_attrs['_comm_default'] = getattr(Widget, '_comm_default', undefined) Widget._comm_default = lambda self: DummyComm() _widget_attrs['_ipython_display_'] = Widget._ipython_display_ def raise_not_implemented(*args, **kwargs): raise NotImplementedError() Widget._ipython_display_ = raise_not_implemented def teardown(): for attr, value in _widget_attrs.items(): if value is undefined: delattr(Widget, attr) else: setattr(Widget, attr, value) def f(**kwargs): pass def clear_display(): global displayed displayed = [] def record_display(*args): displayed.extend(args) # End code taken from ipywidgets ##################### # Distributed stuff # ##################### from operator import add import re from toolz import valmap from distributed.client import Client, wait from distributed.worker import dumps_task from distributed.utils_test import (cluster, inc, dec, throws, gen_cluster) from distributed.utils_test import loop # flake8: noqa from distributed.utils import sync from distributed.diagnostics.progressbar import (ProgressWidget, MultiProgressWidget, progress) @gen_cluster() def test_progressbar_widget(s, a, b): s.update_graph(tasks=valmap(dumps_task, {'x': (inc, 1), 'y': (inc, 'x'), 'z': (inc, 'y')}), keys=['z'], dependencies={'y': {'x'}, 'z': {'y'}}) progress = ProgressWidget(['z'], scheduler=(s.ip, s.port)) yield progress.listen() assert progress.bar.value == 1.0 assert '3 / 3' in progress.bar_text.value progress = ProgressWidget(['z'], scheduler=(s.ip, s.port)) yield progress.listen() @gen_cluster() def test_multi_progressbar_widget(s, a, b): s.update_graph(tasks=valmap(dumps_task, {'x-1': (inc, 1), 'x-2': (inc, 'x-1'), 'x-3': (inc, 'x-2'), 'y-1': (dec, 'x-3'), 'y-2': (dec, 'y-1'), 'e': (throws, 'y-2'), 'other': (inc, 123)}), keys=['e'], dependencies={'x-2': ['x-1'], 'x-3': ['x-2'], 'y-1': ['x-3'], 'y-2': ['y-1'], 'e': ['y-2']}) p = MultiProgressWidget(['e'], scheduler=(s.ip, s.port)) yield p.listen() assert p.bars['x'].value == 1.0 assert p.bars['y'].value == 1.0 assert p.bars['e'].value == 0.0 assert '3 / 3' in p.bar_texts['x'].value assert '2 / 2' in p.bar_texts['y'].value assert '0 / 1' in p.bar_texts['e'].value assert p.bars['x'].bar_style == 'success' assert p.bars['y'].bar_style == 'success' assert p.bars['e'].bar_style == 'danger' assert p.status == 'error' assert 'Exception' in p.elapsed_time.value capacities = [int(re.search(r'\d+ / \d+', row.children[0].value) .group().split(' / ')[1]) for row in p.bar_widgets.children] assert sorted(capacities, reverse=True) == capacities @gen_cluster() def test_multi_progressbar_widget_after_close(s, a, b): s.update_graph(tasks=valmap(dumps_task, {'x-1': (inc, 1), 'x-2': (inc, 'x-1'), 'x-3': (inc, 'x-2'), 'y-1': (dec, 'x-3'), 'y-2': (dec, 'y-1'), 'e': (throws, 'y-2'), 'other': (inc, 123)}), keys=['e'], dependencies={'x-2': {'x-1'}, 'x-3': {'x-2'}, 'y-1': {'x-3'}, 'y-2': {'y-1'}, 'e': {'y-2'}}) p = MultiProgressWidget(['x-1', 'x-2', 'x-3'], scheduler=(s.ip, s.port)) yield p.listen() assert 'x' in p.bars def test_values(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: L = [c.submit(inc, i) for i in range(5)] wait(L) p = MultiProgressWidget(L) sync(loop, p.listen) assert set(p.bars) == {'inc'} assert p.status == 'finished' assert p.comm.closed() assert '5 / 5' in p.bar_texts['inc'].value assert p.bars['inc'].value == 1.0 x = c.submit(throws, 1) p = MultiProgressWidget([x]) sync(loop, p.listen) assert p.status == 'error' def test_progressbar_done(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: L = [c.submit(inc, i) for i in range(5)] wait(L) p = ProgressWidget(L) sync(loop, p.listen) assert p.status == 'finished' assert p.bar.value == 1.0 assert p.bar.bar_style == 'success' assert 'Finished' in p.elapsed_time.value f = c.submit(throws, L) wait([f]) p = ProgressWidget([f]) sync(loop, p.listen) assert p.status == 'error' assert p.bar.value == 0.0 assert p.bar.bar_style == 'danger' assert 'Exception' in p.elapsed_time.value def test_progressbar_cancel(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: import time L = [c.submit(lambda: time.sleep(0.3), i) for i in range(5)] p = ProgressWidget(L) sync(loop, p.listen) L[-1].cancel() wait(L[:-1]) assert p.status == 'error' assert p.bar.value == 0 # no tasks finish before cancel is called @gen_cluster() def test_multibar_complete(s, a, b): s.update_graph(tasks=valmap(dumps_task, {'x-1': (inc, 1), 'x-2': (inc, 'x-1'), 'x-3': (inc, 'x-2'), 'y-1': (dec, 'x-3'), 'y-2': (dec, 'y-1'), 'e': (throws, 'y-2'), 'other': (inc, 123)}), keys=['e'], dependencies={'x-2': {'x-1'}, 'x-3': {'x-2'}, 'y-1': {'x-3'}, 'y-2': {'y-1'}, 'e': {'y-2'}}) p = MultiProgressWidget(['e'], scheduler=(s.ip, s.port), complete=True) yield p.listen() assert p._last_response['all'] == {'x': 3, 'y': 2, 'e': 1} assert all(b.value == 1.0 for k, b in p.bars.items() if k != 'e') assert '3 / 3' in p.bar_texts['x'].value assert '2 / 2' in p.bar_texts['y'].value def test_fast(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: L = c.map(inc, range(100)) L2 = c.map(dec, L) L3 = c.map(add, L, L2) p = progress(L3, multi=True, complete=True, notebook=True) sync(loop, p.listen) assert set(p._last_response['all']) == {'inc', 'dec', 'add'} distributed-1.20.2/distributed/diskutils.py000066400000000000000000000162731321233345200210340ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import errno import glob import logging import os import shutil import stat import tempfile from . import locket from . import config from .compatibility import finalize logger = logging.getLogger(__name__) DIR_LOCK_EXT = '.dirlock' def is_locking_enabled(): return config.get('use-file-locking', True) class WorkDir(object): """ A temporary work directory inside a WorkSpace. """ def __init__(self, workspace, name=None, prefix=None): assert name is None or prefix is None if name is None: self.dir_path = tempfile.mkdtemp(prefix=prefix, dir=workspace.base_dir) else: self.dir_path = os.path.join(workspace.base_dir, name) os.mkdir(self.dir_path) # it shouldn't already exist if is_locking_enabled(): try: self._lock_path = os.path.join(self.dir_path + DIR_LOCK_EXT) assert not os.path.exists(self._lock_path) logger.debug("Locking %r...", self._lock_path) # Avoid a race condition before locking the file # by taking the global lock with workspace._global_lock(): self._lock_file = locket.lock_file(self._lock_path) self._lock_file.acquire() except Exception: shutil.rmtree(self.dir_path, ignore_errors=True) raise workspace._known_locks.add(self._lock_path) self._finalizer = finalize(self, self._finalize, workspace, self._lock_path, self._lock_file, self.dir_path) else: self._finalizer = finalize(self, self._finalize, workspace, None, None, self.dir_path) def release(self): """ Dispose of this directory. """ self._finalizer() @classmethod def _finalize(cls, workspace, lock_path, lock_file, dir_path): try: workspace._purge_directory(dir_path) finally: if lock_file is not None: lock_file.release() if lock_path is not None: workspace._known_locks.remove(lock_path) os.unlink(lock_path) class WorkSpace(object): """ An on-disk workspace that tracks disposable work directories inside it. If a process crash or another event left stale directories behind, this will be detected and the directories purged. """ # Keep track of all locks known to this process, to avoid several # WorkSpaces to step on each other's toes _known_locks = set() def __init__(self, base_dir): self.base_dir = os.path.abspath(base_dir) self._init_workspace() self._global_lock_path = os.path.join(self.base_dir, 'global.lock') self._purge_lock_path = os.path.join(self.base_dir, 'purge.lock') def _init_workspace(self): try: os.mkdir(self.base_dir) except EnvironmentError as e: if e.errno != errno.EEXIST: raise def _global_lock(self, **kwargs): return locket.lock_file(self._global_lock_path, **kwargs) def _purge_lock(self, **kwargs): return locket.lock_file(self._purge_lock_path, **kwargs) def _purge_leftovers(self): if not is_locking_enabled(): return [] # List candidates with the global lock taken, to avoid purging # a lock file that was just created but not yet locked # (see WorkDir.__init__) lock = self._global_lock(timeout=0) try: lock.acquire() except locket.LockError: # No need to waste time here if someone else is busy doing # something on this workspace return [] else: try: candidates = list(self._list_unknown_locks()) finally: lock.release() # No need to hold the global lock here, especially as purging # can take time. Instead take the purge lock to avoid two # processes purging at once. purged = [] lock = self._purge_lock(timeout=0) try: lock.acquire() except locket.LockError: # No need for two processes to purge one after another pass else: try: for path in candidates: if self._check_lock_or_purge(path): purged.append(path) finally: lock.release() return purged def _list_unknown_locks(self): for p in glob.glob(os.path.join(self.base_dir, '*' + DIR_LOCK_EXT)): try: st = os.stat(p) except EnvironmentError: # May have been removed in the meantime pass else: # XXX restrict to files owned by current user? if stat.S_ISREG(st.st_mode): yield p def _purge_directory(self, dir_path): shutil.rmtree(dir_path, onerror=self._on_remove_error) def _check_lock_or_purge(self, lock_path): """ Try locking the given path, if it fails it's in use, otherwise the corresponding directory is deleted. Return True if the lock was stale. """ assert lock_path.endswith(DIR_LOCK_EXT) if lock_path in self._known_locks: # Avoid touching a lock that we know is already taken return False logger.debug("Checking lock file %r...", lock_path) lock = locket.lock_file(lock_path, timeout=0) try: lock.acquire() except locket.LockError: # Lock file still in use, ignore return False try: # Lock file is stale, therefore purge corresponding directory dir_path = lock_path[:-len(DIR_LOCK_EXT)] if os.path.exists(dir_path): logger.warning("Found stale lock file and directory %r, purging", dir_path) self._purge_directory(dir_path) finally: lock.release() # Clean up lock file after we released it try: os.unlink(lock_path) except EnvironmentError as e: # Perhaps it was removed by someone else? if e.errno != errno.ENOENT: logger.error("Failed to remove %r", str(e)) return True def _on_remove_error(self, func, path, exc_info): typ, exc, tb = exc_info logger.error("Failed to remove %r (failed in %r): %s", path, func, str(exc)) def new_work_dir(self, **kwargs): """ Create and return a new WorkDir in this WorkSpace. Either the *prefix* or *name* parameter should be given (*prefix* is preferred as it avoids potential collisions) Parameters ---------- prefix: str (optional) The prefix of the temporary subdirectory name for the workdir name: str (optional) The subdirectory name for the workdir """ self._purge_leftovers() return WorkDir(self, **kwargs) distributed-1.20.2/distributed/joblib.py000066400000000000000000000126521321233345200202570ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from distutils.version import LooseVersion from uuid import uuid4 from tornado import gen from .client import Client, _wait from .utils import ignoring, funcname, itemgetter # A user could have installed joblib, sklearn, both, or neither. Further, only # joblib >= 0.10.0 supports backends, so we also need to check for that. This # bit of logic is to ensure that we create and register the backend for all # viable installations of joblib. joblib = sk_joblib = None with ignoring(ImportError): import joblib if LooseVersion(joblib.__version__) < '0.10.2': joblib = None with ignoring(ImportError): import sklearn.externals.joblib as sk_joblib if LooseVersion(sk_joblib.__version__) < '0.10.2': sk_joblib = None _bases = [] if joblib: from joblib.parallel import AutoBatchingMixin, ParallelBackendBase _bases.append(ParallelBackendBase) if sk_joblib: from sklearn.externals.joblib.parallel import (AutoBatchingMixin, # noqa ParallelBackendBase) _bases.append(ParallelBackendBase) if not _bases: raise RuntimeError("Joblib backend requires either `joblib` >= '0.10.2' " " or `sklearn` > '0.17.1'. Please install or upgrade") def joblib_funcname(x): try: # Can't do isinstance, since joblib is often bundled in packages, and # separate installs will have non-equivalent types. if type(x).__name__ == 'BatchedCalls': x = x.items[0][0] except Exception: pass return funcname(x) class Batch(object): def __init__(self, tasks): self.tasks = tasks def __call__(self, *data): results = [] for func, args, kwargs in self.tasks: args = [a(data) if isinstance(a, itemgetter) else a for a in args] kwargs = {k: v(data) if isinstance(v, itemgetter) else v for (k, v) in kwargs.items()} results.append(func(*args, **kwargs)) return results def __reduce__(self): return (Batch, (self.tasks,)) class DaskDistributedBackend(ParallelBackendBase, AutoBatchingMixin): MIN_IDEAL_BATCH_DURATION = 0.2 MAX_IDEAL_BATCH_DURATION = 1.0 def __init__(self, scheduler_host='127.0.0.1:8786', scatter=None, loop=None): if scatter is not None and not isinstance(scatter, (list, tuple)): raise TypeError("scatter must be a list/tuple, got " "`%s`" % type(scatter).__name__) self.client = Client(scheduler_host, loop=loop, set_as_default=False) if scatter is not None: # Keep a reference to the scattered data to keep the ids the same self._scatter = list(scatter) scattered = self.client.scatter(scatter, broadcast=True) self.data_to_future = {id(x): f for (x, f) in zip(scatter, scattered)} else: self._scatter = [] self.data_to_future = {} self.futures = set() def configure(self, n_jobs=1, parallel=None, **backend_args): return self.effective_n_jobs(n_jobs) def effective_n_jobs(self, n_jobs): return sum(self.client.ncores().values()) def _to_func_args(self, func): if not self.data_to_future: return func, () args2 = [] lookup = dict(self.data_to_future) def maybe_to_futures(args): for x in args: id_x = id(x) if id_x in lookup: x = lookup[id_x] if type(x) is not itemgetter: x = itemgetter(len(args2)) args2.append(lookup[id_x]) lookup[id_x] = x yield x tasks = [] for f, args, kwargs in func.items: args = list(maybe_to_futures(args)) kwargs = dict(zip(kwargs.keys(), maybe_to_futures(kwargs.values()))) tasks.append((f, args, kwargs)) if not args2: return func, () return Batch(tasks), args2 def apply_async(self, func, callback=None): key = '%s-batch-%s' % (joblib_funcname(func), uuid4().hex) func, args = self._to_func_args(func) future = self.client.submit(func, *args, key=key) self.futures.add(future) @gen.coroutine def callback_wrapper(): result = yield _wait([future]) self.futures.remove(future) if callback is not None: callback(result) # gets called in separate thread self.client.loop.add_callback(callback_wrapper) future.get = future.result # monkey patch to achieve AsyncResult API return future def abort_everything(self, ensure_ready=True): # Tell the client to cancel any task submitted via this instance # as joblib.Parallel will never access those results. self.client.cancel(self.futures) self.futures.clear() for base in _bases: base.register(DaskDistributedBackend) DistributedBackend = DaskDistributedBackend # Register the backend with any available versions of joblib if joblib: joblib.register_parallel_backend('distributed', DaskDistributedBackend) joblib.register_parallel_backend('dask.distributed', DaskDistributedBackend) if sk_joblib: sk_joblib.register_parallel_backend('distributed', DaskDistributedBackend) sk_joblib.register_parallel_backend('dask.distributed', DaskDistributedBackend) distributed-1.20.2/distributed/lock.py000066400000000000000000000105321321233345200177410ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import defaultdict, deque from datetime import timedelta import logging import uuid from tornado import gen import tornado.locks from .client import _get_global_client from .utils import log_errors from .worker import get_worker logger = logging.getLogger(__name__) class LockExtension(object): """ An extension for the scheduler to manage Locks This adds the following routes to the scheduler * lock_acquire * lock_release """ def __init__(self, scheduler): self.scheduler = scheduler self.events = defaultdict(deque) self.ids = dict() self.scheduler.handlers.update({'lock_acquire': self.acquire, 'lock_release': self.release}) self.scheduler.extensions['locks'] = self @gen.coroutine def acquire(self, stream=None, name=None, id=None, timeout=None): with log_errors(): if name not in self.ids: result = True else: while name in self.ids: event = tornado.locks.Event() self.events[name].append(event) future = event.wait() if timeout is not None: future = gen.with_timeout(timedelta(seconds=timeout), future) try: yield future except gen.TimeoutError: result = False break else: result = True finally: event2 = self.events[name].popleft() assert event is event2 if result: assert name not in self.ids self.ids[name] = id raise gen.Return(result) def release(self, stream=None, name=None, id=None): with log_errors(): if self.ids.get(name) != id: raise ValueError("This lock has not yet been acquired") del self.ids[name] if self.events[name]: self.scheduler.loop.add_callback(self.events[name][0].set) else: del self.events[name] class Lock(object): """ Distributed Centralized Lock Parameters ---------- name: string Name of the lock to acquire. Choosing the same name allows two disconnected processes to coordinate a lock. Examples -------- >>> lock = Lock('x') # doctest: +SKIP >>> lock.acquire(timeout=1) # doctest: +SKIP >>> # do things with protected resource >>> lock.release() # doctest: +SKIP """ def __init__(self, name=None, client=None): self.client = client or _get_global_client() or get_worker().client self.name = name or 'variable-' + uuid.uuid4().hex self.id = uuid.uuid4().hex self._locked = False def acquire(self, timeout=None): """ Acquire the lock Parameters ---------- timeout: number Seconds to wait on the lock in the scheduler. This does not include local coroutine time, network transfer time, etc.. Examples -------- >>> lock = Lock('x') # doctest: +SKIP >>> lock.acquire(timeout=1) # doctest: +SKIP Returns ------- True or False whether or not it sucessfully acquired the lock """ result = self.client.sync(self.client.scheduler.lock_acquire, name=self.name, id=self.id, timeout=timeout) self._locked = True return result def release(self): """ Release the lock if already acquired """ if not self.locked(): raise ValueError("Lock is not yet acquired") result = self.client.sync(self.client.scheduler.lock_release, name=self.name, id=self.id) self._locked = False return result def locked(self): return self._locked def __enter__(self): self.acquire() return self def __exit__(self, *args, **kwargs): self.release() @gen.coroutine def __aenter__(self): yield self.acquire() raise gen.Return(self) @gen.coroutine def __aexit__(self, *args, **kwargs): yield self.release() distributed-1.20.2/distributed/locket.py000066400000000000000000000124551321233345200203000ustar00rootroot00000000000000# Vendored up-to-date copy of locket.py # Based on https://github.com/mwilliamson/locket.py/pull/8 # flake8: noqa import time import errno import threading import weakref __all__ = ["lock_file"] try: import fcntl except ImportError: try: import ctypes import ctypes.wintypes import msvcrt except ImportError: raise ImportError("Platform not supported (failed to import fcntl, ctypes, msvcrt)") else: _kernel32 = ctypes.WinDLL('kernel32', use_last_error=True) _WinAPI_LockFile = _kernel32.LockFile _WinAPI_LockFile.restype = ctypes.wintypes.BOOL _WinAPI_LockFile.argtypes = [ctypes.wintypes.HANDLE] + [ctypes.wintypes.DWORD] * 4 _WinAPI_UnlockFile = _kernel32.UnlockFile _WinAPI_UnlockFile.restype = ctypes.wintypes.BOOL _WinAPI_UnlockFile.argtypes = [ctypes.wintypes.HANDLE] + [ctypes.wintypes.DWORD] * 4 _lock_file_blocking_available = False def _lock_file_non_blocking(file_): res = _WinAPI_LockFile(msvcrt.get_osfhandle(file_.fileno()), 0, 0, 1, 0) if res: return True else: err = ctypes.get_last_error() # 33 = ERROR_LOCK_VIOLATION if err != 33: raise ctypes.WinError(err) return False def _unlock_file(file_): _WinAPI_UnlockFile(msvcrt.get_osfhandle(file_.fileno()), 0, 0, 1, 0) else: _lock_file_blocking_available = True def _lock_file_blocking(file_): fcntl.flock(file_.fileno(), fcntl.LOCK_EX) def _lock_file_non_blocking(file_): try: fcntl.flock(file_.fileno(), fcntl.LOCK_EX | fcntl.LOCK_NB) return True except IOError as error: if error.errno in [errno.EACCES, errno.EAGAIN]: return False else: raise def _unlock_file(file_): fcntl.flock(file_.fileno(), fcntl.LOCK_UN) _locks_lock = threading.Lock() _locks = weakref.WeakValueDictionary() def lock_file(path, **kwargs): _locks_lock.acquire() try: lock = _locks.get(path) if lock is None: lock = _create_lock_file(path) _locks[path] = lock finally: _locks_lock.release() return _Locker(lock, **kwargs) def _create_lock_file(path): thread_lock = _ThreadLock(path) file_lock = _LockFile(path) return _LockSet([thread_lock, file_lock]) class LockError(Exception): pass def _acquire_non_blocking(acquire, timeout, retry_period, path): if retry_period is None: retry_period = 0.05 start_time = time.time() while True: success = acquire() if success: return elif (timeout is not None and time.time() - start_time > timeout): raise LockError("Couldn't lock {0}".format(path)) else: time.sleep(retry_period) class _LockSet(object): def __init__(self, locks): self._locks = locks def acquire(self, timeout, retry_period): acquired_locks = [] try: for lock in self._locks: lock.acquire(timeout, retry_period) acquired_locks.append(lock) except: for acquired_lock in reversed(acquired_locks): # TODO: handle exceptions acquired_lock.release() raise def release(self): for lock in reversed(self._locks): # TODO: Handle exceptions lock.release() class _ThreadLock(object): def __init__(self, path): self._path = path self._lock = threading.Lock() def acquire(self, timeout=None, retry_period=None): if timeout is None: self._lock.acquire() else: _acquire_non_blocking( acquire=lambda: self._lock.acquire(False), timeout=timeout, retry_period=retry_period, path=self._path, ) def release(self): self._lock.release() class _LockFile(object): def __init__(self, path): self._path = path self._file = None self._thread_lock = threading.Lock() def acquire(self, timeout=None, retry_period=None): if self._file is None: self._file = open(self._path, "wb") if timeout is None and _lock_file_blocking_available: _lock_file_blocking(self._file) else: _acquire_non_blocking( acquire=lambda: _lock_file_non_blocking(self._file), timeout=timeout, retry_period=retry_period, path=self._path, ) def release(self): _unlock_file(self._file) self._file.close() self._file = None class _Locker(object): """ A lock wrapper to always apply the given *timeout* and *retry_period* to acquire() calls. """ def __init__(self, lock, timeout=None, retry_period=None): self._lock = lock self._timeout = timeout self._retry_period = retry_period def acquire(self): self._lock.acquire(self._timeout, self._retry_period) def release(self): self._lock.release() def __enter__(self): self.acquire() return self def __exit__(self, *args): self.release() distributed-1.20.2/distributed/metrics.py000077500000000000000000000044671321233345200204740ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import collections from functools import wraps import sys import time as timemod _empty_namedtuple = collections.namedtuple("_empty_namedtuple", ()) def _psutil_caller(method_name, default=_empty_namedtuple): try: import psutil except ImportError: return default meth = getattr(psutil, method_name) @wraps(meth) def wrapper(): try: return meth() except RuntimeError: # This can happen on some systems (e.g. no physical disk in worker) return default() return wrapper disk_io_counters = _psutil_caller("disk_io_counters") net_io_counters = _psutil_caller("net_io_counters") class _WindowsTime(object): """ Combine time.time() and time.perf_counter() to get an absolute clock with fine resolution. """ # Resync every N seconds, to avoid drifting RESYNC_EVERY = 600 def __init__(self): self.delta = None self.last_resync = float("-inf") if sys.version_info >= (3,): perf_counter = timemod.perf_counter else: perf_counter = timemod.clock def time(self): delta = self.delta cur = self.perf_counter() if cur - self.last_resync >= self.RESYNC_EVERY: delta = self.resync() self.last_resync = cur return delta + cur def resync(self): _time = timemod.time _perf_counter = self.perf_counter min_samples = 5 while True: times = [(_time(), _perf_counter()) for i in range(min_samples * 2)] abs_times = collections.Counter(t[0] for t in times) first, nfirst = abs_times.most_common()[0] if nfirst < min_samples: # System too noisy? Start again continue else: perf_times = [t[1] for t in times if t[0] == first][:-1] assert len(perf_times) >= min_samples - 1, perf_times self.delta = first - sum(perf_times) / len(perf_times) return self.delta # A high-resolution wall clock timer measuring the seconds since Unix epoch if sys.platform.startswith('win'): time = _WindowsTime().time else: # Under modern Unices, time.time() should be good enough time = timemod.time distributed-1.20.2/distributed/nanny.py000066400000000000000000000436321321233345200201430ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from datetime import timedelta import logging from multiprocessing.queues import Empty import os import psutil import shutil import threading from tornado import gen from tornado.ioloop import IOLoop, TimeoutError from tornado.locks import Event from .comm import get_address_host, get_local_address_for, unparse_host_port from .config import config from .core import rpc, RPCClosed, CommClosedError, coerce_to_address from .metrics import time from .node import ServerNode from .process import AsyncProcess from .proctitle import enable_proctitle_on_children from .security import Security from .utils import (get_ip, mp_context, silence_logging, json_load_robust, PeriodicCallback) from .worker import _ncores, run, parse_memory_limit logger = logging.getLogger(__name__) class Nanny(ServerNode): """ A process to manage worker processes The nanny spins up Worker processes, watches then, and kills or restarts them as necessary. """ process = None status = None def __init__(self, scheduler_ip=None, scheduler_port=None, scheduler_file=None, worker_port=0, ncores=None, loop=None, local_dir=None, services=None, name=None, memory_limit='auto', reconnect=True, validate=False, quiet=False, resources=None, silence_logs=None, death_timeout=None, preload=(), security=None, contact_address=None, listen_address=None, **kwargs): if scheduler_file: cfg = json_load_robust(scheduler_file) self.scheduler_addr = cfg['address'] elif scheduler_port is None: 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.preload = preload self.contact_address = contact_address self.memory_terminate_fraction = config.get('worker-memory-terminate', 0.95) 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 self.auto_restart = True self.memory_limit = parse_memory_limit(memory_limit, self.ncores) if silence_logs: silence_logging(level=silence_logs) self.silence_logs = silence_logs handlers = {'instantiate': self.instantiate, 'kill': self.kill, 'restart': self.restart, # cannot call it 'close' on the rpc side for naming conflict 'terminate': self._close, 'run': self.run} super(Nanny, self).__init__(handlers, io_loop=self.loop, connection_args=self.connection_args, **kwargs) if self.memory_limit: pc = PeriodicCallback(self.memory_monitor, 100) self.periodic_callbacks['memory'] = pc self._listen_address = listen_address self.status = 'init' def __repr__(self): return "" % (self.worker_address, self.ncores) @gen.coroutine def _unregister(self, timeout=10): if self.process is None: return worker_address = self.process.worker_address if worker_address is None: return allowed_errors = (gen.TimeoutError, CommClosedError, EnvironmentError, RPCClosed) try: yield gen.with_timeout(timedelta(seconds=timeout), self.scheduler.unregister(address=self.worker_address), quiet_exceptions=allowed_errors) except allowed_errors: pass @property def worker_address(self): return None if self.process is None else self.process.worker_address @property def worker_dir(self): return None if self.process is None else self.process.worker_dir @gen.coroutine def _start(self, addr_or_port=0): """ Start nanny, start local process, start watching """ # XXX Factor this out if not addr_or_port: # Default address is the required one to reach the scheduler self.listen(get_local_address_for(self.scheduler.address), listen_args=self.listen_args) self.ip = get_address_host(self.address) elif isinstance(addr_or_port, int): # addr_or_port is an integer => assume TCP self.ip = get_ip( get_address_host(self.scheduler.address) ) self.listen((self.ip, addr_or_port), listen_args=self.listen_args) else: self.listen(addr_or_port, listen_args=self.listen_args) self.ip = get_address_host(self.address) logger.info(' Start Nanny at: %r', self.address) response = yield self.instantiate() if response == 'OK': assert self.worker_address self.status = 'running' self.start_periodic_callbacks() def start(self, addr_or_port=0): self.loop.add_callback(self._start, addr_or_port) @gen.coroutine def kill(self, comm=None, timeout=2): """ Kill the local worker process Blocks until both the process is down and the scheduler is properly informed """ self.auto_restart = False if self.process is None: raise gen.Return('OK') 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): """ Start a local worker process Blocks until the process is up and the scheduler is properly informed """ if self._listen_address: start_arg = self._listen_address else: host = self.listener.bound_address[0] start_arg = self.listener.prefix + unparse_host_port(host, self._given_worker_port) if self.process is None: self.process = WorkerProcess( worker_args=(self.scheduler_addr,), worker_kwargs=dict(ncores=self.ncores, local_dir=self.local_dir, services=self.services, service_ports={'nanny': self.port}, name=self.name, memory_limit=self.memory_limit, reconnect=self.reconnect, resources=self.resources, validate=self.validate, silence_logs=self.silence_logs, death_timeout=self.death_timeout, preload=self.preload, security=self.security, contact_address=self.contact_address), worker_start_args=(start_arg,), silence_logs=self.silence_logs, on_exit=self._on_exit, ) self.auto_restart = True if self.death_timeout: try: yield gen.with_timeout(timedelta(seconds=self.death_timeout), self.process.start()) except gen.TimeoutError: yield self._close(timeout=self.death_timeout) raise gen.Return('timed out') else: yield self.process.start() raise gen.Return('OK') @gen.coroutine def restart(self, comm=None, timeout=2): start = time() @gen.coroutine def _(): if self.process is not None: yield self.kill() yield self.instantiate() try: yield gen.with_timeout(timedelta(seconds=timeout), _()) except gen.TimeoutError: logger.error("Restart timed out, returning before finished") raise gen.Return('timed out') else: raise gen.Return('OK') def memory_monitor(self): """ Track worker's memory. Restart if it goes above 95% """ if self.status != 'running': return memory = psutil.Process(self.process.pid).memory_info().rss frac = memory / self.memory_limit if self.memory_terminate_fraction and frac > self.memory_terminate_fraction: logger.warning("Worker exceeded 95% memory budget. Restarting") self.process.process.terminate() def is_alive(self): return self.process is not None and self.process.status == 'running' def run(self, *args, **kwargs): return run(self, *args, **kwargs) @gen.coroutine def _on_exit(self, exitcode): if self.status not in ('closing', 'closed'): try: yield self.scheduler.unregister(address=self.worker_address) except (EnvironmentError, CommClosedError): if not self.reconnect: yield self._close() return try: if self.status not in ('closing', 'closed'): if self.auto_restart: logger.warning("Restarting worker") yield self.instantiate() except Exception: logger.error("Failed to restart worker after its process exited", exc_info=True) @property def pid(self): return self.process and self.process.pid @gen.coroutine def _close(self, comm=None, timeout=5, report=None): """ Close the worker process, stop all comms. """ if self.status in ('closing', 'closed'): raise gen.Return('OK') self.status = 'closing' logger.info("Closing Nanny at %r", self.address) self.stop() try: if self.process is not None: yield self.kill(timeout=timeout) except Exception: pass self.process = None self.rpc.close() self.scheduler.close_rpc() self.status = 'closed' raise gen.Return('OK') class WorkerProcess(object): def __init__(self, worker_args, worker_kwargs, worker_start_args, silence_logs, on_exit): self.status = 'init' self.silence_logs = silence_logs self.worker_args = worker_args self.worker_kwargs = worker_kwargs self.worker_start_args = worker_start_args self.on_exit = on_exit self.process = None # Initialized when worker is ready self.worker_dir = None self.worker_address = None @gen.coroutine def start(self): """ Ensure the worker process is started. """ enable_proctitle_on_children() if self.status == 'running': return if self.status == 'starting': yield self.running.wait() return self.init_result_q = mp_context.Queue() self.child_stop_q = mp_context.Queue() self.process = AsyncProcess( target=self._run, kwargs=dict(worker_args=self.worker_args, worker_kwargs=self.worker_kwargs, worker_start_args=self.worker_start_args, silence_logs=self.silence_logs, init_result_q=self.init_result_q, child_stop_q=self.child_stop_q), ) self.process.daemon = True self.process.set_exit_callback(self._on_exit) self.running = Event() self.stopped = Event() self.status = 'starting' yield self.process.start() if self.status == 'starting': yield self._wait_until_running() def _on_exit(self, proc): if proc is not self.process: # Ignore exit of old process instance return self.mark_stopped() def _death_message(self, pid, exitcode): assert exitcode is not None if exitcode == 255: return "Worker process %d was killed by unknown signal" % (pid,) elif exitcode >= 0: return "Worker process %d exited with status %d" % (pid, exitcode,) else: return "Worker process %d was killed by signal %d" % (pid, -exitcode,) def is_alive(self): return self.process is not None and self.process.is_alive() @property def pid(self): return (self.process.pid if self.process and self.process.is_alive() else None) def mark_stopped(self): if self.status != 'stopped': r = self.process.exitcode assert r is not None if r != 0: msg = self._death_message(self.process.pid, r) logger.warning(msg) self.status = 'stopped' self.stopped.set() # Release resources self.process.close() self.init_result_q = None self.child_stop_q = None self.process = None # Best effort to clean up worker directory if self.worker_dir and os.path.exists(self.worker_dir): shutil.rmtree(self.worker_dir, ignore_errors=True) self.worker_dir = None # User hook if self.on_exit is not None: self.on_exit(r) @gen.coroutine def kill(self, timeout=2): """ Ensure the worker process is stopped, waiting at most *timeout* seconds before terminating it abruptly. """ loop = IOLoop.current() deadline = loop.time() + timeout if self.status == 'stopped': return if self.status == 'stopping': yield self.stopped.wait() return assert self.status in ('starting', 'running') self.status = 'stopping' process = self.process self.child_stop_q.put({'op': 'stop', 'timeout': max(0, deadline - loop.time()) * 0.8, }) while process.is_alive() and loop.time() < deadline: yield gen.sleep(0.05) if process.is_alive(): logger.warning("Worker process still alive after %d seconds, killing", timeout) try: yield process.terminate() except Exception as e: logger.error("Failed to kill worker process: %s", e) @gen.coroutine def _wait_until_running(self): delay = 0.05 while True: if self.status != 'starting': return try: msg = self.init_result_q.get_nowait() except Empty: yield gen.sleep(delay) continue if isinstance(msg, Exception): yield self.process.join() raise msg else: self.worker_address = msg['address'] self.worker_dir = msg['dir'] assert self.worker_address self.status = 'running' self.running.set() raise gen.Return(msg) @classmethod def _run(cls, worker_args, worker_kwargs, worker_start_args, silence_logs, init_result_q, child_stop_q): # pragma: no cover from distributed import Worker try: from dask.multiprocessing import initialize_worker_process except ImportError: # old Dask version pass else: initialize_worker_process() if silence_logs: logger.setLevel(silence_logs) IOLoop.clear_instance() loop = IOLoop() loop.make_current() worker = Worker(*worker_args, **worker_kwargs) @gen.coroutine def do_stop(timeout): try: yield worker._close(report=False, nanny=False) finally: loop.stop() def watch_stop_q(): """ Wait for an incoming stop message and then stop the worker cleanly. """ while True: try: msg = child_stop_q.get(timeout=1000) except Empty: pass else: assert msg['op'] == 'stop' loop.add_callback(do_stop, msg['timeout']) break t = threading.Thread(target=watch_stop_q, name="Nanny stop queue watch") t.daemon = True t.start() @gen.coroutine def run(): """ Try to start worker and inform parent of outcome. """ try: yield worker._start(*worker_start_args) except Exception as e: logger.exception("Failed to start worker") init_result_q.put(e) else: assert worker.address init_result_q.put({'address': worker.address, 'dir': worker.local_dir}) yield worker.wait_until_closed() logger.info("Worker closed") try: loop.run_sync(run) except TimeoutError: # Loop was stopped before wait_until_closed() returned, ignore pass except KeyboardInterrupt: pass distributed-1.20.2/distributed/node.py000066400000000000000000000025351321233345200177420ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from tornado.ioloop import IOLoop from .core import Server, ConnectionPool class Node(object): """ Base class for nodes in a distributed cluster. """ def __init__(self, connection_limit=512, deserialize=True, connection_args=None, io_loop=None): self.io_loop = io_loop or IOLoop.current() self.rpc = ConnectionPool(limit=connection_limit, deserialize=deserialize, connection_args=connection_args) class ServerNode(Node, Server): """ Base class for server nodes in a distributed cluster. """ # TODO factor out security, listening, services, etc. here # XXX avoid inheriting from Server? there is some large potential for confusion # between base and derived attribute namespaces... def __init__(self, handlers, connection_limit=512, deserialize=True, connection_args=None, io_loop=None): Node.__init__(self, deserialize=deserialize, connection_limit=connection_limit, connection_args=connection_args, io_loop=io_loop) Server.__init__(self, handlers, connection_limit=connection_limit, deserialize=deserialize, io_loop=self.io_loop) distributed-1.20.2/distributed/preloading.py000066400000000000000000000030531321233345200211350ustar00rootroot00000000000000import atexit import logging import os import shutil import sys from importlib import import_module from .utils import import_file logger = logging.getLogger(__name__) def preload_modules(names, parameter=None, file_dir=None): """ Imports modules, handles `dask_setup` and `dask_teardown` functions Parameters ---------- names: list of strings Module names or file paths parameter: object Parameter passed to `dask_setup` and `dask_teardown` file_dir: string Path of a directory where files should be copied """ for name in names: # import if name.endswith(".py"): # name is a file path if file_dir is not None: basename = os.path.basename(name) copy_dst = os.path.join(file_dir, basename) if os.path.exists(copy_dst): logger.error("File name collision: %s", basename) shutil.copy(name, copy_dst) module = import_file(copy_dst)[0] else: module = import_file(name)[0] else: # name is a module name if name not in sys.modules: import_module(name) module = sys.modules[name] # handle special functions dask_setup = getattr(module, 'dask_setup', None) dask_teardown = getattr(module, 'dask_teardown', None) if dask_setup is not None: dask_setup(parameter) if dask_teardown is not None: atexit.register(dask_teardown, parameter) distributed-1.20.2/distributed/process.py000066400000000000000000000260061321233345200204720ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import atexit from datetime import timedelta import logging import os import re import sys import threading import weakref from .compatibility import finalize, Queue as PyQueue, PY2 from .utils import mp_context from tornado import gen from tornado.concurrent import Future from tornado.ioloop import IOLoop logger = logging.getLogger(__name__) def _loop_add_callback(loop, func, *args): """ Helper to silence "IOLoop is closing" exception on IOLoop.add_callback. """ try: loop.add_callback(func, *args) except RuntimeError as exc: if not re.search("IOLoop is clos(ed|ing)", str(exc)): raise def _call_and_set_future(loop, future, func, *args, **kwargs): try: res = func(*args, **kwargs) except Exception: # Tornado futures are not thread-safe, need to # set_result() / set_exc_info() from the loop's thread _loop_add_callback(loop, future.set_exc_info, sys.exc_info()) else: _loop_add_callback(loop, future.set_result, res) class _ProcessState(object): is_alive = False pid = None exitcode = None class AsyncProcess(object): """ A coroutine-compatible multiprocessing.Process-alike. All normally blocking methods are wrapped in Tornado coroutines. """ def __init__(self, loop=None, target=None, name=None, args=(), kwargs={}): if not callable(target): raise TypeError("`target` needs to be callable, not %r" % (type(target),)) self._state = _ProcessState() self._loop = loop or IOLoop.current(instance=False) # _keep_child_alive is the write side of a pipe, which, when it is # closed, causes the read side of the pipe to unblock for reading. Note # that it is never closed directly. The write side is closed by the # kernel when our process exits, or possibly by the garbage collector # closing the file descriptor when the last reference to # _keep_child_alive goes away. We can take advantage of this fact to # monitor from the child and exit when the parent goes away unexpectedly # (for example due to SIGKILL). This variable is otherwise unused except # for the assignment here. parent_alive_pipe, self._keep_child_alive = mp_context.Pipe(duplex=False) self._process = mp_context.Process(target=self._run, name=name, args=(target, args, kwargs, parent_alive_pipe, self._keep_child_alive)) _dangling.add(self._process) self._name = self._process.name self._watch_q = PyQueue() self._exit_future = Future() self._exit_callback = None self._closed = False self._start_threads() def __repr__(self): return "<%s %s>" % (self.__class__.__name__, self._name) def _check_closed(self): if self._closed: raise ValueError("invalid operation on closed AsyncProcess") def _start_threads(self): self._watch_message_thread = threading.Thread( target=self._watch_message_queue, name="AsyncProcess %s watch message queue" % self.name, args=(weakref.ref(self), self._process, self._loop, self._state, self._watch_q, self._exit_future,)) self._watch_message_thread.daemon = True self._watch_message_thread.start() def stop_thread(q): q.put_nowait({'op': 'stop'}) # We don't join the thread here as a finalizer can be called # asynchronously from anywhere self._finalizer = finalize(self, stop_thread, q=self._watch_q) self._finalizer.atexit = False def _on_exit(self, exitcode): # Called from the event loop when the child process exited self._process = None if self._exit_callback is not None: self._exit_callback(self) self._exit_future.set_result(exitcode) @classmethod def _immediate_exit_when_closed(cls, parent_alive_pipe): """ Immediately exit the process when parent_alive_pipe is closed. """ def monitor_parent(): try: # The parent_alive_pipe should be held open as long as the # parent is alive and wants us to stay alive. Nothing writes to # it, so the read will block indefinitely. parent_alive_pipe.recv() except EOFError: # Parent process went away unexpectedly. Exit immediately. Could # consider other exiting approches here. My initial preference # is to unconditionally and immediately exit. If we're in this # state it is possible that a "clean" process exit won't work # anyway - if, for example, the system is getting bogged down # due to the running out of memory, exiting sooner rather than # later might be needed to restore normal system function. # If this is in appropriate for your use case, please file a # bug. os._exit(-1) else: # If we get here, something odd is going on. File descriptors # got crossed? raise RuntimeError("unexpected state: should be unreachable") t = threading.Thread(target=monitor_parent) t.daemon = True t.start() @staticmethod def reset_logger_locks(): """ Python 2's logger's locks don't survive a fork event https://github.com/dask/distributed/issues/1491 """ if PY2: for name in logging.Logger.manager.loggerDict.keys(): for handler in logging.getLogger(name).handlers: handler.createLock() @classmethod def _run(cls, target, args, kwargs, parent_alive_pipe, _keep_child_alive): # On Python 2 with the fork method, we inherit the _keep_child_alive fd, # whether it is passed or not. Therefore, pass it unconditionally and # close it here, so that there are no other references to the pipe lying # around. cls.reset_logger_locks() _keep_child_alive.close() # Child process entry point cls._immediate_exit_when_closed(parent_alive_pipe) threading.current_thread().name = "MainThread" target(*args, **kwargs) @classmethod def _watch_message_queue(cls, selfref, process, loop, state, q, exit_future): # As multiprocessing.Process is not thread-safe, we run all # blocking operations from this single loop and ship results # back to the caller when needed. r = repr(selfref()) name = selfref().name def _start(): process.start() thread = threading.Thread( target=AsyncProcess._watch_process, name="AsyncProcess %s watch process join" % name, args=(selfref, process, state, q)) thread.daemon = True thread.start() state.is_alive = True state.pid = process.pid logger.debug("[%s] created process with pid %r" % (r, state.pid)) while True: msg = q.get() logger.debug("[%s] got message %r" % (r, msg)) op = msg['op'] if op == 'start': _call_and_set_future(loop, msg['future'], _start) elif op == 'terminate': _call_and_set_future(loop, msg['future'], process.terminate) elif op == 'stop': break else: assert 0, msg @classmethod def _watch_process(cls, selfref, process, state, q): r = repr(selfref()) process.join() exitcode = process.exitcode assert exitcode is not None logger.debug("[%s] process %r exited with code %r", r, state.pid, exitcode) state.is_alive = False state.exitcode = exitcode # Make sure the process is removed from the global list # (see _children in multiprocessing/process.py) # Then notify the Process object self = selfref() # only keep self alive when required try: if self is not None: _loop_add_callback(self._loop, self._on_exit, exitcode) finally: self = None # lose reference def start(self): """ Start the child process. This method is a coroutine. """ self._check_closed() fut = Future() self._watch_q.put_nowait({'op': 'start', 'future': fut}) return fut def terminate(self): """ Terminate the child process. This method is a coroutine. """ self._check_closed() fut = Future() self._watch_q.put_nowait({'op': 'terminate', 'future': fut}) return fut @gen.coroutine def join(self, timeout=None): """ Wait for the child process to exit. This method is a coroutine. """ self._check_closed() assert self._state.pid is not None, 'can only join a started process' if self._state.exitcode is not None: return if timeout is None: yield self._exit_future else: try: yield gen.with_timeout(timedelta(seconds=timeout), self._exit_future) except gen.TimeoutError: pass def close(self): """ Stop helper thread and release resources. This method returns immediately and does not ensure the child process has exited. """ if not self._closed: self._finalizer() self._process = None self._closed = True def set_exit_callback(self, func): """ Set a function to be called by the event loop when the process exits. The function is called with the AsyncProcess as sole argument. The function may be a coroutine function. """ # XXX should this be a property instead? assert callable(func), "exit callback should be callable" assert self._state.pid is None, "cannot set exit callback when process already started" self._exit_callback = func def is_alive(self): return self._state.is_alive @property def pid(self): return self._state.pid @property def exitcode(self): return self._state.exitcode @property def name(self): return self._name @property def daemon(self): return self._process.daemon @daemon.setter def daemon(self, value): self._process.daemon = value _dangling = weakref.WeakSet() @atexit.register def _cleanup_dangling(): for proc in list(_dangling): if proc.daemon and proc.is_alive(): try: logger.warning("reaping stray process %s" % (proc,)) proc.terminate() except OSError: pass distributed-1.20.2/distributed/proctitle.py000066400000000000000000000016551321233345200210240ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import os try: import setproctitle as setproctitle_mod except ImportError: setproctitle_mod = None _enabled = False def enable_proctitle_on_children(): """ Enable setting the process title on this process' children and grandchildren. """ os.environ['DASK_PARENT'] = str(os.getpid()) def enable_proctitle_on_current(): """ Enable setting the process title on this process. """ global _enabled _enabled = True def setproctitle(title): """ Change this process' title, as displayed in various utilities such as `ps`. """ if setproctitle_mod is None: return enabled = _enabled if not enabled: try: enabled = int(os.environ.get('DASK_PARENT', '')) != os.getpid() except ValueError: pass if enabled: setproctitle_mod.setproctitle(title) distributed-1.20.2/distributed/profile.py000066400000000000000000000145441321233345200204600ustar00rootroot00000000000000""" This module contains utility functions to construct and manipulate counting data structures for frames. When performing statistical profiling we obtain many call stacks. We aggregate these call stacks into data structures that maintain counts of how many times each function in that call stack has been called. Because these stacks will overlap this aggregation counting structure forms a tree, such as is commonly visualized by profiling tools. We represent this tree as a nested dictionary with the following form: { 'identifier': 'root', 'description': 'A long description of the line of code being run.', 'count': 10 # the number of times we have seen this line 'children': { # callers of this line. Recursive dicts 'ident-b': {'description': ... 'identifier': 'ident-a', 'count': ... 'children': {...}}, 'ident-b': {'description': ... 'identifier': 'ident-b', 'count': ... 'children': {...}}} } """ from collections import defaultdict import linecache import itertools import toolz from .utils import format_time def identifier(frame): """ A string identifier from a frame Strings are cheaper to use as indexes into dicts than tuples or dicts """ if frame is None: return 'None' else: return ';'.join((frame.f_code.co_name, frame.f_code.co_filename, str(frame.f_code.co_firstlineno))) def repr_frame(frame): """ Render a frame as a line for inclusion into a text traceback """ co = frame.f_code text = ' File "%s", line %s, in %s' % (co.co_filename, frame.f_lineno, co.co_name) line = linecache.getline(co.co_filename, frame.f_lineno, frame.f_globals).lstrip() return text + '\n\t' + line def info_frame(frame): co = frame.f_code line = linecache.getline(co.co_filename, frame.f_lineno, frame.f_globals).lstrip() return {'filename': co.co_filename, 'name': co.co_name, 'line_number': frame.f_lineno, 'line': line} def process(frame, child, state, stop=None): """ Add counts from a frame stack onto existing state This recursively adds counts to the existing state dictionary and creates new entries for new functions. Example ------- >>> import sys, threading >>> ident = threading.get_ident() # replace with your thread of interest >>> frame = sys._current_frames()[ident] >>> state = {'children': {}, 'count': 0, 'description': 'root', ... 'identifier': 'root'} >>> process(frame, None, state) >>> state {'count': 1, 'identifier': 'root', 'description': 'root', 'children': {'...'}} """ if frame.f_back is not None and (stop is None or not frame.f_back.f_code.co_filename.endswith(stop)): state = process(frame.f_back, frame, state, stop=stop) ident = identifier(frame) try: d = state['children'][ident] except KeyError: d = {'count': 0, 'description': info_frame(frame), 'children': {}, 'identifier': ident} state['children'][ident] = d state['count'] += 1 if child is not None: return d else: d['count'] += 1 def merge(*args): """ Merge multiple frame states together """ if not args: return create() s = {arg['identifier'] for arg in args} if len(s) != 1: raise ValueError("Expected identifiers, got %s" % str(s)) children = defaultdict(list) for arg in args: for child in arg['children']: children[child].append(arg['children'][child]) children = {k: merge(*v) for k, v in children.items()} count = sum(arg['count'] for arg in args) return {'description': args[0]['description'], 'children': dict(children), 'count': count, 'identifier': args[0]['identifier']} def create(): return {'count': 0, 'children': {}, 'identifier': 'root', 'description': {'filename': '', 'name': '', 'line_number': 0, 'line': ''}} def call_stack(frame): """ Create a call text stack from a frame Returns ------- list of strings """ L = [] while frame: L.append(repr_frame(frame)) frame = frame.f_back return L[::-1] def plot_data(state, profile_interval=0.010): """ Convert a profile state into data useful by Bokeh See Also -------- distributed.bokeh.components.ProfilePlot """ starts = [] stops = [] heights = [] widths = [] colors = [] states = [] times = [] filenames = [] lines = [] line_numbers = [] names = [] def traverse(state, start, stop, height): if not state['count']: return starts.append(start) stops.append(stop) heights.append(height) width = stop - start widths.append(width) states.append(state) times.append(format_time(state['count'] * profile_interval)) desc = state['description'] filenames.append(desc['filename']) lines.append(desc['line']) line_numbers.append(desc['line_number']) names.append(desc['name']) ident = state['identifier'] try: colors.append(color_of(desc['filename'])) except IndexError: colors.append(palette[-1]) delta = (stop - start) / state['count'] x = start for name, child in state['children'].items(): width = child['count'] * delta traverse(child, x, x + width, height + 1) x += width traverse(state, 0, 1, 0) return {'left': starts, 'right': stops, 'bottom': heights, 'width': widths, 'top': [x + 1 for x in heights], 'color': colors, 'states': states, 'filename': filenames, 'line': lines, 'line_number': line_numbers, 'name': names, 'time': times} try: from bokeh.palettes import viridis except ImportError: palette = ['red', 'green', 'blue', 'yellow'] else: palette = viridis(10) counter = itertools.count() @toolz.memoize def color_of(x): return palette[next(counter) % len(palette)] distributed-1.20.2/distributed/protocol/000077500000000000000000000000001321233345200202775ustar00rootroot00000000000000distributed-1.20.2/distributed/protocol/__init__.py000066400000000000000000000017111321233345200224100ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from functools import partial from .compression import compressions, default_compression from .core import (dumps, loads, maybe_compress, decompress, msgpack) from .serialize import ( serialize, deserialize, nested_deserialize, Serialize, Serialized, to_serialize, register_serialization, register_serialization_lazy, serialize_bytes, deserialize_bytes, serialize_bytelist) from ..utils import ignoring @partial(register_serialization_lazy, "numpy") def _register_numpy(): from . import numpy @partial(register_serialization_lazy, "h5py") def _register_h5py(): from . import h5py @partial(register_serialization_lazy, "netCDF4") def _register_netcdf4(): from . import netcdf4 @partial(register_serialization_lazy, "keras") def _register_keras(): from . import keras @partial(register_serialization_lazy, "sparse") def _register_sparse(): from . import sparse distributed-1.20.2/distributed/protocol/compression.py000066400000000000000000000125751321233345200232240ustar00rootroot00000000000000""" Record known compressors Includes utilities for determining whether or not to compress """ from __future__ import print_function, division, absolute_import import logging import random from dask.context import _globals from toolz import identity, partial try: import blosc n = blosc.set_nthreads(2) if hasattr('blosc', 'releasegil'): blosc.set_releasegil(True) except ImportError: blosc = False from ..config import config from ..utils import ignoring, ensure_bytes compressions = {None: {'compress': identity, 'decompress': identity}} compressions[False] = compressions[None] # alias default_compression = None logger = logging.getLogger(__name__) with ignoring(ImportError): import zlib compressions['zlib'] = {'compress': zlib.compress, 'decompress': zlib.decompress} with ignoring(ImportError): import snappy def _fixed_snappy_decompress(data): # snappy.decompress() doesn't accept memoryviews if isinstance(data, memoryview): data = data.tobytes() return snappy.decompress(data) compressions['snappy'] = {'compress': snappy.compress, 'decompress': _fixed_snappy_decompress} default_compression = 'snappy' with ignoring(ImportError): import lz4 try: # try using the new lz4 API lz4_compress = lz4.block.compress lz4_decompress = lz4.block.decompress except AttributeError as err: # fall back to old one lz4_compress = lz4.LZ4_compress lz4_decompress = lz4.LZ4_uncompress # helper to bypass missing memoryview support in current lz4 # (fixed in later versions) def _fixed_lz4_compress(data): try: return lz4_compress(data) except TypeError: if isinstance(data, memoryview): return lz4_compress(data.tobytes()) else: raise def _fixed_lz4_decompress(data): try: return lz4_decompress(data) except (ValueError, TypeError): if isinstance(data, memoryview): return lz4_decompress(data.tobytes()) else: raise compressions['lz4'] = {'compress': _fixed_lz4_compress, 'decompress': _fixed_lz4_decompress} default_compression = 'lz4' with ignoring(ImportError): import blosc compressions['blosc'] = {'compress': partial(blosc.compress, clevel=5, cname='lz4'), 'decompress': blosc.decompress} default = config.get('compression', 'auto') if default != 'auto': if default in compressions: default_compression = default else: raise ValueError("Default compression '%s' not found.\n" "Choices include auto, %s" % ( default, ', '.join(sorted(map(str, compressions))))) def byte_sample(b, size, n): """ Sample a bytestring from many locations Parameters ---------- b: bytes or memoryview size: int size of each sample to collect n: int number of samples to collect """ starts = [random.randint(0, len(b) - size) for j in range(n)] ends = [] for i, start in enumerate(starts[:-1]): ends.append(min(start + size, starts[i + 1])) ends.append(starts[-1] + size) parts = [b[start:end] for start, end in zip(starts, ends)] return b''.join(map(ensure_bytes, parts)) def maybe_compress(payload, min_size=1e4, sample_size=1e4, nsamples=5): """ Maybe compress payload 1. We don't compress small messages 2. We sample the payload in a few spots, compress that, and if it doesn't do any good we return the original 3. We then compress the full original, it it doesn't compress well then we return the original 4. We return the compressed result """ compression = _globals.get('compression', default_compression) if not compression: return None, payload if len(payload) < min_size: return None, payload if len(payload) > 2**31: # Too large, compression libraries often fail return None, payload min_size = int(min_size) sample_size = int(sample_size) compress = compressions[compression]['compress'] # Compress a sample, return original if not very compressed sample = byte_sample(payload, sample_size, nsamples) if len(compress(sample)) > 0.9 * len(sample): # sample not very compressible return None, payload if type(payload) is memoryview: nbytes = payload.itemsize * len(payload) else: nbytes = len(payload) if default_compression and blosc and type(payload) is memoryview: # Blosc does itemsize-aware shuffling, resulting in better compression compressed = blosc.compress(payload, typesize=payload.itemsize, cname='lz4', clevel=5) compression = 'blosc' else: compressed = compress(ensure_bytes(payload)) if len(compressed) > 0.9 * nbytes: # full data not very compressible return None, payload else: return compression, compressed def decompress(header, frames): """ Decompress frames according to information in the header """ return [compressions[c]['decompress'](frame) for c, frame in zip(header['compression'], frames)] distributed-1.20.2/distributed/protocol/core.py000066400000000000000000000122311321233345200216000ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging import msgpack try: from cytoolz import get_in except ImportError: from toolz import get_in from .compression import compressions, maybe_compress, decompress from .serialize import (serialize, deserialize, Serialize, Serialized, extract_serialize) from .utils import frame_split_size, merge_frames from ..utils import nbytes _deserialize = deserialize logger = logging.getLogger(__name__) def dumps(msg): """ Transform Python message to bytestream suitable for communication """ try: data = {} # Only lists and dicts can contain serialized values if isinstance(msg, (list, dict)): msg, data, bytestrings = extract_serialize(msg) small_header, small_payload = dumps_msgpack(msg) if not data: # fast path without serialized data return small_header, small_payload pre = {key: (value.header, value.frames) for key, value in data.items() if type(value) is Serialized} data = {key: serialize(value.data) for key, value in data.items() if type(value) is Serialize} header = {'headers': {}, 'keys': [], 'bytestrings': list(bytestrings)} out_frames = [] for key, (head, frames) in data.items(): if 'lengths' not in head: head['lengths'] = tuple(map(nbytes, frames)) if 'compression' not in head: frames = frame_split_size(frames) if frames: compression, frames = zip(*map(maybe_compress, frames)) else: compression = [] head['compression'] = compression head['count'] = len(frames) header['headers'][key] = head header['keys'].append(key) out_frames.extend(frames) for key, (head, frames) in pre.items(): if 'lengths' not in head: head['lengths'] = tuple(map(nbytes, frames)) head['count'] = len(frames) header['headers'][key] = head header['keys'].append(key) out_frames.extend(frames) for i, frame in enumerate(out_frames): if type(frame) is memoryview and frame.strides != (1,): try: frame = frame.cast('b') except TypeError: frame = frame.tobytes() out_frames[i] = frame return [small_header, small_payload, msgpack.dumps(header, use_bin_type=True)] + out_frames except Exception: logger.critical("Failed to Serialize", exc_info=True) raise def loads(frames, deserialize=True): """ Transform bytestream back into Python value """ frames = frames[::-1] # reverse order to improve pop efficiency if not isinstance(frames, list): frames = list(frames) try: small_header = frames.pop() small_payload = frames.pop() msg = loads_msgpack(small_header, small_payload) if not frames: return msg header = frames.pop() header = msgpack.loads(header, encoding='utf8', use_list=False) keys = header['keys'] headers = header['headers'] bytestrings = set(header['bytestrings']) for key in keys: head = headers[key] count = head['count'] if count: fs = frames[-count::][::-1] del frames[-count:] else: fs = [] if deserialize or key in bytestrings: if 'compression' in head: fs = decompress(head, fs) fs = merge_frames(head, fs) value = _deserialize(head, fs) else: value = Serialized(head, fs) get_in(key[:-1], msg)[key[-1]] = value return msg except Exception: logger.critical("Failed to deserialize", exc_info=True) raise def dumps_msgpack(msg): """ Dump msg into header and payload, both bytestrings All of the message must be msgpack encodable See Also: loads_msgpack """ header = {} payload = msgpack.dumps(msg, use_bin_type=True) fmt, payload = maybe_compress(payload) if fmt: header['compression'] = fmt if header: header_bytes = msgpack.dumps(header, use_bin_type=True) else: header_bytes = b'' return [header_bytes, payload] def loads_msgpack(header, payload): """ Read msgpack header and payload back to Python object See Also: dumps_msgpack """ if header: header = msgpack.loads(header, encoding='utf8') else: header = {} if header.get('compression'): try: decompress = compressions[header['compression']]['decompress'] payload = decompress(payload) except KeyError: raise ValueError("Data is compressed as %s but we don't have this" " installed" % str(header['compression'])) return msgpack.loads(payload, encoding='utf8') distributed-1.20.2/distributed/protocol/h5py.py000066400000000000000000000017521321233345200215430ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from .serialize import register_serialization def serialize_h5py_file(f): if f.mode != 'r': raise ValueError("Can only serialize read-only h5py files") return {'filename': f.filename}, [] def deserialize_h5py_file(header, frames): import h5py return h5py.File(header['filename'], mode='r') register_serialization('h5py._hl.files.File', serialize_h5py_file, deserialize_h5py_file) def serialize_h5py_dataset(x): header, _ = serialize_h5py_file(x.file) header['name'] = x.name return header, [] def deserialize_h5py_dataset(header, frames): file = deserialize_h5py_file(header, frames) return file[header['name']] register_serialization('h5py._hl.dataset.Dataset', serialize_h5py_dataset, deserialize_h5py_dataset) register_serialization('h5py._hl.group.Group', serialize_h5py_dataset, deserialize_h5py_dataset) distributed-1.20.2/distributed/protocol/keras.py000066400000000000000000000024141321233345200217570ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from .serialize import register_serialization, serialize, deserialize def serialize_keras_model(model): import keras if keras.__version__ < '1.2.0': raise ImportError("Need Keras >= 1.2.0. " "Try pip install keras --upgrade --no-deps") header = model._updated_config() weights = model.get_weights() headers, frames = list(zip(*map(serialize, weights))) header['headers'] = headers header['nframes'] = [len(L) for L in frames] frames = [frame for L in frames for frame in L] return header, frames def deserialize_keras_model(header, frames): from keras.models import model_from_config n = 0 weights = [] for head, length in zip(header['headers'], header['nframes']): x = deserialize(head, frames[n: n + length]) weights.append(x) n += length model = model_from_config(header) model.set_weights(weights) return model for module in ['keras', 'tensorflow.contrib.keras.python.keras']: for name in ['engine.training.Model', 'models.Model', 'models.Sequential']: register_serialization('.'.join([module, name]), serialize_keras_model, deserialize_keras_model) distributed-1.20.2/distributed/protocol/netcdf4.py000066400000000000000000000030621321233345200222010ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from .serialize import register_serialization, serialize, deserialize try: import netCDF4 HAS_NETCDF4 = True except ImportError: HAS_NETCDF4 = False def serialize_netcdf4_dataset(ds): # assume mode is read-only return {'filename': ds.filepath()}, [] def deserialize_netcdf4_dataset(header, frames): import netCDF4 return netCDF4.Dataset(header['filename'], mode='r') if HAS_NETCDF4: register_serialization(netCDF4.Dataset, serialize_netcdf4_dataset, deserialize_netcdf4_dataset) def serialize_netcdf4_variable(x): header, _ = serialize(x.group()) header['parent-type'] = header['type'] header['name'] = x.name return header, [] def deserialize_netcdf4_variable(header, frames): header['type'] = header['parent-type'] parent = deserialize(header, frames) return parent.variables[header['name']] if HAS_NETCDF4: register_serialization(netCDF4.Variable, serialize_netcdf4_variable, deserialize_netcdf4_variable) def serialize_netcdf4_group(g): parent = g while parent.parent: parent = parent.parent header, _ = serialize_netcdf4_dataset(parent) header['path'] = g.path return header, [] def deserialize_netcdf4_group(header, frames): file = deserialize_netcdf4_dataset(header, frames) return file[header['path']] if HAS_NETCDF4: register_serialization(netCDF4.Group, serialize_netcdf4_group, deserialize_netcdf4_group) distributed-1.20.2/distributed/protocol/numpy.py000066400000000000000000000056711321233345200220320ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import fractions import numpy as np try: import blosc n = blosc.set_nthreads(2) except ImportError: blosc = False from .utils import frame_split_size, merge_frames from .serialize import register_serialization from . import pickle from ..utils import log_errors def itemsize(dt): """ Itemsize of dtype Try to return the itemsize of the base element, return 8 as a fallback """ result = dt.base.itemsize if result > 255: result = 8 return result def serialize_numpy_ndarray(x): if x.dtype.hasobject: header = {'pickle': True} frames = [pickle.dumps(x)] return header, frames # We cannot blindly pickle the dtype as some may fail pickling, # so we have a mixture of strategies. if x.dtype.kind == 'V': # Preserving all the information works best when pickling try: # Only use stdlib pickle as cloudpickle is slow when failing # (microseconds instead of nanoseconds) dt = (1, pickle.pickle.dumps(x.dtype)) pickle.loads(dt[1]) # does it unpickle fine? except Exception: # dtype fails pickling => fall back on the descr if reasonable. if x.dtype.type is not np.void or x.dtype.alignment != 1: raise else: dt = (0, x.dtype.descr) else: dt = (0, x.dtype.str) if not x.shape: # 0d array strides = x.strides data = x.ravel() elif x.flags.c_contiguous or x.flags.f_contiguous: # Avoid a copy and respect order when unserializing strides = x.strides data = x.ravel(order='K') else: x = np.ascontiguousarray(x) strides = x.strides data = x.ravel() if data.dtype.fields or data.dtype.itemsize > 8: data = data.view('u%d' % fractions.gcd(x.dtype.itemsize, 8)) try: data = data.data except ValueError: # "ValueError: cannot include dtype 'M' in a buffer" data = data.view('u%d' % fractions.gcd(x.dtype.itemsize, 8)).data header = {'dtype': dt, 'shape': x.shape, 'strides': strides} if x.nbytes > 1e5: frames = frame_split_size([data]) else: frames = [data] header['lengths'] = [x.nbytes] return header, frames def deserialize_numpy_ndarray(header, frames): with log_errors(): if len(frames) > 1: frames = merge_frames(header, frames) if header.get('pickle'): return pickle.loads(frames[0]) is_custom, dt = header['dtype'] if is_custom: dt = pickle.loads(dt) else: dt = np.dtype(dt) x = np.ndarray(header['shape'], dtype=dt, buffer=frames[0], strides=header['strides']) return x register_serialization(np.ndarray, serialize_numpy_ndarray, deserialize_numpy_ndarray) distributed-1.20.2/distributed/protocol/pickle.py000066400000000000000000000032631321233345200221240ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging import sys import cloudpickle if sys.version_info.major == 2: import cPickle as pickle else: import pickle logger = logging.getLogger(__name__) def _always_use_pickle_for(x): mod, _, _ = x.__class__.__module__.partition('.') if mod == 'numpy': import numpy as np return isinstance(x, np.ndarray) elif mod == 'pandas': import pandas as pd return isinstance(x, pd.core.generic.NDFrame) elif mod == 'builtins': return isinstance(x, (str, bytes)) else: return False def dumps(x): """ Manage between cloudpickle and pickle 1. Try pickle 2. If it is short then check if it contains __main__ 3. If it is long, then first check type, then check __main__ """ try: result = pickle.dumps(x, protocol=pickle.HIGHEST_PROTOCOL) if len(result) < 1000: if b'__main__' in result: return cloudpickle.dumps(x, protocol=pickle.HIGHEST_PROTOCOL) else: return result else: if _always_use_pickle_for(x) or b'__main__' not in result: return result else: return cloudpickle.dumps(x, protocol=pickle.HIGHEST_PROTOCOL) except Exception: try: return cloudpickle.dumps(x, protocol=pickle.HIGHEST_PROTOCOL) except Exception as e: logger.info("Failed to serialize %s. Exception: %s", x, e) raise def loads(x): try: return pickle.loads(x) except Exception: logger.info("Failed to deserialize %s", x[:10000], exc_info=True) raise distributed-1.20.2/distributed/protocol/serialize.py000066400000000000000000000226151321233345200226460ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from functools import partial from dask.base import normalize_token try: from cytoolz import valmap, get_in except ImportError: from toolz import valmap, get_in import msgpack from . import pickle from ..compatibility import PY2 from .compression import maybe_compress, decompress from .utils import unpack_frames, pack_frames_prelude, frame_split_size serializers = {} deserializers = {None: lambda header, frames: pickle.loads(b''.join(frames))} lazy_registrations = {} def register_serialization(cls, serialize, deserialize): """ Register a new class for custom serialization Parameters ---------- cls: type serialize: function deserialize: function Examples -------- >>> class Human(object): ... def __init__(self, name): ... self.name = name >>> def serialize(human): ... header = {} ... frames = [human.name.encode()] ... return header, frames >>> def deserialize(header, frames): ... return Human(frames[0].decode()) >>> register_serialization(Human, serialize, deserialize) >>> serialize(Human('Alice')) ({}, [b'Alice']) See Also -------- serialize deserialize """ if isinstance(cls, type): name = typename(cls) elif isinstance(cls, str): name = cls serializers[name] = serialize deserializers[name] = deserialize def register_serialization_lazy(toplevel, func): """Register a registration function to be called if *toplevel* module is ever loaded. """ lazy_registrations[toplevel] = func def typename(typ): """ Return name of type Examples -------- >>> from distributed import Scheduler >>> typename(Scheduler) 'distributed.scheduler.Scheduler' """ return typ.__module__ + '.' + typ.__name__ def _find_lazy_registration(typename): toplevel, _, _ = typename.partition('.') if toplevel in lazy_registrations: lazy_registrations.pop(toplevel)() return True else: return False def serialize(x): r""" Convert object to a header and list of bytestrings This takes in an arbitrary Python object and returns a msgpack serializable header and a list of bytes or memoryview objects. By default this uses pickle/cloudpickle but can use special functions if they have been pre-registered. Examples -------- >>> serialize(1) ({}, [b'\x80\x04\x95\x03\x00\x00\x00\x00\x00\x00\x00K\x01.']) >>> serialize(b'123') # some special types get custom treatment ({'type': 'builtins.bytes'}, [b'123']) >>> deserialize(*serialize(1)) 1 Returns ------- header: dictionary containing any msgpack-serializable metadata frames: list of bytes or memoryviews, commonly of length one See Also -------- deserialize: Convert header and frames back to object to_serialize: Mark that data in a message should be serialized register_serialization: Register custom serialization functions """ if isinstance(x, Serialized): return x.header, x.frames typ = type(x) name = typename(typ) if name in serializers: header, frames = serializers[name](x) header['type'] = name else: if _find_lazy_registration(name): return serialize(x) # recurse header, frames = {}, [pickle.dumps(x)] return header, frames def deserialize(header, frames): """ Convert serialized header and list of bytestrings back to a Python object Parameters ---------- header: dict frames: list of bytes See Also -------- serialize """ name = header.get('type') if name not in deserializers: if _find_lazy_registration(name): return deserialize(header, frames) # recurse f = deserializers[header.get('type')] return f(header, frames) class Serialize(object): """ Mark an object that should be serialized Example ------- >>> msg = {'op': 'update', 'data': to_serialize(123)} >>> msg # doctest: +SKIP {'op': 'update', 'data': } See also -------- distributed.protocol.dumps """ def __init__(self, data): self.data = data def __repr__(self): return "" % str(self.data) def __eq__(self, other): return (isinstance(other, Serialize) and other.data == self.data) def __ne__(self, other): return not (self == other) def __hash__(self): return hash(self.data) to_serialize = Serialize class Serialized(object): """ An object that is already serialized into header and frames Normal serialization operations pass these objects through. This is typically used within the scheduler which accepts messages that contain data without actually unpacking that data. """ def __init__(self, header, frames): self.header = header self.frames = frames def deserialize(self): from .core import decompress frames = decompress(self.header, self.frames) return deserialize(self.header, frames) def __eq__(self, other): return (isinstance(other, Serialized) and other.header == self.header and other.frames == self.frames) def __ne__(self, other): return not (self == other) def container_copy(c): typ = type(c) if typ is list: return list(map(container_copy, c)) if typ is dict: return valmap(container_copy, c) return c def extract_serialize(x): """ Pull out Serialize objects from message This also remove large bytestrings from the message into a second dictionary. Examples -------- >>> from distributed.protocol import to_serialize >>> msg = {'op': 'update', 'data': to_serialize(123)} >>> extract_serialize(msg) ({'op': 'update'}, {('data',): }, set()) """ ser = {} _extract_serialize(x, ser) if ser: x = container_copy(x) for path in ser: t = get_in(path[:-1], x) if isinstance(t, dict): del t[path[-1]] else: t[path[-1]] = None bytestrings = set() for k, v in ser.items(): if type(v) is bytes: ser[k] = to_serialize(v) bytestrings.add(k) return x, ser, bytestrings def _extract_serialize(x, ser, path=()): if type(x) is dict: for k, v in x.items(): typ = type(v) if typ is list or typ is dict: _extract_serialize(v, ser, path + (k,)) elif (typ is Serialize or typ is Serialized or typ is bytes and len(v) > 2**16): ser[path + (k,)] = v elif type(x) is list: for k, v in enumerate(x): typ = type(v) if typ is list or typ is dict: _extract_serialize(v, ser, path + (k,)) elif (typ is Serialize or typ is Serialized or typ is bytes and len(v) > 2**16): ser[path + (k,)] = v def nested_deserialize(x): """ Replace all Serialize and Serialized values nested in *x* with the original values. Returns a copy of *x*. >>> msg = {'op': 'update', 'data': to_serialize(123)} >>> nested_deserialize(msg) {'op': 'update', 'data': 123} """ def replace_inner(x): if type(x) is dict: x = x.copy() for k, v in x.items(): typ = type(v) if typ is dict or typ is list: x[k] = replace_inner(v) elif typ is Serialize: x[k] = v.data elif typ is Serialized: x[k] = deserialize(v.header, v.frames) elif type(x) is list: x = list(x) for k, v in enumerate(x): typ = type(v) if typ is dict or typ is list: x[k] = replace_inner(v) elif typ is Serialize: x[k] = v.data elif typ is Serialized: x[k] = deserialize(v.header, v.frames) return x return replace_inner(x) @partial(normalize_token.register, Serialized) def normalize_Serialized(o): return [o.header] + o.frames # for dask.base.tokenize # Teach serialize how to handle bytestrings def _serialize_bytes(obj): header = {} # no special metadata frames = [obj] return header, frames def _deserialize_bytes(header, frames): return frames[0] register_serialization(bytes, _serialize_bytes, _deserialize_bytes) def serialize_bytelist(x): header, frames = serialize(x) frames = frame_split_size(frames) if frames: compression, frames = zip(*map(maybe_compress, frames)) else: compression = [] header['compression'] = compression header['count'] = len(frames) header = msgpack.dumps(header, use_bin_type=True) frames2 = [header] + list(frames) return [pack_frames_prelude(frames2)] + frames2 def serialize_bytes(x): L = serialize_bytelist(x) if PY2: L = [bytes(y) for y in L] return b''.join(L) def deserialize_bytes(b): frames = unpack_frames(b) header, frames = frames[0], frames[1:] if header: header = msgpack.loads(header, encoding='utf8') else: header = {} frames = decompress(header, frames) return deserialize(header, frames) distributed-1.20.2/distributed/protocol/sparse.py000066400000000000000000000016611321233345200221520ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from .serialize import register_serialization, serialize, deserialize def serialize_sparse(x): coords_header, coords_frames = serialize(x.coords) data_header, data_frames = serialize(x.data) header = {'coords-header': coords_header, 'data-header': data_header, 'shape': x.shape, 'nframes': [len(coords_frames), len(data_frames)]} return header, coords_frames + data_frames def deserialize_sparse(header, frames): import sparse coords_frames = frames[:header['nframes'][0]] data_frames = frames[header['nframes'][0]:] coords = deserialize(header['coords-header'], coords_frames) data = deserialize(header['data-header'], data_frames) shape = header['shape'] return sparse.COO(coords, data, shape=shape) register_serialization('sparse.core.COO', serialize_sparse, deserialize_sparse) distributed-1.20.2/distributed/protocol/tests/000077500000000000000000000000001321233345200214415ustar00rootroot00000000000000distributed-1.20.2/distributed/protocol/tests/test_h5py.py000066400000000000000000000073671321233345200237540ustar00rootroot00000000000000import functools import traceback import pytest h5py = pytest.importorskip('h5py') from distributed.protocol import deserialize, serialize from distributed.utils import PY3, tmpfile def silence_h5py_issue775(func): @functools.wraps(func) def wrapper(): if PY3: try: func() except RuntimeError as e: # https://github.com/h5py/h5py/issues/775 if str(e) != "dictionary changed size during iteration": raise tb = traceback.extract_tb(e.__traceback__) filename, lineno, _, _ = tb[-1] if not filename.endswith("h5py/_objects.pyx"): raise else: func() return wrapper @silence_h5py_issue775 def test_serialize_deserialize_file(): with tmpfile() as fn: with h5py.File(fn, mode='a') as f: f.create_dataset('/x', shape=(2, 2), dtype='i4') with h5py.File(fn, mode='r') as f: g = deserialize(*serialize(f)) assert f.filename == g.filename assert isinstance(g, h5py.File) assert f.mode == g.mode assert g['x'].shape == (2, 2) @silence_h5py_issue775 def test_serialize_deserialize_group(): with tmpfile() as fn: with h5py.File(fn, mode='a') as f: f.create_dataset('/group1/group2/x', shape=(2, 2), dtype='i4') with h5py.File(fn, mode='r') as f: group = f['/group1/group2'] group2 = deserialize(*serialize(group)) assert isinstance(group2, h5py.Group) assert group.file.filename == group2.file.filename assert group2['x'].shape == (2, 2) @silence_h5py_issue775 def test_serialize_deserialize_dataset(): with tmpfile() as fn: with h5py.File(fn, mode='a') as f: x = f.create_dataset('/group1/group2/x', shape=(2, 2), dtype='i4') with h5py.File(fn, mode='r') as f: x = f['group1/group2/x'] y = deserialize(*serialize(x)) assert isinstance(y, h5py.Dataset) assert x.name == y.name assert x.file.filename == y.file.filename assert (x[:] == y[:]).all() @silence_h5py_issue775 def test_raise_error_on_serialize_write_permissions(): with tmpfile() as fn: with h5py.File(fn, mode='a') as f: x = f.create_dataset('/x', shape=(2, 2), dtype='i4') f.flush() with pytest.raises(ValueError): serialize(x) with pytest.raises(ValueError): serialize(f) from distributed.utils_test import gen_cluster import dask.array as da @silence_h5py_issue775 @gen_cluster(client=True) def test_h5py_serialize(c, s, a, b): from dask.utils import SerializableLock lock = SerializableLock('hdf5') with tmpfile() as fn: with h5py.File(fn, mode='a') as f: x = f.create_dataset('/group/x', shape=(4,), dtype='i4', chunks=(2,)) x[:] = [1, 2, 3, 4] with h5py.File(fn, mode='r') as f: dset = f['/group/x'] x = da.from_array(dset, chunks=dset.chunks, lock=lock) y = c.compute(x) y = yield y assert (y[:] == dset[:]).all() @gen_cluster(client=True) def test_h5py_serialize_2(c, s, a, b): with tmpfile() as fn: with h5py.File(fn, mode='a') as f: x = f.create_dataset('/group/x', shape=(12,), dtype='i4', chunks=(4,)) x[:] = [1, 2, 3, 4] * 3 with h5py.File(fn, mode='r') as f: dset = f['/group/x'] x = da.from_array(dset, chunks=(3,)) y = c.compute(x.sum()) y = yield y assert y == (1 + 2 + 3 + 4) * 3 distributed-1.20.2/distributed/protocol/tests/test_keras.py000066400000000000000000000014051321233345200241570ustar00rootroot00000000000000 import numpy as np from numpy.testing import assert_allclose import pytest keras = pytest.importorskip('keras') from distributed.protocol import serialize, deserialize, dumps, loads, to_serialize def test_serialize_deserialize_model(): model = keras.models.Sequential() model.add(keras.layers.Dense(5, input_dim=3)) model.add(keras.layers.Dense(2)) model.compile(optimizer='sgd', loss='mse') x = np.random.random((1, 3)) y = np.random.random((1, 2)) model.train_on_batch(x, y) loaded = deserialize(*serialize(model)) assert_allclose(loaded.predict(x), model.predict(x)) data = {'model': to_serialize(model)} frames = dumps(data) result = loads(frames) assert_allclose(result['model'].predict(x), model.predict(x)) distributed-1.20.2/distributed/protocol/tests/test_netcdf4.py000066400000000000000000000054301321233345200244030ustar00rootroot00000000000000import pytest netCDF4 = pytest.importorskip('netCDF4') np = pytest.importorskip('numpy') from distributed.protocol import deserialize, serialize from distributed.utils import tmpfile def create_test_dataset(fn): with netCDF4.Dataset(fn, mode='w') as ds: ds.createDimension('x', 3) v = ds.createVariable('x', np.int32, ('x',)) v[:] = np.arange(3) g = ds.createGroup('group') g2 = ds.createGroup('group/group1') v2 = ds.createVariable('group/y', np.int32, ('x',)) v2[:] = np.arange(3) + 1 v3 = ds.createVariable('group/group1/z', np.int32, ('x',)) v3[:] = np.arange(3) + 2 def test_serialize_deserialize_dataset(): with tmpfile() as fn: create_test_dataset(fn) with netCDF4.Dataset(fn, mode='r') as f: g = deserialize(*serialize(f)) assert f.filepath() == g.filepath() assert isinstance(g, netCDF4.Dataset) assert g.variables['x'].dimensions == ('x',) assert g.variables['x'].dtype == np.int32 assert (g.variables['x'][:] == np.arange(3)).all() def test_serialize_deserialize_variable(): with tmpfile() as fn: create_test_dataset(fn) with netCDF4.Dataset(fn, mode='r') as f: x = f.variables['x'] y = deserialize(*serialize(x)) assert isinstance(y, netCDF4.Variable) assert y.dimensions == ('x',) assert (x.dtype == y.dtype) assert (x[:] == y[:]).all() def test_serialize_deserialize_group(): with tmpfile() as fn: create_test_dataset(fn) with netCDF4.Dataset(fn, mode='r') as f: for path in ['group', 'group/group1']: g = f[path] h = deserialize(*serialize(g)) assert isinstance(h, netCDF4.Group) assert h.name == g.name assert list(g.groups) == list(h.groups) assert list(g.variables) == list(h.variables) vars = [f.variables['x'], f['group'].variables['y'], f['group/group1'].variables['z']] for x in vars: y = deserialize(*serialize(x)) assert isinstance(y, netCDF4.Variable) assert y.dimensions == ('x',) assert (x.dtype == y.dtype) assert (x[:] == y[:]).all() from distributed.utils_test import gen_cluster import dask.array as da @gen_cluster(client=True) def test_netcdf4_serialize(c, s, a, b): with tmpfile() as fn: create_test_dataset(fn) with netCDF4.Dataset(fn, mode='r') as f: dset = f.variables['x'] x = da.from_array(dset, chunks=2) y = c.compute(x) y = yield y assert (y[:] == dset[:]).all() distributed-1.20.2/distributed/protocol/tests/test_numpy.py000066400000000000000000000175641321233345200242370ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import sys from zlib import crc32 import numpy as np import pytest from distributed.protocol import (serialize, deserialize, decompress, dumps, loads, to_serialize, msgpack) from distributed.protocol.utils import BIG_BYTES_SHARD_SIZE from distributed.utils import tmpfile, nbytes from distributed.utils_test import slow, gen_cluster from distributed.protocol.numpy import itemsize from distributed.protocol.compression import maybe_compress def test_serialize(): x = np.ones((5, 5)) header, frames = serialize(x) assert header['type'] assert len(frames) == 1 if 'compression' in header: frames = decompress(header, frames) result = deserialize(header, frames) assert (result == x).all() @pytest.mark.parametrize('x', [np.ones(5), np.array(5), np.random.random((5, 5)), np.random.random((5, 5))[::2, :], np.random.random((5, 5))[:, ::2], np.asfortranarray(np.random.random((5, 5))), np.asfortranarray(np.random.random((5, 5)))[::2, :], np.asfortranarray(np.random.random((5, 5)))[:, ::2], np.random.random(5).astype('f4'), np.random.random(5).astype('>i8'), np.random.random(5).astype(' 2 result = loads(frames) assert result == {'x': 1, 'data': 123} result2 = loads(frames, deserialize=False) assert result2['x'] == 1 assert isinstance(result2['data'], Serialized) assert any(a is b for a in result2['data'].frames for b in frames) frames2 = dumps(result2) assert all(map(eq_frames, frames, frames2)) result3 = loads(frames2) assert result == result3 def test_dumps_loads_Serialized(): msg = {'x': 1, 'data': Serialized(*serialize(123)), } frames = dumps(msg) assert len(frames) > 2 result = loads(frames) assert result == {'x': 1, 'data': 123} result2 = loads(frames, deserialize=False) assert result2 == msg frames2 = dumps(result2) assert all(map(eq_frames, frames, frames2)) result3 = loads(frames2) assert result == result3 @pytest.mark.skipif(sys.version_info[0] < 3, reason='NumPy doesnt use memoryviews') def test_maybe_compress_memoryviews(): np = pytest.importorskip('numpy') pytest.importorskip('lz4') x = np.arange(1000000, dtype='int64') compression, payload = maybe_compress(x.data) try: import blosc # flake8: noqa except ImportError: assert compression == 'lz4' assert len(payload) < x.nbytes * 0.75 else: assert compression == 'blosc' assert len(payload) < x.nbytes / 10 distributed-1.20.2/distributed/protocol/tests/test_protocol_utils.py000066400000000000000000000013001321233345200261250ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from distributed.protocol.utils import merge_frames, pack_frames, unpack_frames from distributed.utils import ensure_bytes def test_merge_frames(): result = merge_frames({'lengths': [3, 4]}, [b'12', b'34', b'567']) expected = [b'123', b'4567'] assert list(map(ensure_bytes, result)) == expected b = b'123' assert merge_frames({'lengths': [3]}, [b])[0] is b L = [b'123', b'456'] assert merge_frames({'lengths': [3, 3]}, L) is L def test_pack_frames(): frames = [b'123', b'asdf'] b = pack_frames(frames) assert isinstance(b, bytes) frames2 = unpack_frames(b) assert frames == frames2 distributed-1.20.2/distributed/protocol/tests/test_serialize.py000066400000000000000000000103501321233345200250400ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import copy import pickle import numpy as np import pytest from toolz import identity from distributed.protocol import (register_serialization, serialize, deserialize, nested_deserialize, Serialize, Serialized, to_serialize, serialize_bytes, deserialize_bytes, serialize_bytelist,) from distributed.utils import nbytes class MyObj(object): def __init__(self, data): self.data = data def __getstate__(self): raise Exception('Not picklable') def serialize_myobj(x): return {}, [pickle.dumps(x.data)] def deserialize_myobj(header, frames): return MyObj(pickle.loads(frames[0])) register_serialization(MyObj, serialize_myobj, deserialize_myobj) def test_dumps_serialize(): for x in [123, [1, 2, 3]]: header, frames = serialize(x) assert not header assert len(frames) == 1 result = deserialize(header, frames) assert result == x x = MyObj(123) header, frames = serialize(x) assert header['type'] assert len(frames) == 1 result = deserialize(header, frames) assert result.data == x.data def test_serialize_bytestrings(): b = b'123' header, frames = serialize(b) assert frames[0] is b def test_Serialize(): s = Serialize(123) assert '123' in str(s) assert s.data == 123 t = Serialize((1, 2)) assert str(t) u = Serialize(123) assert s == u assert not (s != u) assert s != t assert not (s == t) assert hash(s) == hash(u) assert hash(s) != hash(t) # most probably def test_Serialized(): s = Serialized(*serialize(123)) t = Serialized(*serialize((1, 2))) u = Serialized(*serialize(123)) assert s == u assert not (s != u) assert s != t assert not (s == t) def test_nested_deserialize(): x = {'op': 'update', 'x': [to_serialize(123), to_serialize(456), 789], 'y': {'a': ['abc', Serialized(*serialize('def'))], 'b': 'ghi'} } x_orig = copy.deepcopy(x) assert nested_deserialize(x) == {'op': 'update', 'x': [123, 456, 789], 'y': {'a': ['abc', 'def'], 'b': 'ghi'} } assert x == x_orig # x wasn't mutated from distributed.utils_test import gen_cluster from dask import delayed @gen_cluster(client=True) def test_object_in_graph(c, s, a, b): o = MyObj(123) v = delayed(o) v2 = delayed(identity)(v) future = c.compute(v2) result = yield future assert isinstance(result, MyObj) assert result.data == 123 @gen_cluster(client=True) def test_scatter(c, s, a, b): o = MyObj(123) [future] = yield c._scatter([o]) yield c._replicate(o) o2 = yield c._gather(future) assert isinstance(o2, MyObj) assert o2.data == 123 @gen_cluster(client=True) def test_inter_worker_comms(c, s, a, b): o = MyObj(123) [future] = yield c._scatter([o], workers=a.address) future2 = c.submit(identity, future, workers=b.address) o2 = yield c._gather(future2) assert isinstance(o2, MyObj) assert o2.data == 123 class Empty(object): def __getstate__(self): raise Exception('Not picklable') def serialize_empty(x): return {}, [] def deserialize_empty(header, frames): return Empty() register_serialization(Empty, serialize_empty, deserialize_empty) def test_empty(): e = Empty() e2 = deserialize(*serialize(e)) assert isinstance(e2, Empty) def test_empty_loads(): from distributed.protocol import loads, dumps e = Empty() e2 = loads(dumps([to_serialize(e)])) assert isinstance(e2[0], Empty) def test_serialize_bytes(): for x in [1, 'abc', np.arange(5)]: b = serialize_bytes(x) assert isinstance(b, bytes) y = deserialize_bytes(b) assert str(x) == str(y) def test_serialize_list_compress(): pytest.importorskip('lz4') x = np.ones(1000000) L = serialize_bytelist(x) assert sum(map(nbytes, L)) < x.nbytes / 2 b = b''.join(L) y = deserialize_bytes(b) assert (x == y).all() distributed-1.20.2/distributed/protocol/tests/test_sparse.py000066400000000000000000000014071321233345200243510ustar00rootroot00000000000000 import numpy as np from numpy.testing import assert_allclose import pytest sparse = pytest.importorskip('sparse') from distributed.protocol import deserialize, serialize def test_serialize_deserialize_sparse(): x = np.random.random((2, 3, 4, 5)) x[x < 0.8] = 0 y = sparse.COO(x) header, frames = serialize(y) assert 'sparse' in header['type'] z = deserialize(*serialize(y)) assert_allclose(y.data, z.data) assert_allclose(y.coords, z.coords) assert_allclose(y.todense(), z.todense()) @pytest.mark.slow def test_serialize_deserialize_sparse_large(): n = 100000000 x = np.arange(n) data = np.ones(n, dtype=np.int16) s = sparse.COO([x], data) header, frames = serialize(s) s2 = deserialize(header, frames) distributed-1.20.2/distributed/protocol/utils.py000066400000000000000000000060631321233345200220160ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import struct from ..utils import ensure_bytes, nbytes BIG_BYTES_SHARD_SIZE = 2**26 def frame_split_size(frames, n=BIG_BYTES_SHARD_SIZE): """ Split a list of frames into a list of frames of maximum size This helps us to avoid passing around very large bytestrings. Examples -------- >>> frame_split_size([b'12345', b'678'], n=3) # doctest: +SKIP [b'123', b'45', b'678'] """ if not frames: return frames if max(map(nbytes, frames)) <= n: return frames out = [] for frame in frames: if nbytes(frame) > n: if isinstance(frame, bytes): frame = memoryview(frame) try: itemsize = frame.itemsize except AttributeError: itemsize = 1 for i in range(0, nbytes(frame) // itemsize, n // itemsize): out.append(frame[i: i + n // itemsize]) else: out.append(frame) return out def merge_frames(header, frames): """ Merge frames into original lengths Examples -------- >>> merge_frames({'lengths': [3, 3]}, [b'123456']) [b'123', b'456'] >>> merge_frames({'lengths': [6]}, [b'123', b'456']) [b'123456'] """ lengths = list(header['lengths']) if not frames: return frames assert sum(lengths) == sum(map(nbytes, frames)) if all(len(f) == l for f, l in zip(frames, lengths)): return frames frames = frames[::-1] lengths = lengths[::-1] out = [] while lengths: l = lengths.pop() L = [] while l: frame = frames.pop() if nbytes(frame) <= l: L.append(frame) l -= nbytes(frame) else: mv = memoryview(frame) L.append(mv[:l]) frames.append(mv[l:]) l = 0 out.append(b''.join(map(ensure_bytes, L))) return out def pack_frames_prelude(frames): lengths = [len(f) for f in frames] lengths = ([struct.pack('Q', len(frames))] + [struct.pack('Q', nbytes(frame)) for frame in frames]) return b''.join(lengths) def pack_frames(frames): """ Pack frames into a byte-like object This prepends length information to the front of the bytes-like object See Also -------- unpack_frames """ prelude = [pack_frames_prelude(frames)] if not isinstance(frames, list): frames = list(frames) return b''.join(prelude + frames) def unpack_frames(b): """ Unpack bytes into a sequence of frames This assumes that length information is at the front of the bytestring, as performed by pack_frames See Also -------- pack_frames """ (n_frames,) = struct.unpack('Q', b[:8]) frames = [] start = 8 + n_frames * 8 for i in range(n_frames): (length,) = struct.unpack('Q', b[(i + 1) * 8: (i + 2) * 8]) frame = b[start: start + length] frames.append(frame) start += length return frames distributed-1.20.2/distributed/publish.py000066400000000000000000000045041321233345200204610ustar00rootroot00000000000000from collections import MutableMapping from distributed.utils import log_errors class PublishExtension(object): """ An extension for the scheduler to manage collections * publish-list * publish-put * publish-get * publish-delete """ def __init__(self, scheduler): self.scheduler = scheduler self.datasets = dict() handlers = {'publish_list': self.list, 'publish_put': self.put, 'publish_get': self.get, 'publish_delete': self.delete} self.scheduler.handlers.update(handlers) self.scheduler.extensions['publish'] = self def put(self, stream=None, keys=None, data=None, name=None, client=None): with log_errors(): if name in self.datasets: raise KeyError("Dataset %s already exists" % name) self.scheduler.client_wants_keys(keys, 'published-%s' % name) self.datasets[name] = {'data': data, 'keys': keys} return {'status': 'OK', 'name': name} def delete(self, stream=None, name=None): with log_errors(): out = self.datasets.pop(name, {'keys': []}) self.scheduler.client_releases_keys(out['keys'], 'published-%s' % name) def list(self, *args): with log_errors(): return list(sorted(self.datasets.keys())) def get(self, stream, name=None, client=None): with log_errors(): if name in self.datasets: return self.datasets[name] else: raise KeyError("Dataset '%s' not found" % name) class Datasets(MutableMapping): """A dict-like wrapper around :class:`Client` dataset methods. Parameters ---------- client : distributed.client.Client """ def __init__(self, client): self.__client = client def __getitem__(self, key): return self.__client.get_dataset(key) def __setitem__(self, key, value): self.__client.publish_dataset(**{key: value}) def __delitem__(self, key): self.__client.unpublish_dataset(key) def __contains__(self, key): return key in self.__client.list_datasets() def __iter__(self): for key in self.__client.list_datasets(): yield key def __len__(self): return len(self.__client.list_datasets()) distributed-1.20.2/distributed/pytest_resourceleaks.py000066400000000000000000000330621321233345200232730ustar00rootroot00000000000000# -*- coding: utf-8 -*- """ A pytest plugin to trace resource leaks. """ from __future__ import print_function, division import collections import gc import time import os import sys import threading import pytest def pytest_addoption(parser): group = parser.getgroup('resource leaks') group.addoption( '-L', '--leaks', action='store', dest='leaks', help='''\ List of resources to monitor for leaks before and after each test. Can be 'all' or a comma-separated list of resource names (possible values: {known_checkers}). '''.format(known_checkers=', '.join(sorted("'%s'" % s for s in all_checkers))) ) group.addoption( '--leaks-timeout', action='store', type='float', dest='leaks_timeout', default=0.5, help='''\ Wait at most this number of seconds to mark a test leaking (default: %(default)s). ''' ) group.addoption( '--leaks-fail', action='store_true', dest='leaks_mark_failed', default=False, help='''Mark leaked tests failed.''' ) group.addoption( '--leak-retries', action='store', type=int, dest='leak_retries', default=1, help='''\ Max number of times to retry a test when it leaks, to ignore warmup-related issues (default: 1). ''' ) def pytest_configure(config): leaks = config.getvalue('leaks') if leaks: if leaks == 'all': leaks = sorted(all_checkers) else: leaks = leaks.split(',') unknown = sorted(set(leaks) - set(all_checkers)) if unknown: raise ValueError("unknown resources: %r" % (unknown,)) checkers = [all_checkers[leak]() for leak in leaks] checker = LeakChecker(checkers=checkers, grace_delay=config.getvalue('leaks_timeout'), mark_failed=config.getvalue('leaks_mark_failed'), max_retries=config.getvalue('leak_retries'), ) config.pluginmanager.register(checker, 'leaks_checker') all_checkers = {} def register_checker(name): def decorate(cls): assert issubclass(cls, ResourceChecker), cls assert name not in all_checkers all_checkers[name] = cls return cls return decorate class ResourceChecker(object): def on_start_test(self): pass def on_stop_test(self): pass def on_retry(self): pass def measure(self): raise NotImplementedError def has_leak(self, before, after): raise NotImplementedError def format(self, before, after): raise NotImplementedError @register_checker('fds') class FDChecker(ResourceChecker): def measure(self): if os.name == 'posix': import psutil return psutil.Process().num_fds() else: return 0 def has_leak(self, before, after): return after > before def format(self, before, after): return "leaked %d file descriptor(s)" % (after - before) @register_checker('memory') class RSSMemoryChecker(ResourceChecker): def measure(self): import psutil return psutil.Process().memory_info().rss def has_leak(self, before, after): return after > before + 1e7 def format(self, before, after): return "leaked %d MB of RSS memory" % ((after - before) / 1e6) @register_checker('threads') class ActiveThreadsChecker(ResourceChecker): def measure(self): return set(threading.enumerate()) def has_leak(self, before, after): return not after <= before def format(self, before, after): leaked = after - before assert leaked return ("leaked %d Python threads: %s" % (len(leaked), sorted(leaked, key=str))) class _ChildProcess(collections.namedtuple('_ChildProcess', ('pid', 'name', 'cmdline'))): @classmethod def from_process(cls, p): return cls(p.pid, p.name(), p.cmdline()) @register_checker('processes') class ChildProcessesChecker(ResourceChecker): def measure(self): import psutil # We use pid and creation time as keys to disambiguate between # processes (and protect against pid reuse) # Other properties such as cmdline may change for a given process children = {} p = psutil.Process() for c in p.children(recursive=True): try: with c.oneshot(): if c.ppid() == p.pid and os.path.samefile(c.exe(), sys.executable): cmdline = c.cmdline() if any(a.startswith('from multiprocessing.semaphore_tracker import main') for a in cmdline): # Skip multiprocessing semaphore tracker continue if any(a.startswith('from multiprocessing.forkserver import main') for a in cmdline): # Skip forkserver process, the forkserver's children # however will be recorded normally continue children[(c.pid, c.create_time())] = _ChildProcess.from_process(c) except psutil.NoSuchProcess: pass return children def has_leak(self, before, after): return not set(after) <= set(before) def format(self, before, after): leaked = set(after) - set(before) assert leaked formatted = [] for key in sorted(leaked): p = after[key] formatted.append(' - pid={p.pid}, name={p.name!r}, cmdline={p.cmdline!r}' .format(p=p)) return ("leaked %d processes:\n%s" % (len(leaked), '\n'.join(formatted))) @register_checker('tracemalloc') class TracemallocMemoryChecker(ResourceChecker): def __init__(self): global tracemalloc import tracemalloc def on_start_test(self): tracemalloc.start(1) def on_stop_test(self): tracemalloc.stop() def measure(self): import tracemalloc current, peak = tracemalloc.get_traced_memory() snap = tracemalloc.take_snapshot() return current, snap def has_leak(self, before, after): return after[0] > before[0] + 1e6 def format(self, before, after): bytes_before, snap_before = before bytes_after, snap_after = after diff = snap_after.compare_to(snap_before, 'traceback') ndiff = 5 min_size_diff = 2e5 lines = [] lines += ["leaked %.1f MB of traced Python memory" % ((bytes_after - bytes_before) / 1e6)] for stat in diff[:ndiff]: size_diff = stat.size_diff or stat.size if size_diff < min_size_diff: break count = stat.count_diff or stat.count lines += [" - leaked %.1f MB in %d calls at:" % (size_diff / 1e6, count)] lines += [" " + line for line in stat.traceback.format()] return "\n".join(lines) class LeakChecker(object): def __init__(self, checkers, grace_delay, mark_failed, max_retries): self.checkers = checkers self.grace_delay = grace_delay self.mark_failed = mark_failed self.max_retries = max_retries # {nodeid: {checkers}} self.skip_checkers = {} # {nodeid: {checker: [(before, after)]}} self.counters = {} # {nodeid: [(checker, before, after)]} self.leaks = {} # {nodeid: {outcomes}} self.outcomes = collections.defaultdict(set) # Reentrancy guard self._retrying = False def cleanup(self): gc.collect() def checks_for_item(self, nodeid): return [c for c in self.checkers if c not in self.skip_checkers.get(nodeid, ())] def measure(self, nodeid): # Return items in order return [(c, c.measure()) for c in self.checks_for_item(nodeid)] def measure_before_test(self, nodeid): for checker in self.checks_for_item(nodeid): checker.on_start_test() for checker, before in self.measure(nodeid): assert before is not None self.counters[nodeid][checker].append((before, None)) def measure_after_test(self, nodeid): outcomes = self.outcomes[nodeid] assert outcomes if outcomes != {'passed'}: # Test failed or skipped return def run_measurements(): leaks = [] for checker, after in self.measure(nodeid): assert after is not None c = self.counters[nodeid][checker] before, _ = c[-1] c[-1] = (before, after) if checker.has_leak(before, after): leaks.append((checker, before, after)) return leaks t1 = time.time() deadline = t1 + self.grace_delay leaks = run_measurements() if leaks: self.cleanup() for c, _, _ in leaks: c.on_retry() leaks = run_measurements() while leaks and time.time() < deadline: time.sleep(0.1) self.cleanup() for c, _, _ in leaks: c.on_retry() leaks = run_measurements() if leaks: self.leaks[nodeid] = leaks else: self.leaks.pop(nodeid, None) for checker in self.checks_for_item(nodeid): checker.on_stop_test() def maybe_retry(self, item, nextitem=None): def run_test_again(): # This invokes our setup/teardown hooks again # Inspired by https://pypi.python.org/pypi/pytest-rerunfailures from _pytest.runner import runtestprotocol item._initrequest() # Re-init fixtures reports = runtestprotocol(item, nextitem=nextitem, log=False) nodeid = item.nodeid leaks = self.leaks.get(nodeid) if leaks: self._retrying = True try: for i in range(self.max_retries): run_test_again() except Exception as e: print("--- Exception when re-running test ---") import traceback traceback.print_exc() else: leaks = self.leaks.get(nodeid) finally: self._retrying = False return leaks # Note on hook execution order: # pytest_runtest_protocol # pytest_runtest_setup # pytest_report_teststatus # pytest_runtest_call # pytest_report_teststatus # pytest_runtest_teardown # pytest_report_teststatus # See also https://github.com/abalkin/pytest-leaks/blob/master/pytest_leaks.py @pytest.hookimpl(hookwrapper=True) def pytest_runtest_protocol(self, item, nextitem): if not self._retrying: nodeid = item.nodeid assert nodeid not in self.counters self.counters[nodeid] = {c: [] for c in self.checkers} leaking = item.get_marker('leaking') if leaking is not None: unknown = sorted(set(leaking.args) - set(all_checkers)) if unknown: raise ValueError("pytest.mark.leaking: unknown resources %r" % (unknown,)) classes = tuple(all_checkers[a] for a in leaking.args) self.skip_checkers[nodeid] = {c for c in self.checkers if isinstance(c, classes)} yield @pytest.hookimpl(hookwrapper=True) def pytest_runtest_setup(self, item): self.measure_before_test(item.nodeid) yield @pytest.hookimpl(hookwrapper=True, trylast=True) def pytest_runtest_teardown(self, item): yield self.measure_after_test(item.nodeid) if not self._retrying: leaks = self.maybe_retry(item) if leaks and self.mark_failed: # Trigger fail here to allow stopping with `-x` pytest.fail() @pytest.hookimpl(hookwrapper=True, trylast=True) def pytest_report_teststatus(self, report): nodeid = report.nodeid outcomes = self.outcomes[nodeid] outcomes.add(report.outcome) outcome = yield if not self._retrying: if report.when == 'teardown': leaks = self.leaks.get(report.nodeid) if leaks: if self.mark_failed: outcome.force_result(('failed', 'L', 'LEAKED')) report.outcome = 'failed' report.longrepr = "\n".join( ["%s %s" % (nodeid, checker.format(before, after)) for checker, before, after in leaks]) else: outcome.force_result(('leaked', 'L', 'LEAKED')) # XXX should we log retried tests @pytest.hookimpl def pytest_terminal_summary(self, terminalreporter, exitstatus): tr = terminalreporter leaked = tr.getreports('leaked') if leaked: # If mark_failed is False, leaks are output as a separate # results section tr.write_sep("=", 'RESOURCE LEAKS') for rep in leaked: nodeid = rep.nodeid for checker, before, after in self.leaks[nodeid]: tr.line("%s %s" % (rep.nodeid, checker.format(before, after))) distributed-1.20.2/distributed/queues.py000066400000000000000000000222561321233345200203260ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import defaultdict import logging import uuid from tornado import gen import tornado.queues from .client import Future, _get_global_client, Client from .utils import tokey, sync from .worker import get_client logger = logging.getLogger(__name__) class QueueExtension(object): """ An extension for the scheduler to manage queues This adds the following routes to the scheduler * queue_create * queue_release * queue_put * queue_get * queue_size """ def __init__(self, scheduler): self.scheduler = scheduler self.queues = dict() self.client_refcount = dict() self.future_refcount = defaultdict(lambda: 0) self.scheduler.handlers.update({'queue_create': self.create, 'queue_release': self.release, 'queue_put': self.put, 'queue_get': self.get, 'queue_qsize': self.qsize}) self.scheduler.client_handlers['queue-future-release'] = self.future_release self.scheduler.extensions['queues'] = self def create(self, stream=None, name=None, client=None, maxsize=0): if name not in self.queues: self.queues[name] = tornado.queues.Queue(maxsize=maxsize) self.client_refcount[name] = 1 else: self.client_refcount[name] += 1 def release(self, stream=None, name=None, client=None): self.client_refcount[name] -= 1 if self.client_refcount[name] == 0: del self.client_refcount[name] futures = self.queues[name].queue del self.queues[name] self.scheduler.client_releases_keys(keys=[f.key for f in futures], client='queue-%s' % name) @gen.coroutine def put(self, stream=None, name=None, key=None, data=None, client=None, timeout=None): if key is not None: record = {'type': 'Future', 'value': key} self.future_refcount[name, key] += 1 self.scheduler.client_desires_keys(keys=[key], client='queue-%s' % name) else: record = {'type': 'msgpack', 'value': data} yield self.queues[name].put(record, timeout=timeout) def future_release(self, name=None, key=None, client=None): self.future_refcount[name, key] -= 1 if self.future_refcount[name, key] == 0: self.scheduler.client_releases_keys(keys=[key], client='queue-%s' % name) del self.future_refcount[name, key] @gen.coroutine def get(self, stream=None, name=None, client=None, timeout=None, batch=False): def process(record): """ Add task status if known """ if record['type'] == 'Future': record = record.copy() key = record['value'] try: state = self.scheduler.task_state[key] except KeyError: state = 'lost' record['state'] = state if state == 'erred': record['exception'] = self.scheduler.exceptions[self.scheduler.exceptions_blame[key]] record['traceback'] = self.scheduler.tracebacks[self.scheduler.exceptions_blame[key]] return record if batch: q = self.queues[name] out = [] if batch is True: while not q.empty(): record = yield q.get() out.append(record) else: if timeout is not None: msg = ("Dask queues don't support simultaneous use of " "integer batch sizes and timeouts") raise NotImplementedError(msg) for i in range(batch): record = yield q.get() out.append(record) out = [process(o) for o in out] raise gen.Return(out) else: record = yield self.queues[name].get(timeout=timeout) record = process(record) raise gen.Return(record) def qsize(self, stream=None, name=None, client=None): return self.queues[name].qsize() class Queue(object): """ Distributed Queue This allows multiple clients to share futures or small bits of data between each other with a multi-producer/multi-consumer queue. All metadata is sequentialized through the scheduler. Elements of the Queue must be either Futures or msgpack-encodable data (ints, strings, lists, dicts). All data is sent through the scheduler so it is wise not to send large objects. To share large objects scatter the data and share the future instead. .. warning:: This object is experimental and has known issues in Python 2 Examples -------- >>> from dask.distributed import Client, Queue # doctest: +SKIP >>> client = Client() # doctest: +SKIP >>> queue = Queue('x') # doctest: +SKIP >>> future = client.submit(f, x) # doctest: +SKIP >>> queue.put(future) # doctest: +SKIP See Also -------- Variable: shared variable between clients """ def __init__(self, name=None, client=None, maxsize=0): self.client = client or _get_global_client() self.name = name or 'queue-' + uuid.uuid4().hex if self.client.asynchronous: self._started = self.client.scheduler.queue_create(name=self.name, maxsize=maxsize) else: sync(self.client.loop, self.client.scheduler.queue_create, name=self.name, maxsize=maxsize) self._started = gen.moment def __await__(self): @gen.coroutine def _(): yield self._started raise gen.Return(self) return _().__await__() @gen.coroutine def _put(self, value, timeout=None): if isinstance(value, Future): yield self.client.scheduler.queue_put(key=tokey(value.key), timeout=timeout, name=self.name) else: yield self.client.scheduler.queue_put(data=value, timeout=timeout, name=self.name) def put(self, value, timeout=None, **kwargs): """ Put data into the queue """ return self.client.sync(self._put, value, timeout=timeout, **kwargs) def get(self, timeout=None, batch=False, **kwargs): """ Get data from the queue Parameters ---------- timeout: Number (optional) Time in seconds to wait before timing out batch: boolean, int (optional) If True then return all elements currently waiting in the queue. If an integer than return that many elements from the queue If False (default) then return one item at a time """ return self.client.sync(self._get, timeout=timeout, batch=batch, **kwargs) def qsize(self, **kwargs): """ Current number of elements in the queue """ return self.client.sync(self._qsize, **kwargs) @gen.coroutine def _get(self, timeout=None, batch=False): resp = yield self.client.scheduler.queue_get(timeout=timeout, name=self.name, batch=batch) def process(d): if d['type'] == 'Future': value = Future(d['value'], self.client, inform=True, state=d['state']) if d['state'] == 'erred': value._state.set_error(d['exception'], d['traceback']) self.client._send_to_scheduler({'op': 'queue-future-release', 'name': self.name, 'key': d['value']}) else: value = d['value'] return value if batch is False: result = process(resp) else: result = list(map(process, resp)) raise gen.Return(result) @gen.coroutine def _qsize(self): result = yield self.client.scheduler.queue_qsize(name=self.name) raise gen.Return(result) def _release(self): if self.client.status == 'running': # TODO: can leave zombie futures self.client._send_to_scheduler({'op': 'queue_release', 'name': self.name}) def __del__(self): self._release() def __getstate__(self): return (self.name, self.client.scheduler.address) def __setstate__(self, state): name, address = state try: client = get_client(address) assert client.address == address except (AttributeError, AssertionError): client = Client(address, set_as_default=False) self.__init__(name=name, client=client) distributed-1.20.2/distributed/recreate_exceptions.py000066400000000000000000000147661321233345200230610ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging from tornado import gen from .client import futures_of, wait from .utils import sync, tokey from .utils_comm import pack_data from .worker import _deserialize logger = logging.getLogger(__name__) class ReplayExceptionScheduler(object): """ A plugin for the scheduler to recreate exceptions locally This adds the following routes to the scheduler * cause_of_failure """ def __init__(self, scheduler): self.scheduler = scheduler self.scheduler.handlers['cause_of_failure'] = self.cause_of_failure self.scheduler.extensions['exceptions'] = self def cause_of_failure(self, *args, **kwargs): """ Return details of first failed task required by set of keys Parameters ---------- keys: list of keys known to the scheduler Returns ------- Dictionary with: cause: the key that failed task: the definition of that key deps: keys that the task depends on """ keys = kwargs.pop('keys', []) for key in keys: if isinstance(key, list): key = tuple(key) # ensure not a list from msgpack key = tokey(key) if key in self.scheduler.exceptions_blame: cause = self.scheduler.exceptions_blame[key] # cannot serialize sets return {'deps': list(self.scheduler.dependencies[cause]), 'cause': cause, 'task': self.scheduler.tasks[cause]} class ReplayExceptionClient(object): """ A plugin for the client allowing replay of remote exceptions locally Adds the following methods (and their async variants)to the given client: - ``recreate_error_locally``: main user method - ``get_futures_error``: gets the task, its details and dependencies, responsible for failure of the given future. """ def __init__(self, client): self.client = client self.client.extensions['exceptions'] = self # monkey patch self.client.recreate_error_locally = self.recreate_error_locally self.client._recreate_error_locally = self._recreate_error_locally self.client._get_futures_error = self._get_futures_error self.client.get_futures_error = self.get_futures_error @property def scheduler(self): return self.client.scheduler @gen.coroutine def _get_futures_error(self, future): # only get errors for futures that errored. futures = [f for f in futures_of(future) if f.status == 'error'] if not futures: raise ValueError("No errored futures passed") out = yield self.scheduler.cause_of_failure( keys=[f.key for f in futures]) deps, task = out['deps'], out['task'] if isinstance(task, dict): function, args, kwargs = _deserialize(**task) raise gen.Return((function, args, kwargs, deps)) else: function, args, kwargs = _deserialize(task=task) raise gen.Return((function, args, kwargs, [])) def get_futures_error(self, future): """ Ask the scheduler details of the sub-task of the given failed future When a future evaluates to a status of "error", i.e., an exception was raised in a task within its graph, we an get information from the scheduler. This function gets the details of the specific task that raised the exception and led to the error, but does not fetch data from the cluster or execute the function. Parameters ---------- future : future that failed, having ``status=="error"``, typically after an attempt to ``gather()`` shows a stack-stace. Returns ------- Tuple: - the function that raised an exception - argument list (a tuple), may include values and keys - keyword arguments (a dictionary), may include values and keys - list of keys that the function requires to be fetched to run See Also -------- ReplayExceptionClient.recreate_error_locally """ return self.client.sync(self._get_futures_error, future) @gen.coroutine def _recreate_error_locally(self, future): yield wait(future) out = yield self._get_futures_error(future) function, args, kwargs, deps = out futures = self.client._graph_to_futures({}, deps) data = yield self.client._gather(futures) args = pack_data(args, data) kwargs = pack_data(kwargs, data) raise gen.Return((function, args, kwargs)) def recreate_error_locally(self, future): """ For a failed calculation, perform the blamed task locally for debugging. This operation should be performed after a future (result of ``gather``, ``compute``, etc) comes back with a status of "error", if the stack- trace is not informative enough to diagnose the problem. The specific task (part of the graph pointing to the future) responsible for the error will be fetched from the scheduler, together with the values of its inputs. The function will then be executed, so that ``pdb`` can be used for debugging. Examples -------- >>> future = c.submit(div, 1, 0) # doctest: +SKIP >>> future.status # doctest: +SKIP 'error' >>> c.recreate_error_locally(future) # doctest: +SKIP ZeroDivisionError: division by zero If you're in IPython you might take this opportunity to use pdb >>> %pdb # doctest: +SKIP Automatic pdb calling has been turned ON >>> c.recreate_error_locally(future) # doctest: +SKIP ZeroDivisionError: division by zero 1 def div(x, y): ----> 2 return x / y ipdb> Parameters ---------- future : future or collection that failed The same thing as was given to ``gather``, but came back with an exception/stack-trace. Can also be a (persisted) dask collection containing any errored futures. Returns ------- Nothing; the function runs and should raise an exception, allowing the debugger to run. """ func, args, kwargs = sync(self.client.loop, self._recreate_error_locally, future) func(*args, **kwargs) distributed-1.20.2/distributed/scheduler.py000066400000000000000000004222021321233345200207700ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import defaultdict, deque, OrderedDict from datetime import timedelta from functools import partial import itertools import json import logging import os import pickle import random import six from sortedcontainers import SortedSet try: from cytoolz import frequencies, merge, pluck, merge_sorted, first except ImportError: from toolz import frequencies, merge, pluck, merge_sorted, first from toolz import memoize, valmap, first, second, concat from tornado import gen from tornado.gen import Return from tornado.ioloop import IOLoop from dask.core import reverse_dict from dask.order import order from .batched import BatchedSend from .comm import (normalize_address, resolve_address, get_address_host, unparse_host_port) from .compatibility import finalize from .config import config, log_format from .core import (rpc, connect, Server, send_recv, error_message, clean_exception, CommClosedError) from . import profile from .metrics import time from .node import ServerNode from .proctitle import setproctitle from .security import Security from .utils import (All, ignoring, get_ip, get_fileno_limit, log_errors, key_split, validate_key, no_default, DequeHandler) from .utils_comm import (scatter_to_workers, gather_from_workers) from .versions import get_versions from .publish import PublishExtension from .queues import QueueExtension from .recreate_exceptions import ReplayExceptionScheduler from .lock import LockExtension from .stealing import WorkStealing from .variable import VariableExtension logger = logging.getLogger(__name__) BANDWIDTH = config.get('bandwidth', 100e6) ALLOWED_FAILURES = config.get('allowed-failures', 3) LOG_PDB = config.get('pdb-on-err') or os.environ.get('DASK_ERROR_PDB', False) DEFAULT_DATA_SIZE = config.get('default-data-size', 1000) DEFAULT_EXTENSIONS = [ LockExtension, PublishExtension, ReplayExceptionScheduler, QueueExtension, VariableExtension, ] if config.get('work-stealing', True): DEFAULT_EXTENSIONS.append(WorkStealing) class Scheduler(ServerNode): """ Dynamic distributed task scheduler The scheduler tracks the current state of workers, data, and computations. The scheduler listens for events and responds by controlling workers appropriately. It continuously tries to use the workers to execute an ever growing dask graph. All events are handled quickly, in linear time with respect to their input (which is often of constant size) and generally within a millisecond. To accomplish this the scheduler tracks a lot of state. Every operation maintains the consistency of this state. The scheduler communicates with the outside world through Comm objects. It maintains a consistent and valid view of the world even when listening to several clients at once. A Scheduler is typically started either with the ``dask-scheduler`` executable:: $ dask-scheduler Scheduler started at 127.0.0.1:8786 Or within a LocalCluster a Client starts up without connection information:: >>> c = Client() # doctest: +SKIP >>> c.cluster.scheduler # doctest: +SKIP Scheduler(...) Users typically do not interact with the scheduler directly but rather with the client object ``Client``. **State** The scheduler contains the following state variables. Each variable is listed along with what it stores and a brief description. * **tasks:** ``{key: task}``: Dictionary mapping key to a serialized task like the following: ``{'function': b'...', 'args': b'...'}`` or ``{'task': b'...'}`` * **dependencies:** ``{key: {keys}}``: Dictionary showing which keys depend on which others * **dependents:** ``{key: {keys}}``: Dictionary showing which keys are dependent on which others * **task_state:** ``{key: string}``: Dictionary listing the current state of every task among the following: released, waiting, queue, no-worker, processing, memory, erred * **priority:** ``{key: tuple}``: A score per key that determines its priority * **waiting:** ``{key: {key}}``: Dictionary like dependencies but excludes keys already computed * **waiting_data:** ``{key: {key}}``: Dictionary like dependents but excludes keys already computed * **ready:** ``deque(key)`` Keys that are ready to run, but not yet assigned to a worker * **processing:** ``{worker: {key: cost}}``: Set of keys currently in execution on each worker and their expected duration * **rprocessing:** ``{key: worker}``: The worker currently executing a particular task * **who_has:** ``{key: {worker}}``: Where each key lives. The current state of distributed memory. * **has_what:** ``{worker: {key}}``: What worker has what keys. The transpose of who_has. * **released:** ``{keys}`` Set of keys that are known, but released from memory * **unrunnable:** ``{key}`` Keys that we are unable to run * **host_restrictions:** ``{key: {hostnames}}``: A set of hostnames per key of where that key can be run. Usually this is empty unless a key has been specifically restricted to only run on certain hosts. * **worker_restrictions:** ``{key: {workers}}``: Like host_restrictions except that these include specific host:port worker names * **loose_restrictions:** ``{key}``: Set of keys for which we are allow to violate restrictions (see above) if not valid workers are present. * **resource_restrictions:** ``{key: {str: Number}}``: Resources required by a task, such as ``{'GPU': 1}`` or ``{'memory': 1e9}``. These names must match resources specified when creating workers. * **worker_resources:** ``{worker: {str: Number}}``: The available resources on each worker like ``{'gpu': 2, 'mem': 1e9}``. These are abstract quantities that constrain certain tasks from running at the same time. * **used_resources:** ``{worker: {str: Number}}``: The sum of each resource used by all tasks allocated to a particular worker. * **exceptions:** ``{key: Exception}``: A dict mapping keys to remote exceptions * **tracebacks:** ``{key: list}``: A dict mapping keys to remote tracebacks stored as a list of strings * **exceptions_blame:** ``{key: key}``: A dict mapping a key to another key on which it depends that has failed * **suspicious_tasks:** ``{key: int}`` Number of times a task has been involved in a worker failure * **retries:** ``{key: int}`` Number of times a task may be automatically retried after failing * **deleted_keys:** ``{key: {workers}}`` Locations of workers that have keys that should be deleted * **wants_what:** ``{client: {key}}``: What keys are wanted by each client.. The transpose of who_wants. * **who_wants:** ``{key: {client}}``: Which clients want each key. The active targets of computation. * **nbytes:** ``{key: int}``: Number of bytes for a key as reported by workers holding that key. * **ncores:** ``{worker: int}``: Number of cores owned by each worker * **idle:** ``{worker}``: Set of workers that are not fully utilized * **worker_info:** ``{worker: {str: data}}``: Information about each worker * **host_info:** ``{hostname: dict}``: Information about each worker host * **worker_bytes:** ``{worker: int}``: Number of bytes in memory on each worker * **occupancy:** ``{worker: time}`` Expected runtime for all tasks currently processing on a worker * **services:** ``{str: port}``: Other services running on this scheduler, like Bokeh * **loop:** ``IOLoop``: The running Tornado IOLoop * **comms:** ``[Comm]``: A list of Comms from which we both accept stimuli and report results * **task_duration:** ``{key-prefix: time}`` Time we expect certain functions to take, e.g. ``{'sum': 0.25}`` * **coroutines:** ``[Futures]``: A list of active futures that control operation """ default_port = 8786 def __init__( self, center=None, loop=None, delete_interval=500, synchronize_worker_interval=60000, services=None, allowed_failures=ALLOWED_FAILURES, extensions=None, validate=False, scheduler_file=None, security=None, **kwargs): self._setup_logging() # Attributes self.allowed_failures = allowed_failures self.validate = validate self.status = None self.delete_interval = delete_interval self.synchronize_worker_interval = synchronize_worker_interval self.digests = None self.service_specs = services or {} self.services = {} self.scheduler_file = scheduler_file self.security = security or Security() assert isinstance(self.security, Security) self.connection_args = self.security.get_connection_args('scheduler') self.listen_args = self.security.get_listen_args('scheduler') # Communication state self.loop = loop or IOLoop.current() self.worker_comms = dict() self.comms = dict() self.coroutines = [] self._worker_coroutines = [] self._ipython_kernel = None # Task state self.tasks = dict() self.task_state = dict() self.dependencies = dict() self.dependents = dict() self.generation = 0 self.released = set() self.priority = dict() self.nbytes = dict() self.worker_bytes = dict() self.processing = dict() self.rprocessing = dict() self.task_duration = {prefix: 0.00001 for prefix in fast_tasks} self.unknown_durations = defaultdict(set) self.host_restrictions = dict() self.worker_restrictions = dict() self.resource_restrictions = dict() self.loose_restrictions = set() self.retries = dict() self.suspicious_tasks = defaultdict(lambda: 0) self.waiting = dict() self.waiting_data = dict() self.ready = deque() self.unrunnable = set() self.idle = set() self.who_has = dict() self.has_what = dict() self.who_wants = defaultdict(set) self.wants_what = defaultdict(set) self.exceptions = dict() self.tracebacks = dict() self.exceptions_blame = dict() self.datasets = dict() self.n_tasks = 0 self.task_metadata = dict() self.idle = SortedSet() self.saturated = set() self._task_collections = [self.tasks, self.dependencies, self.dependents, self.waiting, self.waiting_data, self.released, self.priority, self.nbytes, self.host_restrictions, self.worker_restrictions, self.loose_restrictions, self.ready, self.who_wants, self.wants_what, self.unknown_durations, self.rprocessing, self.resource_restrictions, self.retries] # Worker state self.ncores = dict() self.workers = SortedSet() self.total_ncores = 0 self.total_occupancy = 0 self.worker_info = dict() self.host_info = defaultdict(dict) self.worker_resources = dict() self.used_resources = dict() self.resources = defaultdict(dict) self.aliases = dict() self.occupancy = dict() self._worker_collections = [self.ncores, self.workers, self.worker_info, self.host_info, self.worker_resources, self.worker_restrictions, self.host_restrictions, self.resource_restrictions, self.used_resources, self.resources, self.aliases, self.occupancy, self.idle, self.saturated, self.processing, self.rprocessing, self.has_what, self.who_has] self.extensions = {} self.plugins = [] self.transition_log = deque(maxlen=config.get('transition-log-length', 100000)) self.log = deque(maxlen=config.get('transition-log-length', 100000)) self.worker_handlers = {'task-finished': self.handle_task_finished, 'task-erred': self.handle_task_erred, 'release': self.handle_release_data, 'release-worker-data': self.release_worker_data, 'add-keys': self.add_keys, 'missing-data': self.handle_missing_data, 'long-running': self.handle_long_running, 'reschedule': self.reschedule} self.client_handlers = {'update-graph': self.update_graph, 'client-desires-keys': self.client_desires_keys, 'update-data': self.update_data, 'report-key': self.report_on_key, 'client-releases-keys': self.client_releases_keys, 'restart': self.restart} self.handlers = {'register-client': self.add_client, 'scatter': self.scatter, 'register': self.add_worker, 'unregister': self.remove_worker, 'gather': self.gather, 'cancel': self.stimulus_cancel, 'feed': self.feed, 'terminate': self.close, 'broadcast': self.broadcast, 'ncores': self.get_ncores, 'has_what': self.get_has_what, 'who_has': self.get_who_has, 'processing': self.get_processing, 'call_stack': self.get_call_stack, 'profile': self.get_profile, 'logs': self.get_logs, 'worker_logs': self.get_worker_logs, 'nbytes': self.get_nbytes, 'versions': self.get_versions, 'add_keys': self.add_keys, 'rebalance': self.rebalance, 'replicate': self.replicate, 'start_ipython': self.start_ipython, 'run_function': self.run_function, 'update_data': self.update_data, 'set_resources': self.add_resources, 'retire_workers': self.retire_workers, 'get_metadata': self.get_metadata, 'set_metadata': self.set_metadata, 'get_task_status': self.get_task_status} self._transitions = { ('released', 'waiting'): self.transition_released_waiting, ('waiting', 'released'): self.transition_waiting_released, ('waiting', 'processing'): self.transition_waiting_processing, ('waiting', 'memory'): self.transition_waiting_memory, ('processing', 'released'): self.transition_processing_released, ('processing', 'memory'): self.transition_processing_memory, ('processing', 'erred'): self.transition_processing_erred, ('no-worker', 'released'): self.transition_no_worker_released, ('no-worker', 'waiting'): self.transition_no_worker_waiting, ('released', 'forgotten'): self.transition_released_forgotten, ('memory', 'forgotten'): self.transition_memory_forgotten, ('erred', 'forgotten'): self.transition_released_forgotten, ('memory', 'released'): self.transition_memory_released, ('released', 'erred'): self.transition_released_erred } connection_limit = get_fileno_limit() / 2 super(Scheduler, self).__init__( handlers=self.handlers, io_loop=self.loop, connection_limit=connection_limit, deserialize=False, connection_args=self.connection_args, **kwargs) if extensions is None: extensions = DEFAULT_EXTENSIONS for ext in extensions: ext(self) setproctitle("dask-scheduler [not started]") ################## # Administration # ################## def __repr__(self): return '' % ( self.address, len(self.workers), self.total_ncores) def identity(self, comm=None): """ Basic information about ourselves and our cluster """ d = {'type': type(self).__name__, 'id': str(self.id), 'address': self.address, 'services': {key: v.port for (key, v) in self.services.items()}, 'workers': dict(self.worker_info)} return d def get_worker_service_addr(self, worker, service_name): """ Get the (host, port) address of the named service on the *worker*. Returns None if the service doesn't exist. """ info = self.worker_info[worker] port = info['services'].get(service_name) if port is None: return None else: return info['host'], port def get_versions(self, comm): """ Basic information about ourselves and our cluster """ return get_versions() def start_services(self, listen_ip): for k, v in self.service_specs.items(): if isinstance(k, tuple): k, port = k else: port = 0 if listen_ip == '0.0.0.0': listen_ip = '' # for IPv6 try: service = v(self, io_loop=self.loop) service.listen((listen_ip, port)) self.services[k] = service except Exception as e: logger.info("Could not launch service: %r", (k, port), exc_info=True) def stop_services(self): for service in self.services.values(): service.stop() def start(self, addr_or_port=8786, start_queues=True): """ Clear out old state and restart all running coroutines """ for collection in self._task_collections: collection.clear() with ignoring(AttributeError): for c in self._worker_coroutines: c.cancel() for cor in self.coroutines: if cor.done(): exc = cor.exception() if exc: raise exc if self.status != 'running': if isinstance(addr_or_port, int): # Listen on all interfaces. `get_ip()` is not suitable # as it would prevent connecting via 127.0.0.1. self.listen(('', addr_or_port), listen_args=self.listen_args) self.ip = get_ip() listen_ip = '' else: self.listen(addr_or_port, listen_args=self.listen_args) self.ip = get_address_host(self.listen_address) listen_ip = self.ip if listen_ip == '0.0.0.0': listen_ip = '' if isinstance(addr_or_port, str) and addr_or_port.startswith('inproc://'): listen_ip = 'localhost' # Services listen on all addresses self.start_services(listen_ip) self.status = 'running' logger.info(" Scheduler at: %25s", self.address) for k, v in self.services.items(): logger.info("%11s at: %25s", k, '%s:%d' % (listen_ip, v.port)) if self.scheduler_file: with open(self.scheduler_file, 'w') as f: json.dump(self.identity(), f, indent=2) fn = self.scheduler_file # remove file when we close the process def del_scheduler_file(): if os.path.exists(fn): os.remove(fn) finalize(self, del_scheduler_file) self.loop.add_callback(self.reevaluate_occupancy) self.start_periodic_callbacks() setproctitle("dask-scheduler [%s]" % (self.address,)) return self.finished() @gen.coroutine def finished(self): """ Wait until all coroutines have ceased """ while any(not c.done() for c in self.coroutines): yield All(self.coroutines) def close_comms(self): """ Close all active Comms.""" for comm in self.comms.values(): comm.abort() self.rpc.close() @gen.coroutine def close(self, comm=None, fast=False): """ Send cleanup signal to all coroutines then wait until finished See Also -------- Scheduler.cleanup """ if self.status == 'closed': return logger.info("Scheduler closing...") setproctitle("dask-scheduler [closing]") self.stop_services() for ext in self.extensions: with ignoring(AttributeError): ext.teardown() logger.info("Scheduler closing all comms") yield self.cleanup() if not fast: yield self.finished() self.close_comms() self.status = 'closed' self.stop() yield super(Scheduler, self).close() setproctitle("dask-scheduler [closed]") @gen.coroutine def close_worker(self, stream=None, worker=None, safe=None): """ Remove a worker from the cluster This both removes the worker from our local state and also sends a signal to the worker to shut down. This works regardless of whether or not the worker has a nanny process restarting it """ logger.info("Closing worker %s", worker) with log_errors(): self.log_event(worker, {'action': 'close-worker'}) nanny_addr = self.get_worker_service_addr(worker, 'nanny') address = nanny_addr or worker self.remove_worker(address=worker, safe=safe) with rpc(address, connection_args=self.connection_args) as r: try: yield r.terminate(report=False) except EnvironmentError as e: logger.info("Exception from worker while closing: %s", e) self.remove_worker(address=worker, safe=safe) @gen.coroutine def cleanup(self): """ Clean up queues and coroutines, prepare to stop """ if self.status == 'closing': raise gen.Return() self.status = 'closing' logger.debug("Cleaning up coroutines") futures = [] for w, comm in list(self.worker_comms.items()): with ignoring(AttributeError): futures.append(comm.close()) for future in futures: yield future def _setup_logging(self): self._deque_handler = DequeHandler(n=config.get('log-length', 10000)) self._deque_handler.setFormatter(logging.Formatter(log_format)) logger.addHandler(self._deque_handler) finalize(self, logger.removeHandler, self._deque_handler) ########### # Stimuli # ########### def add_worker(self, comm=None, address=None, keys=(), ncores=None, name=None, resolve_address=True, nbytes=None, now=None, resources=None, host_info=None, **info): """ Add a new worker to the cluster """ with log_errors(): local_now = time() now = now or time() info = info or {} host_info = host_info or {} address = self.coerce_address(address, resolve_address) host = get_address_host(address) self.host_info[host]['last-seen'] = local_now address = normalize_address(address) if address not in self.worker_info: self.worker_info[address] = dict() if info: self.worker_info[address].update(info) if host_info: self.host_info[host].update(host_info) self.worker_info[address]['ncores'] = ncores delay = time() - now self.worker_info[address]['time-delay'] = delay self.worker_info[address]['last-seen'] = time() if resources: self.add_resources(worker=address, resources=resources) self.worker_info[address]['resources'] = resources if address in self.workers: self.log_event(address, merge({'action': 'heartbeat'}, info)) return {'status': 'OK', 'time': time(), 'heartbeat-interval': heartbeat_interval(len(self.workers))} name = name or address if name in self.aliases: return {'status': 'error', 'message': 'name taken, %s' % name, 'time': time()} if 'addresses' not in self.host_info[host]: self.host_info[host].update({'addresses': set(), 'cores': 0}) self.host_info[host]['addresses'].add(address) self.host_info[host]['cores'] += ncores self.ncores[address] = ncores self.workers.add(address) self.total_ncores += ncores self.aliases[name] = address self.worker_info[address]['name'] = name self.worker_info[address]['host'] = host if address not in self.processing: self.has_what[address] = set() self.worker_bytes[address] = 0 self.processing[address] = dict() self.occupancy[address] = 0 # Do not need to adjust self.total_occupancy as self.occupancy[address] cannot exist before this. self.check_idle_saturated(address) # for key in keys: # TODO # self.mark_key_in_memory(key, [address]) self.worker_comms[address] = BatchedSend(interval=5, loop=self.loop) self._worker_coroutines.append(self.handle_worker(address)) if self.ncores[address] > len(self.processing[address]): self.idle.add(address) for plugin in self.plugins[:]: try: plugin.add_worker(scheduler=self, worker=address) except Exception as e: logger.exception(e) if nbytes: for key in nbytes: state = self.task_state.get(key) if state in ('processing', 'waiting'): recommendations = self.transition(key, 'memory', worker=address, nbytes=nbytes[key]) self.transitions(recommendations) recommendations = {} for key in list(self.unrunnable): valid = self.valid_workers(key) if valid is True or address in valid or name in valid: recommendations[key] = 'waiting' if recommendations: self.transitions(recommendations) self.log_event(address, {'action': 'add-worker'}) self.log_event('all', {'action': 'add-worker', 'worker': address}) logger.info("Register %s", str(address)) return {'status': 'OK', 'time': time(), 'heartbeat-interval': heartbeat_interval(len(self.workers))} def update_graph(self, client=None, tasks=None, keys=None, dependencies=None, restrictions=None, priority=None, loose_restrictions=None, resources=None, submitting_task=None, retries=None): """ Add new computations to the internal dask graph This happens whenever the Client calls submit, map, get, or compute. """ start = time() keys = set(keys) self.client_desires_keys(keys=keys, client=client) if len(tasks) > 1: self.log_event(['all', client], {'action': 'update_graph', 'count': len(tasks)}) for k in list(tasks): if tasks[k] is k: del tasks[k] if k in self.tasks: del tasks[k] n = 0 while len(tasks) != n: # walk thorough new tasks, cancel any bad deps n = len(tasks) for k, deps in list(dependencies.items()): if any(dep not in self.dependencies and dep not in tasks for dep in deps): # bad key logger.info('User asked for computation on lost data, %s', k) del tasks[k] del dependencies[k] if k in keys: keys.remove(k) self.report({'op': 'cancelled-key', 'key': k}) self.client_releases_keys(keys=[k], client=client) stack = list(keys) touched = set() while stack: k = stack.pop() if k in self.dependencies: continue touched.add(k) if k not in self.tasks and k in tasks: self.tasks[k] = tasks[k] self.dependencies[k] = set(dependencies.get(k, ())) self.released.add(k) self.task_state[k] = 'released' for dep in self.dependencies[k]: if dep not in self.dependents: self.dependents[dep] = set() self.dependents[dep].add(k) if k not in self.dependents: self.dependents[k] = set() stack.extend(self.dependencies[k]) recommendations = OrderedDict() new_priority = priority or order(tasks) # TODO: define order wrt old graph if submitting_task: # sub-tasks get better priority than parent tasks try: generation = self.priority[submitting_task][0] - 0.01 except KeyError: # super-task already cleaned up generation = self.generation else: self.generation += 1 # older graph generations take precedence generation = self.generation for key in set(new_priority) & touched: if key not in self.priority: self.priority[key] = (generation, new_priority[key]) # prefer old if restrictions: # *restrictions* is a dict keying task ids to lists of # restriction specifications (either worker names or addresses) worker_restrictions = defaultdict(set) host_restrictions = defaultdict(set) for k, v in restrictions.items(): if v is None: continue for w in v: try: w = self.coerce_address(w) except ValueError: # Not a valid address, but perhaps it's a hostname host_restrictions[k].add(w) else: worker_restrictions[k].add(w) self.worker_restrictions.update(worker_restrictions) self.host_restrictions.update(host_restrictions) if loose_restrictions: self.loose_restrictions |= set(loose_restrictions) if resources: self.resource_restrictions.update(resources) if retries: self.retries.update(retries) for key in sorted(touched | keys, key=self.priority.get): if self.task_state[key] == 'released': recommendations[key] = 'waiting' for key in touched | keys: for dep in self.dependencies[key]: if dep in self.exceptions_blame: self.exceptions_blame[key] = self.exceptions_blame[dep] recommendations[key] = 'erred' break self.transitions(recommendations) for plugin in self.plugins[:]: try: plugin.update_graph(self, client=client, tasks=tasks, keys=keys, restrictions=restrictions or {}, dependencies=dependencies, loose_restrictions=loose_restrictions) except Exception as e: logger.exception(e) for key in keys: if self.task_state[key] in ('memory', 'erred'): self.report_on_key(key, client=client) end = time() if self.digests is not None: self.digests['update-graph-duration'].add(end - start) # TODO: balance workers def stimulus_task_finished(self, key=None, worker=None, **kwargs): """ Mark that a task has finished execution on a particular worker """ logger.debug("Stimulus task finished %s, %s", key, worker) if key not in self.task_state: return {} if self.task_state[key] == 'processing': recommendations = self.transition(key, 'memory', worker=worker, **kwargs) if self.task_state.get(key) == 'memory': if key not in self.has_what[worker]: self.worker_bytes[worker] += self.nbytes.get(key, DEFAULT_DATA_SIZE) self.who_has[key].add(worker) self.has_what[worker].add(key) else: logger.debug("Received already computed task, worker: %s, state: %s" ", key: %s, who_has: %s", worker, self.task_state.get(key), key, self.who_has.get(key)) if worker not in self.who_has.get(key, ()): self.worker_send(worker, {'op': 'release-task', 'key': key}) recommendations = {} return recommendations def stimulus_task_erred(self, key=None, worker=None, exception=None, traceback=None, **kwargs): """ Mark that a task has erred on a particular worker """ logger.debug("Stimulus task erred %s, %s", key, worker) if key not in self.task_state: return {} if self.task_state[key] == 'processing': retries = self.retries.get(key, 0) if retries > 0: self.retries[key] = retries - 1 recommendations = self.transition(key, 'waiting') else: recommendations = self.transition(key, 'erred', cause=key, exception=exception, traceback=traceback, worker=worker, **kwargs) else: recommendations = {} return recommendations def stimulus_missing_data(self, cause=None, key=None, worker=None, ensure=True, **kwargs): """ Mark that certain keys have gone missing. Recover. """ with log_errors(): logger.debug("Stimulus missing data %s, %s", key, worker) if key and self.task_state.get(key) in (None, 'memory'): return {} recommendations = OrderedDict() if self.task_state.get(cause) == 'memory': # couldn't find this for w in set(self.who_has[cause]): # TODO: this behavior is extreme self.has_what[w].remove(cause) self.who_has[cause].remove(w) self.worker_bytes[w] -= self.nbytes.get(cause, DEFAULT_DATA_SIZE) recommendations[cause] = 'released' if key: recommendations[key] = 'released' self.transitions(recommendations) if self.validate: assert cause not in self.who_has return {} def remove_worker(self, comm=None, address=None, safe=False, close=True): """ Remove worker from cluster We do this when a worker reports that it plans to leave or when it appears to be unresponsive. This may send its tasks back to a released state. """ with log_errors(): if self.status == 'closed': return if address not in self.processing: return 'already-removed' address = self.coerce_address(address) host = get_address_host(address) self.log_event(['all', address], {'action': 'remove-worker', 'worker': address, 'processing-tasks': self.processing[address]}) logger.info("Remove worker %s", address) if close: with ignoring(AttributeError, CommClosedError): self.worker_comms[address].send({'op': 'close'}) self.host_info[host]['cores'] -= self.ncores[address] self.host_info[host]['addresses'].remove(address) self.total_ncores -= self.ncores[address] if not self.host_info[host]['addresses']: del self.host_info[host] del self.worker_comms[address] del self.ncores[address] self.workers.remove(address) del self.aliases[self.worker_info[address]['name']] del self.worker_info[address] if address in self.idle: self.idle.remove(address) if address in self.saturated: self.saturated.remove(address) recommendations = OrderedDict() in_flight = set(self.processing.pop(address)) for k in list(in_flight): # del self.rprocessing[k] if not safe: self.suspicious_tasks[k] += 1 if not safe and self.suspicious_tasks[k] > self.allowed_failures: e = pickle.dumps(KilledWorker(k, address)) r = self.transition(k, 'erred', exception=e, cause=k) recommendations.update(r) in_flight.remove(k) else: recommendations[k] = 'released' self.total_occupancy -= self.occupancy.pop(address) del self.worker_bytes[address] self.remove_resources(address) for key in self.has_what.pop(address): self.who_has[key].remove(address) if not self.who_has[key]: if key in self.tasks: recommendations[key] = 'released' else: recommendations[key] = 'forgotten' self.transitions(recommendations) if self.validate: assert all(self.who_has.values()), len(self.who_has) for plugin in self.plugins[:]: try: plugin.remove_worker(scheduler=self, worker=address) except Exception as e: logger.exception(e) if not self.processing: logger.info("Lost all workers") logger.debug("Removed worker %s", address) return 'OK' def stimulus_cancel(self, comm, keys=None, client=None, force=False): """ Stop execution on a list of keys """ logger.info("Client %s requests to cancel %d keys", client, len(keys)) if client: self.log_event(client, {'action': 'cancel', 'count': len(keys), 'force': force}) for key in keys: self.cancel_key(key, client, force=force) 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 if key not in self.who_wants: # no key yet, lets try again in 500ms if retries: self.loop.add_future(gen.sleep(0.2), lambda _: self.cancel_key(key, client, retries - 1)) return if force or self.who_wants[key] == {client}: # no one else wants this key for dep in list(self.dependents[key]): self.cancel_key(dep, client, force=force) logger.info("Scheduler cancels key %s. Force=%s", key, force) self.report({'op': 'cancelled-key', 'key': key}) clients = list(self.who_wants[key]) if force else [client] for c in clients: self.client_releases_keys(keys=[key], client=c) def client_desires_keys(self, keys=None, client=None): for k in keys: self.who_wants[k].add(client) self.wants_what[client].add(k) if self.task_state.get(k) in ('memory', 'erred'): self.report_on_key(k, client=client) def client_releases_keys(self, keys=None, client=None): """ Remove keys from client desired list """ keys2 = set() for key in list(keys): if key in self.wants_what[client]: self.wants_what[client].remove(key) s = self.who_wants[key] s.remove(client) if not s: del self.who_wants[key] keys2.add(key) for key in keys2: if key in self.waiting_data and not self.waiting_data[key]: r = self.transition(key, 'released') self.transitions(r) if key in self.dependents and not self.dependents[key]: r = self.transition(key, 'forgotten') self.transitions(r) def client_wants_keys(self, keys=None, client=None): for k in keys: self.who_wants[k].add(client) self.wants_what[client].add(k) ###################################### # Task Validation (currently unused) # ###################################### def validate_released(self, key): assert key in self.dependencies assert self.task_state[key] == 'released' assert key not in self.waiting_data assert key not in self.who_has assert key not in self.rprocessing # assert key not in self.ready assert key not in self.waiting assert not any(key in self.waiting_data.get(dep, ()) for dep in self.dependencies[key]) assert key in self.released def validate_waiting(self, key): assert key in self.waiting assert key in self.waiting_data assert key not in self.who_has assert key not in self.rprocessing assert key not in self.released for dep in self.dependencies[key]: assert (dep in self.who_has) + (dep in self.waiting[key]) == 1 assert key in self.waiting_data[dep] def validate_processing(self, key): assert key not in self.waiting assert key in self.waiting_data assert key in self.rprocessing w = self.rprocessing[key] assert key in self.processing[w] assert key not in self.who_has for dep in self.dependencies[key]: assert dep in self.who_has assert key in self.waiting_data[dep] def validate_memory(self, key): assert key in self.who_has assert key not in self.rprocessing assert key not in self.waiting assert key not in self.released for dep in self.dependents[key]: assert (dep in self.who_has) + (dep in self.waiting_data[key]) == 1 def validate_queue(self, key): # assert key in self.ready assert key not in self.released assert key not in self.rprocessing assert key not in self.who_has assert key not in self.waiting for dep in self.dependencies[key]: assert dep in self.who_has def validate_no_worker(self, key): assert key in self.unrunnable assert key not in self.waiting assert key not in self.released assert key not in self.rprocessing assert key not in self.who_has for dep in self.dependencies[key]: assert dep in self.who_has def validate_erred(self, key): assert key in self.exceptions_blame assert key not in self.who_has def validate_key(self, key): try: try: func = getattr(self, 'validate_' + self.task_state[key].replace('-', '_')) except KeyError: logger.debug("Key lost: %s", key) except AttributeError: logger.error("self.validate_%s not found", self.task_state[key].replace('-', '_')) else: func(key) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def validate_state(self, allow_overlap=False): validate_state(self.dependencies, self.dependents, self.waiting, self.waiting_data, self.ready, self.who_has, self.processing, None, self.released, self.who_wants, self.wants_what, tasks=self.tasks, erred=self.exceptions_blame, allow_overlap=allow_overlap) if not (set(self.ncores) == set(self.workers) == set(self.has_what) == set(self.processing) == set(self.worker_info) == set(self.worker_comms) == set(self.occupancy)): raise ValueError("Workers not the same in all collections") a = self.worker_bytes b = {w: sum(self.nbytes[k] for k in keys) for w, keys in self.has_what.items()} assert a == b, (a, b) for key, workers in self.who_has.items(): for worker in workers: assert key in self.has_what[worker] for worker, keys in self.has_what.items(): for key in keys: assert worker in self.who_has[key] assert all(self.who_has.values()) for worker, occ in self.occupancy.items(): # for d in self.extensions['stealing'].in_flight.values(): # if worker in (d['thief'], d['victim']): # continue assert abs(sum(self.processing[worker].values()) - occ) < 1e-8 assert abs(sum(self.occupancy.values()) - self.total_occupancy) < 1e-8 ################### # Manage Messages # ################### def report(self, msg, client=None): """ Publish updates to all listening Queues and Comms If the message contains a key then we only send the message to those comms that care about the key. """ if client is not None: try: comm = self.comms[client] comm.send(msg) except CommClosedError: if self.status == 'running': logger.critical("Tried writing to closed comm: %s", msg) except KeyError: pass if 'key' in msg: if msg['key'] not in self.who_wants: return comms = [self.comms[c] for c in self.who_wants.get(msg['key'], ()) if c in self.comms] else: comms = self.comms.values() for c in comms: try: c.send(msg) # logger.debug("Scheduler sends message to client %s", msg) except CommClosedError: if self.status == 'running': logger.critical("Tried writing to closed comm: %s", msg) @gen.coroutine def add_client(self, comm, client=None): """ Add client to network We listen to all future messages from this Comm. """ logger.info("Receive client connection: %s", client) self.log_event(['all', client], {'action': 'add-client', 'client': client}) try: yield self.handle_client(comm, client=client) finally: if not comm.closed(): self.comms[client].send({'op': 'stream-closed'}) try: yield self.comms[client].close() del self.comms[client] logger.info("Close client connection: %s", client) except TypeError: # comm becomes None during GC pass def remove_client(self, client=None): """ Remove client from network """ logger.info("Remove client %s", client) self.log_event(['all', client], {'action': 'remove-client', 'client': client}) self.client_releases_keys(self.wants_what.get(client, ()), client) with ignoring(KeyError): del self.wants_what[client] @gen.coroutine def handle_client(self, comm, client=None): """ Listen and respond to messages from clients This runs once per Client Comm or Queue. See Also -------- Scheduler.worker_stream: The equivalent function for workers """ bcomm = BatchedSend(interval=2, loop=self.loop) bcomm.start(comm) self.comms[client] = bcomm try: bcomm.send({'op': 'stream-start'}) breakout = False while True: try: msgs = yield comm.read() except (CommClosedError, AssertionError, GeneratorExit): logger.info("Connection to client %s broken", str(client)) break except Exception as e: logger.exception(e) bcomm.send(error_message(e, status='scheduler-error')) continue if self.status == 'closed': return if not isinstance(msgs, list): msgs = [msgs] for msg in msgs: # logger.debug("scheduler receives message %s", msg) try: op = msg.pop('op') except Exception as e: logger.exception(e) bcomm.end(error_message(e, status='scheduler-error')) if op == 'close-stream': breakout = True break elif op == 'close': breakout = True self.close() break elif op in self.client_handlers: try: handler = self.client_handlers[op] if 'client' not in msg: msg['client'] = client result = handler(**msg) if isinstance(result, gen.Future): yield result except Exception as e: logger.exception(e) raise else: logger.warning("Bad message: op=%s, %s", op, msg, exc_info=True) if op == 'close': breakout = True break if breakout: break self.remove_client(client=client) logger.debug('Finished handle_client coroutine') except Exception: try: logger.error("Exception in handle_client", exc_info=True) except TypeError: pass def send_task_to_worker(self, worker, key): """ Send a single computational task to a worker """ try: msg = {'op': 'compute-task', 'key': key, 'priority': self.priority[key], 'duration': self.get_task_duration(key)} if key in self.resource_restrictions: msg['resource_restrictions'] = self.resource_restrictions[key] deps = self.dependencies[key] if deps: msg['who_has'] = {dep: list(self.who_has[dep]) for dep in deps} msg['nbytes'] = {dep: self.nbytes.get(dep) for dep in deps} if self.validate and deps: assert all(msg['who_has'].values()) task = self.tasks[key] if type(task) is dict: msg.update(task) else: msg['task'] = task self.worker_send(worker, msg) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def handle_uncaught_error(self, **msg): logger.exception(clean_exception(**msg)[1]) def handle_task_finished(self, key=None, **msg): validate_key(key) r = self.stimulus_task_finished(key=key, **msg) self.transitions(r) def handle_task_erred(self, key=None, **msg): r = self.stimulus_task_erred(key=key, **msg) self.transitions(r) def handle_release_data(self, key=None, worker=None, client=None, **msg): if self.rprocessing.get(key) != worker: return r = self.stimulus_missing_data(key=key, ensure=False, **msg) self.transitions(r) if self.validate: assert all(self.who_has.values()) def handle_missing_data(self, key=None, errant_worker=None, **kwargs): logger.debug("handle missing data key=%s worker=%s", key, errant_worker) self.log.append(('missing', key, errant_worker)) if key not in self.who_has: return if errant_worker in self.who_has[key]: self.who_has[key].remove(errant_worker) self.has_what[errant_worker].remove(key) self.worker_bytes[errant_worker] -= self.nbytes.get(key, DEFAULT_DATA_SIZE) if not self.who_has[key]: if key in self.tasks: self.transitions({key: 'released'}) else: self.transitions({key: 'forgotten'}) def release_worker_data(self, stream=None, keys=None, worker=None): hw = self.has_what[worker] recommendations = dict() for key in set(keys) & hw: hw.remove(key) wh = self.who_has[key] wh.remove(worker) if not wh: recommendations[key] = 'released' if recommendations: self.transitions(recommendations) def handle_long_running(self, key=None, worker=None, compute_duration=None): """ A task has seceded from the thread pool We stop the task from being stolen in the future, and change task duration accounting as if the task has stopped. """ if 'stealing' in self.extensions: self.extensions['stealing'].remove_key_from_stealable(key) try: actual_worker = self.rprocessing[key] except KeyError: logger.debug("Received long-running signal from duplicate task. " "Ignoring.") return if compute_duration: ks = key_split(key) old_duration = self.task_duration.get(ks, 0) new_duration = compute_duration if not old_duration: avg_duration = new_duration else: avg_duration = (0.5 * old_duration + 0.5 * new_duration) self.task_duration[ks] = avg_duration worker = self.rprocessing[key] self.occupancy[actual_worker] -= self.processing[actual_worker][key] self.total_occupancy -= self.processing[actual_worker][key] self.processing[actual_worker][key] = 0 @gen.coroutine def handle_worker(self, worker): """ Listen to responses from a single worker This is the main loop for scheduler-worker interaction See Also -------- Scheduler.handle_client: Equivalent coroutine for clients """ try: comm = yield connect(worker, connection_args=self.connection_args) except Exception as e: logger.error("Failed to connect to worker %r: %s", worker, e) self.remove_worker(address=worker) return yield comm.write({'op': 'compute-stream', 'reply': False}) worker_comm = self.worker_comms[worker] worker_comm.start(comm) logger.info("Starting worker compute stream, %s", worker) io_error = None try: while True: msgs = yield comm.read() start = time() if not isinstance(msgs, list): msgs = [msgs] if worker in self.worker_info and not comm.closed(): self.counters['worker-message-length'].add(len(msgs)) for msg in msgs: if msg == 'OK': # from close break if 'status' in msg and 'error' in msg['status']: try: logger.error("error from worker %s: %s", worker, clean_exception(**msg)[1]) except Exception: logger.error("error from worker %s", worker) op = msg.pop('op') if op: handler = self.worker_handlers[op] handler(worker=worker, **msg) end = time() if self.digests is not None: self.digests['handle-worker-duration'].add(end - start) except (CommClosedError, EnvironmentError) as e: io_error = e except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise finally: if worker in self.worker_comms: # Worker didn't send us a close message if io_error: logger.info("Worker %r failed from closed comm: %s", worker, io_error) worker_comm.abort() self.remove_worker(address=worker) else: assert comm.closed() worker_comm.abort() def add_plugin(self, plugin): """ Add external plugin to scheduler See https://distributed.readthedocs.io/en/latest/plugins.html """ self.plugins.append(plugin) def remove_plugin(self, plugin): """ Remove external plugin from scheduler """ self.plugins.remove(plugin) def worker_send(self, worker, msg): """ Send message to worker This also handles connection failures by adding a callback to remove the worker on the next cycle. """ try: self.worker_comms[worker].send(msg) except (CommClosedError, AttributeError): self.loop.add_callback(self.remove_worker, address=worker) ############################ # Less common interactions # ############################ @gen.coroutine def scatter(self, comm=None, data=None, workers=None, client=None, broadcast=False, timeout=2): """ Send data out to workers See also -------- Scheduler.broadcast: """ start = time() while not self.workers: yield gen.sleep(0.2) if time() > start + timeout: raise gen.TimeoutError("No workers found") if workers is None: ncores = self.ncores else: workers = [self.coerce_address(w) for w in workers] ncores = {w: self.ncores[w] for w in workers} assert isinstance(data, dict) keys, who_has, nbytes = yield scatter_to_workers(ncores, data, rpc=self.rpc, report=False) self.update_data(who_has=who_has, nbytes=nbytes, client=client) if broadcast: if broadcast == True: # flake8: noqa n = len(ncores) else: n = broadcast yield self.replicate(keys=keys, workers=workers, n=n) self.log_event([client, 'all'], {'action': 'scatter', 'client': client, 'count': len(data)}) raise gen.Return(keys) @gen.coroutine def gather(self, comm=None, keys=None): """ Collect data in from workers """ keys = list(keys) who_has = {key: self.who_has.get(key, ()) for key in keys} data, missing_keys, missing_workers = yield gather_from_workers( who_has, rpc=self.rpc, close=False) if not missing_keys: result = {'status': 'OK', 'data': data} else: logger.debug("Couldn't gather keys %s state: %s workers: %s", missing_keys, [self.task_state.get(key) for key in missing_keys], missing_workers) result = {'status': 'error', 'keys': missing_keys} with log_errors(): for worker in missing_workers: self.remove_worker(address=worker) # this is extreme for key, workers in missing_keys.items(): if not workers: continue logger.exception("Workers don't have promised key. " "This should never occur: %s, %s", str(workers), str(key)) for worker in workers: if worker in self.workers and key in self.has_what[worker]: self.has_what[worker].remove(key) self.who_has[key].remove(worker) self.worker_bytes[worker] -= self.nbytes.get(key, DEFAULT_DATA_SIZE) self.transitions({key: 'released'}) self.log_event('all', {'action': 'gather', 'count': len(keys)}) raise gen.Return(result) def clear_task_state(self): logger.info("Clear task state") for collection in self._task_collections: collection.clear() for collection in self._worker_collections: collection.clear() @gen.coroutine def restart(self, client=None, timeout=3): """ Restart all workers. Reset local state. """ with log_errors(): n_workers = len(self.workers) logger.info("Send lost future signal to clients") for client, keys in self.wants_what.items(): self.client_releases_keys(keys=keys, client=client) nannies = {addr: self.get_worker_service_addr(addr, 'nanny') for addr in self.workers} for addr in list(self.workers): try: # Ask the worker to close if it doesn't have a nanny, # otherwise the nanny will kill it anyway self.remove_worker(address=addr, close=addr not in nannies) except Exception as e: logger.info("Exception while restarting. This is normal", exc_info=True) self.clear_task_state() for plugin in self.plugins[:]: try: plugin.restart(self) except Exception as e: logger.exception(e) logger.debug("Send kill signal to nannies: %s", nannies) nannies = [rpc(nanny_address, connection_args=self.connection_args) for nanny_address in nannies.values() if nanny_address is not None] try: resps = All([nanny.restart(close=True, timeout=timeout * 0.8) for nanny in nannies]) resps = yield gen.with_timeout(timedelta(seconds=timeout), resps) assert all(resp == 'OK' for resp in resps) except gen.TimeoutError: logger.error("Nannies didn't report back restarted within " "timeout. Continuuing with restart process") finally: for nanny in nannies: nanny.close_rpc() self.start() self.log_event([client, 'all'], {'action': 'restart', 'client': client}) start = time() while time() < start + 10 and len(self.workers) < n_workers: yield gen.sleep(0.01) self.report({'op': 'restart'}) @gen.coroutine def broadcast(self, comm=None, msg=None, workers=None, hosts=None, nanny=False): """ Broadcast message to workers, return all results """ if workers is None: if hosts is None: workers = list(self.workers) else: workers = [] if hosts is not None: for host in hosts: if host in self.host_info: workers.extend(self.host_info[host]['addresses']) # TODO replace with worker_list if nanny: addresses = [self.get_worker_service_addr(w, 'nanny') for w in workers] else: addresses = workers @gen.coroutine def send_message(addr): comm = yield connect(addr, deserialize=self.deserialize, connection_args=self.connection_args) resp = yield send_recv(comm, close=True, **msg) raise gen.Return(resp) results = yield All([send_message(self.coerce_address(address)) for address in addresses if address is not None]) raise Return(dict(zip(workers, results))) @gen.coroutine def rebalance(self, comm=None, keys=None, workers=None): """ Rebalance keys so that each worker stores roughly equal bytes **Policy** This orders the workers by what fraction of bytes of the existing keys they have. It walks down this list from most-to-least. At each worker it sends the largest results it can find and sends them to the least occupied worker until either the sender or the recipient are at the average expected load. """ with log_errors(): keys = set(keys or self.who_has) workers = set(workers or self.workers) if not keys.issubset(self.who_has): raise Return({'status': 'missing-data', 'keys': list(keys - set(self.who_has))}) workers_by_key = {k: self.who_has.get(k, set()) & workers for k in keys} keys_by_worker = {w: set() for w in workers} for k, v in workers_by_key.items(): for vv in v: keys_by_worker[vv].add(k) worker_bytes = {w: sum(self.nbytes.get(k, DEFAULT_DATA_SIZE) for k in v) for w, v in keys_by_worker.items()} avg = sum(worker_bytes.values()) / len(worker_bytes) sorted_workers = list(map(first, sorted(worker_bytes.items(), key=second, reverse=True))) recipients = iter(reversed(sorted_workers)) recipient = next(recipients) msgs = [] # (sender, recipient, key) for sender in sorted_workers[:len(workers) // 2]: sender_keys = {k: self.nbytes.get(k, DEFAULT_DATA_SIZE) for k in keys_by_worker[sender]} sender_keys = iter(sorted(sender_keys.items(), key=second, reverse=True)) try: while worker_bytes[sender] > avg: while (worker_bytes[recipient] < avg and worker_bytes[sender] > avg): k, nb = next(sender_keys) if k not in keys_by_worker[recipient]: keys_by_worker[recipient].add(k) # keys_by_worker[sender].remove(k) msgs.append((sender, recipient, k)) worker_bytes[sender] -= nb worker_bytes[recipient] += nb if worker_bytes[sender] > avg: recipient = next(recipients) except StopIteration: break to_recipients = defaultdict(lambda: defaultdict(list)) to_senders = defaultdict(list) for sender, recipient, key in msgs: to_recipients[recipient][key].append(sender) to_senders[sender].append(key) result = yield {r: self.rpc(addr=r).gather(who_has=v) for r, v in to_recipients.items()} for r, v in to_recipients.items(): self.log_event(r, {'action': 'rebalance', 'who_has': v}) self.log_event('all', {'action': 'rebalance', 'total-keys': len(keys), 'senders': valmap(len, to_senders), 'recipients': valmap(len, to_recipients), 'moved_keys': len(msgs)}) if not all(r['status'] == 'OK' for r in result.values()): raise Return({'status': 'missing-data', 'keys': sum([r['keys'] for r in result if 'keys' in r], [])}) for sender, recipient, key in msgs: self.who_has[key].add(recipient) self.has_what[recipient].add(key) self.worker_bytes[recipient] += self.nbytes.get(key, DEFAULT_DATA_SIZE) self.log.append(('rebalance', key, time(), sender, recipient)) result = yield {r: self.rpc(addr=r).delete_data(keys=v, report=False) for r, v in to_senders.items()} for sender, recipient, key in msgs: self.who_has[key].remove(sender) self.has_what[sender].remove(key) self.worker_bytes[sender] -= self.nbytes.get(key, DEFAULT_DATA_SIZE) raise Return({'status': 'OK'}) @gen.coroutine def replicate(self, comm=None, keys=None, n=None, workers=None, branching_factor=2, delete=True): """ Replicate data throughout cluster This performs a tree copy of the data throughout the network individually on each piece of data. Parameters ---------- keys: Iterable list of keys to replicate n: int Number of replications we expect to see within the cluster branching_factor: int, optional The number of workers that can copy data in each generation See also -------- Scheduler.rebalance """ workers = set(self.workers_list(workers)) if n is None: n = len(workers) n = min(n, len(workers)) keys = set(keys) if n == 0: raise ValueError("Can not use replicate to delete data") if not keys.issubset(self.who_has): raise Return({'status': 'missing-data', 'keys': list(keys - set(self.who_has))}) # Delete extraneous data if delete: del_keys = {k: random.sample(self.who_has[k] & workers, len(self.who_has[k] & workers) - n) for k in keys if len(self.who_has[k] & workers) > n} del_workers = {k: v for k, v in reverse_dict(del_keys).items() if v} yield [self.rpc(addr=worker).delete_data(keys=list(keys), report=False) for worker, keys in del_workers.items()] for worker, keys in del_workers.items(): self.has_what[worker] -= keys for key in keys: self.who_has[key].remove(worker) self.worker_bytes[worker] -= self.nbytes.get(key, DEFAULT_DATA_SIZE) self.log_event(worker, {'action': 'replicate-remove', 'keys': keys}) keys = {k for k in keys if len(self.who_has[k] & workers) < n} # Copy not-yet-filled data while keys: gathers = defaultdict(dict) for k in list(keys): missing = workers - self.who_has[k] count = min(max(n - len(self.who_has[k] & workers), 0), branching_factor * len(self.who_has[k])) if not count: keys.remove(k) else: sample = random.sample(missing, count) for w in sample: gathers[w][k] = list(self.who_has[k]) results = yield {w: self.rpc(addr=w).gather(who_has=who_has) for w, who_has in gathers.items()} for w, v in results.items(): if v['status'] == 'OK': self.add_keys(worker=w, keys=list(gathers[w])) else: logger.warning("Communication failed during replication: %s", v) self.log_event(w, {'action': 'replicate-add', 'keys': gathers[w]}) self.log_event('all', {'action': 'replicate', 'workers': list(workers), 'key-count': len(keys), 'branching-factor': branching_factor}) def workers_to_close(self, memory_ratio=2): """ Find workers that we can close with low cost This returns a list of workers that are good candidates to retire. These workers are not running anything and are storing relatively little data relative to their peers. If all workers are idle then we still maintain enough workers to have enough RAM to store our data, with a comfortable buffer. This is for use with systems like ``distributed.deploy.adaptive``. Parameters ---------- memory_factor: Number Amount of extra space we want to have for our stored data. Defaults two 2, or that we want to have twice as much memory as we currently have data. Returns ------- to_close: list of workers that are OK to close """ with log_errors(): if all(self.processing.values()): return [] limit_bytes = {w: self.worker_info[w]['memory_limit'] for w in self.worker_info} worker_bytes = self.worker_bytes limit = sum(limit_bytes.values()) total = sum(worker_bytes.values()) idle = sorted([worker for worker in self.idle if not self.processing[worker]], key=worker_bytes.get, reverse=True) to_close = [] while idle: w = idle.pop() limit -= limit_bytes[w] if limit >= memory_ratio * total: # still plenty of space to_close.append(w) else: break return to_close @gen.coroutine def retire_workers(self, comm=None, workers=None, remove=True, close=False, close_workers=False): if close: logger.warning("The keyword close= has been deprecated. " "Use close_workers= instead") close_workers = close_workers or close with log_errors(): if workers is None: while True: try: workers = self.workers_to_close() if workers: yield self.retire_workers(workers=workers, remove=remove, close_workers=close_workers) raise gen.Return(list(workers)) except KeyError: # keys left during replicate pass workers = set(workers) if len(workers) > 0: keys = set.union(*[self.has_what[w] for w in workers]) keys = {k for k in keys if self.who_has[k].issubset(workers)} else: keys = set() other_workers = set(self.worker_info) - workers if keys: if other_workers: yield self.replicate(keys=keys, workers=other_workers, n=1, delete=False) else: raise gen.Return([]) if close_workers and workers: yield [self.close_worker(worker=w, safe=True) for w in workers] if remove: for w in workers: self.remove_worker(address=w, safe=True) self.log_event('all', {'action': 'retire-workers', 'workers': workers, 'moved-keys': len(keys)}) self.log_event(list(workers), {'action': 'retired'}) raise gen.Return(list(workers)) def add_keys(self, comm=None, worker=None, keys=()): """ Learn that a worker has certain keys This should not be used in practice and is mostly here for legacy reasons. """ if worker not in self.worker_info: return 'not found' for key in keys: if key in self.who_has: if key not in self.has_what[worker]: self.worker_bytes[worker] += self.nbytes.get(key, DEFAULT_DATA_SIZE) self.has_what[worker].add(key) self.who_has[key].add(worker) else: self.worker_send(worker, {'op': 'delete-data', 'keys': [key], 'report': False}) return 'OK' def update_data(self, comm=None, who_has=None, nbytes=None, client=None): """ Learn that new data has entered the network from an external source See Also -------- Scheduler.mark_key_in_memory """ with log_errors(): who_has = {k: [self.coerce_address(vv) for vv in v] for k, v in who_has.items()} logger.debug("Update data %s", who_has) if client: self.client_wants_keys(keys=list(who_has), client=client) # for key, workers in who_has.items(): # TODO # self.mark_key_in_memory(key, workers) self.nbytes.update(nbytes) for key, workers in who_has.items(): if key not in self.dependents: self.dependents[key] = set() if key not in self.dependencies: self.dependencies[key] = set() self.task_state[key] = 'memory' if key not in self.who_has: self.who_has[key] = set() for w in workers: if key not in self.has_what[w]: self.worker_bytes[w] += self.nbytes.get(key, DEFAULT_DATA_SIZE) self.has_what[w].add(key) self.who_has[key].add(w) if key not in self.waiting_data: self.waiting_data[key] = set() self.report({'op': 'key-in-memory', 'key': key, 'workers': list(workers)}) def report_on_key(self, key, client=None): if key not in self.task_state: self.report({'op': 'cancelled-key', 'key': key}, client=client) elif self.task_state[key] == 'memory': self.report({'op': 'key-in-memory', 'key': key}, client=client) elif self.task_state[key] == 'erred': failing_key = self.exceptions_blame[key] self.report({'op': 'task-erred', 'key': key, 'exception': self.exceptions[failing_key], 'traceback': self.tracebacks.get(failing_key, None)}, client=client) @gen.coroutine def feed(self, comm, function=None, setup=None, teardown=None, interval=1, **kwargs): """ Provides a data Comm to external requester Caution: this runs arbitrary Python code on the scheduler. This should eventually be phased out. It is mostly used by diagnostics. """ import pickle with log_errors(): if function: function = pickle.loads(function) if setup: setup = pickle.loads(setup) if teardown: teardown = pickle.loads(teardown) state = setup(self) if setup else None if isinstance(state, gen.Future): state = yield state try: while self.status == 'running': if state is None: response = function(self) else: response = function(self, state) yield comm.write(response) yield gen.sleep(interval) except (EnvironmentError, CommClosedError): pass finally: if teardown: teardown(self, state) def get_processing(self, comm=None, workers=None): if workers is not None: workers = set(map(self.coerce_address, workers)) return {w: list(self.processing[w]) for w in workers} else: return valmap(list, self.processing) def get_who_has(self, comm=None, keys=None): if keys is not None: return {k: list(self.who_has.get(k, [])) for k in keys} else: return valmap(list, self.who_has) def get_has_what(self, comm=None, workers=None): if workers is not None: workers = map(self.coerce_address, workers) return {w: list(self.has_what.get(w, ())) for w in workers} else: return valmap(list, self.has_what) def get_ncores(self, comm=None, workers=None): if workers is not None: workers = map(self.coerce_address, workers) return {w: self.ncores.get(w, None) for w in workers} else: return self.ncores @gen.coroutine def get_call_stack(self, comm=None, keys=None): if keys is not None: stack = list(keys) processing = set() while stack: key = stack.pop() state = self.task_state[key] if state == 'waiting': stack.extend(self.dependencies[key]) elif state == 'processing': processing.add(key) workers = defaultdict(list) for key in processing: if key in self.rprocessing: workers[self.rprocessing[key]].append(key) else: workers = {w: None for w in self.workers} if not workers: raise gen.Return({}) else: response = yield {w: self.rpc(w).call_stack(keys=v) for w, v in workers.items()} response = {k: v for k, v in response.items() if v} raise gen.Return(response) def get_nbytes(self, comm=None, keys=None, summary=True): with log_errors(): if keys is not None: result = {k: self.nbytes[k] for k in keys} else: result = self.nbytes if summary: out = defaultdict(lambda: 0) for k, v in result.items(): out[key_split(k)] += v result = out return result def get_comm_cost(self, key, worker): """ Get the estimated communication cost (in s.) to compute key on the given worker. """ return (sum(self.nbytes[d] for d in self.dependencies[key] - self.has_what[worker]) / BANDWIDTH) def get_task_duration(self, key, default=0.5): """ Get the estimated computation cost of the given key (not including any communication cost). """ ks = key_split(key) try: return self.task_duration[ks] except KeyError: self.unknown_durations[ks].add(key) return default def run_function(self, stream, function, args=(), kwargs={}): """ Run a function within this process See Also -------- Client.run_on_scheduler: """ from .worker import run self.log_event('all', {'action': 'run-function', 'function': function}) return run(self, stream, function=function, args=args, kwargs=kwargs) def set_metadata(self, stream=None, keys=None, value=None): try: metadata = self.task_metadata for key in keys[:-1]: if key not in metadata or not isinstance(metadata[key], (dict, list)): metadata[key] = dict() metadata = metadata[key] metadata[keys[-1]] = value except Exception as e: import pdb; pdb.set_trace() def get_metadata(self, stream=None, keys=None, default=no_default): metadata = self.task_metadata for key in keys[:-1]: metadata = metadata[key] try: return metadata[keys[-1]] except KeyError: if default != no_default: return default else: raise def get_task_status(self, stream=None, keys=None): return {key: self.task_state.get(key) for key in keys} ##################### # State Transitions # ##################### def transition_released_waiting(self, key): try: if self.validate: assert key in self.tasks assert key in self.dependencies assert key in self.dependents assert key not in self.waiting assert key not in self.who_has assert key not in self.rprocessing # assert all(dep in self.task_state # for dep in self.dependencies[key]) if not all(dep in self.task_state for dep in self.dependencies[key]): return {key: 'forgotten'} recommendations = OrderedDict() for dep in self.dependencies[key]: if dep in self.exceptions_blame: self.exceptions_blame[key] = self.exceptions_blame[dep] recommendations[key] = 'erred' return recommendations self.waiting[key] = set() for dep in self.dependencies[key]: if dep not in self.who_has: self.waiting[key].add(dep) if dep in self.released: recommendations[dep] = 'waiting' else: self.waiting_data[dep].add(key) self.waiting_data[key] = {dep for dep in self.dependents[key] if dep not in self.who_has and dep not in self.released and dep not in self.exceptions_blame} if not self.waiting[key]: if self.workers: self.task_state[key] = 'waiting' recommendations[key] = 'processing' else: self.unrunnable.add(key) del self.waiting[key] self.task_state[key] = 'no-worker' else: self.task_state[key] = 'waiting' if self.validate: if self.task_state[key] == 'waiting': assert key in self.waiting self.released.remove(key) return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_no_worker_waiting(self, key): try: if self.validate: assert key in self.unrunnable assert key not in self.waiting assert key not in self.who_has assert key not in self.rprocessing self.unrunnable.remove(key) if not all(dep in self.task_state for dep in self.dependencies[key]): return {key: 'forgotten'} recommendations = OrderedDict() self.waiting[key] = set() for dep in self.dependencies[key]: if dep not in self.who_has: self.waiting[key].add(dep) if dep in self.released: recommendations[dep] = 'waiting' else: self.waiting_data[dep].add(key) self.task_state[key] = 'waiting' if not self.waiting[key]: if self.workers: recommendations[key] = 'processing' else: self.task_state[key] = 'no-worker' return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def decide_worker(self, key): valid_workers = self.valid_workers(key) if not valid_workers and key not in self.loose_restrictions and self.ncores: self.unrunnable.add(key) self.task_state[key] = 'no-worker' return {} if self.dependencies.get(key, None) or valid_workers is not True: worker = decide_worker(self.dependencies, self.occupancy, self.who_has, valid_workers, self.loose_restrictions, partial(self.worker_objective, key), key) elif self.idle: if len(self.idle) < 20: # smart but linear in small case worker = min(self.idle, key=self.occupancy.get) else: # dumb but fast in large case worker = self.idle[self.n_tasks % len(self.idle)] else: if len(self.workers) < 20: # smart but linear in small case worker = min(self.workers, key=self.occupancy.get) else: # dumb but fast in large case worker = self.workers[self.n_tasks % len(self.workers)] assert worker return worker def transition_waiting_processing(self, key): try: if self.validate: assert key in self.waiting assert not self.waiting[key] assert key not in self.who_has assert key not in self.exceptions_blame assert key not in self.rprocessing # assert key not in self.readyset assert key not in self.unrunnable assert all(dep in self.who_has for dep in self.dependencies[key]) if any(not self.who_has[dep] for dep in self.dependencies[key]): return {} del self.waiting[key] worker = self.decide_worker(key) if not worker: return worker ks = key_split(key) duration = self.get_task_duration(ks) comm = self.get_comm_cost(key, worker) self.processing[worker][key] = duration + comm self.rprocessing[key] = worker self.occupancy[worker] += duration + comm self.total_occupancy += duration + comm self.task_state[key] = 'processing' self.consume_resources(key, worker) self.check_idle_saturated(worker) self.n_tasks += 1 # logger.debug("Send job to worker: %s, %s", worker, key) self.send_task_to_worker(worker, key) if self.validate: assert key not in self.waiting return {} except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_waiting_memory(self, key, nbytes=None, worker=None, **kwargs): try: if self.validate: assert key not in self.rprocessing assert key in self.waiting assert self.task_state[key] == 'waiting' del self.waiting[key] if nbytes is not None: self.nbytes[key] = nbytes if key not in self.who_has: self.who_has[key] = set() self.who_has[key].add(worker) self.has_what[worker].add(key) self.worker_bytes[worker] += nbytes or DEFAULT_DATA_SIZE self.check_idle_saturated(worker) recommendations = OrderedDict() deps = self.dependents.get(key, []) if len(deps) > 1: deps = sorted(deps, key=self.priority.get, reverse=True) for dep in deps: if dep in self.waiting: s = self.waiting[dep] s.remove(key) if not s: # new task ready to run recommendations[dep] = 'processing' for dep in self.dependencies.get(key, []): if dep in self.waiting_data: s = self.waiting_data[dep] s.remove(key) if (not s and dep and dep not in self.who_wants and not self.waiting_data.get(dep)): recommendations[dep] = 'released' if (not self.waiting_data.get(key) and key not in self.who_wants): recommendations[key] = 'released' else: msg = {'op': 'key-in-memory', 'key': key} self.report(msg) self.task_state[key] = 'memory' if self.validate: assert key not in self.rprocessing assert key not in self.waiting assert self.who_has[key] return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_processing_memory(self, key, nbytes=None, type=None, worker=None, startstops=None, **kwargs): try: if self.validate: assert key in self.rprocessing w = self.rprocessing[key] assert key in self.processing[w] assert key not in self.waiting assert key not in self.who_has assert key not in self.exceptions_blame # assert all(dep in self.waiting_data[key ] for dep in # self.dependents[key] if self.task_state[dep] in # ['waiting', 'queue', 'stacks']) # assert key not in self.nbytes assert self.task_state[key] == 'processing' if worker not in self.processing: return {key: 'released'} if worker != self.rprocessing[key]: # someone else has this task logger.warning("Unexpected worker completed task, likely due to" " work stealing. Expected: %s, Got: %s, Key: %s", self.rprocessing[key], worker, key) return {} if startstops: L = [(b, c) for a, b, c in startstops if a == 'compute'] if L: compute_start, compute_stop = L[0] else: # This is very rare compute_start = compute_stop = None else: compute_start = compute_stop = None ############################# # Update Timing Information # ############################# if compute_start and self.processing[worker].get(key, True): # Update average task duration for worker info = self.worker_info[worker] ks = key_split(key) old_duration = self.task_duration.get(ks, 0) new_duration = compute_stop - compute_start if not old_duration: avg_duration = new_duration else: avg_duration = (0.5 * old_duration + 0.5 * new_duration) self.task_duration[ks] = avg_duration if ks in self.unknown_durations: for k in self.unknown_durations.pop(ks): if k in self.rprocessing: w = self.rprocessing[k] old = self.processing[w][k] comm = self.get_comm_cost(k, w) self.processing[w][k] = avg_duration + comm self.occupancy[w] += avg_duration + comm - old self.total_occupancy += avg_duration + comm - old info['last-task'] = compute_stop ############################ # Update State Information # ############################ if nbytes is not None: self.nbytes[key] = nbytes self.who_has[key] = set() self.release_resources(key, worker) assert worker self.who_has[key].add(worker) self.has_what[worker].add(key) self.worker_bytes[worker] += self.nbytes.get(key, DEFAULT_DATA_SIZE) w = self.rprocessing.pop(key) duration = self.processing[w].pop(key) if not self.processing[w]: self.total_occupancy -= self.occupancy[w] self.occupancy[w] = 0 else: self.total_occupancy -= duration self.occupancy[w] -= duration self.check_idle_saturated(w) recommendations = OrderedDict() deps = self.dependents.get(key, []) if len(deps) > 1: deps = sorted(deps, key=self.priority.get, reverse=True) for dep in deps: if dep in self.waiting: s = self.waiting[dep] s.remove(key) if not s: # new task ready to run recommendations[dep] = 'processing' for dep in self.dependencies.get(key, []): if dep in self.waiting_data: s = self.waiting_data[dep] s.remove(key) if (not s and dep and dep not in self.who_wants and not self.waiting_data.get(dep)): recommendations[dep] = 'released' if (not self.waiting_data.get(key) and key not in self.who_wants): recommendations[key] = 'released' else: msg = {'op': 'key-in-memory', 'key': key} if type is not None: msg['type'] = type self.report(msg) self.task_state[key] = 'memory' if key in self.wants_what['fire-and-forget']: self.client_releases_keys(client='fire-and-forget', keys=[key]) if self.validate: assert key not in self.rprocessing assert key not in self.waiting return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_memory_released(self, key, safe=False): try: if self.validate: assert key in self.who_has assert key not in self.released # assert key not in self.readyset assert key not in self.waiting assert key not in self.rprocessing if safe: assert not self.waiting_data.get(key) # assert key not in self.who_wants recommendations = OrderedDict() for dep in self.waiting_data.get(key, ()): # lost dependency dep_state = self.task_state[dep] if dep_state in ('no-worker', 'processing'): recommendations[dep] = 'waiting' if dep_state == 'waiting': self.waiting[dep].add(key) workers = self.who_has.pop(key) for w in workers: if w in self.worker_info: # in case worker has died self.has_what[w].remove(key) self.worker_bytes[w] -= self.nbytes.get(key, DEFAULT_DATA_SIZE) self.worker_send(w, {'op': 'delete-data', 'keys': [key], 'report': False}) self.released.add(key) self.task_state[key] = 'released' self.report({'op': 'lost-data', 'key': key}) if key not in self.tasks: # pure data recommendations[key] = 'forgotten' elif not all(dep in self.task_state for dep in self.dependencies[key]): recommendations[key] = 'forgotten' elif key in self.who_wants or self.waiting_data.get(key): recommendations[key] = 'waiting' if key in self.waiting_data: del self.waiting_data[key] if self.validate: assert key not in self.waiting return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_released_erred(self, key): try: if self.validate: with log_errors(pdb=LOG_PDB): assert key in self.exceptions_blame assert key not in self.who_has assert key not in self.waiting assert key not in self.waiting_data recommendations = {} failing_key = self.exceptions_blame[key] for dep in self.dependents[key]: self.exceptions_blame[dep] = failing_key if dep not in self.who_has: recommendations[dep] = 'erred' self.report({'op': 'task-erred', 'key': key, 'exception': self.exceptions[failing_key], 'traceback': self.tracebacks.get(failing_key, None)}) self.task_state[key] = 'erred' self.released.remove(key) # TODO: waiting data? return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_waiting_released(self, key): try: if self.validate: assert key in self.waiting assert key in self.waiting_data assert key not in self.who_has assert key not in self.rprocessing recommendations = {} del self.waiting[key] for dep in self.dependencies[key]: if dep in self.waiting_data: if key in self.waiting_data[dep]: self.waiting_data[dep].remove(key) if not self.waiting_data[dep] and dep not in self.who_wants: recommendations[dep] = 'released' assert self.task_state[dep] != 'erred' self.task_state[key] = 'released' self.released.add(key) if self.validate: assert not any(key in self.waiting_data.get(dep, ()) for dep in self.dependencies[key]) if any(dep not in self.task_state for dep in self.dependencies[key]): recommendations[key] = 'forgotten' elif (key not in self.exceptions_blame and (key in self.who_wants or self.waiting_data.get(key))): recommendations[key] = 'waiting' del self.waiting_data[key] return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_processing_released(self, key): try: if self.validate: assert key in self.rprocessing assert key not in self.who_has assert key not in self.waiting assert self.task_state[key] == 'processing' w = self.rprocessing.pop(key) if w in self.workers: duration = self.processing[w].pop(key) if not self.processing[w]: self.total_occupancy -= self.occupancy[w] self.occupancy[w] = 0 else: self.total_occupancy -= duration self.occupancy[w] -= duration self.check_idle_saturated(w) self.release_resources(key, w) self.worker_send(w, {'op': 'release-task', 'key': key}) self.released.add(key) self.task_state[key] = 'released' recommendations = OrderedDict() if any(dep not in self.task_state for dep in self.dependencies[key]): recommendations[key] = 'forgotten' elif self.waiting_data[key] or key in self.who_wants: recommendations[key] = 'waiting' else: for dep in self.dependencies[key]: if dep not in self.released: assert key in self.waiting_data[dep] self.waiting_data[dep].remove(key) if not self.waiting_data[dep] and dep not in self.who_wants: recommendations[dep] = 'released' del self.waiting_data[key] if self.validate: assert key not in self.rprocessing return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_processing_erred(self, key, cause=None, exception=None, traceback=None, **kwargs): try: if self.validate: assert cause or key in self.exceptions_blame assert key in self.rprocessing assert key not in self.who_has assert key not in self.waiting if exception: self.exceptions[key] = exception if traceback: self.tracebacks[key] = traceback if cause: self.exceptions_blame[key] = cause failing_key = self.exceptions_blame[key] recommendations = {} for dep in self.dependents[key]: self.exceptions_blame[dep] = key recommendations[dep] = 'erred' for dep in self.dependencies.get(key, []): if dep in self.waiting_data: s = self.waiting_data[dep] if key in s: s.remove(key) if (not s and dep and dep not in self.who_wants and not self.waiting_data.get(dep)): recommendations[dep] = 'released' w = self.rprocessing.pop(key) if w in self.processing: duration = self.processing[w].pop(key) if not self.processing[w]: self.total_occupancy -= self.occupancy[w] self.occupancy[w] = 0 else: self.total_occupancy -= duration self.occupancy[w] -= duration self.check_idle_saturated(w) self.release_resources(key, w) del self.waiting_data[key] # do anything with this? self.task_state[key] = 'erred' self.report({'op': 'task-erred', 'key': key, 'exception': self.exceptions[failing_key], 'traceback': self.tracebacks.get(failing_key)}) if key in self.wants_what['fire-and-forget']: self.client_releases_keys(client='fire-and-forget', keys=[key]) if self.validate: assert key not in self.rprocessing return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def remove_key(self, key): if key in self.tasks: del self.tasks[key] del self.task_state[key] if key in self.dependencies: del self.dependencies[key] del self.dependents[key] if key in self.worker_restrictions: del self.worker_restrictions[key] if key in self.host_restrictions: del self.host_restrictions[key] if key in self.loose_restrictions: self.loose_restrictions.remove(key) if key in self.priority: del self.priority[key] if key in self.exceptions: del self.exceptions[key] if key in self.exceptions_blame: del self.exceptions_blame[key] if key in self.tracebacks: del self.tracebacks[key] if key in self.released: self.released.remove(key) if key in self.waiting_data: del self.waiting_data[key] if key in self.suspicious_tasks: del self.suspicious_tasks[key] if key in self.retries: del self.retries[key] if key in self.nbytes: del self.nbytes[key] if key in self.resource_restrictions: del self.resource_restrictions[key] if key in self.task_metadata: del self.task_metadata[key] def transition_memory_forgotten(self, key): try: if self.validate: assert key in self.dependents assert self.task_state[key] == 'memory' assert key in self.waiting_data assert key in self.who_has assert key not in self.rprocessing # assert key not in self.ready assert key not in self.waiting recommendations = {} for dep in self.waiting_data[key]: recommendations[dep] = 'forgotten' for dep in self.dependents[key]: if self.task_state[dep] == 'released': recommendations[dep] = 'forgotten' for dep in self.dependencies.get(key, ()): try: s = self.dependents[dep] s.remove(key) if not s and dep not in self.who_wants: assert dep is not key recommendations[dep] = 'forgotten' except KeyError: pass workers = self.who_has.pop(key) for w in workers: if w in self.worker_info: # in case worker has died self.has_what[w].remove(key) self.worker_bytes[w] -= self.nbytes.get(key, DEFAULT_DATA_SIZE) self.worker_send(w, {'op': 'delete-data', 'keys': [key], 'report': False}) if self.validate: assert all(key not in self.dependents[dep] for dep in self.dependencies[key] if dep in self.task_state) assert all(key not in self.waiting_data.get(dep, ()) for dep in self.dependencies[key] if dep in self.task_state) self.remove_key(key) self.report_on_key(key) return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_no_worker_released(self, key): try: if self.validate: assert self.task_state[key] == 'no-worker' assert key not in self.who_has assert key not in self.waiting self.unrunnable.remove(key) self.released.add(key) self.task_state[key] = 'released' for dep in self.dependencies[key]: try: self.waiting_data[dep].remove(key) except KeyError: # dep may also be released pass del self.waiting_data[key] return {} except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_released_forgotten(self, key): try: if self.validate: assert key in self.dependencies assert self.task_state[key] in ('released', 'erred') # assert not self.waiting_data[key] if key in self.tasks and self.dependencies[key].issubset(self.task_state): assert key not in self.who_wants assert not self.dependents[key] assert not any(key in self.waiting_data.get(dep, ()) for dep in self.dependencies[key]) assert key not in self.who_has assert key not in self.rprocessing # assert key not in self.ready assert key not in self.waiting recommendations = {} for dep in self.dependencies[key]: try: s = self.dependents[dep] s.remove(key) if not s and dep not in self.who_wants: assert dep is not key recommendations[dep] = 'forgotten' except KeyError: pass for dep in self.dependents[key]: if self.task_state[dep] not in ('memory', 'error'): recommendations[dep] = 'forgotten' for dep in self.dependents[key]: if self.task_state[dep] == 'released': recommendations[dep] = 'forgotten' for dep in self.dependencies[key]: try: self.waiting_data[dep].remove(key) except KeyError: pass if self.validate: assert all(key not in self.dependents[dep] for dep in self.dependencies[key] if dep in self.task_state) assert all(key not in self.waiting_data.get(dep, ()) for dep in self.dependencies[key] if dep in self.task_state) self.remove_key(key) self.report_on_key(key) return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition(self, key, finish, *args, **kwargs): """ Transition a key from its current state to the finish state Examples -------- >>> self.transition('x', 'waiting') {'x': 'processing'} Returns ------- Dictionary of recommendations for future transitions See Also -------- Scheduler.transitions: transitive version of this function """ try: try: start = self.task_state[key] except KeyError: return {} if start == finish: return {} if (start, finish) in self._transitions: func = self._transitions[start, finish] recommendations = func(key, *args, **kwargs) else: func = self._transitions['released', finish] assert not args and not kwargs a = self.transition(key, 'released') if key in a: func = self._transitions['released', a[key]] b = func(key) a = a.copy() a.update(b) recommendations = a start = 'released' finish2 = self.task_state.get(key, 'forgotten') self.transition_log.append((key, start, finish2, recommendations, time())) if self.validate: logger.debug("Transition %s->%s: %s New: %s", start, finish2, key, recommendations) for plugin in self.plugins: try: plugin.transition(key, start, finish2, *args, **kwargs) except Exception: logger.info("Plugin failed with exception", exc_info=True) return recommendations except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transitions(self, recommendations): """ Process transitions until none are left This includes feedback from previous transitions and continues until we reach a steady state """ keys = set() recommendations = recommendations.copy() while recommendations: key, finish = recommendations.popitem() keys.add(key) new = self.transition(key, finish) recommendations.update(new) if self.validate: for key in keys: self.validate_key(key) def story(self, *keys): """ Get all transitions that touch one of the input keys """ keys = set(keys) return [t for t in self.transition_log if t[0] in keys or keys.intersection(t[3])] transition_story = story def reschedule(self, key=None, worker=None): """ Reschedule a task Things may have shifted and this task may now be better suited to run elsewhere """ if self.task_state[key] != 'processing': return if worker and self.rprocessing[key] != worker: return self.transitions({key: 'released'}) ############################## # Assigning Tasks to Workers # ############################## def check_idle_saturated(self, worker, occ=None): if occ is None: occ = self.occupancy[worker] nc = self.ncores[worker] p = len(self.processing[worker]) avg = self.total_occupancy / self.total_ncores if p < nc or occ / nc < avg / 2: self.idle.add(worker) if worker in self.saturated: self.saturated.remove(worker) else: if worker in self.idle: self.idle.remove(worker) pending = occ * (p - nc) / p / nc if p > nc and pending > 0.4 and pending > 1.9 * avg: self.saturated.add(worker) elif worker in self.saturated: self.saturated.remove(worker) def valid_workers(self, key): """ Return set of currently valid worker addresses for key If all workers are valid then this returns ``True``. This checks tracks the following state: * worker_restrictions * host_restrictions * resource_restrictions """ s = True if key in self.worker_restrictions: s = {w for w in self.worker_restrictions[key] if w in self.worker_info} if key in self.host_restrictions: # Resolve the alias here rather than early, for the worker # may not be connected when host_restrictions is populated hr = [self.coerce_hostname(h) for h in self.host_restrictions[key]] ss = [self.host_info[h]['addresses'] for h in hr if h in self.host_info] ss = set.union(*ss) if ss else set() if s is True: s = ss else: s |= ss if self.resource_restrictions.get(key): w = {resource: {w for w, supplied in self.resources[resource].items() if supplied >= required} for resource, required in self.resource_restrictions[key].items()} ww = set.intersection(*w.values()) if s is True: s = ww else: s &= ww return s def consume_resources(self, key, worker): if key in self.resource_restrictions: for r, required in self.resource_restrictions[key].items(): self.used_resources[worker][r] += required def release_resources(self, key, worker): if key in self.resource_restrictions: for r, required in self.resource_restrictions[key].items(): self.used_resources[worker][r] -= required ##################### # Utility functions # ##################### def add_resources(self, stream=None, worker=None, resources=None): if worker not in self.worker_resources: self.worker_resources[worker] = resources self.used_resources[worker] = resources for resource, quantity in resources.items(): self.resources[resource][worker] = quantity return 'OK' def remove_resources(self, worker): if worker in self.worker_resources: del self.used_resources[worker] for resource, quantity in self.worker_resources.pop(worker).items(): del self.resources[resource][worker] def coerce_address(self, addr, resolve=True): """ Coerce possible input addresses to canonical form. *resolve* can be disabled for testing with fake hostnames. Handles strings, tuples, or aliases. """ # XXX how many address-parsing routines do we have? if addr in self.aliases: addr = self.aliases[addr] if isinstance(addr, tuple): addr = unparse_host_port(*addr) if not isinstance(addr, six.string_types): raise TypeError("addresses should be strings or tuples, got %r" % (addr,)) if resolve: addr = resolve_address(addr) else: addr = normalize_address(addr) return addr def coerce_hostname(self, host): """ Coerce the hostname of a worker. """ if host in self.aliases: return self.worker_info[self.aliases[host]]['host'] else: return host def workers_list(self, workers): """ List of qualifying workers Takes a list of worker addresses or hostnames. Returns a list of all worker addresses that match """ if workers is None: return list(self.workers) out = set() for w in workers: if ':' in w: out.add(w) else: out.update({ww for ww in self.workers if w in ww}) # TODO: quadratic return list(out) def start_ipython(self, comm=None): """Start an IPython kernel Returns Jupyter connection info dictionary. """ from ._ipython_utils import start_ipython if self._ipython_kernel is None: self._ipython_kernel = start_ipython( ip=self.ip, ns={'scheduler': self}, log=logger, ) return self._ipython_kernel.get_connection_info() def worker_objective(self, key, worker): """ Objective function to determine which worker should get the key Minimize expected start time. If a tie then break with data storate. """ comm_bytes = sum([self.nbytes.get(k, DEFAULT_DATA_SIZE) for k in self.dependencies[key] if worker not in self.who_has[k]]) stack_time = self.occupancy[worker] / self.ncores[worker] start_time = comm_bytes / BANDWIDTH + stack_time return (start_time, self.worker_bytes[worker]) @gen.coroutine def get_profile(self, comm=None, workers=None, merge_workers=True, start=None, stop=None, key=None): if workers is None: workers = self.workers else: workers = self.workers & workers result = yield {w: self.rpc(w).profile(start=start, stop=stop, key=key) for w in workers} if merge_workers: result = profile.merge(*result.values()) raise gen.Return(result) @gen.coroutine def get_profile_metadata(self, comm=None, workers=None, merge_workers=True, start=None, stop=None, profile_cycle_interval=None): dt = profile_cycle_interval or config.get('profile-cycle-interval', 1000) / 1000 if workers is None: workers = self.workers else: workers = self.workers & workers result = yield {w: self.rpc(w).profile_metadata(start=start, stop=stop) for w in workers} counts = [v['counts'] for v in result.values()] counts = itertools.groupby(merge_sorted(*counts), lambda t: t[0] // dt * dt) counts = [(time, sum(pluck(1, group))) for time, group in counts] keys = set() for v in result.values(): for t, d in v['keys']: for k in d: keys.add(k) keys = {k: [] for k in keys} groups1 = [v['keys'] for v in result.values()] groups2 = list(merge_sorted(*groups1, key=first)) last = 0 for t, d in groups2: tt = t // dt * dt if tt > last: last = tt for k, v in keys.items(): v.append([tt, 0]) for k, v in d.items(): keys[k][-1][1] += v 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) raise gen.Return(results) ########### # Cleanup # ########### @gen.coroutine def reevaluate_occupancy(self): """ Periodically reassess task duration time The expected duration of a task can change over time. Unfortunately we don't have a good constant-time way to propagate the effects of these changes out to the summaries that they affect, like the total expected runtime of each of the workers, or what tasks are stealable. In this coroutine we walk through all of the workers and re-align their estimates with the current state of tasks. We do this periodically rather than at every transition, and we only do it if the scheduler process isn't under load (using psutil.Process.cpu_percent()). This lets us avoid this fringe optimization when we have better things to think about. """ DELAY = 0.1 try: import psutil proc = psutil.Process() last = time() while self.status != 'closed': yield gen.sleep(DELAY) while not self.rprocessing: yield gen.sleep(DELAY) last = time() for w, processing in list(self.processing.items()): while proc.cpu_percent() > 50: yield gen.sleep(DELAY) last = time() if w not in self.workers or not processing: continue self._reevaluate_occupancy_worker(w) duration = time() - last if duration > 0.005: # 5ms since last release yield gen.sleep(duration * 5) # 25ms gap last = time() except Exception: logger.error("Error in reevaluate occupancy", exc_info=True) raise def _reevaluate_occupancy_worker(self, worker): """ See reevaluate_occupancy """ w = worker processing = self.processing[w] if not processing or w not in self.workers or self.status == 'closed': return old = self.occupancy[w] new = 0 nbytes = 0 for key in processing: duration = self.get_task_duration(key) comm = self.get_comm_cost(key, worker) processing[key] = duration + comm new += duration + comm self.occupancy[w] = new self.total_occupancy += new - old self.check_idle_saturated(w) # significant increase in duration if (new > old * 1.3) and ('stealing' in self.extensions): steal = self.extensions['stealing'] for key in processing: steal.remove_key_from_stealable(key) steal.put_key_in_stealable(key) def decide_worker(dependencies, occupancy, who_has, valid_workers, loose_restrictions, objective, key): """ Decide which worker should take task >>> dependencies = {'c': {'b'}, 'b': {'a'}} >>> occupancy = {'alice:8000': 0, 'bob:8000': 0} >>> who_has = {'a': {'alice:8000'}} >>> nbytes = {'a': 100} >>> ncores = {'alice:8000': 1, 'bob:8000': 1} >>> valid_workers = True >>> loose_restrictions = set() We choose the worker that has the data on which 'b' depends (alice has 'a') >>> decide_worker(dependencies, occupancy, who_has, has_what, ... valid_workers, loose_restrictions, nbytes, ncores, 'b') 'alice:8000' If both Alice and Bob have dependencies then we choose the less-busy worker >>> who_has = {'a': {'alice:8000', 'bob:8000'}} >>> has_what = {'alice:8000': {'a'}, 'bob:8000': {'a'}} >>> decide_worker(dependencies, who_has, has_what, ... valid_workers, loose_restrictions, nbytes, ncores, 'b') 'bob:8000' Optionally provide valid workers of where jobs are allowed to occur >>> valid_workers = {'alice:8000', 'charlie:8000'} >>> decide_worker(dependencies, who_has, has_what, ... valid_workers, loose_restrictions, nbytes, ncores, 'b') 'alice:8000' If the task requires data communication, then we choose to minimize the number of bytes sent between workers. This takes precedence over worker occupancy. >>> dependencies = {'c': {'a', 'b'}} >>> who_has = {'a': {'alice:8000'}, 'b': {'bob:8000'}} >>> has_what = {'alice:8000': {'a'}, 'bob:8000': {'b'}} >>> nbytes = {'a': 1, 'b': 1000} >>> decide_worker(dependencies, who_has, has_what, ... {}, set(), nbytes, ncores, 'c') 'bob:8000' """ deps = dependencies[key] assert all(d in who_has for d in deps) workers = frequencies([w for dep in deps for w in who_has[dep]]) if not workers: workers = occupancy if valid_workers is not True: workers = valid_workers & set(workers) if not workers: workers = valid_workers if not workers: if key in loose_restrictions: return decide_worker(dependencies, occupancy, who_has, True, set(), objective, key) else: return None if not workers or not occupancy: return None if len(workers) == 1: return first(workers) return min(workers, key=objective) def validate_state(dependencies, dependents, waiting, waiting_data, ready, who_has, processing, finished_results, released, who_wants, wants_what, tasks=None, allow_overlap=False, erred=None, **kwargs): """ Validate a current runtime state This performs a sequence of checks on the entire graph, running in about linear time. This raises assert errors if anything doesn't check out. """ in_processing = {k for v in processing.values() for k in v} keys = {key for key in dependents if not dependents[key]} ready_set = set(ready) assert set(waiting).issubset(dependencies), "waiting not subset of deps" assert set(waiting_data).issubset(dependents), "waiting_data not subset" if tasks is not None: assert ready_set.issubset(tasks), "All ready tasks are tasks" assert set(dependents).issubset(set(tasks) | set(who_has)), "all dependents tasks" assert set(dependencies).issubset(set(tasks) | set(who_has)), "all dependencies tasks" for k, v in waiting.items(): assert v, "waiting on empty set" assert v.issubset(dependencies[k]), "waiting set not dependencies" for vv in v: assert vv not in who_has, ("waiting dependency in memory", k, vv) assert vv not in released, ("dependency released", k, vv) for dep in dependencies[k]: assert dep in v or who_has.get(dep), ("dep missing", k, dep) for k, v in waiting_data.items(): for vv in v: if vv in released: raise ValueError('dependent not in play', k, vv) if not (vv in waiting or vv in in_processing): raise ValueError('dependent not in play2', k, vv) for v in concat(processing.values()): assert v in dependencies, "all processing keys in dependencies" for key in who_has: assert key in waiting_data or key in who_wants @memoize def check_key(key): """ Validate a single key, recurse downwards """ vals = ([key in waiting, key in ready, key in in_processing, not not who_has.get(key), key in released, key in erred]) if ((allow_overlap and sum(vals) < 1) or (not allow_overlap and sum(vals) != 1)): raise ValueError("Key exists in wrong number of places", key, vals) for dep in dependencies[key]: if dep in dependents: check_key(dep) # Recursive case if who_has.get(key): assert not any(key in waiting.get(dep, ()) for dep in dependents.get(key, ())) assert not waiting.get(key) if key in in_processing: if not all(who_has.get(dep) for dep in dependencies[key]): raise ValueError("Key in processing without all deps", key) assert not waiting.get(key) assert key not in ready if finished_results is not None: if key in finished_results: assert who_has.get(key) assert key in keys if key in keys and who_has.get(key): assert key in finished_results for key, s in who_wants.items(): assert s, "empty who_wants" for client in s: assert key in wants_what[client] if key in waiting: assert waiting[key], 'waiting empty' if key in ready: assert key not in waiting return True assert all(map(check_key, keys)) _round_robin = [0] fast_tasks = {'rechunk-split', 'shuffle-split'} def heartbeat_interval(n): """ Interval in seconds that we desire heartbeats based on number of workers """ if n <= 10: return 0.5 elif n < 50: return 1 elif n < 200: return 2 else: return 5 class KilledWorker(Exception): pass distributed-1.20.2/distributed/security.py000066400000000000000000000116061321233345200206630ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import try: import ssl except ImportError: ssl = None from . import config _roles = ['client', 'scheduler', 'worker'] _tls_per_role_fields = ['key', 'cert'] _tls_fields = ['ca_file', 'ciphers'] _misc_fields = ['require_encryption'] _fields = set(_misc_fields + ['tls_%s' % field for field in _tls_fields] + ['tls_%s_%s' % (role, field) for role in _roles for field in _tls_per_role_fields] ) def _field_to_config_key(field): return field.replace('_', '-') class Security(object): """ An object to gather and pass around security configuration. Default values are gathered from the global ``config`` object and can be overriden by constructor args. Supported fields: - require_encryption - tls_ca_file - tls_ciphers - tls_client_key - tls_client_cert - tls_scheduler_key - tls_scheduler_cert - tls_worker_key - tls_worker_cert """ __slots__ = tuple(_fields) def __init__(self, **kwargs): self._init_from_dict(config) for k, v in kwargs.items(): if v is not None: setattr(self, k, v) for k in _fields: if not hasattr(self, k): setattr(self, k, None) def _init_from_dict(self, d): """ Initialize Security from nested dict. """ self._init_fields_from_dict(d, '', _misc_fields, {}) self._init_fields_from_dict(d, 'tls', _tls_fields, _tls_per_role_fields) def _init_fields_from_dict(self, d, category, fields, per_role_fields): if category: d = d.get(category, {}) category_prefix = category + '_' else: category_prefix = '' for field in fields: k = _field_to_config_key(field) if k in d: setattr(self, '%s%s' % (category_prefix, field), d[k]) for role in _roles: dd = d.get(role, {}) for field in per_role_fields: k = _field_to_config_key(field) if k in dd: setattr(self, '%s%s_%s' % (category_prefix, role, field), dd[k]) def __repr__(self): items = sorted((k, getattr(self, k)) for k in _fields) return ("Security(" + ", ".join("%s=%r" % (k, v) for k, v in items if v is not None) + ")") def get_tls_config_for_role(self, role): """ Return the TLS configuration for the given role, as a flat dict. """ return self._get_config_for_role('tls', role, _tls_fields, _tls_per_role_fields) def _get_config_for_role(self, category, role, fields, per_role_fields): if role not in _roles: raise ValueError("unknown role %r" % (role,)) d = {} for field in fields: k = '%s_%s' % (category, field) d[field] = getattr(self, k) for field in per_role_fields: k = '%s_%s_%s' % (category, role, field) d[field] = getattr(self, k) return d def _get_tls_context(self, tls, purpose): if tls.get('ca_file') and tls.get('cert'): try: ctx = ssl.create_default_context(purpose=purpose, cafile=tls['ca_file']) except AttributeError: raise RuntimeError("TLS functionality requires Python 2.7.9+") ctx.verify_mode = ssl.CERT_REQUIRED # We expect a dedicated CA for the cluster and people using # IP addresses rather than hostnames ctx.check_hostname = False ctx.load_cert_chain(tls['cert'], tls.get('key')) if tls.get('ciphers'): ctx.set_ciphers(tls.get('ciphers')) return ctx def get_connection_args(self, role): """ Get the *connection_args* argument for a connect() call with the given *role*. """ d = {} tls = self.get_tls_config_for_role(role) # Ensure backwards compatibility (ssl.Purpose is Python 2.7.9+ only) purpose = ssl.Purpose.SERVER_AUTH if hasattr(ssl, "Purpose") else None d['ssl_context'] = self._get_tls_context(tls, purpose) d['require_encryption'] = self.require_encryption return d def get_listen_args(self, role): """ Get the *connection_args* argument for a listen() call with the given *role*. """ d = {} tls = self.get_tls_config_for_role(role) # Ensure backwards compatibility (ssl.Purpose is Python 2.7.9+ only) purpose = ssl.Purpose.CLIENT_AUTH if hasattr(ssl, "Purpose") else None d['ssl_context'] = self._get_tls_context(tls, purpose) d['require_encryption'] = self.require_encryption return d distributed-1.20.2/distributed/sizeof.py000066400000000000000000000050741321233345200203150ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging import sys from dask.utils import Dispatch try: # PyPy does not support sys.getsizeof sys.getsizeof(1) getsizeof = sys.getsizeof except (AttributeError, TypeError): # Monkey patch def getsizeof(x): return 100 logger = logging.getLogger(__name__) sizeof = Dispatch() @sizeof.register(object) def sizeof_default(o): return getsizeof(o) @sizeof.register(list) @sizeof.register(tuple) @sizeof.register(set) @sizeof.register(frozenset) def sizeof_python_collection(seq): return getsizeof(seq) + sum(map(sizeof, seq)) @sizeof.register_lazy("numpy") def register_numpy(): import numpy as np @sizeof.register(np.ndarray) def sizeof_numpy_ndarray(x): return int(x.nbytes) @sizeof.register_lazy("pandas") def register_pandas(): import pandas as pd import numpy as np def object_size(x): if not len(x): return 0 sample = np.random.choice(x, size=20, replace=True) sample = list(map(sizeof, sample)) return sum(sample) / 20 * len(x) @sizeof.register(pd.DataFrame) def sizeof_pandas_dataframe(df): p = sizeof(df.index) for name, col in df.iteritems(): p += col.memory_usage(index=False) if col.dtype == object: p += object_size(col._values) return int(p) + 1000 @sizeof.register(pd.Series) def sizeof_pandas_series(s): p = int(s.memory_usage(index=True)) if s.dtype == object: p += object_size(s._values) if s.index.dtype == object: p += object_size(s.index) return int(p) + 1000 @sizeof.register(pd.Index) def sizeof_pandas_index(i): p = int(i.memory_usage()) if i.dtype == object: p += object_size(i) return int(p) + 1000 @sizeof.register_lazy("scipy") def register_spmatrix(): from scipy import sparse @sizeof.register(sparse.dok_matrix) def sizeof_spmatrix_dok(s): return s.__sizeof__() @sizeof.register(sparse.spmatrix) def sizeof_spmatrix(s): return sum( sizeof(v) for v in s.__dict__.values() ) def safe_sizeof(obj, default_size=1e6): """ Safe variant of sizeof that captures and logs exceptions This returns a default size of 1e6 if the sizeof function fails """ try: return sizeof(obj) except Exception: logger.warning('Sizeof calculation failed. Defaulting to 1MB', exc_info=True) return int(default_size) distributed-1.20.2/distributed/stealing.py000066400000000000000000000355451321233345200206320ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import defaultdict, deque import logging from math import log import os from time import time from .config import config from .core import CommClosedError from .diagnostics.plugin import SchedulerPlugin from .utils import key_split, log_errors, PeriodicCallback try: from cytoolz import topk except ImportError: from toolz import topk BANDWIDTH = 100e6 LATENCY = 10e-3 log_2 = log(2) logger = logging.getLogger(__name__) LOG_PDB = config.get('pdb-on-err') or os.environ.get('DASK_ERROR_PDB', False) class WorkStealing(SchedulerPlugin): def __init__(self, scheduler): self.scheduler = scheduler self.stealable_all = [set() for i in range(15)] self.stealable = dict() self.key_stealable = dict() self.stealable_unknown_durations = defaultdict(set) self.cost_multipliers = [1 + 2 ** (i - 6) for i in range(15)] self.cost_multipliers[0] = 1 for worker in scheduler.workers: self.add_worker(worker=worker) pc = PeriodicCallback(callback=self.balance, callback_time=100) self._pc = pc self.scheduler.periodic_callbacks['stealing'] = pc self.scheduler.plugins.append(self) self.scheduler.extensions['stealing'] = self self.scheduler.events['stealing'] = deque(maxlen=100000) self.count = 0 self.in_flight = dict() self.in_flight_occupancy = defaultdict(lambda: 0) self.scheduler.worker_handlers['steal-response'] = self.move_task_confirm @property def log(self): return self.scheduler.events['stealing'] def add_worker(self, scheduler=None, worker=None): self.stealable[worker] = [set() for i in range(15)] def remove_worker(self, scheduler=None, worker=None): del self.stealable[worker] def teardown(self): self._pc.stop() def transition(self, key, start, finish, compute_start=None, compute_stop=None, *args, **kwargs): if finish == 'processing': self.put_key_in_stealable(key) if start == 'processing': self.remove_key_from_stealable(key) if finish == 'memory': ks = key_split(key) if ks in self.stealable_unknown_durations: for k in self.stealable_unknown_durations.pop(ks): if k in self.in_flight: continue if self.scheduler.task_state[k] == 'processing': self.put_key_in_stealable(k, split=ks) else: if key in self.in_flight: del self.in_flight[key] def put_key_in_stealable(self, key, split=None): worker = self.scheduler.rprocessing[key] cost_multiplier, level = self.steal_time_ratio(key, split=split) self.log.append(('add-stealable', key, worker, level)) if cost_multiplier is not None: self.stealable_all[level].add(key) self.stealable[worker][level].add(key) self.key_stealable[key] = (worker, level) def remove_key_from_stealable(self, key): result = self.key_stealable.pop(key, None) if result is None: return worker, level = result self.log.append(('remove-stealable', key, worker, level)) try: self.stealable[worker][level].remove(key) except KeyError: pass try: self.stealable_all[level].remove(key) except KeyError: pass def steal_time_ratio(self, key, split=None): """ The compute to communication time ratio of a key Returns ------- cost_multiplier: The increased cost from moving this task as a factor. For example a result of zero implies a task without dependencies. level: The location within a stealable list to place this value """ if (key not in self.scheduler.loose_restrictions and (key in self.scheduler.host_restrictions or key in self.scheduler.worker_restrictions) or key in self.scheduler.resource_restrictions): return None, None # don't steal if not self.scheduler.dependencies[key]: # no dependencies fast path return 0, 0 nbytes = sum(self.scheduler.nbytes.get(k, 1000) for k in self.scheduler.dependencies[key]) transfer_time = nbytes / BANDWIDTH + LATENCY split = split or key_split(key) if split in fast_tasks: return None, None try: worker = self.scheduler.rprocessing[key] compute_time = self.scheduler.processing[worker][key] except KeyError: self.stealable_unknown_durations[split].add(key) return None, None else: if compute_time < 0.005: # 5ms, just give up return None, None cost_multiplier = transfer_time / compute_time if cost_multiplier > 100: return None, None level = int(round(log(cost_multiplier) / log_2 + 6, 0)) level = max(1, level) return cost_multiplier, level def move_task_request(self, key, victim, thief): try: if self.scheduler.validate: if victim != self.scheduler.rprocessing[key]: import pdb pdb.set_trace() self.remove_key_from_stealable(key) logger.debug("Request move %s, %s: %2f -> %s: %2f", key, victim, self.scheduler.occupancy[victim], thief, self.scheduler.occupancy[thief]) victim_duration = self.scheduler.processing[victim][key] thief_duration = self.scheduler.task_duration.get(key_split(key), 0.5) thief_duration += sum(self.scheduler.nbytes[key] for key in self.scheduler.dependencies[key] - self.scheduler.has_what[thief]) / BANDWIDTH self.scheduler.worker_comms[victim].send({'op': 'steal-request', 'key': key}) self.in_flight[key] = {'victim': victim, 'thief': thief, 'victim_duration': victim_duration, 'thief_duration': thief_duration} self.in_flight_occupancy[victim] -= victim_duration self.in_flight_occupancy[thief] += thief_duration except CommClosedError: logger.info("Worker comm closed while stealing: %s", victim) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def move_task_confirm(self, key=None, worker=None, state=None): try: try: d = self.in_flight.pop(key) except KeyError: return thief = d['thief'] victim = d['victim'] logger.debug("Confirm move %s, %s -> %s. State: %s", key, victim, thief, state) self.in_flight_occupancy[thief] -= d['thief_duration'] self.in_flight_occupancy[victim] += d['victim_duration'] if not self.in_flight: self.in_flight_occupancy = defaultdict(lambda: 0) if (self.scheduler.task_state.get(key) != 'processing' or self.scheduler.rprocessing[key] != victim): old_thief = self.scheduler.occupancy[thief] new_thief = sum(self.scheduler.processing[thief].values()) old_victim = self.scheduler.occupancy[victim] new_victim = sum(self.scheduler.processing[victim].values()) self.scheduler.occupancy[thief] = new_thief self.scheduler.occupancy[victim] = new_victim self.scheduler.total_occupancy += new_thief - old_thief + new_victim - old_victim return # One of the pair has left, punt and reschedule if (thief not in self.scheduler.workers or victim not in self.scheduler.workers): self.scheduler.reschedule(key) return # Victim had already started execution, reverse stealing if state in ('memory', 'executing', 'long-running', None): self.log.append(('already-computing', key, victim, thief)) self.scheduler.check_idle_saturated(thief) self.scheduler.check_idle_saturated(victim) # Victim was waiting, has given up task, enact steal elif state in ('waiting', 'ready'): self.remove_key_from_stealable(key) self.scheduler.rprocessing[key] = thief duration = self.scheduler.processing[victim][key] self.scheduler.occupancy[victim] -= duration self.scheduler.total_occupancy -= duration del self.scheduler.processing[victim][key] self.scheduler.processing[thief][key] = d['thief_duration'] self.scheduler.occupancy[thief] += d['thief_duration'] self.scheduler.total_occupancy += d['thief_duration'] self.put_key_in_stealable(key) try: self.scheduler.send_task_to_worker(thief, key) except CommClosedError: self.scheduler.remove_worker(thief) self.log.append(('confirm', key, victim, thief)) else: raise ValueError("Unexpected task state: %s" % state) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def balance(self): s = self.scheduler def combined_occupancy(w): return s.occupancy[w] + self.in_flight_occupancy[w] with log_errors(): i = 0 idle = s.idle saturated = s.saturated if not idle or len(idle) == len(s.workers): return log = list() start = time() seen = False acted = False if not s.saturated: saturated = topk(10, s.workers, key=combined_occupancy) saturated = [w for w in saturated if combined_occupancy(w) > 0.2 and len(s.processing[w]) > s.ncores[w]] elif len(s.saturated) < 20: saturated = sorted(saturated, key=combined_occupancy, reverse=True) if len(idle) < 20: idle = sorted(idle, key=combined_occupancy) for level, cost_multiplier in enumerate(self.cost_multipliers): if not idle: break for sat in list(saturated): stealable = self.stealable[sat][level] if not stealable or not idle: continue else: seen = True for key in list(stealable): if key not in self.key_stealable: stealable.remove(key) continue i += 1 if not idle: break idl = idle[i % len(idle)] durations = s.processing[sat] try: duration = durations[key] except KeyError: stealable.remove(key) continue if (combined_occupancy(idl) + cost_multiplier * duration <= combined_occupancy(sat) - duration / 2): self.move_task_request(key, sat, idl) log.append((start, level, key, duration, sat, combined_occupancy(sat), idl, combined_occupancy(idl))) s.check_idle_saturated(sat, occ=combined_occupancy(sat)) s.check_idle_saturated(idl, occ=combined_occupancy(idl)) seen = True if self.cost_multipliers[level] < 20: # don't steal from public at cost stealable = self.stealable_all[level] if stealable: seen = True for key in list(stealable): if not idle: break if key not in self.key_stealable: stealable.remove(key) continue try: sat = s.rprocessing[key] except KeyError: stealable.remove(key) continue if combined_occupancy(sat) < 0.2: continue if len(s.processing[sat]) <= s.ncores[sat]: continue i += 1 idl = idle[i % len(idle)] duration = s.processing[sat][key] if (combined_occupancy(idl) + cost_multiplier * duration <= combined_occupancy(sat) - duration / 2): self.move_task_request(key, sat, idl) log.append((start, level, key, duration, sat, combined_occupancy(sat), idl, combined_occupancy(idl))) s.check_idle_saturated(sat, occ=combined_occupancy(sat)) s.check_idle_saturated(idl, occ=combined_occupancy(idl)) seen = True if seen and not acted: break if log: self.log.append(log) self.count += 1 stop = time() if s.digests: s.digests['steal-duration'].add(stop - start) def restart(self, scheduler): for stealable in self.stealable.values(): for s in stealable: s.clear() for s in self.stealable_all: s.clear() self.key_stealable.clear() self.stealable_unknown_durations.clear() def story(self, *keys): keys = set(keys) out = [] for L in self.log: if not isinstance(L, list): L = [L] for t in L: if any(x in keys for x in t): out.append(t) return out fast_tasks = {'shuffle-split'} distributed-1.20.2/distributed/submit.py000066400000000000000000000061641321233345200203220ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging import os import socket import subprocess import tempfile import sys from tornado import gen from tornado.ioloop import IOLoop from distributed import rpc from distributed.compatibility import unicode from distributed.core import Server from distributed.security import Security from distributed.utils import get_ip logger = logging.getLogger('distributed.remote') class RemoteClient(Server): def __init__(self, ip=None, local_dir=tempfile.mkdtemp(prefix='client-'), loop=None, security=None, **kwargs): self.ip = ip or get_ip() self.loop = loop or IOLoop.current() self.local_dir = local_dir handlers = {'upload_file': self.upload_file, 'execute': self.execute} self.security = security or Security() assert isinstance(self.security, Security) self.listen_args = self.security.get_listen_args('scheduler') super(RemoteClient, self).__init__(handlers, io_loop=self.loop, **kwargs) @gen.coroutine def _start(self, port=0): self.listen(port, listen_args=self.listen_args) def start(self, port=0): self.loop.add_callback(self._start, port) logger.info("Remote Client is running at {0}:{1}".format(self.ip, port)) @gen.coroutine def execute(self, stream=None, filename=None): script_path = os.path.join(self.local_dir, filename) cmd = '{0} {1}'.format(sys.executable, script_path) process = subprocess.Popen(cmd, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE) out, err = process.communicate() return_code = process.returncode raise gen.Return({'stdout': out, 'stderr': err, 'returncode': return_code}) def upload_file(self, stream, filename=None, file_payload=None): out_filename = os.path.join(self.local_dir, filename) if isinstance(file_payload, unicode): file_payload = file_payload.encode() with open(out_filename, 'wb') as f: f.write(file_payload) return {'status': 'OK', 'nbytes': len(file_payload)} @gen.coroutine def _close(self): self.stop() def _remote(host, port, loop=IOLoop.current(), client=RemoteClient): host = host or get_ip() if ':' in host and port == 8788: host, port = host.rsplit(':', 1) port = int(port) ip = socket.gethostbyname(host) remote_client = client(ip=ip, loop=loop) remote_client.start(port=port) loop.start() loop.close() remote_client.stop() logger.info("End remote client at %s:%d", host, port) @gen.coroutine def _submit(remote_client_address, filepath, connection_args=None): rc = rpc(remote_client_address, connection_args=connection_args) remote_file = os.path.basename(filepath) with open(filepath, 'rb') as f: bytes_read = f.read() yield rc.upload_file(filename=remote_file, file_payload=bytes_read) result = yield rc.execute(filename=remote_file) raise gen.Return((result['stdout'], result['stderr'])) distributed-1.20.2/distributed/system_monitor.py000066400000000000000000000062001321233345200221010ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import deque import psutil from .compatibility import WINDOWS from .metrics import time class SystemMonitor(object): def __init__(self, n=10000): self.proc = psutil.Process() self.time = deque(maxlen=n) self.cpu = deque(maxlen=n) self.memory = deque(maxlen=n) self.count = 0 self.quantities = {'cpu': self.cpu, 'memory': self.memory, 'time': self.time} try: ioc = psutil.net_io_counters() except Exception: self._collect_net_io_counters = False else: self.last_time = time() self.read_bytes = deque(maxlen=n) self.write_bytes = deque(maxlen=n) self.quantities['read_bytes'] = self.read_bytes self.quantities['write_bytes'] = self.write_bytes self._last_io_counters = ioc self._collect_net_io_counters = True if not WINDOWS: self.num_fds = deque(maxlen=n) self.quantities['num_fds'] = self.num_fds self.update() def recent(self): try: return {k: v[-1] for k, v in self.quantities.items()} except IndexError: return {k: None for k, v in self.quantities.items()} def update(self): cpu = self.proc.cpu_percent() memory = self.proc.memory_info().rss now = time() self.cpu.append(cpu) self.memory.append(memory) self.time.append(now) self.count += 1 result = {'cpu': cpu, 'memory': memory, 'time': now, 'count': self.count} if self._collect_net_io_counters: try: ioc = psutil.net_io_counters() except Exception: pass else: last = self._last_io_counters duration = now - self.last_time read_bytes = (ioc.bytes_recv - last.bytes_recv) / (duration or 0.5) write_bytes = (ioc.bytes_sent - last.bytes_sent) / (duration or 0.5) self.last_time = now self._last_io_counters = ioc self.read_bytes.append(read_bytes) self.write_bytes.append(write_bytes) result['read_bytes'] = read_bytes result['write_bytes'] = write_bytes if not WINDOWS: num_fds = self.proc.num_fds() self.num_fds.append(num_fds) result['num_fds'] = num_fds return result def __repr__(self): return '' % ( self.cpu[-1], self.memory[-1] / 1e6, -1 if WINDOWS else self.num_fds[-1]) def range_query(self, start): if start == self.count: return {k: [] for k in self.quantities} istart = start - (self.count - len(self.cpu)) istart = max(0, istart) seq = [i for i in range(istart, len(self.cpu))] d = {k: [v[i] for i in seq] for k, v in self.quantities.items()} return d distributed-1.20.2/distributed/tests/000077500000000000000000000000001321233345200176005ustar00rootroot00000000000000distributed-1.20.2/distributed/tests/make_tls_certs.py000066400000000000000000000120251321233345200231510ustar00rootroot00000000000000"""Make the custom certificate and private key files used by TLS tests. Code heavily borrowed from Lib/tests/make_ssl_certs.py in CPython. """ import os import shutil import tempfile import subprocess req_template = """ [req] distinguished_name = req_distinguished_name x509_extensions = req_x509_extensions prompt = no [req_distinguished_name] C = XY L = Dask-distributed O = Dask CN = {hostname} [req_x509_extensions] subjectAltName = @san [san] DNS.1 = {hostname} [ca] default_ca = CA_default [CA_default] dir = cadir database = $dir/index.txt crlnumber = $dir/crl.txt default_md = sha1 default_days = 3600 default_crl_days = 3600 certificate = tls-ca-cert.pem private_key = tls-ca-key.pem serial = $dir/serial RANDFILE = $dir/.rand policy = policy_match [policy_match] countryName = match stateOrProvinceName = optional organizationName = match organizationalUnitName = optional commonName = supplied emailAddress = optional [policy_anything] countryName = optional stateOrProvinceName = optional localityName = optional organizationName = optional organizationalUnitName = optional commonName = supplied emailAddress = optional [v3_ca] subjectKeyIdentifier=hash authorityKeyIdentifier=keyid:always,issuer basicConstraints = CA:true """ here = os.path.abspath(os.path.dirname(__file__)) def make_cert_key(hostname, sign=False): print("creating cert for " + hostname) tempnames = [] for i in range(3): with tempfile.NamedTemporaryFile(delete=False) as f: tempnames.append(f.name) req_file, cert_file, key_file = tempnames try: req = req_template.format(hostname=hostname) with open(req_file, 'w') as f: f.write(req) args = ['req', '-new', '-days', '3650', '-nodes', '-newkey', 'rsa:1024', '-keyout', key_file, '-config', req_file] if sign: with tempfile.NamedTemporaryFile(delete=False) as f: tempnames.append(f.name) reqfile = f.name args += ['-out', reqfile] else: args += ['-x509', '-out', cert_file] subprocess.check_call(['openssl'] + args) if sign: args = ['ca', '-config', req_file, '-out', cert_file, '-outdir', 'cadir', '-policy', 'policy_anything', '-batch', '-infiles', reqfile] subprocess.check_call(['openssl'] + args) with open(cert_file, 'r') as f: cert = f.read() with open(key_file, 'r') as f: key = f.read() return cert, key finally: for name in tempnames: os.remove(name) TMP_CADIR = 'cadir' def unmake_ca(): shutil.rmtree(TMP_CADIR) def make_ca(): os.mkdir(TMP_CADIR) with open(os.path.join('cadir', 'index.txt'), 'a+') as f: pass # empty file # with open(os.path.join('cadir','crl.txt'),'a+') as f: # f.write("00") with open(os.path.join('cadir', 'index.txt.attr'), 'w+') as f: f.write('unique_subject = no') with tempfile.NamedTemporaryFile("w") as t: t.write(req_template.format(hostname='our-ca-server')) t.flush() with tempfile.NamedTemporaryFile() as f: args = ['req', '-new', '-days', '3650', '-extensions', 'v3_ca', '-nodes', '-newkey', 'rsa:2048', '-keyout', 'tls-ca-key.pem', '-out', f.name, '-subj', '/C=XY/L=Dask-distributed/O=Dask CA/CN=our-ca-server'] subprocess.check_call(['openssl'] + args) args = ['ca', '-config', t.name, '-create_serial', '-out', 'tls-ca-cert.pem', '-batch', '-outdir', TMP_CADIR, '-keyfile', 'tls-ca-key.pem', '-days', '3650', '-selfsign', '-extensions', 'v3_ca', '-infiles', f.name] subprocess.check_call(['openssl'] + args) #args = ['ca', '-config', t.name, '-gencrl', '-out', 'revocation.crl'] #subprocess.check_call(['openssl'] + args) if __name__ == '__main__': os.chdir(here) cert, key = make_cert_key('localhost') with open('tls-self-signed-cert.pem', 'w') as f: f.write(cert) with open('tls-self-signed-key.pem', 'w') as f: f.write(key) # For certificate matching tests make_ca() with open('tls-ca-cert.pem', 'r') as f: ca_cert = f.read() cert, key = make_cert_key('localhost', sign=True) with open('tls-cert.pem', 'w') as f: f.write(cert) with open('tls-cert-chain.pem', 'w') as f: f.write(cert) f.write(ca_cert) with open('tls-key.pem', 'w') as f: f.write(key) with open('tls-key-cert.pem', 'w') as f: f.write(key) f.write(cert) unmake_ca() distributed-1.20.2/distributed/tests/mytestegg-1.0.0-py3.4.egg000066400000000000000000000034371321233345200236100ustar00rootroot00000000000000PKoGӉRsEGG-INFO/PKG-INFOM-ILI,I K-*ϳR03KMR(I-.IMOBJ&UZ)yqy$"KK29QԼb$m.E%``9%iEPKoG[ EGG-INFO/top_level.txt+I-.IMOPKoGI|rEGG-INFO/SOURCES.txtm 0{%7RV=/be1l՛s7s]ۄg2^ʐ Rt2߹oRpM'(d\W,%܋ˌR-)E<7PKoG2EGG-INFO/not-zip-safePKoG2EGG-INFO/dependency_links.txtPKG testegg/__init__.pyKIMSKԴRԒҢ 0 assert [value == 2 for value in results.values()] results = await c.run(inc, 1, workers=[]) assert results == {} @coro_test async def test_asyncio_run_on_scheduler(): def f(dask_scheduler=None): return dask_scheduler.address async with AioClient(processes=False) as c: address = await c.run_on_scheduler(f) assert address == c.cluster.scheduler.address with pytest.raises(ZeroDivisionError): await c.run_on_scheduler(div, 1, 0) @coro_test async def test_asyncio_run_coroutine(): async def aioinc(x, delay=0.02): await asyncio.sleep(delay) return x + 1 async def aiothrows(x, delay=0.02): await asyncio.sleep(delay) raise RuntimeError('hello') async with AioClient(processes=False) as c: results = await c.run_coroutine(aioinc, 1, delay=0.05) assert len(results) > 0 assert [value == 2 for value in results.values()] results = await c.run_coroutine(aioinc, 1, workers=[]) assert results == {} with pytest.raises(RuntimeError) as exc_info: await c.run_coroutine(aiothrows, 1) assert "hello" in str(exc_info) @slow @coro_test async def test_asyncio_restart(): c = await AioClient(processes=False) assert c.status == 'running' x = c.submit(inc, 1) assert x.key in c.refcount await c.restart() assert x.key not in c.refcount key = x.key del x import gc gc.collect() assert key not in c.refcount await c.shutdown() @coro_test async def test_asyncio_nanny_workers(): async with AioClient(n_workers=2) as c: assert await c.submit(inc, 1) == 2 @coro_test async def test_asyncio_variable(): c = await AioClient(processes=False) s = c.cluster.scheduler x = Variable('x') xx = Variable('x') assert x.client is c future = c.submit(inc, 1) await x.set(future) future2 = await xx.get() assert future.key == future2.key del future, future2 await asyncio.sleep(0.1) assert s.task_state # future still present x.delete() start = time() while s.task_state: await asyncio.sleep(0.01) assert time() < start + 5 distributed-1.20.2/distributed/tests/py3_test_client.py000066400000000000000000000046321321233345200232670ustar00rootroot00000000000000from time import sleep import pytest from tornado import gen from distributed.utils_test import div, gen_cluster, inc, loop from distributed import as_completed, Client, Lock @gen_cluster(client=True) def test_await_future(c, s, a, b): future = c.submit(inc, 1) async def f(): # flake8: noqa result = await future assert result == 2 yield f() future = c.submit(div, 1, 0) async def f(): with pytest.raises(ZeroDivisionError): await future yield f() @gen_cluster(client=True) def test_as_completed_async_for(c, s, a, b): futures = c.map(inc, range(10)) ac = as_completed(futures) results = [] async def f(): async for future in ac: result = await future results.append(result) yield f() assert set(results) == set(range(1, 11)) @gen_cluster(client=True) def test_as_completed_async_for_results(c, s, a, b): futures = c.map(inc, range(10)) ac = as_completed(futures, with_results=True) results = [] async def f(): async for future, result in ac: results.append(result) yield f() assert set(results) == set(range(1, 11)) assert not s.counters['op'].components[0]['gather'] @gen_cluster(client=True) def test_as_completed_async_for_cancel(c, s, a, b): x = c.submit(inc, 1) y = c.submit(sleep, 0.3) ac = as_completed([x, y]) async def _(): await gen.sleep(0.1) await y.cancel(asynchronous=True) c.loop.add_callback(_) L = [] async def f(): async for future in ac: L.append(future) yield f() assert L == [x] def test_async_with(loop): result = None client = None cluster = None async def f(): async with Client(processes=False, asynchronous=True) as c: nonlocal result, client, cluster result = await c.submit(lambda x: x + 1, 10) client = c cluster = c.cluster loop.run_sync(f) assert result == 11 assert client.status == 'closed' assert cluster.status == 'closed' def test_locks(loop): async def f(): async with Client(processes=False, asynchronous=True) as c: assert c.asynchronous == True async with Lock('x'): lock2 = Lock('x') result = await lock2.acquire(timeout=0.1) assert result is False loop.run_sync(f) distributed-1.20.2/distributed/tests/py3_test_utils_tst.py000066400000000000000000000004731321233345200240420ustar00rootroot00000000000000from distributed.utils_test import gen_cluster from distributed import Client @gen_cluster() async def test_gen_cluster_async(s, a, b): # flake8: noqa async with Client(s.address, asynchronous=True) as c: future = c.submit(lambda x: x + 1, 1) result = await future assert result == 2 distributed-1.20.2/distributed/tests/test_as_completed.py000066400000000000000000000104111321233345200236450ustar00rootroot00000000000000from collections import Iterator from operator import add import random from time import sleep import pytest from tornado import gen from distributed import Client from distributed.client import _as_completed, as_completed, _first_completed from distributed.compatibility import Empty from distributed.utils_test import cluster, gen_cluster, inc from distributed.utils_test import loop # flake8: noqa from distributed.compatibility import Queue @gen_cluster(client=True) def test__as_completed(c, s, a, b): x = c.submit(inc, 1) y = c.submit(inc, 1) z = c.submit(inc, 2) queue = Queue() yield _as_completed([x, y, z], queue) assert queue.qsize() == 3 assert {queue.get(), queue.get(), queue.get()} == {x, y, z} result = yield _first_completed([x, y, z]) assert result in [x, y, z] def test_as_completed(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(inc, 1) y = c.submit(inc, 2) z = c.submit(inc, 1) seq = as_completed([x, y, z]) assert isinstance(seq, Iterator) assert set(seq) == {x, y, z} assert list(as_completed([])) == [] def test_as_completed_with_non_futures(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop): with pytest.raises(TypeError): list(as_completed([1, 2, 3])) def test_as_completed_add(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: total = 0 expected = sum(map(inc, range(10))) futures = c.map(inc, range(10)) ac = as_completed(futures) for future in ac: result = future.result() total += result if random.random() < 0.5: future = c.submit(add, future, 10) ac.add(future) expected += result + 10 assert total == expected def test_as_completed_update(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: total = 0 todo = list(range(10)) expected = sum(map(inc, todo)) ac = as_completed([]) while todo or not ac.is_empty(): if todo: work, todo = todo[:4], todo[4:] ac.update(c.map(inc, work)) batch = ac.next_batch(block=True) total += sum(r.result() for r in batch) assert total == expected def test_as_completed_repeats(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: ac = as_completed() x = c.submit(inc, 1) ac.add(x) ac.add(x) assert next(ac) is x assert next(ac) is x with pytest.raises(StopIteration): next(ac) ac.add(x) assert next(ac) is x def test_as_completed_is_empty(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: ac = as_completed() assert ac.is_empty() x = c.submit(inc, 1) ac.add(x) assert not ac.is_empty() assert next(ac) is x assert ac.is_empty() def test_as_completed_cancel(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(sleep, 1) y = c.submit(inc, 1) ac = as_completed([x, y]) x.cancel() assert next(ac) is y with pytest.raises(Empty): ac.queue.get(timeout=0.1) assert list(as_completed([x, y, x])) == [y] def test_as_completed_cancel_last(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: w = c.submit(sleep, 0.3) x = c.submit(inc, 1) y = c.submit(sleep, 0.3) @gen.coroutine def _(): yield gen.sleep(0.1) yield w.cancel(asynchronous=True) yield y.cancel(asynchronous=True) loop.add_callback(_) ac = as_completed([x, y]) result = list(ac) assert result == [x] distributed-1.20.2/distributed/tests/test_asyncio.py000066400000000000000000000001611321233345200226540ustar00rootroot00000000000000import sys if sys.version_info >= (3, 5): from distributed.tests.py3_test_asyncio import * # flake8: noqa distributed-1.20.2/distributed/tests/test_asyncprocess.py000066400000000000000000000272631321233345200237370ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from datetime import timedelta import gc import os import signal import sys import threading from time import sleep import weakref import pytest from tornado import gen from tornado.locks import Event from distributed.metrics import time from distributed.process import AsyncProcess from distributed.utils import mp_context from distributed.utils_test import gen_test, pristine_loop, nodebug def feed(in_q, out_q): obj = in_q.get(timeout=5) out_q.put(obj) def exit(q): sys.exit(q.get()) def exit_now(rc=0): sys.exit(rc) def exit_with_signal(signum): signal.signal(signal.SIGINT, signal.SIG_DFL) while True: os.kill(os.getpid(), signum) sleep(0.01) def wait(): while True: sleep(0.01) def threads_info(q): q.put(len(threading.enumerate())) q.put(threading.current_thread().name) @nodebug @gen_test() def test_simple(): to_child = mp_context.Queue() from_child = mp_context.Queue() proc = AsyncProcess(target=feed, args=(to_child, from_child)) assert not proc.is_alive() assert proc.pid is None assert proc.exitcode is None assert not proc.daemon proc.daemon = True assert proc.daemon wr1 = weakref.ref(proc) wr2 = weakref.ref(proc._process) # join() before start() with pytest.raises(AssertionError): yield proc.join() yield proc.start() assert proc.is_alive() assert proc.pid is not None assert proc.exitcode is None t1 = time() yield proc.join(timeout=0.02) dt = time() - t1 assert 0.2 >= dt >= 0.01 assert proc.is_alive() assert proc.pid is not None assert proc.exitcode is None # setting daemon attribute after start() with pytest.raises(AssertionError): proc.daemon = False to_child.put(5) assert from_child.get() == 5 # child should be stopping now t1 = time() yield proc.join(timeout=10) dt = time() - t1 assert dt <= 1.0 assert not proc.is_alive() assert proc.pid is not None assert proc.exitcode == 0 # join() again t1 = time() yield proc.join() dt = time() - t1 assert dt <= 0.6 del proc gc.collect() if wr1() is not None: # Help diagnosing from types import FrameType p = wr1() if p is not None: rc = sys.getrefcount(p) refs = gc.get_referrers(p) del p print("refs to proc:", rc, refs) frames = [r for r in refs if isinstance(r, FrameType)] for i, f in enumerate(frames): print("frames #%d:" % i, f.f_code.co_name, f.f_code.co_filename, sorted(f.f_locals)) pytest.fail("AsyncProcess should have been destroyed") t1 = time() while wr2() is not None: yield gen.sleep(0.01) gc.collect() dt = time() - t1 assert dt < 2.0 @gen_test() def test_exitcode(): q = mp_context.Queue() proc = AsyncProcess(target=exit, kwargs={'q': q}) proc.daemon = True assert not proc.is_alive() assert proc.exitcode is None yield proc.start() assert proc.is_alive() assert proc.exitcode is None q.put(5) yield proc.join(timeout=3.0) assert not proc.is_alive() assert proc.exitcode == 5 @pytest.mark.skipif(os.name == 'nt', reason="POSIX only") @gen_test() def test_signal(): proc = AsyncProcess(target=exit_with_signal, args=(signal.SIGINT,)) proc.daemon = True assert not proc.is_alive() assert proc.exitcode is None yield proc.start() yield proc.join(timeout=3.0) assert not proc.is_alive() # Can be 255 with forkserver, see https://bugs.python.org/issue30589 assert proc.exitcode in (-signal.SIGINT, 255) proc = AsyncProcess(target=wait) yield proc.start() os.kill(proc.pid, signal.SIGTERM) yield proc.join(timeout=3.0) assert not proc.is_alive() assert proc.exitcode in (-signal.SIGTERM, 255) @gen_test() def test_terminate(): proc = AsyncProcess(target=wait) proc.daemon = True yield proc.start() yield proc.terminate() yield proc.join(timeout=3.0) assert not proc.is_alive() assert proc.exitcode in (-signal.SIGTERM, 255) @gen_test() def test_close(): proc = AsyncProcess(target=exit_now) proc.close() with pytest.raises(ValueError): yield proc.start() proc = AsyncProcess(target=exit_now) yield proc.start() proc.close() with pytest.raises(ValueError): yield proc.terminate() proc = AsyncProcess(target=exit_now) yield proc.start() yield proc.join() proc.close() with pytest.raises(ValueError): yield proc.join() proc.close() @gen_test() def test_exit_callback(): to_child = mp_context.Queue() from_child = mp_context.Queue() evt = Event() @gen.coroutine def on_stop(_proc): assert _proc is proc yield gen.moment evt.set() # Normal process exit proc = AsyncProcess(target=feed, args=(to_child, from_child)) evt.clear() proc.set_exit_callback(on_stop) proc.daemon = True yield proc.start() yield gen.sleep(0.05) assert proc.is_alive() assert not evt.is_set() to_child.put(None) yield evt.wait(timedelta(seconds=3)) assert evt.is_set() assert not proc.is_alive() # Process terminated proc = AsyncProcess(target=wait) evt.clear() proc.set_exit_callback(on_stop) proc.daemon = True yield proc.start() yield gen.sleep(0.05) assert proc.is_alive() assert not evt.is_set() yield proc.terminate() yield evt.wait(timedelta(seconds=3)) assert evt.is_set() @gen_test() def test_child_main_thread(): """ The main thread in the child should be called "MainThread". """ q = mp_context.Queue() proc = AsyncProcess(target=threads_info, args=(q,)) yield proc.start() yield proc.join() n_threads = q.get() main_name = q.get() assert n_threads == 2 assert main_name == "MainThread" q.close() q._reader.close() q._writer.close() @pytest.mark.skipif(sys.platform.startswith('win'), reason="num_fds not supported on windows") @gen_test() def test_num_fds(): psutil = pytest.importorskip('psutil') # Warm up proc = AsyncProcess(target=exit_now) proc.daemon = True yield proc.start() yield proc.join() p = psutil.Process() before = p.num_fds() proc = AsyncProcess(target=exit_now) proc.daemon = True yield proc.start() yield proc.join() assert not proc.is_alive() assert proc.exitcode == 0 start = time() while p.num_fds() > before: yield gen.sleep(0.1) print("fds:", before, p.num_fds()) assert time() < start + 10 @gen_test() def test_terminate_after_stop(): proc = AsyncProcess(target=sleep, args=(0,)) yield proc.start() yield gen.sleep(0.1) yield proc.terminate() def _worker_process(worker_ready, child_pipe): # child_pipe is the write-side of the children_alive pipe held by the # test process. When this _worker_process exits, this file descriptor should # have no references remaining anywhere and be closed by the kernel. The # test will therefore be able to tell that this process has exited by # reading children_alive. # Signal to parent process that this process has started and made it this # far. This should cause the parent to exit rapidly after this statement. worker_ready.set() # The parent exiting should cause this process to os._exit from a monitor # thread. This sleep should never return. shorter_timeout = 2.5 # timeout shorter than that in the spawning test. sleep(shorter_timeout) # Unreachable if functioning correctly. child_pipe.send("child should have exited by now") def _parent_process(child_pipe): """ Simulate starting an AsyncProcess and then dying. The child_alive pipe is held open for as long as the child is alive, and can be used to determine if it exited correctly. """ def parent_process_coroutine(): worker_ready = mp_context.Event() worker = AsyncProcess(target=_worker_process, args=(worker_ready, child_pipe)) yield worker.start() # Wait for the child process to have started. worker_ready.wait() # Exit immediately, without doing any process teardown (including atexit # and 'finally:' blocks) as if by SIGKILL. This should cause # worker_process to also exit. os._exit(255) with pristine_loop() as loop: try: loop.run_sync(gen.coroutine(parent_process_coroutine), timeout=10) finally: loop.stop() raise RuntimeError("this should be unreachable due to os._exit") def test_asyncprocess_child_teardown_on_parent_exit(): """ Check that a child process started by AsyncProcess exits if its parent exits. The motivation is to ensure that if an AsyncProcess is created and the creator process dies unexpectedly (e.g, via Out-of-memory SIGKILL), the child process and resources held by it should not be leaked. The child should monitor its parent and exit promptly if the parent exits. [test process] -> [parent using AsyncProcess (dies)] -> [worker process] \ / \________ <-- child_pipe <-- ________/ """ # When child_pipe is closed, the children_alive pipe unblocks. children_alive, child_pipe = mp_context.Pipe(duplex=False) try: parent = mp_context.Process(target=_parent_process, args=(child_pipe,)) parent.start() # Close our reference to child_pipe so that the child has the only one. child_pipe.close() # Wait for the parent to exit. By the time join returns, the child # process is orphaned, and should be in the process of exiting by # itself. parent.join() # By the time we reach here,the parent has exited. The parent only exits # when the child is ready to enter the sleep, so all of the slow things # (process startup, etc) should have happened by now, even on a busy # system. A short timeout should therefore be appropriate. short_timeout = 5. # Poll is used to allow other tests to proceed after this one in case of # test failure. try: readable = children_alive.poll(short_timeout) except EnvironmentError: # Windows can raise BrokenPipeError. EnvironmentError is caught for # Python2/3 portability. assert sys.platform.startswith('win'), "should only raise on windows" # Broken pipe implies closed, which is readable. readable = True # If this assert fires, then something went wrong. Either the child # should write into the pipe, or it should exit and the pipe should be # closed (which makes it become readable). assert readable try: # This won't block due to the above 'assert readable'. result = children_alive.recv() except EOFError: pass # Test passes. except EnvironmentError: # Windows can raise BrokenPipeError. EnvironmentError is caught for # Python2/3 portability. assert sys.platform.startswith('win'), "should only raise on windows" # Test passes. else: # Oops, children_alive read something. It should be closed. If # something was read, it's a message from the child telling us they # are still alive! raise RuntimeError("unreachable: {}".format(result)) finally: # Cleanup. children_alive.close() distributed-1.20.2/distributed/tests/test_batched.py000066400000000000000000000147241321233345200226130ustar00rootroot00000000000000 from contextlib import contextmanager from datetime import timedelta import gc import random import sys import weakref import pytest from toolz import assoc from tornado import gen from distributed.batched import BatchedSend from distributed.core import listen, connect, CommClosedError from distributed.metrics import time from distributed.utils import All from distributed.utils_test import gen_test, slow, gen_cluster class EchoServer(object): count = 0 @gen.coroutine def handle_comm(self, comm): while True: try: msg = yield comm.read() self.count += 1 yield comm.write(msg) except CommClosedError as e: return def listen(self): listener = listen('', self.handle_comm) listener.start() self.address = listener.contact_address self.stop = listener.stop @contextmanager def echo_server(): server = EchoServer() server.listen() try: yield server finally: server.stop() @gen_test() def test_BatchedSend(): with echo_server() as e: comm = yield connect(e.address) b = BatchedSend(interval=10) assert str(len(b.buffer)) in str(b) assert str(len(b.buffer)) in repr(b) b.start(comm) yield gen.sleep(0.020) b.send('hello') b.send('hello') b.send('world') yield gen.sleep(0.020) b.send('HELLO') b.send('HELLO') result = yield comm.read() assert result == ['hello', 'hello', 'world'] result = yield comm.read() assert result == ['HELLO', 'HELLO'] assert b.byte_count > 1 @gen_test() def test_send_before_start(): with echo_server() as e: comm = yield connect(e.address) b = BatchedSend(interval=10) b.send('hello') b.send('world') b.start(comm) result = yield comm.read() assert result == ['hello', 'world'] @gen_test() def test_send_after_stream_start(): with echo_server() as e: comm = yield connect(e.address) b = BatchedSend(interval=10) b.start(comm) b.send('hello') b.send('world') result = yield comm.read() if len(result) < 2: result += yield comm.read() assert result == ['hello', 'world'] @gen_test() def test_send_before_close(): with echo_server() as e: comm = yield connect(e.address) b = BatchedSend(interval=10) b.start(comm) cnt = int(e.count) b.send('hello') yield b.close() # close immediately after sending assert not b.buffer start = time() while e.count != cnt + 1: yield gen.sleep(0.01) assert time() < start + 5 with pytest.raises(CommClosedError): b.send('123') @gen_test() def test_close_closed(): with echo_server() as e: comm = yield connect(e.address) b = BatchedSend(interval=10) b.start(comm) b.send(123) comm.close() # external closing yield b.close() @gen_test() def test_close_not_started(): b = BatchedSend(interval=10) yield b.close() @gen_test() def test_close_twice(): with echo_server() as e: comm = yield connect(e.address) b = BatchedSend(interval=10) b.start(comm) yield b.close() yield b.close() @slow @gen_test(timeout=50) def test_stress(): with echo_server() as e: comm = yield connect(e.address) L = [] @gen.coroutine def send(): b = BatchedSend(interval=3) b.start(comm) for i in range(0, 10000, 2): b.send(i) b.send(i + 1) yield gen.sleep(0.00001 * random.randint(1, 10)) @gen.coroutine def recv(): while True: result = yield gen.with_timeout(timedelta(seconds=1), comm.read()) L.extend(result) if result[-1] == 9999: break yield All([send(), recv()]) assert L == list(range(0, 10000, 1)) comm.close() @gen.coroutine def run_traffic_jam(nsends, nbytes): # This test eats `nsends * nbytes` bytes in RAM np = pytest.importorskip('numpy') from distributed.protocol import to_serialize data = bytes(np.random.randint(0, 255, size=(nbytes,)).astype('u1').data) with echo_server() as e: comm = yield connect(e.address) b = BatchedSend(interval=0.01) b.start(comm) msg = {'x': to_serialize(data)} for i in range(nsends): b.send(assoc(msg, 'i', i)) if np.random.random() > 0.5: yield gen.sleep(0.001) results = [] count = 0 while len(results) < nsends: # If this times out then I think it's a backpressure issue # Somehow we're able to flood the socket so that the receiving end # loses some of our messages L = yield gen.with_timeout(timedelta(seconds=5), comm.read()) count += 1 results.extend(r['i'] for r in L) assert count == b.batch_count == e.count assert b.message_count == nsends assert results == list(range(nsends)) comm.close() # external closing yield b.close() @gen_test() def test_sending_traffic_jam(): yield run_traffic_jam(50, 300000) @slow @gen_test() def test_large_traffic_jam(): yield run_traffic_jam(500, 1500000) @gen_cluster(client=True) def test_dont_hold_on_to_large_messages(c, s, a, b): np = pytest.importorskip('numpy') da = pytest.importorskip('dask.array') x = np.random.random(1000000) xr = weakref.ref(x) d = da.from_array(x, chunks=(100000,)) d = d.persist() del x start = time() while xr() is not None: if time() > start + 1: # Help diagnosing from types import FrameType x = xr() if x is not None: del x rc = sys.getrefcount(xr()) refs = gc.get_referrers(xr()) print("refs to x:", rc, refs, gc.isenabled()) frames = [r for r in refs if isinstance(r, FrameType)] for i, f in enumerate(frames): print("frames #%d:" % i, f.f_code.co_name, f.f_code.co_filename, sorted(f.f_locals)) pytest.fail("array should have been destroyed") yield gen.sleep(0.05) distributed-1.20.2/distributed/tests/test_client.py000066400000000000000000004244121321233345200224760ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from operator import add from collections import Iterator, deque from concurrent.futures import CancelledError import gc import itertools import logging import os import pickle import random import sys import threading from threading import Thread, Semaphore from time import sleep import traceback import warnings import weakref import zipfile import pytest from toolz import (identity, isdistinct, concat, pluck, valmap, partial, first) from tornado import gen from tornado.ioloop import IOLoop import dask from dask import delayed from dask.context import _globals from distributed import (Worker, Nanny, fire_and_forget, get_client, secede, get_worker, Executor, profile) from distributed.comm import CommClosedError from distributed.client import (Client, Future, wait, as_completed, tokenize, _get_global_client, default_client, ensure_default_get, futures_of, temp_default_client) from distributed.compatibility import PY3 from distributed.metrics import time from distributed.scheduler import Scheduler, KilledWorker from distributed.sizeof import sizeof from distributed.utils import ignoring, mp_context, sync, tmp_text, tokey from distributed.utils_test import (cluster, slow, slowinc, slowadd, slowdec, randominc, inc, dec, div, throws, geninc, asyncinc, gen_cluster, gen_test, double, deep, popen, captured_logger, varying, map_varying, wait_for, async_wait_for) from distributed.utils_test import loop, loop_in_thread, nodebug # flake8: noqa @gen_cluster(client=True, timeout=None) def test_submit(c, s, a, b): x = c.submit(inc, 10) assert not x.done() assert isinstance(x, Future) assert x.client is c result = yield x assert result == 11 assert x.done() y = c.submit(inc, 20) z = c.submit(add, x, y) result = yield z assert result == 11 + 21 s.validate_state() @gen_cluster(client=True) def test_map(c, s, a, b): L1 = c.map(inc, range(5)) assert len(L1) == 5 assert isdistinct(x.key for x in L1) assert all(isinstance(x, Future) for x in L1) result = yield L1[0] assert result == inc(0) assert len(s.tasks) == 5 L2 = c.map(inc, L1) result = yield L2[1] assert result == inc(inc(1)) assert len(s.tasks) == 10 # assert L1[0].key in s.tasks[L2[0].key] total = c.submit(sum, L2) result = yield total assert result == sum(map(inc, map(inc, range(5)))) L3 = c.map(add, L1, L2) result = yield L3[1] assert result == inc(1) + inc(inc(1)) L4 = c.map(add, range(3), range(4)) results = yield c.gather(L4) if sys.version_info[0] >= 3: assert results == list(map(add, range(3), range(4))) def f(x, y=10): return x + y L5 = c.map(f, range(5), y=5) results = yield c.gather(L5) assert results == list(range(5, 10)) y = c.submit(f, 10) L6 = c.map(f, range(5), y=y) results = yield c.gather(L6) assert results == list(range(20, 25)) s.validate_state() @gen_cluster(client=True) def test_map_empty(c, s, a, b): L1 = c.map(inc, [], pure=False) assert len(L1) == 0 results = yield c.gather(L1) assert results == [] @gen_cluster(client=True) def test_map_keynames(c, s, a, b): futures = c.map(inc, range(4), key='INC') assert all(f.key.startswith('INC') for f in futures) assert isdistinct(f.key for f in futures) futures2 = c.map(inc, [5, 6, 7, 8], key='INC') assert [f.key for f in futures] != [f.key for f in futures2] keys = ['inc-1', 'inc-2', 'inc-3', 'inc-4'] futures = c.map(inc, range(4), key=keys) assert [f.key for f in futures] == keys @gen_cluster(client=True) def test_map_retries(c, s, a, b): args = [[ZeroDivisionError("one"), 2, 3], [4, 5, 6], [ZeroDivisionError("seven"), ZeroDivisionError("eight"), 9]] x, y, z = c.map(*map_varying(args), retries=2) assert (yield x) == 2 assert (yield y) == 4 assert (yield z) == 9 x, y, z = c.map(*map_varying(args), retries=1, pure=False) assert (yield x) == 2 assert (yield y) == 4 with pytest.raises(ZeroDivisionError) as exc_info: yield z exc_info.match("eight") x, y, z = c.map(*map_varying(args), retries=0, pure=False) with pytest.raises(ZeroDivisionError) as exc_info: yield x exc_info.match("one") assert (yield y) == 4 with pytest.raises(ZeroDivisionError) as exc_info: yield z exc_info.match("seven") @gen_cluster(client=True) def test_compute_retries(c, s, a, b): args = [ZeroDivisionError("one"), ZeroDivisionError("two"), 3] # Sanity check for varying() use x = c.compute(delayed(varying(args))()) with pytest.raises(ZeroDivisionError) as exc_info: yield x exc_info.match("one") # Same retries for all x = c.compute(delayed(varying(args))(), retries=1) with pytest.raises(ZeroDivisionError) as exc_info: yield x exc_info.match("two") x = c.compute(delayed(varying(args))(), retries=2) assert (yield x) == 3 args.append(4) x = c.compute(delayed(varying(args))(), retries=2) assert (yield x) == 3 # Per-future retries xargs = [ZeroDivisionError("one"), ZeroDivisionError("two"), 30, 40] yargs = [ZeroDivisionError("five"), ZeroDivisionError("six"), 70] zargs = [80, 90, 100] x, y = [delayed(varying(args))() for args in (xargs, yargs)] x, y = c.compute([x, y], retries={x: 2}) gc.collect() assert (yield x) == 30 with pytest.raises(ZeroDivisionError) as exc_info: yield y exc_info.match("five") x, y, z = [delayed(varying(args))() for args in (xargs, yargs, zargs)] x, y, z = c.compute([x, y, z], retries={(y, z): 2}) with pytest.raises(ZeroDivisionError) as exc_info: yield x exc_info.match("one") assert (yield y) == 70 assert (yield z) == 80 @gen_cluster(client=True) def test_compute_persisted_retries(c, s, a, b): args = [ZeroDivisionError("one"), ZeroDivisionError("two"), 3] # Sanity check x = c.persist(delayed(varying(args))()) fut = c.compute(x) with pytest.raises(ZeroDivisionError) as exc_info: yield fut exc_info.match("one") x = c.persist(delayed(varying(args))()) fut = c.compute(x, retries=1) with pytest.raises(ZeroDivisionError) as exc_info: yield fut exc_info.match("two") x = c.persist(delayed(varying(args))()) fut = c.compute(x, retries=2) assert (yield fut) == 3 args.append(4) x = c.persist(delayed(varying(args))()) fut = c.compute(x, retries=3) assert (yield fut) == 3 @gen_cluster(client=True) def test_persist_retries(c, s, a, b): # Same retries for all args = [ZeroDivisionError("one"), ZeroDivisionError("two"), 3] x = c.persist(delayed(varying(args))(), retries=1) x = c.compute(x) with pytest.raises(ZeroDivisionError) as exc_info: yield x exc_info.match("two") x = c.persist(delayed(varying(args))(), retries=2) x = c.compute(x) assert (yield x) == 3 # Per-key retries xargs = [ZeroDivisionError("one"), ZeroDivisionError("two"), 30, 40] yargs = [ZeroDivisionError("five"), ZeroDivisionError("six"), 70] zargs = [80, 90, 100] x, y, z = [delayed(varying(args))() for args in (xargs, yargs, zargs)] x, y, z = c.persist([x, y, z], retries={(y, z): 2}) x, y, z = c.compute([x, y, z]) with pytest.raises(ZeroDivisionError) as exc_info: yield x exc_info.match("one") assert (yield y) == 70 assert (yield z) == 80 @gen_cluster(client=True) def test_future_repr(c, s, a, b): for func in [repr, lambda x: x._repr_html_()]: x = c.submit(inc, 10) assert str(x.key) in func(x) assert str(x.status) in func(x) assert str(x.status) in repr(c.futures[x.key]) @gen_cluster(client=True) def test_Future_exception(c, s, a, b): x = c.submit(div, 1, 0) result = yield x.exception() assert isinstance(result, ZeroDivisionError) x = c.submit(div, 1, 1) result = yield x.exception() assert result is None def test_Future_exception_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(div, 1, 0) assert isinstance(x.exception(), ZeroDivisionError) x = c.submit(div, 1, 1) assert x.exception() is None @gen_cluster(client=True) def test_Future_release(c, s, a, b): # Released Futures should be removed timely from the Client x = c.submit(div, 1, 1) yield x x.release() yield gen.moment assert not c.futures x = c.submit(slowinc, 1, delay=0.5) x.release() yield gen.moment assert not c.futures x = c.submit(div, 1, 0) yield x.exception() x.release() yield gen.moment assert not c.futures def test_Future_release_sync(loop): # Released Futures should be removed timely from the Client with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(div, 1, 1) x.result() x.release() wait_for(lambda: not c.futures, timeout=0.3) x = c.submit(slowinc, 1, delay=0.8) x.release() wait_for(lambda: not c.futures, timeout=0.3) x = c.submit(div, 1, 0) x.exception() x.release() wait_for(lambda: not c.futures, timeout=0.3) def test_short_tracebacks(loop): tblib = pytest.importorskip('tblib') with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: future = c.submit(div, 1, 0) try: future.result() except Exception: _, _, tb = sys.exc_info() tb = tblib.Traceback(tb).to_dict() n = 0 while tb != None: n += 1 tb = tb['tb_next'] assert n < 5 @gen_cluster(client=True) def test_map_naming(c, s, a, b): L1 = c.map(inc, range(5)) L2 = c.map(inc, range(5)) assert [x.key for x in L1] == [x.key for x in L2] L3 = c.map(inc, [1, 1, 1, 1]) assert len({x._state for x in L3}) == 1 L4 = c.map(inc, [1, 1, 1, 1], pure=False) assert len({x._state for x in L4}) == 4 @gen_cluster(client=True) def test_submit_naming(c, s, a, b): a = c.submit(inc, 1) b = c.submit(inc, 1) assert a._state is b._state c = c.submit(inc, 1, pure=False) assert c.key != a.key @gen_cluster(client=True) def test_exceptions(c, s, a, b): x = c.submit(div, 1, 2) result = yield x assert result == 1 / 2 x = c.submit(div, 1, 0) with pytest.raises(ZeroDivisionError): result = yield x x = c.submit(div, 10, 2) # continues to operate result = yield x assert result == 10 / 2 @gen_cluster() def test_gc(s, a, b): c = yield Client((s.ip, s.port), asynchronous=True) x = c.submit(inc, 10) yield x assert s.who_has[x.key] x.__del__() yield async_wait_for(lambda: x.key not in s.who_has, timeout=0.3) yield c.close() def test_thread(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(inc, 1) assert x.result() == 2 x = c.submit(slowinc, 1, delay=0.3) with pytest.raises(gen.TimeoutError): x.result(timeout=0.01) assert x.result() == 2 def test_sync_exceptions(loop): with cluster() as (s, [a, b]): c = Client(s['address'], loop=loop) x = c.submit(div, 10, 2) assert x.result() == 5 y = c.submit(div, 10, 0) try: y.result() assert False except ZeroDivisionError: pass z = c.submit(div, 10, 5) assert z.result() == 2 c.close() @gen_cluster(client=True) def test_gather(c, s, a, b): x = c.submit(inc, 10) y = c.submit(inc, x) result = yield c.gather(x) assert result == 11 result = yield c.gather([x]) assert result == [11] result = yield c.gather({'x': x, 'y': [y]}) assert result == {'x': 11, 'y': [12]} @gen_cluster(client=True) def test_gather_lost(c, s, a, b): [x] = yield c.scatter([1], workers=a.address) y = c.submit(inc, 1, workers=b.address) yield a._close() with pytest.raises(Exception): yield c.gather([x, y]) def test_gather_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(inc, 1) assert c.gather(x) == 2 y = c.submit(div, 1, 0) with pytest.raises(ZeroDivisionError): c.gather([x, y]) [xx] = c.gather([x, y], errors='skip') assert xx == 2 @gen_cluster(client=True) def test_gather_strict(c, s, a, b): x = c.submit(div, 2, 1) y = c.submit(div, 1, 0) with pytest.raises(ZeroDivisionError): yield c.gather([x, y]) [xx] = yield c.gather([x, y], errors='skip') assert xx == 2 @gen_cluster(client=True, timeout=None) def test_get(c, s, a, b): future = c.get({'x': (inc, 1)}, 'x', sync=False) assert isinstance(future, Future) result = yield future assert result == 2 futures = c.get({'x': (inc, 1)}, ['x'], sync=False) assert isinstance(futures[0], Future) result = yield futures assert result == [2] result = yield c.get({}, [], sync=False) assert result == [] result = yield c.get({('x', 1): (inc, 1), ('x', 2): (inc, ('x', 1))}, ('x', 2), sync=False) assert result == 3 def test_get_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: assert c.get({'x': (inc, 1)}, 'x') == 2 def test_no_future_references(loop): from weakref import WeakSet ws = WeakSet() with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: futures = c.map(inc, range(10)) ws.update(futures) del futures import gc gc.collect() start = time() while list(ws): sleep(0.01) assert time() < start + 2 def test_get_sync_optimize_graph_passes_through(loop): import dask.bag as db import dask bag = db.range(10, npartitions=3).map(inc) with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: dask.compute(bag.sum(), optimize_graph=False, get=c.get) @gen_cluster(client=True) def test_gather_errors(c, s, a, b): def f(a, b): raise TypeError def g(a, b): raise AttributeError future_f = c.submit(f, 1, 2) future_g = c.submit(g, 1, 2) with pytest.raises(TypeError): yield c.gather(future_f) with pytest.raises(AttributeError): yield c.gather(future_g) yield a._close() @gen_cluster(client=True) def test_wait(c, s, a, b): x = c.submit(inc, 1) y = c.submit(inc, 1) z = c.submit(inc, 2) done, not_done = yield wait([x, y, z]) assert done == {x, y, z} assert not_done == set() assert x.status == y.status == 'finished' @gen_cluster(client=True, timeout=2) def test_wait_timeout(c, s, a, b): future = c.submit(sleep, 0.3) with pytest.raises(gen.TimeoutError): yield wait(future, timeout=0.01) def test_wait_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(inc, 1) y = c.submit(inc, 2) done, not_done = wait([x, y]) assert done == {x, y} assert not_done == set() assert x.status == y.status == 'finished' future = c.submit(sleep, 0.3) with pytest.raises(gen.TimeoutError): wait(future, timeout=0.01) def test_wait_informative_error_for_timeouts(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(inc, 1) y = c.submit(inc, 2) try: wait(x, y) except Exception as e: assert "timeout" in str(e) assert "list" in str(e) @gen_cluster(client=True) def test_garbage_collection(c, s, a, b): x = c.submit(inc, 1) y = c.submit(inc, 1) assert c.refcount[x.key] == 2 x.__del__() yield gen.moment assert c.refcount[x.key] == 1 z = c.submit(inc, y) y.__del__() yield gen.moment result = yield z assert result == 3 ykey = y.key y.__del__() yield gen.moment assert ykey not in c.futures @gen_cluster(client=True) def test_garbage_collection_with_scatter(c, s, a, b): [future] = yield c.scatter([1]) assert future.key in c.futures assert future.status == 'finished' assert s.who_wants[future.key] == {c.id} assert c.refcount[future.key] == 1 future.__del__() yield gen.moment assert c.refcount[future.key] == 0 start = time() while True: if future.key not in s.who_has: break else: assert time() < start + 3 yield gen.sleep(0.1) @gen_cluster(timeout=1000, client=True) def test_recompute_released_key(c, s, a, b): x = c.submit(inc, 100) result1 = yield x xkey = x.key del x import gc gc.collect() yield gen.moment assert c.refcount[xkey] == 0 # 1 second batching needs a second action to trigger while xkey in s.who_has or xkey in a.data or xkey in b.data: yield gen.sleep(0.1) x = c.submit(inc, 100) assert x.key in c.futures result2 = yield x assert result1 == result2 @slow @gen_cluster(client=True) def test_long_tasks_dont_trigger_timeout(c, s, a, b): from time import sleep x = c.submit(sleep, 3) yield x @pytest.mark.skip @gen_cluster(client=True) def test_missing_data_heals(c, s, a, b): a.validate = False b.validate = False x = c.submit(inc, 1) y = c.submit(inc, x) z = c.submit(inc, y) yield wait([x, y, z]) # Secretly delete y's key if y.key in a.data: del a.data[y.key] a.release_key(y.key) if y.key in b.data: del b.data[y.key] b.release_key(y.key) yield gen.moment w = c.submit(add, y, z) result = yield w assert result == 3 + 4 @pytest.mark.xfail(reason="Test creates inconsistent scheduler state") @slow @gen_cluster() def test_missing_worker(s, a, b): bad = 'bad-host:8788' s.ncores[bad] = 4 s.who_has['b'] = {bad} s.has_what[bad] = {'b'} c = yield Client((s.ip, s.port), asynchronous=True) dsk = {'a': 1, 'b': (inc, 'a'), 'c': (inc, 'b')} result = yield c.get(dsk, 'c', sync=False) assert result == 3 assert bad not in s.ncores yield c.close() @pytest.mark.skip @gen_cluster(client=True) def test_gather_robust_to_missing_data(c, s, a, b): a.validate = False b.validate = False x, y, z = c.map(inc, range(3)) yield wait([x, y, z]) # everything computed for f in [x, y]: for w in [a, b]: if f.key in w.data: del w.data[f.key] yield gen.moment w.release_key(f.key) xx, yy, zz = yield c.gather([x, y, z]) assert (xx, yy, zz) == (1, 2, 3) @pytest.mark.skip @gen_cluster(client=True) def test_gather_robust_to_nested_missing_data(c, s, a, b): a.validate = False b.validate = False w = c.submit(inc, 1) x = c.submit(inc, w) y = c.submit(inc, x) z = c.submit(inc, y) yield wait([z]) for worker in [a, b]: for datum in [y, z]: if datum.key in worker.data: del worker.data[datum.key] yield gen.moment worker.release_key(datum.key) result = yield c.gather([z]) assert result == [inc(inc(inc(inc(1))))] @gen_cluster(client=True) def test_tokenize_on_futures(c, s, a, b): x = c.submit(inc, 1) y = c.submit(inc, 1) tok = tokenize(x) assert tokenize(x) == tokenize(x) assert tokenize(x) == tokenize(y) c.futures[x.key].finish() assert tok == tokenize(y) @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster([('127.0.0.1', 1), ('127.0.0.2', 2)], client=True) def test_restrictions_submit(c, s, a, b): x = c.submit(inc, 1, workers={a.ip}) y = c.submit(inc, x, workers={b.ip}) yield wait([x, y]) assert s.host_restrictions[x.key] == {a.ip} assert x.key in a.data assert s.host_restrictions[y.key] == {b.ip} assert y.key in b.data @gen_cluster(client=True) def test_restrictions_ip_port(c, s, a, b): x = c.submit(inc, 1, workers={a.address}) y = c.submit(inc, x, workers={b.address}) yield wait([x, y]) assert s.worker_restrictions[x.key] == {a.address} assert x.key in a.data assert s.worker_restrictions[y.key] == {b.address} assert y.key in b.data @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster([('127.0.0.1', 1), ('127.0.0.2', 2)], client=True) def test_restrictions_map(c, s, a, b): L = c.map(inc, range(5), workers={a.ip}) yield wait(L) assert set(a.data) == {x.key for x in L} assert not b.data for x in L: assert s.host_restrictions[x.key] == {a.ip} L = c.map(inc, [10, 11, 12], workers=[{a.ip}, {a.ip, b.ip}, {b.ip}]) yield wait(L) assert s.host_restrictions[L[0].key] == {a.ip} assert s.host_restrictions[L[1].key] == {a.ip, b.ip} assert s.host_restrictions[L[2].key] == {b.ip} with pytest.raises(ValueError): c.map(inc, [10, 11, 12], workers=[{a.ip}]) @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster([('127.0.0.1', 1), ('127.0.0.2', 2)], client=True) def test_restrictions_get(c, s, a, b): dsk = {'x': 1, 'y': (inc, 'x'), 'z': (inc, 'y')} restrictions = {'y': {a.ip}, 'z': {b.ip}} futures = c.get(dsk, ['y', 'z'], restrictions, sync=False) result = yield futures assert result == [2, 3] assert 'y' in a.data assert 'z' in b.data @gen_cluster(client=True) def dont_test_bad_restrictions_raise_exception(c, s, a, b): z = c.submit(inc, 2, workers={'bad-address'}) try: yield z assert False except ValueError as e: assert 'bad-address' in str(e) assert z.key in str(e) @gen_cluster(client=True, timeout=None) def test_remove_worker(c, s, a, b): L = c.map(inc, range(20)) yield wait(L) yield b._close() assert b.address not in s.worker_info result = yield c.gather(L) assert result == list(map(inc, range(20))) @gen_cluster(ncores=[('127.0.0.1', 1)], client=True) def test_errors_dont_block(c, s, w): L = [c.submit(inc, 1), c.submit(throws, 1), c.submit(inc, 2), c.submit(throws, 2)] start = time() while not (L[0].status == L[2].status == 'finished'): assert time() < start + 5 yield gen.sleep(0.01) result = yield c.gather([L[0], L[2]]) assert result == [2, 3] @gen_cluster(client=True) def test_submit_quotes(c, s, a, b): def assert_list(x, z=[]): return isinstance(x, list) and isinstance(z, list) x = c.submit(assert_list, [1, 2, 3]) result = yield x assert result x = c.submit(assert_list, [1, 2, 3], z=[4, 5, 6]) result = yield x assert result x = c.submit(inc, 1) y = c.submit(inc, 2) z = c.submit(assert_list, [x, y]) result = yield z assert result @gen_cluster(client=True) def test_map_quotes(c, s, a, b): def assert_list(x, z=[]): return isinstance(x, list) and isinstance(z, list) L = c.map(assert_list, [[1, 2, 3], [4]]) result = yield c.gather(L) assert all(result) L = c.map(assert_list, [[1, 2, 3], [4]], z=[10]) result = yield c.gather(L) assert all(result) L = c.map(assert_list, [[1, 2, 3], [4]], [[]] * 3) result = yield c.gather(L) assert all(result) @gen_cluster() def test_two_consecutive_clients_share_results(s, a, b): c = yield Client((s.ip, s.port), asynchronous=True) x = c.submit(random.randint, 0, 1000, pure=True) xx = yield x f = yield Client((s.ip, s.port), asynchronous=True) y = f.submit(random.randint, 0, 1000, pure=True) yy = yield y assert xx == yy yield c.close() yield f.close() @gen_cluster(client=True) def test_submit_then_get_with_Future(c, s, a, b): x = c.submit(slowinc, 1) dsk = {'y': (inc, x)} result = yield c.get(dsk, 'y', sync=False) assert result == 3 @gen_cluster(client=True) def test_aliases(c, s, a, b): x = c.submit(inc, 1) dsk = {'y': x} result = yield c.get(dsk, 'y', sync=False) assert result == 2 @gen_cluster(client=True) def test_aliases_2(c, s, a, b): dsk_keys = [ ({'x': (inc, 1), 'y': 'x', 'z': 'x', 'w': (add, 'y', 'z')}, ['y', 'w']), ({'x': 'y', 'y': 1}, ['x']), ({'x': 1, 'y': 'x', 'z': 'y', 'w': (inc, 'z')}, ['w'])] for dsk, keys in dsk_keys: result = yield c.get(dsk, keys, sync=False) assert list(result) == list(dask.get(dsk, keys)) yield gen.moment @gen_cluster(client=True) def test__scatter(c, s, a, b): d = yield c.scatter({'y': 20}) assert isinstance(d['y'], Future) assert a.data.get('y') == 20 or b.data.get('y') == 20 assert (a.address in s.who_has['y'] or b.address in s.who_has['y']) assert s.who_has['y'] assert s.nbytes == {'y': sizeof(20)} yy = yield c.gather([d['y']]) assert yy == [20] [x] = yield c.scatter([10]) assert isinstance(x, Future) assert a.data.get(x.key) == 10 or b.data.get(x.key) == 10 xx = yield c.gather([x]) assert s.who_has[x.key] assert (a.address in s.who_has[x.key] or b.address in s.who_has[x.key]) assert s.nbytes == {'y': sizeof(20), x.key: sizeof(10)} assert xx == [10] z = c.submit(add, x, d['y']) # submit works on Future result = yield z assert result == 10 + 20 result = yield c.gather([z, x]) assert result == [30, 10] @gen_cluster(client=True) def test__scatter_types(c, s, a, b): d = yield c.scatter({'x': 1}) assert isinstance(d, dict) assert list(d) == ['x'] for seq in [[1], (1,), {1}, frozenset([1])]: L = yield c.scatter(seq) assert isinstance(L, type(seq)) assert len(L) == 1 s.validate_state() seq = yield c.scatter(range(5)) assert isinstance(seq, list) assert len(seq) == 5 s.validate_state() @gen_cluster(client=True) def test__scatter_non_list(c, s, a, b): x = yield c.scatter(1) assert isinstance(x, Future) result = yield x assert result == 1 @gen_cluster(client=True) def test_scatter_hash(c, s, a, b): [a] = yield c.scatter([1]) [b] = yield c.scatter([1]) assert a.key == b.key s.validate_state() @gen_cluster(client=True) def test_scatter_tokenize_local(c, s, a, b): from dask.base import normalize_token class MyObj(object): pass L = [] @normalize_token.register(MyObj) def f(x): L.append(x) return 'x' obj = MyObj() future = yield c.scatter(obj) assert L and L[0] is obj @gen_cluster(client=True) def test_scatter_singletons(c, s, a, b): np = pytest.importorskip('numpy') pd = pytest.importorskip('pandas') for x in [1, np.ones(5), pd.DataFrame({'x': [1, 2, 3]})]: future = yield c.scatter(x) result = yield future assert str(result) == str(x) @gen_cluster(client=True) def test_scatter_typename(c, s, a, b): future = yield c.scatter(123) assert future.key.startswith('int') @gen_cluster(client=True) def test_scatter_hash(c, s, a, b): x = yield c.scatter(123) y = yield c.scatter(123) assert x.key == y.key z = yield c.scatter(123, hash=False) assert z.key != y.key @gen_cluster(client=True) def test_get_releases_data(c, s, a, b): [x] = yield c.get({'x': (inc, 1)}, ['x'], sync=False) import gc gc.collect() yield gen.moment assert c.refcount['x'] == 0 def test_Current(): with cluster() as (s, [a, b]): with Client(s['address']) as c: assert Client.current() is c with pytest.raises(ValueError): Client.current() with Client(s['address']) as c: assert Client.current() is c def test_global_clients(loop): assert _get_global_client() is None with pytest.raises(ValueError): default_client() with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: assert _get_global_client() is c assert default_client() is c with Client(s['address'], loop=loop) as f: assert _get_global_client() is f assert default_client() is f assert default_client(c) is c assert default_client(f) is f assert _get_global_client() is None @gen_cluster(client=True) def test_exception_on_exception(c, s, a, b): x = c.submit(lambda: 1 / 0) y = c.submit(inc, x) with pytest.raises(ZeroDivisionError): yield y z = c.submit(inc, y) with pytest.raises(ZeroDivisionError): yield z @gen_cluster(client=True) def test_nbytes(c, s, a, b): [x] = yield c.scatter([1]) assert s.nbytes == {x.key: sizeof(1)} y = c.submit(inc, x) yield y assert s.nbytes == {x.key: sizeof(1), y.key: sizeof(2)} @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster([('127.0.0.1', 1), ('127.0.0.2', 2)], client=True) def test_nbytes_determines_worker(c, s, a, b): x = c.submit(identity, 1, workers=[a.ip]) y = c.submit(identity, tuple(range(100)), workers=[b.ip]) yield c.gather([x, y]) z = c.submit(lambda x, y: None, x, y) yield z assert s.who_has[z.key] == {b.address} @gen_cluster(client=True) def test_if_intermediates_clear_on_error(c, s, a, b): x = delayed(div, pure=True)(1, 0) y = delayed(div, pure=True)(1, 2) z = delayed(add, pure=True)(x, y) f = c.compute(z) with pytest.raises(ZeroDivisionError): yield f s.validate_state() assert not s.who_has @gen_cluster(client=True) def test_pragmatic_move_small_data_to_large_data(c, s, a, b): np = pytest.importorskip('numpy') lists = c.map(np.ones, [10000] * 10, pure=False) sums = c.map(np.sum, lists) total = c.submit(sum, sums) def f(x, y): return None s.task_duration['f'] = 0.001 results = c.map(f, lists, [total] * 10) yield wait([total]) yield wait(results) assert sum(s.who_has[r.key].issubset(s.who_has[l.key]) for l, r in zip(lists, results)) >= 9 @gen_cluster(client=True) def test_get_with_non_list_key(c, s, a, b): dsk = {('x', 0): (inc, 1), 5: (inc, 2)} x = yield c.get(dsk, ('x', 0), sync=False) y = yield c.get(dsk, 5, sync=False) assert x == 2 assert y == 3 @gen_cluster(client=True) def test_get_with_error(c, s, a, b): dsk = {'x': (div, 1, 0), 'y': (inc, 'x')} with pytest.raises(ZeroDivisionError): yield c.get(dsk, 'y', sync=False) def test_get_with_error_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: dsk = {'x': (div, 1, 0), 'y': (inc, 'x')} with pytest.raises(ZeroDivisionError): c.get(dsk, 'y') @gen_cluster(client=True) def test_directed_scatter(c, s, a, b): yield c.scatter([1, 2, 3], workers=[a.address]) assert len(a.data) == 3 assert not b.data yield c.scatter([4, 5], workers=[b.name]) assert len(b.data) == 2 def test_directed_scatter_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: futures = c.scatter([1, 2, 3], workers=[b['address']]) has_what = sync(loop, c.scheduler.has_what) assert len(has_what[b['address']]) == len(futures) assert len(has_what[a['address']]) == 0 def test_iterator_scatter(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as 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(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as ee: from distributed.compatibility import Queue q = Queue() for d in range(10): q.put(d) futures = ee.scatter(q) assert isinstance(futures, Queue) a = futures.get() assert ee.gather(a) == 0 def test_queue_scatter_gather_maxsize(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as 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(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as ee: from distributed.compatibility import Queue q = Queue() qin = list(range(10)) for d in qin: q.put(d) futures = ee.scatter(q) assert isinstance(futures, Queue) ff = ee.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(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as ee: i_in = list(range(10)) g = (d for d in i_in) futures = ee.scatter(g) assert isinstance(futures, Iterator) ff = ee.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 = ee.scatter(g) ff = ee.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) assert future.key in a.data or future.key in b.data assert s.who_has[future.key] assert future.status == 'finished' result = yield future assert result == 123 assert not s.counters['op'].components[0]['scatter'] result = yield future assert not s.counters['op'].components[0]['gather'] result = yield c.gather(future) assert not s.counters['op'].components[0]['gather'] @gen_cluster(client=True) def test_scatter_direct_numpy(c, s, a, b): np = pytest.importorskip('numpy') x = np.ones(5) future = yield c.scatter(x, direct=True) result = yield future assert np.allclose(x, result) assert not s.counters['op'].components[0]['scatter'] @gen_cluster(client=True) def test_scatter_direct_broadcast(c, s, a, b): future2 = yield c.scatter(456, direct=True, broadcast=True) assert future2.key in a.data assert future2.key in b.data assert s.who_has[future2.key] == {a.address, b.address} result = yield future2 assert result == 456 assert not s.counters['op'].components[0]['scatter'] @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 4) def test_scatter_direct_balanced(c, s, *workers): futures = yield c.scatter([1, 2, 3], direct=True) assert sorted([len(w.data) for w in workers]) == [0, 1, 1, 1] @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 4) def test_scatter_direct_broadcast_target(c, s, *workers): futures = yield c.scatter([123, 456], direct=True, workers=workers[0].address) assert futures[0].key in workers[0].data assert futures[1].key in workers[0].data futures = yield c.scatter([123, 456], direct=True, broadcast=True, workers=[w.address for w in workers[:3]]) assert (f.key in w.data and w.address in s.who_has[f.key] for f in futures for w in workers[:3]) @gen_cluster(client=True, ncores=[]) def test_scatter_direct_empty(c, s): with pytest.raises((ValueError, gen.TimeoutError)): yield c.scatter(123, direct=True, timeout=0.1) @gen_cluster(client=True, timeout=None, ncores=[('127.0.0.1', 1)] * 5) def test_scatter_direct_spread_evenly(c, s, *workers): futures = [] for i in range(10): future = yield c.scatter(i, direct=True) futures.append(future) assert all(w.data for w in workers) @pytest.mark.parametrize('direct', [True, False]) @pytest.mark.parametrize('broadcast', [True, False]) def test_scatter_gather_sync(loop, direct, broadcast): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: futures = c.scatter([1, 2, 3], direct=direct, broadcast=broadcast) results = c.gather(futures, direct=direct) assert results == [1, 2, 3] @gen_cluster(client=True) def test_gather_direct(c, s, a, b): futures = yield c.scatter([1, 2, 3]) data = yield c.gather(futures, direct=True) assert data == [1, 2, 3] @gen_cluster(client=True) def test_many_submits_spread_evenly(c, s, a, b): L = [c.submit(inc, i) for i in range(10)] yield wait(L) assert a.data and b.data @gen_cluster(client=True) def test_traceback(c, s, a, b): x = c.submit(div, 1, 0) tb = yield x.traceback() if sys.version_info[0] >= 3: assert any('x / y' in line for line in pluck(3, traceback.extract_tb(tb))) @gen_cluster(client=True) def test_get_traceback(c, s, a, b): try: yield c.get({'x': (div, 1, 0)}, 'x', sync=False) except ZeroDivisionError: exc_type, exc_value, exc_traceback = sys.exc_info() L = traceback.format_tb(exc_traceback) assert any('x / y' in line for line in L) @gen_cluster(client=True) def test_gather_traceback(c, s, a, b): x = c.submit(div, 1, 0) try: yield c.gather(x) except ZeroDivisionError: exc_type, exc_value, exc_traceback = sys.exc_info() L = traceback.format_tb(exc_traceback) assert any('x / y' in line for line in L) def test_traceback_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(div, 1, 0) tb = x.traceback() if sys.version_info[0] >= 3: assert any('x / y' in line for line in concat(traceback.extract_tb(tb)) if isinstance(line, str)) y = c.submit(inc, x) tb2 = y.traceback() assert set(pluck(3, traceback.extract_tb(tb2))).issuperset( set(pluck(3, traceback.extract_tb(tb)))) z = c.submit(div, 1, 2) tb = z.traceback() assert tb is None @gen_cluster(client=True) def test_upload_file(c, s, a, b): def g(): import myfile return myfile.f() try: for value in [123, 456]: with tmp_text('myfile.py', 'def f():\n return {}'.format(value)) as fn: yield c.upload_file(fn) 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) def test_upload_file_zip(c, s, a, b): def g(): import myfile return myfile.f() 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: # 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 @gen_cluster(client=True) def test_upload_large_file(c, s, a, b): assert a.local_dir assert b.local_dir with tmp_text('myfile', 'abc') as fn: with tmp_text('myfile2', 'def') as fn2: yield c._upload_large_file(fn, remote_filename='x') yield c._upload_large_file(fn2) for w in [a, b]: assert os.path.exists(os.path.join(w.local_dir, 'x')) assert os.path.exists(os.path.join(w.local_dir, 'myfile2')) with open(os.path.join(w.local_dir, 'x')) as f: assert f.read() == 'abc' with open(os.path.join(w.local_dir, 'myfile2')) as f: assert f.read() == 'def' def test_upload_file_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: def g(): import myfile return myfile.x with tmp_text('myfile.py', 'x = 123') as fn: c.upload_file(fn) x = c.submit(g) assert x.result() == 123 @gen_cluster(client=True) def test_upload_file_exception(c, s, a, b): with tmp_text('myfile.py', 'syntax-error!') as fn: with pytest.raises(SyntaxError): yield c.upload_file(fn) def test_upload_file_exception_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: with tmp_text('myfile.py', 'syntax-error!') as fn: with pytest.raises(SyntaxError): c.upload_file(fn) @pytest.mark.xfail @gen_cluster() def test_multiple_clients(s, a, b): a = yield Client((s.ip, s.port), asynchronous=True) b = yield Client((s.ip, s.port), asynchronous=True) x = a.submit(inc, 1) y = b.submit(inc, 2) assert x.client is a assert y.client is b xx = yield x yy = yield y assert xx == 2 assert yy == 3 z = a.submit(add, x, y) assert z.client is a zz = yield z assert zz == 5 yield a.close() yield b.close() @gen_cluster(client=True) def test_async_compute(c, s, a, b): from dask.delayed import delayed x = delayed(1) y = delayed(inc)(x) z = delayed(dec)(x) [yy, zz, aa] = c.compute([y, z, 3], sync=False) assert isinstance(yy, Future) assert isinstance(zz, Future) assert aa == 3 result = yield c.gather([yy, zz]) assert result == [2, 0] assert isinstance(c.compute(y), Future) assert isinstance(c.compute([y]), (tuple, list)) @gen_cluster(client=True) def test_async_compute_with_scatter(c, s, a, b): d = yield c.scatter({('x', 1): 1, ('y', 1): 2}) x, y = d[('x', 1)], d[('y', 1)] from dask.delayed import delayed z = delayed(add)(delayed(inc)(x), delayed(inc)(y)) zz = c.compute(z) [result] = yield c.gather([zz]) assert result == 2 + 3 def test_sync_compute(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = delayed(1) y = delayed(inc)(x) z = delayed(dec)(x) yy, zz = c.compute([y, z], sync=True) assert (yy, zz) == (2, 0) @gen_cluster(client=True) def test_remote_scatter_gather(c, s, a, b): x, y, z = yield c.scatter([1, 2, 3]) assert x.key in a.data or x.key in b.data assert y.key in a.data or y.key in b.data assert z.key in a.data or z.key in b.data xx, yy, zz = yield c.gather([x, y, z]) assert (xx, yy, zz) == (1, 2, 3) @gen_cluster(timeout=1000, client=True) def test_remote_submit_on_Future(c, s, a, b): x = c.submit(lambda x: x + 1, 1) y = c.submit(lambda x: x + 1, x) result = yield y assert result == 3 def test_start_is_idempotent(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: c.start() c.start() c.start() x = c.submit(inc, 1) assert x.result() == 2 @gen_cluster(client=True) def test_client_with_scheduler(c, s, a, b): assert s.ncores == {a.address: a.ncores, b.address: b.ncores} x = c.submit(inc, 1) y = c.submit(inc, 2) z = c.submit(add, x, y) result = yield x assert result == 1 + 1 result = yield z assert result == 1 + 1 + 1 + 2 A, B, C = yield c.scatter([1, 2, 3]) AA, BB, xx = yield c.gather([A, B, x]) assert (AA, BB, xx) == (1, 2, 2) result = yield c.get({'x': (inc, 1), 'y': (add, 'x', 10)}, 'y', sync=False) assert result == 12 @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster([('127.0.0.1', 1), ('127.0.0.2', 2)], client=True) def test_allow_restrictions(c, s, a, b): x = c.submit(inc, 1, workers=a.ip) yield x assert s.who_has[x.key] == {a.address} assert not s.loose_restrictions x = c.submit(inc, 2, workers=a.ip, allow_other_workers=True) yield x assert s.who_has[x.key] == {a.address} assert x.key in s.loose_restrictions L = c.map(inc, range(3, 13), workers=a.ip, allow_other_workers=True) yield wait(L) assert all(s.who_has[f.key] == {a.address} for f in L) assert {f.key for f in L}.issubset(s.loose_restrictions) """ x = c.submit(inc, 14, workers='127.0.0.3') with ignoring(gen.TimeoutError): yield gen.with_timeout(timedelta(seconds=0.1), x assert False assert not s.who_has[x.key] assert x.key not in s.loose_restrictions """ x = c.submit(inc, 15, workers='127.0.0.3', allow_other_workers=True) yield x assert s.who_has[x.key] assert x.key in s.loose_restrictions L = c.map(inc, range(15, 25), workers='127.0.0.3', allow_other_workers=True) yield wait(L) assert all(s.who_has[f.key] for f in L) assert {f.key for f in L}.issubset(s.loose_restrictions) with pytest.raises(ValueError): c.submit(inc, 1, allow_other_workers=True) with pytest.raises(ValueError): c.map(inc, [1], allow_other_workers=True) with pytest.raises(TypeError): c.submit(inc, 20, workers='127.0.0.1', allow_other_workers='Hello!') with pytest.raises(TypeError): c.map(inc, [20], workers='127.0.0.1', allow_other_workers='Hello!') @pytest.mark.skipif('True', reason='because') def test_bad_address(): try: Client('123.123.123.123:1234', timeout=0.1) except (IOError, gen.TimeoutError) as e: assert "connect" in str(e).lower() try: Client('127.0.0.1:1234', timeout=0.1) except (IOError, gen.TimeoutError) as e: assert "connect" in str(e).lower() @gen_cluster(client=True) def test_long_error(c, s, a, b): def bad(x): raise ValueError('a' * 100000) x = c.submit(bad, 10) try: yield x except ValueError as e: assert len(str(e)) < 100000 tb = yield x.traceback() assert all(len(line) < 100000 for line in concat(traceback.extract_tb(tb)) if isinstance(line, str)) @gen_cluster(client=True) def test_map_on_futures_with_kwargs(c, s, a, b): def f(x, y=10): return x + y futures = c.map(inc, range(10)) futures2 = c.map(f, futures, y=20) results = yield c.gather(futures2) assert results == [i + 1 + 20 for i in range(10)] future = c.submit(inc, 100) future2 = c.submit(f, future, y=200) result = yield future2 assert result == 100 + 1 + 200 class BadlySerializedObject(object): def __getstate__(self): return 1 def __setstate__(self, state): raise TypeError("hello!") class FatallySerializedObject(object): def __getstate__(self): return 1 def __setstate__(self, state): print("This should never have been deserialized, closing") import sys sys.exit(0) @gen_cluster(client=True) def test_badly_serialized_input(c, s, a, b): o = BadlySerializedObject() future = c.submit(inc, o) futures = c.map(inc, range(10)) L = yield c.gather(futures) assert list(L) == list(map(inc, range(10))) assert future.status == 'error' @pytest.mark.skipif('True', reason="") def test_badly_serialized_input_stderr(capsys, loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: o = BadlySerializedObject() future = c.submit(inc, o) start = time() while True: sleep(0.01) out, err = capsys.readouterr() if 'hello!' in err: break assert time() - start < 20 assert future.status == 'error' def test_repr(loop): funcs = [str, repr, lambda x: x._repr_html_()] with cluster(nworkers=3) as (s, [a, b, c]): with Client(s['address'], loop=loop) as c: for func in funcs: text = func(c) assert c.scheduler.address in text assert '3' in text for func in funcs: text = func(c) assert 'not connected' in text @gen_cluster(client=True) def test_forget_simple(c, s, a, b): x = c.submit(inc, 1, retries=2) y = c.submit(inc, 2) z = c.submit(add, x, y, workers=[a.ip], allow_other_workers=True) yield wait([x, y, z]) assert not s.waiting_data[x.key] assert not s.waiting_data[y.key] assert set(s.tasks) == {x.key, y.key, z.key} s.client_releases_keys(keys=[x.key], client=c.id) assert x.key in s.tasks s.client_releases_keys(keys=[z.key], client=c.id) for coll in [s.tasks, s.dependencies, s.dependents, s.waiting, s.waiting_data, s.who_has, s.worker_restrictions, s.host_restrictions, s.loose_restrictions, s.released, s.priority, s.exceptions, s.tracebacks, s.who_wants, s.exceptions_blame, s.nbytes, s.task_state, s.retries]: assert x.key not in coll assert z.key not in coll assert z.key not in s.dependents[y.key] s.client_releases_keys(keys=[y.key], client=c.id) assert not s.tasks @gen_cluster(client=True) def test_forget_complex(e, s, A, B): a, b, c, d = yield e.scatter(list(range(4))) ab = e.submit(add, a, b) cd = e.submit(add, c, d) ac = e.submit(add, a, c) acab = e.submit(add, ac, ab) yield wait([a, b, c, d, ab, ac, cd, acab]) assert set(s.tasks) == {f.key for f in [ab, ac, cd, acab]} s.client_releases_keys(keys=[ab.key], client=e.id) assert set(s.tasks) == {f.key for f in [ab, ac, cd, acab]} s.client_releases_keys(keys=[b.key], client=e.id) assert set(s.tasks) == {f.key for f in [ac, cd, acab]} s.client_releases_keys(keys=[acab.key], client=e.id) assert set(s.tasks) == {f.key for f in [ac, cd]} assert b.key not in s.who_has start = time() while b.key in A.data or b.key in B.data: yield gen.sleep(0.01) assert time() < start + 10 s.client_releases_keys(keys=[ac.key], client=e.id) assert set(s.tasks) == {f.key for f in [cd]} @gen_cluster(client=True) def test_forget_in_flight(e, s, A, B): delayed2 = partial(delayed, pure=True) a, b, c, d = [delayed2(slowinc)(i) for i in range(4)] ab = delayed2(slowadd)(a, b) cd = delayed2(slowadd)(c, d) ac = delayed2(slowadd)(a, c) acab = delayed2(slowadd)(ac, ab) x, y = e.compute([ac, acab]) s.validate_state() for i in range(5): yield gen.sleep(0.01) s.validate_state() s.client_releases_keys(keys=[y.key], client=e.id) s.validate_state() for k in [acab.key, ab.key, b.key]: assert k not in s.tasks assert k not in s.waiting assert k not in s.who_has @gen_cluster(client=True) def test_forget_errors(c, s, a, b): x = c.submit(div, 1, 0) y = c.submit(inc, x) z = c.submit(inc, y) yield wait([y]) assert x.key in s.exceptions assert x.key in s.exceptions_blame assert y.key in s.exceptions_blame assert z.key in s.exceptions_blame s.client_releases_keys(keys=[z.key], client=c.id) assert x.key in s.exceptions assert x.key in s.exceptions_blame assert y.key in s.exceptions_blame assert z.key not in s.exceptions_blame s.client_releases_keys(keys=[x.key], client=c.id) assert x.key in s.exceptions assert x.key in s.exceptions_blame assert y.key in s.exceptions_blame assert z.key not in s.exceptions_blame s.client_releases_keys(keys=[y.key], client=c.id) assert x.key not in s.exceptions assert x.key not in s.exceptions_blame assert y.key not in s.exceptions_blame assert z.key not in s.exceptions_blame def test_repr_sync(loop): with cluster(nworkers=3) as (s, [a, b, c]): with Client(s['address'], loop=loop) as c: s = str(c) r = repr(c) assert c.scheduler.address in s assert c.scheduler.address in r assert str(3) in s # nworkers assert 'cores' in s @gen_cluster(client=True) def test_waiting_data(c, s, a, b): x = c.submit(inc, 1) y = c.submit(inc, 2) z = c.submit(add, x, y, workers=[a.ip], allow_other_workers=True) yield wait([x, y, z]) assert x.key not in s.waiting_data[x.key] assert y.key not in s.waiting_data[y.key] assert not s.waiting_data[x.key] assert not s.waiting_data[y.key] @gen_cluster() def test_multi_client(s, a, b): c = yield Client((s.ip, s.port), asynchronous=True) f = yield Client((s.ip, s.port), asynchronous=True) assert set(s.comms) == {c.id, f.id} x = c.submit(inc, 1) y = f.submit(inc, 2) y2 = c.submit(inc, 2) assert y.key == y2.key yield wait([x, y]) assert s.wants_what == {c.id: {x.key, y.key}, f.id: {y.key}, 'fire-and-forget': set()} assert s.who_wants == {x.key: {c.id}, y.key: {c.id, f.id}} yield c.close() start = time() while c.id in s.wants_what: yield gen.sleep(0.01) assert time() < start + 5 assert c.id not in s.wants_what assert c.id not in s.who_wants[y.key] assert x.key not in s.who_wants yield f.close() assert not s.tasks def long_running_client_connection(address): from distributed.utils_test import pristine_loop with pristine_loop(): c = Client(address) x = c.submit(lambda x: x + 1, 10) x.result() sleep(100) @gen_cluster() def test_cleanup_after_broken_client_connection(s, a, b): proc = mp_context.Process(target=long_running_client_connection, args=(s.address,)) proc.daemon = True proc.start() start = time() while not s.tasks: yield gen.sleep(0.01) assert time() < start + 5 proc.terminate() start = time() while s.tasks: yield gen.sleep(0.01) assert time() < start + 5 @gen_cluster() def test_multi_garbage_collection(s, a, b): c = yield Client((s.ip, s.port), asynchronous=True) f = yield Client((s.ip, s.port), asynchronous=True) x = c.submit(inc, 1) y = f.submit(inc, 2) y2 = c.submit(inc, 2) assert y.key == y2.key yield wait([x, y]) x.__del__() start = time() while x.key in a.data or x.key in b.data: yield gen.sleep(0.01) assert time() < start + 5 assert s.wants_what == {c.id: {y.key}, f.id: {y.key}, 'fire-and-forget': set()} assert s.who_wants == {y.key: {c.id, f.id}} y.__del__() start = time() while x.key in s.wants_what[f.id]: yield gen.sleep(0.01) assert time() < start + 5 yield gen.sleep(0.1) assert y.key in a.data or y.key in b.data assert s.wants_what == {c.id: {y.key}, f.id: set(), 'fire-and-forget': set()} assert s.who_wants == {y.key: {c.id}} y2.__del__() start = time() while y.key in a.data or y.key in b.data: yield gen.sleep(0.01) assert time() < start + 5 assert not any(v for v in s.wants_what.values()) assert not s.who_wants yield c.close() yield f.close() @gen_cluster(client=True) def test__broadcast(c, s, a, b): x, y = yield c.scatter([1, 2], broadcast=True) assert a.data == b.data == {x.key: 1, y.key: 2} @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 4) def test__broadcast_integer(c, s, *workers): x, y = yield c.scatter([1, 2], broadcast=2) assert len(s.who_has[x.key]) == 2 assert len(s.who_has[y.key]) == 2 @gen_cluster(client=True) def test__broadcast_dict(c, s, a, b): d = yield c.scatter({'x': 1}, broadcast=True) assert a.data == b.data == {'x': 1} def test_broadcast(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x, y = c.scatter([1, 2], broadcast=True) has_what = sync(c.loop, c.scheduler.has_what) assert {k: set(v) for k, v in has_what.items()} == { a['address']: {x.key, y.key}, b['address']: {x.key, y.key}} [z] = c.scatter([3], broadcast=True, workers=[a['address']]) has_what = sync(c.loop, c.scheduler.has_what) assert {k: set(v) for k, v in has_what.items()} == { a['address']: {x.key, y.key, z.key}, b['address']: {x.key, y.key}} @gen_cluster(client=True) def test__cancel(c, s, a, b): x = c.submit(slowinc, 1) y = c.submit(slowinc, x) while y.key not in s.tasks: yield gen.sleep(0.01) yield c.cancel([x]) assert x.cancelled() assert 'cancel' in str(x) s.validate_state() start = time() while not y.cancelled(): yield gen.sleep(0.01) assert time() < start + 5 assert not s.tasks assert not s.who_has s.validate_state() @gen_cluster(client=True) def test__cancel_tuple_key(c, s, a, b): x = c.submit(inc, 1, key=('x', 0, 1)) result = yield x yield c.cancel(x) with pytest.raises(CancelledError): yield x @gen_cluster() def test__cancel_multi_client(s, a, b): c = yield Client((s.ip, s.port), asynchronous=True) f = yield Client((s.ip, s.port), asynchronous=True) x = c.submit(slowinc, 1) y = f.submit(slowinc, 1) assert x.key == y.key yield c.cancel([x]) assert x.cancelled() assert not y.cancelled() start = time() while y.key not in s.tasks: yield gen.sleep(0.01) assert time() < start + 5 out = yield y assert out == 2 with pytest.raises(CancelledError): yield x yield c.close() yield f.close() @gen_cluster(client=True) def test__cancel_collection(c, s, a, b): import dask.bag as db L = c.map(double, [[1], [2], [3]]) x = db.Bag({('b', i): f for i, f in enumerate(L)}, 'b', 3) yield c.cancel(x) yield c.cancel([x]) assert all(f.cancelled() for f in L) assert not s.tasks assert not s.who_has def test_cancel(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(slowinc, 1, key='x') y = c.submit(slowinc, x, key='y') z = c.submit(slowinc, y, key='z') c.cancel([y]) start = time() while not z.cancelled(): sleep(0.01) assert time() < start + 5 assert x.result() == 2 z.cancel() assert z.cancelled() @gen_cluster(client=True) def test_future_type(c, s, a, b): x = c.submit(inc, 1) yield wait([x]) assert x.type == int assert 'int' in str(x) @gen_cluster(client=True) def test_traceback_clean(c, s, a, b): x = c.submit(div, 1, 0) try: yield x except Exception as e: f = e exc_type, exc_value, tb = sys.exc_info() while tb: assert 'scheduler' not in tb.tb_frame.f_code.co_filename assert 'worker' not in tb.tb_frame.f_code.co_filename 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 @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(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as 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 def test_Future_exception_sync_2(loop, capsys): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: ensure_default_get(c) ensure_default_get(c) ensure_default_get(c) ensure_default_get(c) assert _globals['get'] == c.get out, err = capsys.readouterr() assert len(out.strip().split('\n')) == 1 assert _globals.get('get') != c.get @gen_cluster(timeout=60, client=True) def test_async_persist(c, s, a, b): from dask.delayed import delayed, Delayed x = delayed(1) y = delayed(inc)(x) z = delayed(dec)(x) w = delayed(add)(y, z) yy, ww = c.persist([y, w]) assert type(yy) == type(y) assert type(ww) == type(w) assert len(yy.dask) == 1 assert len(ww.dask) == 1 assert len(w.dask) > 1 assert y.__dask_keys__() == yy.__dask_keys__() assert w.__dask_keys__() == ww.__dask_keys__() while y.key not in s.tasks and w.key not in s.tasks: yield gen.sleep(0.01) assert s.who_wants[y.key] == {c.id} assert s.who_wants[w.key] == {c.id} yyf, wwf = c.compute([yy, ww]) yyy, www = yield c.gather([yyf, wwf]) assert yyy == inc(1) assert www == add(inc(1), dec(1)) assert isinstance(c.persist(y), Delayed) assert isinstance(c.persist([y]), (list, tuple)) @gen_cluster(client=True) def test__persist(c, s, a, b): pytest.importorskip('dask.array') import dask.array as da x = da.ones((10, 10), chunks=(5, 10)) y = 2 * (x + 1) assert len(y.dask) == 6 yy = c.persist(y) assert len(y.dask) == 6 assert len(yy.dask) == 2 assert all(isinstance(v, Future) for v in yy.dask.values()) assert yy.__dask_keys__() == y.__dask_keys__() g, h = c.compute([y, yy]) gg, hh = yield c.gather([g, h]) assert (gg == hh).all() def test_persist(loop): pytest.importorskip('dask.array') import dask.array as da with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = da.ones((10, 10), chunks=(5, 10)) y = 2 * (x + 1) assert len(y.dask) == 6 yy = c.persist(y) assert len(y.dask) == 6 assert len(yy.dask) == 2 assert all(isinstance(v, Future) for v in yy.dask.values()) assert yy.__dask_keys__() == y.__dask_keys__() zz = yy.compute(get=c.get) z = y.compute(get=c.get) assert (zz == z).all() @gen_cluster(timeout=60, client=True) def test_long_traceback(c, s, a, b): from distributed.protocol.pickle import dumps n = sys.getrecursionlimit() sys.setrecursionlimit(500) try: x = c.submit(deep, 1000) yield wait([x]) assert len(dumps(c.futures[x.key].traceback)) < 10000 assert isinstance(c.futures[x.key].exception, RuntimeError) finally: sys.setrecursionlimit(n) @gen_cluster(client=True) def test_wait_on_collections(c, s, a, b): import dask.bag as db L = c.map(double, [[1], [2], [3]]) x = db.Bag({('b', i): f for i, f in enumerate(L)}, 'b', 3) yield wait(x) assert all(f.key in a.data or f.key in b.data for f in L) @gen_cluster(client=True) def test_futures_of(c, s, a, b): x, y, z = c.map(inc, [1, 2, 3]) assert set(futures_of(0)) == set() assert set(futures_of(x)) == {x} assert set(futures_of([x, y, z])) == {x, y, z} assert set(futures_of([x, [y], [[z]]])) == {x, y, z} assert set(futures_of({'x': x, 'y': [y]})) == {x, y} import dask.bag as db b = db.Bag({('b', i): f for i, f in enumerate([x, y, z])}, 'b', 3) assert set(futures_of(b)) == {x, y, z} def test_futures_of_class(): da = pytest.importorskip('dask.array') assert futures_of([da.Array]) == [] @gen_cluster(client=True) def test_futures_of_cancelled_raises(c, s, a, b): x = c.submit(inc, 1) yield c.cancel([x]) with pytest.raises(CancelledError): yield x with pytest.raises(CancelledError): yield c.get({'x': (inc, x), 'y': (inc, 2)}, ['x', 'y'], sync=False) with pytest.raises(CancelledError): c.submit(inc, x) with pytest.raises(CancelledError): c.submit(add, 1, y=x) with pytest.raises(CancelledError): c.map(add, [1], y=x) assert 'y' not in s.tasks @pytest.mark.skip @gen_cluster(ncores=[('127.0.0.1', 1)], client=True) def test_dont_delete_recomputed_results(c, s, w): x = c.submit(inc, 1) # compute first time yield wait([x]) x.__del__() # trigger garbage collection yield gen.moment xx = c.submit(inc, 1) # compute second time start = time() while xx.key not in w.data: # data shows up yield gen.sleep(0.01) assert time() < start + 1 while time() < start + (s.delete_interval + 100) / 1000: # and stays assert xx.key in w.data yield gen.sleep(0.01) @gen_cluster(ncores=[], client=True) def test_fatally_serialized_input(c, s): o = FatallySerializedObject() future = c.submit(inc, o) while not s.tasks: yield gen.sleep(0.01) @pytest.mark.xfail(reason='Use fast random selection now') @gen_cluster(client=True) def test_balance_tasks_by_stacks(c, s, a, b): x = c.submit(inc, 1) yield wait(x) y = c.submit(inc, 2) yield wait(y) assert len(a.data) == len(b.data) == 1 @gen_cluster(client=True) def test_run(c, s, a, b): results = yield c.run(inc, 1) assert results == {a.address: 2, b.address: 2} results = yield c.run(inc, 1, workers=[a.address]) assert results == {a.address: 2} results = yield c.run(inc, 1, workers=[]) assert results == {} @gen_cluster(client=True) def test_run_handles_picklable_data(c, s, a, b): futures = c.map(inc, range(10)) yield wait(futures) def func(): return {}, set(), [], (), 1, 'hello', b'100' results = yield c.run_on_scheduler(func) assert results == func() results = yield c.run(func) assert results == {w.address: func() for w in [a, b]} def test_run_sync(loop): def func(x, y=10): return x + y with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: result = c.run(func, 1, y=2) assert result == {a['address']: 3, b['address']: 3} result = c.run(func, 1, y=2, workers=[a['address']]) assert result == {a['address']: 3} @gen_cluster(client=True) def test_run_coroutine(c, s, a, b): results = yield c.run_coroutine(geninc, 1, delay=0.05) assert results == {a.address: 2, b.address: 2} results = yield c.run_coroutine(geninc, 1, delay=0.05, workers=[a.address]) assert results == {a.address: 2} results = yield c.run_coroutine(geninc, 1, workers=[]) assert results == {} with pytest.raises(RuntimeError) as exc_info: yield c.run_coroutine(throws, 1) assert "hello" in str(exc_info) if sys.version_info >= (3, 5): results = yield c.run_coroutine(asyncinc, 2, delay=0.01) assert results == {a.address: 3, b.address: 3} def test_run_coroutine_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: result = c.run_coroutine(geninc, 2, delay=0.01) assert result == {a['address']: 3, b['address']: 3} result = c.run_coroutine(geninc, 2, workers=[a['address']]) assert result == {a['address']: 3} t1 = time() result = c.run_coroutine(geninc, 2, delay=10, wait=False) t2 = time() assert result is None assert t2 - t1 <= 1.0 def test_run_exception(loop): def raise_exception(exc_type, exc_msg): raise exc_type(exc_msg) with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: for exc_type in [ValueError, RuntimeError]: with pytest.raises(exc_type) as excinfo: c.run(raise_exception, exc_type, 'informative message') assert 'informative message' in str(excinfo.value) def test_diagnostic_ui(loop): with cluster() as (s, [a, b]): a_addr = a['address'] b_addr = b['address'] with Client(s['address'], loop=loop) as c: d = c.ncores() assert d == {a_addr: 1, b_addr: 1} d = c.ncores([a_addr]) assert d == {a_addr: 1} d = c.ncores(a_addr) assert d == {a_addr: 1} d = c.ncores(a['address']) assert d == {a_addr: 1} x = c.submit(inc, 1) y = c.submit(inc, 2) z = c.submit(inc, 3) wait([x, y, z]) d = c.who_has() assert set(d) == {x.key, y.key, z.key} assert all(w in [a_addr, b_addr] for v in d.values() for w in v) assert all(d.values()) d = c.who_has([x, y]) assert set(d) == {x.key, y.key} d = c.who_has(x) assert set(d) == {x.key} d = c.has_what() assert set(d) == {a_addr, b_addr} assert all(k in [x.key, y.key, z.key] for v in d.values() for k in v) d = c.has_what([a_addr]) assert set(d) == {a_addr} d = c.has_what(a_addr) assert set(d) == {a_addr} def test_diagnostic_nbytes_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: incs = c.map(inc, [1, 2, 3]) doubles = c.map(double, [1, 2, 3]) wait(incs + doubles) assert c.nbytes(summary=False) == {k.key: sizeof(1) for k in incs + doubles} assert c.nbytes(summary=True) == {'inc': sizeof(1) * 3, 'double': sizeof(1) * 3} @gen_cluster(client=True) def test_diagnostic_nbytes(c, s, a, b): incs = c.map(inc, [1, 2, 3]) doubles = c.map(double, [1, 2, 3]) yield wait(incs + doubles) assert s.get_nbytes(summary=False) == {k.key: sizeof(1) for k in incs + doubles} assert s.get_nbytes(summary=True) == {'inc': sizeof(1) * 3, 'double': sizeof(1) * 3} @gen_test() def test_worker_aliases(): s = Scheduler(validate=True) s.start(0) a = Worker(s.ip, s.port, name='alice') b = Worker(s.ip, s.port, name='bob') w = Worker(s.ip, s.port, name=3) yield [a._start(), b._start(), w._start()] c = yield Client((s.ip, s.port), asynchronous=True) L = c.map(inc, range(10), workers='alice') future = yield c.scatter(123, workers=3) yield wait(L) assert len(a.data) == 10 assert len(b.data) == 0 assert dict(w.data) == {future.key: 123} for i, alias in enumerate([3, [3], 'alice']): result = yield c.submit(lambda x: x + 1, i, workers=alias) assert result == i + 1 yield c.close() yield [a._close(), b._close(), w._close()] yield s.close() def test_persist_get_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: dadd = delayed(add) x, y = delayed(1), delayed(2) xx = delayed(add)(x, x) yy = delayed(add)(y, y) xxyy = delayed(add)(xx, yy) xxyy2 = c.persist(xxyy) xxyy3 = delayed(add)(xxyy2, 10) assert xxyy3.compute(get=c.get) == ((1 + 1) + (2 + 2)) + 10 @gen_cluster(client=True) def test_persist_get(c, s, a, b): dadd = delayed(add) x, y = delayed(1), delayed(2) xx = delayed(add)(x, x) yy = delayed(add)(y, y) xxyy = delayed(add)(xx, yy) xxyy2 = c.persist(xxyy) xxyy3 = delayed(add)(xxyy2, 10) yield gen.sleep(0.5) result = yield c.get(xxyy3.dask, xxyy3.__dask_keys__(), sync=False) assert result[0] == ((1 + 1) + (2 + 2)) + 10 result = yield c.compute(xxyy3) assert result == ((1 + 1) + (2 + 2)) + 10 result = yield c.compute(xxyy3) assert result == ((1 + 1) + (2 + 2)) + 10 result = yield c.compute(xxyy3) assert result == ((1 + 1) + (2 + 2)) + 10 @pytest.mark.skipif(sys.platform.startswith('win'), reason="num_fds not supported on windows") def test_client_num_fds(loop): psutil = pytest.importorskip('psutil') with cluster() as (s, [a, b]): proc = psutil.Process() before = proc.num_fds() with Client(s['address'], loop=loop) as c: during = proc.num_fds() after = proc.num_fds() assert before >= after @gen_cluster() def test_startup_close_startup(s, a, b): c = yield Client((s.ip, s.port), asynchronous=True) yield c.close() c = yield Client((s.ip, s.port), asynchronous=True) yield c.close() def test_startup_close_startup_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: sleep(0.1) with Client(s['address']) as c: pass with Client(s['address']) as c: pass sleep(0.1) with Client(s['address']) as c: pass @gen_cluster(client=True) def test_badly_serialized_exceptions(c, s, a, b): def f(): class BadlySerializedException(Exception): def __reduce__(self): raise TypeError() raise BadlySerializedException('hello world') x = c.submit(f) try: result = yield x except Exception as e: assert 'hello world' in str(e) else: assert False @gen_cluster(client=True) def test_rebalance(c, s, a, b): x, y = yield c.scatter([1, 2], workers=[a.address]) assert len(a.data) == 2 assert len(b.data) == 0 s.validate_state() yield c.rebalance() s.validate_state() assert len(b.data) == 1 assert s.has_what[b.address] == set(b.data) assert b.address in s.who_has[x.key] or b.address in s.who_has[y.key] assert len(a.data) == 1 assert s.has_what[a.address] == set(a.data) assert (a.address not in s.who_has[x.key] or a.address not in s.who_has[y.key]) @gen_cluster(ncores=[('127.0.0.1', 1)] * 4, client=True) def test_rebalance_workers(e, s, a, b, c, d): w, x, y, z = yield e.scatter([1, 2, 3, 4], workers=[a.address]) assert len(a.data) == 4 assert len(b.data) == 0 assert len(c.data) == 0 assert len(d.data) == 0 yield e.rebalance([x, y], workers=[a.address, c.address]) assert len(a.data) == 3 assert len(b.data) == 0 assert len(c.data) == 1 assert len(d.data) == 0 assert c.data == {x.key: 2} or c.data == {y.key: 3} yield e.rebalance() assert len(a.data) == 1 assert len(b.data) == 1 assert len(c.data) == 1 assert len(d.data) == 1 s.validate_state() @gen_cluster(client=True) def test_rebalance_execution(c, s, a, b): futures = c.map(inc, range(10), workers=a.address) yield c.rebalance(futures) assert len(a.data) == len(b.data) == 5 s.validate_state() def test_rebalance_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: futures = c.map(inc, range(10), workers=[a['address']]) c.rebalance(futures) has_what = c.has_what() assert len(has_what) == 2 assert list(valmap(len, has_what).values()) == [5, 5] @gen_cluster(client=True) def test_rebalance_unprepared(c, s, a, b): futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) yield gen.sleep(0.1) yield c.rebalance(futures) s.validate_state() @gen_cluster(client=True) def test_receive_lost_key(c, s, a, b): x = c.submit(inc, 1, workers=[a.address]) result = yield x yield a._close() start = time() while x.status == 'finished': assert time() < start + 5 yield gen.sleep(0.01) @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster([('127.0.0.1', 1), ('127.0.0.2', 2)], client=True) def test_unrunnable_task_runs(c, s, a, b): x = c.submit(inc, 1, workers=[a.ip]) result = yield x yield a._close() start = time() while x.status == 'finished': assert time() < start + 5 yield gen.sleep(0.01) assert x.key in s.unrunnable assert s.task_state[x.key] == 'no-worker' w = Worker(s.ip, s.port, loop=s.loop) yield w._start() start = time() while x.status != 'finished': assert time() < start + 2 yield gen.sleep(0.01) assert x.key not in s.unrunnable result = yield x assert result == 2 yield w._close() @gen_cluster(client=True, ncores=[]) def test_add_worker_after_tasks(c, s): futures = c.map(inc, range(10)) n = Nanny(s.ip, s.port, ncores=2, loop=s.loop) n.start(0) result = yield c.gather(futures) yield n._close() @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster([('127.0.0.1', 1), ('127.0.0.2', 2)], client=True) def test_workers_register_indirect_data(c, s, a, b): [x] = yield c.scatter([1], workers=a.address) y = c.submit(inc, x, workers=b.ip) yield y assert b.data[x.key] == 1 assert s.who_has[x.key] == {a.address, b.address} assert s.has_what[b.address] == {x.key, y.key} s.validate_state() @gen_cluster(client=True) def test_submit_on_cancelled_future(c, s, a, b): x = c.submit(inc, 1) yield x yield c.cancel(x) with pytest.raises(CancelledError): y = c.submit(inc, x) @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10) def test_replicate(c, s, *workers): [a, b] = yield c.scatter([1, 2]) yield s.replicate(keys=[a.key, b.key], n=5) s.validate_state() assert len(s.who_has[a.key]) == 5 assert len(s.who_has[b.key]) == 5 assert sum(a.key in w.data for w in workers) == 5 assert sum(b.key in w.data for w in workers) == 5 @gen_cluster(client=True) def test_replicate_tuple_keys(c, s, a, b): x = delayed(inc)(1, dask_key_name=('x', 1)) f = c.persist(x) yield c.replicate(f, n=5) s.validate_state() assert a.data and b.data yield c.rebalance(f) s.validate_state() @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10) def test_replicate_workers(c, s, *workers): [a, b] = yield c.scatter([1, 2], workers=[workers[0].address]) yield s.replicate(keys=[a.key, b.key], n=5, workers=[w.address for w in workers[:5]]) assert len(s.who_has[a.key]) == 5 assert len(s.who_has[b.key]) == 5 assert sum(a.key in w.data for w in workers[:5]) == 5 assert sum(b.key in w.data for w in workers[:5]) == 5 assert sum(a.key in w.data for w in workers[5:]) == 0 assert sum(b.key in w.data for w in workers[5:]) == 0 yield s.replicate(keys=[a.key, b.key], n=1) assert len(s.who_has[a.key]) == 1 assert len(s.who_has[b.key]) == 1 assert sum(a.key in w.data for w in workers) == 1 assert sum(b.key in w.data for w in workers) == 1 s.validate_state() yield s.replicate(keys=[a.key, b.key], n=None) # all assert len(s.who_has[a.key]) == 10 assert len(s.who_has[b.key]) == 10 s.validate_state() yield s.replicate(keys=[a.key, b.key], n=1, workers=[w.address for w in workers[:5]]) assert sum(a.key in w.data for w in workers[:5]) == 1 assert sum(b.key in w.data for w in workers[:5]) == 1 assert sum(a.key in w.data for w in workers[5:]) == 5 assert sum(b.key in w.data for w in workers[5:]) == 5 s.validate_state() class CountSerialization(object): def __init__(self): self.n = 0 def __setstate__(self, n): self.n = n + 1 def __getstate__(self): return self.n @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10) def test_replicate_tree_branching(c, s, *workers): obj = CountSerialization() [future] = yield c.scatter([obj]) yield s.replicate(keys=[future.key], n=10) max_count = max(w.data[future.key].n for w in workers) assert max_count > 1 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10) def test_client_replicate(c, s, *workers): x = c.submit(inc, 1) y = c.submit(inc, 2) yield c.replicate([x, y], n=5) assert len(s.who_has[x.key]) == 5 assert len(s.who_has[y.key]) == 5 yield c.replicate([x, y], n=3) assert len(s.who_has[x.key]) == 3 assert len(s.who_has[y.key]) == 3 yield c.replicate([x, y]) s.validate_state() assert len(s.who_has[x.key]) == 10 assert len(s.who_has[y.key]) == 10 @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster(client=True, ncores=[('127.0.0.1', 1), ('127.0.0.2', 1), ('127.0.0.2', 1)], timeout=None) def test_client_replicate_host(e, s, a, b, c): x = e.submit(inc, 1, workers='127.0.0.2') yield wait([x]) assert (s.who_has[x.key] == {b.address} or s.who_has[x.key] == {c.address}) yield e.replicate([x], workers=['127.0.0.2']) assert s.who_has[x.key] == {b.address, c.address} yield e.replicate([x], workers=['127.0.0.1']) assert s.who_has[x.key] == {a.address, b.address, c.address} def test_client_replicate_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(inc, 1) y = c.submit(inc, 2) c.replicate([x, y], n=2) who_has = c.who_has() assert len(who_has[x.key]) == len(who_has[y.key]) == 2 with pytest.raises(ValueError): c.replicate([x], n=0) assert y.result() == 3 @pytest.mark.skipif(sys.platform.startswith('win'), reason="Windows timer too coarse-grained") @gen_cluster(client=True, ncores=[('127.0.0.1', 4)] * 1) def test_task_load_adapts_quickly(c, s, a): future = c.submit(slowinc, 1, delay=0.2) # slow yield wait(future) assert 0.15 < s.task_duration['slowinc'] < 0.4 futures = c.map(slowinc, range(10), delay=0) # very fast yield wait(futures) assert 0 < s.task_duration['slowinc'] < 0.1 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_even_load_after_fast_functions(c, s, a, b): x = c.submit(inc, 1, workers=a.address) # very fast y = c.submit(inc, 2, workers=b.address) # very fast yield wait([x, y]) futures = c.map(inc, range(2, 11)) yield wait(futures) assert any(f.key in a.data for f in futures) assert any(f.key in b.data for f in futures) # assert abs(len(a.data) - len(b.data)) <= 3 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_even_load_on_startup(c, s, a, b): x, y = c.map(inc, [1, 2]) yield wait([x, y]) assert len(a.data) == len(b.data) == 1 @pytest.mark.xfail @gen_cluster(client=True, ncores=[('127.0.0.1', 2)] * 2) def test_contiguous_load(c, s, a, b): w, x, y, z = c.map(inc, [1, 2, 3, 4]) yield wait([w, x, y, z]) groups = [set(a.data), set(b.data)] assert {w.key, x.key} in groups assert {y.key, z.key} in groups @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 4) def test_balanced_with_submit(c, s, *workers): L = [c.submit(slowinc, i) for i in range(4)] yield wait(L) for w in workers: assert len(w.data) == 1 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 4) def test_balanced_with_submit_and_resident_data(c, s, *workers): [x] = yield c.scatter([10], broadcast=True) L = [c.submit(slowinc, x, pure=False) for i in range(4)] yield wait(L) for w in workers: assert len(w.data) == 2 @gen_cluster(client=True, ncores=[('127.0.0.1', 20)] * 2) def test_scheduler_saturates_cores(c, s, a, b): for delay in [0, 0.01, 0.1]: futures = c.map(slowinc, range(100), delay=delay) futures = c.map(slowinc, futures, delay=delay / 10) while not s.tasks or s.ready: if s.tasks: assert all(len(p) >= 20 for p in s.processing.values()) yield gen.sleep(0.01) @gen_cluster(client=True, ncores=[('127.0.0.1', 20)] * 2) def test_scheduler_saturates_cores_random(c, s, a, b): for delay in [0, 0.01, 0.1]: futures = c.map(randominc, range(100), scale=0.1) while not s.tasks or s.ready: if s.tasks: assert all(len(p) >= 20 for p in s.processing.values()) yield gen.sleep(0.01) @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 4) def test_cancel_clears_processing(c, s, *workers): da = pytest.importorskip('dask.array') x = c.submit(slowinc, 1, delay=0.2) while not s.tasks: yield gen.sleep(0.01) yield c.cancel(x) start = time() while any(v for v in s.processing.values()): assert time() < start + 0.2 yield gen.sleep(0.01) s.validate_state() def test_default_get(): with cluster() as (s, [a, b]): pre_get = _globals.get('get') pre_shuffle = _globals.get('shuffle') with Client(s['address'], set_as_default=True) as c: assert _globals['get'] == c.get assert _globals['shuffle'] == 'tasks' assert _globals['get'] is pre_get assert _globals['shuffle'] == pre_shuffle c = Client(s['address'], set_as_default=False) assert _globals['get'] is pre_get assert _globals['shuffle'] == pre_shuffle c.close() c = Client(s['address'], set_as_default=True) assert _globals['shuffle'] == 'tasks' assert _globals['get'] == c.get c.close() assert _globals['get'] is pre_get assert _globals['shuffle'] == pre_shuffle with Client(s['address']) as c: assert _globals['get'] == c.get with Client(s['address'], set_as_default=False) as c: assert _globals['get'] != c.get dask.set_options(get=c.get) assert _globals['get'] == c.get assert _globals['get'] != c.get @gen_cluster(client=True) def test_get_processing(c, s, a, b): processing = yield c.scheduler.processing() assert processing == valmap(list, s.processing) futures = c.map(slowinc, range(10), delay=0.1, workers=[a.address], allow_other_workers=True) yield gen.sleep(0.2) x = yield c.scheduler.processing() assert set(x) == {a.address, b.address} x = yield c.scheduler.processing(workers=[a.address]) assert isinstance(x[a.address], list) @gen_cluster(client=True) def test_get_foo(c, s, a, b): futures = c.map(inc, range(10)) yield wait(futures) x = yield c.scheduler.ncores() assert x == s.ncores x = yield c.scheduler.ncores(workers=[a.address]) assert x == {a.address: s.ncores[a.address]} x = yield c.scheduler.has_what() assert x == valmap(list, s.has_what) x = yield c.scheduler.has_what(workers=[a.address]) assert x == {a.address: list(s.has_what[a.address])} x = yield c.scheduler.nbytes(summary=False) assert x == s.nbytes x = yield c.scheduler.nbytes(keys=[futures[0].key], summary=False) assert x == {futures[0].key: s.nbytes[futures[0].key]} x = yield c.scheduler.who_has() assert x == valmap(list, s.who_has) x = yield c.scheduler.who_has(keys=[futures[0].key]) assert x == {futures[0].key: list(s.who_has[futures[0].key])} @slow @gen_cluster(client=True, Worker=Nanny) def test_bad_tasks_fail(c, s, a, b): f = c.submit(sys.exit, 1) with pytest.raises(KilledWorker): yield f def test_get_processing_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: processing = c.processing() assert not any(v for v in processing.values()) futures = c.map(slowinc, range(10), delay=0.1, workers=[a['address']], allow_other_workers=False) sleep(0.2) aa = a['address'] bb = b['address'] processing = c.processing() assert set(c.processing(aa)) == {aa} assert set(c.processing([aa])) == {aa} c.cancel(futures) def test_close_idempotent(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: c.close() c.close() c.close() @nodebug def test_get_returns_early(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: start = time() with ignoring(RuntimeError): result = c.get({'x': (throws, 1), 'y': (sleep, 1)}, ['x', 'y']) assert time() < start + 0.5 # Futures should be released and forgotten wait_for(lambda: not c.futures, timeout=0.1) wait_for(lambda: not any(c.processing().values()), timeout=3) x = c.submit(inc, 1) x.result() with ignoring(RuntimeError): result = c.get({'x': (throws, 1), x.key: (inc, 1)}, ['x', x.key]) assert x.key in c.futures @slow @gen_cluster(Worker=Nanny, client=True) def test_Client_clears_references_after_restart(c, s, a, b): x = c.submit(inc, 1) assert x.key in c.refcount yield c.restart() assert x.key not in c.refcount key = x.key del x import gc gc.collect() yield gen.moment assert key not in c.refcount def test_get_stops_work_after_error(loop): with cluster(active_rpc_timeout=10) as (s, [a, b]): with Client(s['address'], loop=loop) as c: with pytest.raises(RuntimeError): c.get({'x': (throws, 1), 'y': (sleep, 1.5)}, ['x', 'y']) start = time() while any(c.processing().values()): sleep(0.01) assert time() < start + 0.5 def test_as_completed_list(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: seq = c.map(inc, iter(range(5))) seq2 = list(as_completed(seq)) assert set(c.gather(seq2)) == {1, 2, 3, 4, 5} def test_as_completed_results(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: seq = c.map(inc, range(5)) seq2 = list(as_completed(seq, with_results=True)) assert set(pluck(1, seq2)) == {1, 2, 3, 4, 5} assert set(pluck(0, seq2)) == set(seq) @pytest.mark.parametrize('with_results', [True, False]) def test_as_completed_batches(loop, with_results): n = 50 with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: futures = c.map(slowinc, range(n), delay=0.01) out = [] for batch in as_completed(futures, with_results=with_results).batches(): assert isinstance(batch, (tuple, list)) sleep(0.05) out.extend(batch) assert len(out) == n if with_results: assert set(pluck(1, out)) == set(range(1, n + 1)) else: assert set(out) == set(futures) def test_as_completed_next_batch(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: futures = c.map(slowinc, range(2), delay=0.1) ac = as_completed(futures) assert ac.next_batch(block=False) == [] assert set(ac.next_batch(block=True)).issubset(futures) @gen_test() def test_status(): s = Scheduler() s.start(0) c = yield Client((s.ip, s.port), asynchronous=True) assert c.status == 'running' x = c.submit(inc, 1) yield c.close() assert c.status == 'closed' yield s.close() @gen_cluster(client=True) def test_persist_optimize_graph(c, s, a, b): i = 10 import dask.bag as db for method in [c.persist, c.compute]: b = db.range(i, npartitions=2) i += 1 b2 = b.map(inc) b3 = b2.map(inc) b4 = method(b3, optimize_graph=False) yield wait(b4) assert set(map(tokey, b3.__dask_keys__())).issubset(s.tasks) b = db.range(i, npartitions=2) i += 1 b2 = b.map(inc) b3 = b2.map(inc) b4 = method(b3, optimize_graph=True) yield wait(b4) assert not any(tokey(k) in s.tasks for k in b2.__dask_keys__()) @gen_cluster(client=True, ncores=[]) def test_scatter_raises_if_no_workers(c, s): with pytest.raises(gen.TimeoutError): yield c.scatter([1]) @slow def test_reconnect(loop): w = Worker('127.0.0.1', 9393, loop=loop) w.start() scheduler_cli = ['dask-scheduler', '--host', '127.0.0.1', '--port', '9393', '--no-bokeh'] with popen(scheduler_cli) as s: c = Client('127.0.0.1:9393', loop=loop) start = time() while len(c.ncores()) != 1: sleep(0.1) assert time() < start + 3 x = c.submit(inc, 1) assert x.result() == 2 start = time() while c.status != 'connecting': assert time() < start + 5 sleep(0.01) with pytest.raises(Exception): c.ncores() assert x.status == 'cancelled' with pytest.raises(CancelledError): x.result() with popen(scheduler_cli) as s: start = time() while c.status != 'running': sleep(0.1) assert time() < start + 5 start = time() while len(c.ncores()) != 1: sleep(0.05) assert time() < start + 15 x = c.submit(inc, 1) assert x.result() == 2 start = time() while True: try: x.result() assert False except CommClosedError: continue except CancelledError: break assert time() < start + 5 sleep(0.1) sync(loop, w._close) c.close() # On Python 2, heavy process spawning can deadlock (e.g. on a logging IO lock) _params = ([(Worker, 100, 5), (Nanny, 10, 20)] if sys.version_info >= (3,) else [(Worker, 100, 5)]) @slow @pytest.mark.skipif(sys.platform.startswith('win'), reason="num_fds not supported on windows") @pytest.mark.parametrize("worker,count,repeat", _params) 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, []): gc.collect() before = proc.num_fds() done = Semaphore(0) running = weakref.WeakKeyDictionary() @gen.coroutine def start_worker(sleep, duration, repeat=1): for i in range(repeat): yield gen.sleep(sleep) w = worker(s['address'], loop=loop) running[w] = None yield w._start() addr = w.worker_address running[w] = addr yield gen.sleep(duration) yield w._close() del w yield gen.moment done.release() for i in range(count): loop.add_callback(start_worker, random.random() / 5, random.random() / 5, repeat=repeat) with Client(s['address'], loop=loop) as c: sleep(1) for i in range(count): done.acquire() gc.collect() if not running: break start = time() while c.ncores(): sleep(0.2) assert time() < start + 10 start = time() while proc.num_fds() > before: print("fds:", before, proc.num_fds()) sleep(0.1) assert time() < start + 10 @gen_cluster(client=False, timeout=None) def test_idempotence(s, a, b): c = yield Client((s.ip, s.port), asynchronous=True) f = yield Client((s.ip, s.port), asynchronous=True) # Submit x = c.submit(inc, 1) yield x log = list(s.transition_log) len_single_submit = len(log) # see last assert y = f.submit(inc, 1) assert x.key == y.key yield y yield gen.sleep(0.1) log2 = list(s.transition_log) assert log == log2 # Error a = c.submit(div, 1, 0) yield wait(a) assert a.status == 'error' log = list(s.transition_log) b = f.submit(div, 1, 0) assert a.key == b.key yield wait(b) yield gen.sleep(0.1) log2 = list(s.transition_log) assert log == log2 s.transition_log.clear() # Simultaneous Submit d = c.submit(inc, 2) e = c.submit(inc, 2) yield wait([d, e]) assert len(s.transition_log) == len_single_submit yield c.close() yield f.close() def test_scheduler_info(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: info = c.scheduler_info() assert isinstance(info, dict) assert len(info['workers']) == 2 def test_get_versions(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: v = c.get_versions() assert v['scheduler'] is not None assert v['client'] is not None assert len(v['workers']) == 2 for k, v in v['workers'].items(): assert v is not None c.get_versions(check=True) # smoke test for versions # that this does not raise def test_threaded_get_within_distributed(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: import dask.multiprocessing for get in [dask.local.get_sync, dask.multiprocessing.get, dask.threaded.get]: def f(): return get({'x': (lambda: 1,)}, 'x') future = c.submit(f) assert future.result() == 1 @gen_cluster(client=True) def test_lose_scattered_data(c, s, a, b): [x] = yield c.scatter([1], workers=a.address) yield a._close() yield gen.sleep(0.1) assert x.status == 'cancelled' assert x.key not in s.task_state @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_partially_lose_scattered_data(e, s, a, b, c): [x] = yield e.scatter([1], workers=a.address) yield e.replicate(x, n=2) yield a._close() yield gen.sleep(0.1) assert x.status == 'finished' assert s.task_state[x.key] == 'memory' @gen_cluster(client=True) def test_scatter_compute_lose(c, s, a, b): [x] = yield c.scatter([[1, 2, 3, 4]], workers=a.address) y = c.submit(inc, 1, workers=b.address) z = c.submit(slowadd, x, y, delay=0.2) yield gen.sleep(0.1) yield a._close() assert x.status == 'cancelled' assert y.status == 'finished' assert z.status == 'cancelled' with pytest.raises(CancelledError): yield wait(z) @gen_cluster(client=True) def test_scatter_compute_store_lose(c, s, a, b): """ Create irreplaceable data on one machine, cause a dependent computation to occur on another and complete Kill the machine with the irreplaceable data. What happens to the complete result? How about after it GCs and tries to come back? """ x = yield c.scatter(1, workers=a.address) xx = c.submit(inc, x, workers=a.address) y = c.submit(inc, 1) z = c.submit(slowadd, xx, y, delay=0.2, workers=b.address) yield wait(z) yield a._close() start = time() while x.status == 'finished': yield gen.sleep(0.01) assert time() < start + 2 # assert xx.status == 'finished' assert y.status == 'finished' assert z.status == 'finished' zz = c.submit(inc, z) yield wait(zz) zkey = z.key del z start = time() while s.task_state[zkey] != 'released': yield gen.sleep(0.01) assert time() < start + 2 xxkey = xx.key del xx start = time() while (x.key in s.task_state and zkey not in s.task_state and xxkey not in s.task_state): yield gen.sleep(0.01) assert time() < start + 2 @gen_cluster(client=True) def test_scatter_compute_store_lose_processing(c, s, a, b): """ Create irreplaceable data on one machine, cause a dependent computation to occur on another and complete Kill the machine with the irreplaceable data. What happens to the complete result? How about after it GCs and tries to come back? """ [x] = yield c.scatter([1], workers=a.address) y = c.submit(slowinc, x, delay=0.2) z = c.submit(inc, y) yield gen.sleep(0.1) yield a._close() start = time() while x.status == 'finished': yield gen.sleep(0.01) assert time() < start + 2 assert y.status == 'cancelled' assert z.status == 'cancelled' @gen_cluster(client=False) def test_serialize_future(s, a, b): c = yield Client((s.ip, s.port), asynchronous=True) f = yield Client((s.ip, s.port), asynchronous=True) future = c.submit(lambda: 1) result = yield future with temp_default_client(f): future2 = pickle.loads(pickle.dumps(future)) assert future2.client is f assert tokey(future2.key) in f.futures result2 = yield future2 assert result == result2 yield c.close() yield f.close() @gen_cluster(client=False) def test_temp_client(s, a, b): c = yield Client((s.ip, s.port), asynchronous=True) f = yield Client((s.ip, s.port), asynchronous=True) with temp_default_client(c): assert default_client() is c assert default_client(f) is f with temp_default_client(f): assert default_client() is f assert default_client(c) is c yield c.close() yield f.close() @nodebug # test timing is fragile @gen_cluster(ncores=[('127.0.0.1', 1)] * 3, client=True) def test_persist_workers(e, s, a, b, c): L1 = [delayed(inc)(i) for i in range(4)] total = delayed(sum)(L1) L2 = [delayed(add)(i, total) for i in L1] out = e.persist(L1 + L2 + [total], workers={tuple(L1): a.address, total: b.address, tuple(L2): [c.address]}, allow_other_workers=L1 + [total]) yield wait(out) assert all(v.key in a.data for v in L1) assert total.key in b.data assert all(v.key in c.data for v in L2) assert s.loose_restrictions == {total.key} | {v.key for v in L1} @gen_cluster(ncores=[('127.0.0.1', 1)] * 3, client=True) def test_compute_workers(e, s, a, b, c): L1 = [delayed(inc)(i) for i in range(4)] total = delayed(sum)(L1) L2 = [delayed(add)(i, total) for i in L1] out = e.compute(L1 + L2 + [total], workers={tuple(L1): a.address, total: b.address, tuple(L2): [c.address]}, allow_other_workers=L1 + [total]) yield wait(out) for v in L1: assert s.worker_restrictions[v.key] == {a.address} for v in L2: assert s.worker_restrictions[v.key] == {c.address} assert s.worker_restrictions[total.key] == {b.address} assert s.loose_restrictions == {total.key} | {v.key for v in L1} @gen_cluster(client=True) def test_compute_nested_containers(c, s, a, b): da = pytest.importorskip('dask.array') np = pytest.importorskip('numpy') x = da.ones(10, chunks=(5,)) + 1 future = c.compute({'x': [x], 'y': 123}) result = yield future assert isinstance(result, dict) assert (result['x'][0] == np.ones(10) + 1).all() assert result['y'] == 123 def test_get_restrictions(): L1 = [delayed(inc)(i) for i in range(4)] total = delayed(sum)(L1) L2 = [delayed(add)(i, total) for i in L1] r1, loose = Client.get_restrictions(L2, '127.0.0.1', False) assert r1 == {d.key: ['127.0.0.1'] for d in L2} assert not loose r1, loose = Client.get_restrictions(L2, ['127.0.0.1'], True) assert r1 == {d.key: ['127.0.0.1'] for d in L2} assert set(loose) == {d.key for d in L2} r1, loose = Client.get_restrictions(L2, {total: '127.0.0.1'}, True) assert r1 == {total.key: ['127.0.0.1']} assert loose == [total.key] r1, loose = Client.get_restrictions(L2, {(total,): '127.0.0.1'}, True) assert r1 == {total.key: ['127.0.0.1']} assert loose == [total.key] @gen_cluster(client=True) def test_scatter_type(c, s, a, b): [future] = yield c.scatter([1]) assert future.type == int d = yield c.scatter({'x': 1.0}) assert d['x'].type == float @gen_cluster(client=True) def test_retire_workers(c, s, a, b): [x] = yield c.scatter([1], workers=a.address) yield s.retire_workers(workers=[a.address]) assert b.data == {x.key: 1} assert s.who_has == {x.key: {b.address}} assert s.has_what == {b.address: {x.key}} assert a.address not in s.worker_info @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10) def test_retire_many_workers(c, s, *workers): futures = yield c.scatter(list(range(100))) yield s.retire_workers(workers=[w.address for w in workers[:7]]) results = yield c.gather(futures) assert results == list(range(100)) assert len(s.has_what) == len(s.ncores) == 3 for w, keys in s.has_what.items(): assert 20 < len(keys) < 50 @gen_cluster(client=True, ncores=[('127.0.0.1', 3)] * 2) def test_weight_occupancy_against_data_movement(c, s, a, b): s.extensions['stealing']._pc.callback_time = 1000000 s.task_duration['f'] = 0.01 def f(x, y=0, z=0): sleep(0.01) return x y = yield c.scatter([[1, 2, 3, 4]], workers=[a.address]) z = yield c.scatter([1], workers=[b.address]) futures = c.map(f, [1, 2, 3, 4], y=y, z=z) yield wait(futures) assert sum(f.key in a.data for f in futures) >= 2 assert sum(f.key in b.data for f in futures) >= 1 @gen_cluster(client=True, ncores=[('127.0.0.1', 1), ('127.0.0.1', 10)]) def test_distribute_tasks_by_ncores(c, s, a, b): s.task_duration['f'] = 0.01 s.extensions['stealing']._pc.callback_time = 1000000 def f(x, y=0): sleep(0.01) return x y = yield c.scatter([1], broadcast=True) futures = c.map(f, range(20), y=y) yield wait(futures) assert len(b.data) > 2 * len(a.data) @gen_cluster(client=True) def test_add_done_callback(c, s, a, b): S = set() def f(future): future.add_done_callback(g) def g(future): S.add((future.key, future.status)) u = c.submit(inc, 1, key='u') v = c.submit(throws, "hello", key='v') w = c.submit(slowinc, 2, delay=0.3, key='w') x = c.submit(inc, 3, key='x') u.add_done_callback(f) v.add_done_callback(f) w.add_done_callback(f) yield wait((u, v, w, x)) x.add_done_callback(f) t = time() while len(S) < 4 and time() - t < 2.0: yield gen.sleep(0.01) assert S == {(f.key, f.status) for f in (u, v, w, x)} @gen_cluster(client=True) def test_normalize_collection(c, s, a, b): x = delayed(inc)(1) y = delayed(inc)(x) z = delayed(inc)(y) yy = c.persist(y) zz = c.normalize_collection(z) assert len(z.dask) == len(y.dask) + 1 assert isinstance(zz.dask[y.key], Future) assert len(zz.dask) < len(z.dask) @gen_cluster(client=True) def test_normalize_collection_dask_array(c, s, a, b): da = pytest.importorskip('dask.array') x = da.ones(10, chunks=(5,)) y = x + 1 yy = c.persist(y) z = y.sum() zdsk = dict(z.dask) zz = c.normalize_collection(z) assert z.dask == zdsk # do not mutate input assert len(z.dask) > len(zz.dask) assert any(isinstance(v, Future) for v in zz.dask.values()) for k, v in yy.dask.items(): assert zz.dask[k].key == v.key result1 = yield c.compute(z) result2 = yield c.compute(zz) assert result1 == result2 @gen_cluster(client=True) def test_auto_normalize_collection(c, s, a, b): da = pytest.importorskip('dask.array') x = da.ones(10, chunks=5) assert len(x.dask) == 2 with dask.set_options(optimizations=[c._optimize_insert_futures]): y = x.map_blocks(slowinc, delay=1, dtype=x.dtype) yy = c.persist(y) yield wait(yy) start = time() future = c.compute(y.sum()) yield future end = time() assert end - start < 1 start = time() z = c.persist(y + 1) yield wait(z) end = time() assert end - start < 1 def test_auto_normalize_collection_sync(loop): da = pytest.importorskip('dask.array') with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = da.ones(10, chunks=5) y = x.map_blocks(slowinc, delay=1, dtype=x.dtype) yy = c.persist(y) wait(yy) with dask.set_options(optimizations=[c._optimize_insert_futures]): start = time() y.sum().compute() end = time() assert end - start < 1 def assert_no_data_loss(scheduler): for key, start, finish, recommendations, _ in scheduler.transition_log: if start == 'memory' and finish == 'released': for k, v in recommendations.items(): assert not (k == key and v == 'waiting') @gen_cluster(client=True, timeout=None) def test_interleave_computations(c, s, a, b): import distributed distributed.g = s xs = [delayed(slowinc)(i, delay=0.02) for i in range(30)] ys = [delayed(slowdec)(x, delay=0.02) for x in xs] zs = [delayed(slowadd)(x, y, delay=0.02) for x, y in zip(xs, ys)] total = delayed(sum)(zs) future = c.compute(total) done = ('memory', 'released') yield gen.sleep(0.1) while not s.tasks or any(s.processing.values()): yield gen.sleep(0.05) x_done = len([k for k in xs if s.task_state[k.key] in done]) y_done = len([k for k in ys if s.task_state[k.key] in done]) z_done = len([k for k in zs if s.task_state[k.key] in done]) assert x_done >= y_done >= z_done assert x_done < y_done + 10 assert y_done < z_done + 10 assert_no_data_loss(s) @pytest.mark.xfail(reason="Now prefer first-in-first-out") @gen_cluster(client=True, timeout=None) def test_interleave_computations_map(c, s, a, b): xs = c.map(slowinc, range(30), delay=0.02) ys = c.map(slowdec, xs, delay=0.02) zs = c.map(slowadd, xs, ys, delay=0.02) done = ('memory', 'released') while not s.tasks or any(s.processing.values()): yield gen.sleep(0.05) x_done = len([k for k in xs if s.task_state[k.key] in done]) y_done = len([k for k in ys if s.task_state[k.key] in done]) z_done = len([k for k in zs if s.task_state[k.key] in done]) assert x_done >= y_done >= z_done assert x_done < y_done + 10 assert y_done < z_done + 10 @gen_cluster(client=True) def test_scatter_dict_workers(c, s, a, b): yield c.scatter({'a': 10}, workers=[a.address, b.address]) assert 'a' in a.data or 'a' in b.data @slow @gen_test() def test_client_timeout(): loop = IOLoop.current() c = Client('127.0.0.1:57484', asynchronous=True) s = Scheduler(loop=loop) yield gen.sleep(4) try: s.start(('127.0.0.1', 57484)) except EnvironmentError: # port in use return start = time() while not c.scheduler_comm: yield gen.sleep(0.1) assert time() < start + 2 yield c.close() yield s.close() @gen_cluster(client=True) def test_submit_list_kwargs(c, s, a, b): futures = yield c.scatter([1, 2, 3]) def f(L=None): return sum(L) future = c.submit(f, L=futures) result = yield future assert result == 1 + 2 + 3 @gen_cluster(client=True) def test_map_list_kwargs(c, s, a, b): futures = yield c.scatter([1, 2, 3]) def f(i, L=None): return i + sum(L) futures = c.map(f, range(10), L=futures) results = yield c.gather(futures) assert results == [i + 6 for i in range(10)] @gen_cluster(client=True) def test_dont_clear_waiting_data(c, s, a, b): [x] = yield c.scatter([1]) y = c.submit(slowinc, x, delay=0.2) while y.key not in s.task_state: yield gen.sleep(0.01) [x] = yield c.scatter([1]) for i in range(5): assert s.waiting_data[x.key] yield gen.moment @gen_cluster(client=True) def test_get_future_error_simple(c, s, a, b): f = c.submit(div, 1, 0) yield wait(f) assert f.status == 'error' function, args, kwargs, deps = yield c._get_futures_error(f) # args contains only solid values, not keys assert function.__name__ == 'div' with pytest.raises(ZeroDivisionError): function(*args, **kwargs) @gen_cluster(client=True) def test_get_futures_error(c, s, a, b): x0 = delayed(dec)(2) y0 = delayed(dec)(1) x = delayed(div)(1, x0) y = delayed(div)(1, y0) tot = delayed(sum)(x, y) f = c.compute(tot) yield wait(f) assert f.status == 'error' function, args, kwargs, deps = yield c._get_futures_error(f) assert function.__name__ == 'div' assert args == (1, y0.key) @gen_cluster(client=True) def test_recreate_error_delayed(c, s, a, b): x0 = delayed(dec)(2) y0 = delayed(dec)(1) x = delayed(div)(1, x0) y = delayed(div)(1, y0) tot = delayed(sum)(x, y) f = c.compute(tot) assert f.status == 'pending' function, args, kwargs = yield c._recreate_error_locally(f) assert f.status == 'error' assert function.__name__ == 'div' assert args == (1, 0) with pytest.raises(ZeroDivisionError): function(*args, **kwargs) @gen_cluster(client=True) def test_recreate_error_futures(c, s, a, b): x0 = c.submit(dec, 2) y0 = c.submit(dec, 1) x = c.submit(div, 1, x0) y = c.submit(div, 1, y0) tot = c.submit(sum, x, y) f = c.compute(tot) assert f.status == 'pending' function, args, kwargs = yield c._recreate_error_locally(f) assert f.status == 'error' assert function.__name__ == 'div' assert args == (1, 0) with pytest.raises(ZeroDivisionError): function(*args, **kwargs) @gen_cluster(client=True) def test_recreate_error_collection(c, s, a, b): import dask.bag as db b = db.range(10, npartitions=4) b = b.map(lambda x: 1 / x) b = b.persist() f = c.compute(b) function, args, kwargs = yield c._recreate_error_locally(f) with pytest.raises(ZeroDivisionError): function(*args, **kwargs) dd = pytest.importorskip('dask.dataframe') import pandas as pd df = dd.from_pandas(pd.DataFrame({'a': [0, 1, 2, 3, 4]}), chunksize=2) def make_err(x): # because pandas would happily work with NaN if x == 0: raise ValueError return x df2 = df.a.map(make_err) f = c.compute(df2) function, args, kwargs = yield c._recreate_error_locally(f) with pytest.raises(ValueError): function(*args, **kwargs) # with persist df3 = c.persist(df2) function, args, kwargs = yield c._recreate_error_locally(df3) with pytest.raises(ValueError): function(*args, **kwargs) def test_recreate_error_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x0 = c.submit(dec, 2) y0 = c.submit(dec, 1) x = c.submit(div, 1, x0) y = c.submit(div, 1, y0) tot = c.submit(sum, x, y) f = c.compute(tot) with pytest.raises(ZeroDivisionError) as e: c.recreate_error_locally(f) assert f.status == 'error' def test_recreate_error_not_error(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: f = c.submit(dec, 2) with pytest.raises(ValueError) as e: c.recreate_error_locally(f) assert "No errored futures passed" in str(e) @gen_cluster(client=True) def test_retire_workers(c, s, a, b): assert s.workers == {a.address, b.address} yield c.scheduler.retire_workers(workers=[a.address], close_workers=True) assert s.workers == {b.address} start = time() while a.status != 'closed': yield gen.sleep(0.01) assert time() < start + 5 class MyException(Exception): pass @gen_cluster(client=True) def test_robust_unserializable(c, s, a, b): class Foo(object): def __getstate__(self): raise MyException() with pytest.raises(MyException): future = c.submit(identity, Foo()) futures = c.map(inc, range(10)) results = yield c.gather(futures) assert results == list(map(inc, range(10))) assert a.data and b.data @gen_cluster(client=True) def test_robust_undeserializable(c, s, a, b): class Foo(object): def __getstate__(self): return 1 def __setstate__(self, state): raise MyException('hello') future = c.submit(identity, Foo()) with pytest.raises(MyException): yield future futures = c.map(inc, range(10)) results = yield c.gather(futures) assert results == list(map(inc, range(10))) assert a.data and b.data @gen_cluster(client=True) def test_robust_undeserializable_function(c, s, a, b): class Foo(object): def __getstate__(self): return 1 def __setstate__(self, state): raise MyException('hello') def __call__(self, *args): return 1 future = c.submit(Foo(), 1) with pytest.raises(MyException) as e: yield future futures = c.map(inc, range(10)) results = yield c.gather(futures) assert results == list(map(inc, range(10))) assert a.data and b.data @gen_cluster(client=True) def test_fire_and_forget(c, s, a, b): future = c.submit(slowinc, 1, delay=0.1) import distributed def f(x): distributed.foo = 123 try: fire_and_forget(c.submit(f, future)) start = time() while not hasattr(distributed, 'foo'): yield gen.sleep(0.01) assert time() < start + 2 assert distributed.foo == 123 finally: del distributed.foo start = time() while len(s.task_state) > 1: yield gen.sleep(0.01) assert time() < start + 2 assert set(s.who_wants) == {future.key} assert set(s.task_state) == {future.key} @gen_cluster(client=True) def test_fire_and_forget_err(c, s, a, b): fire_and_forget(c.submit(div, 1, 0)) yield gen.sleep(0.1) # erred task should clear out quickly start = time() while s.task_state: yield gen.sleep(0.01) assert time() < start + 1 def test_quiet_client_close(loop): with captured_logger(logging.getLogger('distributed')) as logger: with Client(loop=loop, processes=False, threads_per_worker=4) as c: futures = c.map(slowinc, range(1000), delay=0.01) sleep(0.200) # stop part-way sleep(0.5) # let things settle out = logger.getvalue() assert not out @gen_cluster() def test_close(s, a, b): c = yield Client(s.address, asynchronous=True) future = c.submit(inc, 1) yield wait(future) assert c.id in s.wants_what yield c.close() start = time() while c.id in s.wants_what or s.task_state: yield gen.sleep(0.01) assert time() < start + 5 def test_threadsafe(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: def f(_): d = deque(maxlen=50) for i in range(100): future = c.submit(inc, random.randint(0, 100)) d.append(future) sleep(0.001) c.gather(list(d)) total = c.submit(sum, list(d)) return total.result() from concurrent.futures import ThreadPoolExecutor with ThreadPoolExecutor(20) as e: results = list(e.map(f, range(20))) assert results and all(results) del results @slow def test_threadsafe_get(loop): da = pytest.importorskip('dask.array') x = da.arange(100, chunks=(10,)) with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: def f(_): total = 0 for i in range(20): total += (x + random.randint(0, 20)).sum().compute() sleep(0.001) return total from concurrent.futures import ThreadPoolExecutor e = ThreadPoolExecutor(30) results = list(e.map(f, range(30))) assert results and all(results) @slow def test_threadsafe_compute(loop): da = pytest.importorskip('dask.array') x = da.arange(100, chunks=(10,)) with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: def f(_): total = 0 for i in range(20): future = c.compute((x + random.randint(0, 20)).sum()) total += future.result() sleep(0.001) return total from concurrent.futures import ThreadPoolExecutor e = ThreadPoolExecutor(30) results = list(e.map(f, range(30))) assert results and all(results) @gen_cluster(client=True) def test_identity(c, s, a, b): assert c.id.lower().startswith('client') assert a.id.lower().startswith('worker') assert b.id.lower().startswith('worker') assert s.id.lower().startswith('scheduler') @gen_cluster(client=True, ncores=[('127.0.0.1', 4)] * 2) def test_get_client(c, s, a, b): assert get_client() is c assert c.asynchronous def f(x): client = get_client() future = client.submit(inc, x) import distributed assert not client.asynchronous assert client is distributed.tmp_client return future.result() import distributed distributed.tmp_client = c try: futures = c.map(f, range(5)) results = yield c.gather(futures) assert results == list(map(inc, range(5))) finally: del distributed.tmp_client def test_get_client_no_cluster(): # Clean up any global workers added by other tests. This test requires that # there are no global workers. from distributed.worker import _global_workers del _global_workers[:] msg = 'No global client found and no address provided' with pytest.raises(ValueError, match=r'^{}$'.format(msg)): get_client() @gen_cluster(client=True) def test_serialize_collections(c, s, a, b): da = pytest.importorskip('dask.array') x = da.arange(10, chunks=(5,)).persist() def f(x): assert isinstance(x, da.Array) return x.sum().compute() future = c.submit(f, x) result = yield future assert result == sum(range(10)) @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 1, timeout=100) def test_secede_simple(c, s, a): def f(): client = get_client() secede() return client.submit(inc, 1).result() result = yield c.submit(f) assert result == 2 @slow @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2, timeout=60) def test_secede_balances(c, s, a, b): count = threading.active_count() def f(x): client = get_client() sleep(0.01) # do some work secede() futures = client.map(slowinc, range(10), pure=False, delay=0.01) total = client.submit(sum, futures).result() return total futures = c.map(f, range(100)) start = time() while not all(f.status == 'finished' for f in futures): yield gen.sleep(0.01) assert threading.active_count() < count + 50 # assert 0.005 < s.task_duration['f'] < 0.1 assert len(a.log) < 2 * len(b.log) assert len(b.log) < 2 * len(a.log) results = yield c.gather(futures) assert results == [sum(map(inc, range(10)))] * 100 @gen_cluster(client=True) def test_sub_submit_priority(c, s, a, b): def f(): client = get_client() client.submit(slowinc, 1, delay=0.2) future = c.submit(f) yield gen.sleep(0.1) if len(s.task_state) == 2: f_key = [k for k in s.task_state if k.startswith('f')][0] slowinc_key = [k for k in s.task_state if k.startswith('slowinc')][0] assert s.priorities[f_key] > s.priorities[slowinc_key] # lower values schedule first def test_get_client_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: results = c.run(lambda: get_worker().scheduler.address) assert results == {w['address']: s['address'] for w in [a, b]} results = c.run(lambda: get_client().scheduler.address) assert results == {w['address']: s['address'] for w in [a, b]} @gen_cluster(client=True) def test_serialize_collections_of_futures(c, s, a, b): pd = pytest.importorskip('pandas') dd = pytest.importorskip('dask.dataframe') from dask.dataframe.utils import assert_eq df = pd.DataFrame({'x': [1, 2, 3]}) ddf = dd.from_pandas(df, npartitions=2).persist() future = yield c.scatter(ddf) ddf2 = yield future df2 = yield c.compute(ddf2) assert_eq(df, df2) def test_serialize_collections_of_futures_sync(loop): pd = pytest.importorskip('pandas') dd = pytest.importorskip('dask.dataframe') from dask.dataframe.utils import assert_eq df = pd.DataFrame({'x': [1, 2, 3]}) with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: ddf = dd.from_pandas(df, npartitions=2).persist() future = c.scatter(ddf) result = future.result() assert_eq(result.compute(), df) assert future.type == dd.DataFrame assert c.submit(lambda x, y: assert_eq(x.compute(), y), future, df).result() def _dynamic_workload(x, delay=0.01): if delay == 'random': sleep(random.random() / 2) else: sleep(delay) if x > 4: return 4 secede() client = get_client() futures = client.map(_dynamic_workload, [x + i + 1 for i in range(2)], pure=False, delay=delay) total = client.submit(sum, futures) return total.result() def _test_dynamic_workloads_sync(loop, delay): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: future = c.submit(_dynamic_workload, 0, delay=delay) assert future.result(timeout=40) == 52 def test_dynamic_workloads_sync(loop): _test_dynamic_workloads_sync(loop, delay=0.02) @slow def test_dynamic_workloads_sync_random(loop): _test_dynamic_workloads_sync(loop, delay='random') @gen_cluster(client=True) def test_bytes_keys(c, s, a, b): key = b'inc-123' future = c.submit(inc, 1, key=key) result = yield future assert type(future.key) is bytes assert list(s.task_state)[0] == key assert key in a.data or key in b.data assert result == 2 @gen_cluster(client=True) def test_unicode_ascii_keys(c, s, a, b): # cross-version unicode type (py2: unicode, py3: str) uni_type = type(u"") key = u'inc-123' future = c.submit(inc, 1, key=key) result = yield future assert type(future.key) is uni_type assert list(s.task_state)[0] == key assert key in a.data or key in b.data assert result == 2 @gen_cluster(client=True) def test_unicode_keys(c, s, a, b): # cross-version unicode type (py2: unicode, py3: str) uni_type = type(u"") key = u'inc-123\u03bc' future = c.submit(inc, 1, key=key) result = yield future assert type(future.key) is uni_type assert list(s.task_state)[0] == key assert key in a.data or key in b.data assert result == 2 future2 = c.submit(inc, future) result2 = yield future2 assert result2 == 3 future3 = yield c.scatter({u'data-123': 123}) result3 = yield future3[u'data-123'] assert result3 == 123 def test_use_synchronous_client_in_async_context(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: @gen.coroutine def f(): x = yield c.scatter(123) y = c.submit(inc, x) z = yield c.gather(y) raise gen.Return(z) z = sync(loop, f) assert z == 124 def test_quiet_quit_when_cluster_leaves(loop_in_thread): from distributed import LocalCluster loop = loop_in_thread cluster = LocalCluster(loop=loop, scheduler_port=0, diagnostics_port=None, silence_logs=False) with captured_logger('distributed.comm') as sio: with Client(cluster, loop=loop) as client: futures = client.map(lambda x: x + 1, range(10)) sleep(0.05) cluster.close() sleep(0.05) text = sio.getvalue() assert not text def test_warn_executor(loop): with cluster() as (s, [a, b]): with warnings.catch_warnings(record=True) as record: with Executor(s['address'], loop=loop) as c: pass assert any('Client' in str(r.message) for r in record) @gen_cluster([('127.0.0.1', 4)] * 2, client=True) def test_call_stack_future(c, s, a, b): x = c.submit(slowdec, 1, delay=0.5) future = c.submit(slowinc, 1, delay=0.5) yield gen.sleep(0.1) results = yield [c.call_stack(future), c.call_stack(keys=[future.key])] assert all(list(first(result.values())) == [future.key] for result in results) assert results[0] == results[1] result = results[0] w = a if future.key in a.executing else b assert list(result) == [w.address] assert list(result[w.address]) == [future.key] assert 'slowinc' in str(result) assert 'slowdec' not in str(result) @gen_cluster([('127.0.0.1', 4)] * 2, client=True) def test_call_stack_all(c, s, a, b): future = c.submit(slowinc, 1, delay=0.5) yield gen.sleep(0.1) result = yield c.call_stack() w = a if a.executing else b assert list(result) == [w.address] assert list(result[w.address]) == [future.key] assert 'slowinc' in str(result) @gen_cluster([('127.0.0.1', 4)] * 2, client=True) def test_call_stack_collections(c, s, a, b): da = pytest.importorskip('dask.array') x = da.random.random(100, chunks=(10,)).map_blocks(slowinc, delay=0.5).persist() while not a.executing and not b.executing: yield gen.sleep(0.001) result = yield c.call_stack(x) assert result @gen_cluster([('127.0.0.1', 4)] * 2, client=True) def test_call_stack_collections_all(c, s, a, b): da = pytest.importorskip('dask.array') x = da.random.random(100, chunks=(10,)).map_blocks(slowinc, delay=0.5).persist() while not a.executing and not b.executing: yield gen.sleep(0.001) result = yield c.call_stack() assert result @gen_cluster(client=True, worker_kwargs={'profile_cycle_interval': 100}) def test_profile(c, s, a, b): futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) yield wait(futures) x = yield c.profile(start=time() + 10, stop=time() + 20) assert not x['count'] x = yield c.profile(start=0, stop=time()) assert x['count'] == sum(p['count'] for _, p in a.profile_history) + a.profile_recent['count'] y = yield c.profile(start=time() - 0.300, stop=time()) assert 0 < y['count'] < x['count'] assert not any(p['count'] for _, p in b.profile_history) result = yield c.profile(workers=b.address) assert not result['count'] @gen_cluster(client=True, worker_kwargs={'profile_cycle_interval': 100}) def test_profile_keys(c, s, a, b): x = c.map(slowinc, range(10), delay=0.05, workers=a.address) y = c.map(slowdec, range(10), delay=0.05, workers=a.address) yield wait(x + y) xp = yield c.profile('slowinc') yp = yield c.profile('slowdec') p = yield c.profile() assert p['count'] == xp['count'] + yp['count'] with captured_logger(logging.getLogger('distributed')) as logger: prof = yield c.profile('does-not-exist') assert prof == profile.create() out = logger.getvalue() assert not out @gen_cluster() def test_client_with_name(s, a, b): with captured_logger('distributed.scheduler') as sio: client = yield Client(s.address, asynchronous=True, name='foo', silence_logs=False) assert 'foo' in client.id yield client.close() text = sio.getvalue() assert 'foo' in text @gen_cluster(client=True) def test_future_defaults_to_default_client(c, s, a, b): x = c.submit(inc, 1) yield wait(x) future = Future(x.key) assert future.client is c @gen_cluster(client=True) def test_future_auto_inform(c, s, a, b): x = c.submit(inc, 1) yield wait(x) client = yield Client(s.address, asynchronous=True) future = Future(x.key, client) start = time() while future.status != 'finished': yield gen.sleep(0.01) assert time() < start + 1 yield client.close() def test_client_async_before_loop_starts(): loop = IOLoop() client = Client(asynchronous=True, loop=loop) assert client.asynchronous client.close() # Avoid long wait for cluster close at shutdown loop.close() @slow @gen_cluster(client=True, Worker=Nanny if PY3 else Worker, timeout=60) def test_nested_compute(c, s, a, b): def fib(x): assert get_worker().get_current_task() if x < 2: return x a = delayed(fib)(x - 1) b = delayed(fib)(x - 2) c = a + b return c.compute() future = c.submit(fib, 8) result = yield future assert result == 21 assert len(s.transition_log) > 50 @gen_cluster(client=True) def test_task_metadata(c, s, a, b): yield c.set_metadata('x', 1) result = yield c.get_metadata('x') assert result == 1 future = c.submit(inc, 1) key = future.key yield wait(future) yield c.set_metadata(key, 123) result = yield c.get_metadata(key) assert result == 123 del future while key in s.task_state: yield gen.sleep(0.01) with pytest.raises(KeyError): yield c.get_metadata(key) result = yield c.get_metadata(key, None) assert result is None yield c.set_metadata(['x', 'a'], 1) result = yield c.get_metadata('x') assert result == {'a': 1} yield c.set_metadata(['x', 'b'], 2) result = yield c.get_metadata('x') assert result == {'a': 1, 'b': 2} result = yield c.get_metadata(['x', 'a']) assert result == 1 yield c.set_metadata(['x', 'a', 'c', 'd'], 1) result = yield c.get_metadata('x') assert result == {'a': {'c': {'d': 1}}, 'b': 2} @gen_cluster(client=True) def test_logs(c, s, a, b): yield wait(c.map(inc, range(5))) logs = yield c.get_scheduler_logs(n=5) assert logs for _, msg in logs: assert 'distributed.scheduler' in msg w_logs = yield c.get_worker_logs(n=5) assert set(w_logs.keys()) == {a.address, b.address} for log in w_logs.values(): for _, msg in log: assert 'distributed.worker' in msg @gen_cluster(client=True) def test_avoid_delayed_finalize(c, s, a, b): x = delayed(inc)(1) future = c.compute(x) result = yield future assert result == 2 assert list(s.tasks) == [future.key] == [x.key] @gen_cluster() def test_config_scheduler_address(s, a, b): from distributed import config config['scheduler-address'] = s.address with captured_logger('distributed.client') as sio: c = yield Client(asynchronous=True) assert c.scheduler.address == s.address text = sio.getvalue() assert s.address in text del config['scheduler-address'] yield c.close() if sys.version_info >= (3, 5): from distributed.tests.py3_test_client import * # flake8: noqa distributed-1.20.2/distributed/tests/test_client_executor.py000066400000000000000000000204361321233345200244120ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import random import time from concurrent.futures import ( CancelledError, TimeoutError, Future, wait, as_completed, FIRST_COMPLETED, FIRST_EXCEPTION) import pytest from toolz import take from distributed.client import Client from distributed.utils_test import (slowinc, slowadd, slowdec, inc, throws, cluster) from distributed.utils_test import loop # flake8: noqa def number_of_processing_tasks(client): return sum(len(v) for k, v in client.processing().items()) def test_submit(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: with c.get_executor() as e: f1 = e.submit(slowadd, 1, 2) assert isinstance(f1, Future) f2 = e.submit(slowadd, 3, y=4) f3 = e.submit(throws, "foo") f4 = e.submit(slowadd, x=5, y=6) assert f1.result() == 3 assert f2.result() == 7 with pytest.raises(RuntimeError): f3.result() assert f4.result() == 11 def test_as_completed(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: with c.get_executor() as e: N = 10 fs = [e.submit(slowinc, i, delay=0.02) for i in range(N)] expected = set(range(1, N + 1)) for f in as_completed(fs): res = f.result() assert res in expected expected.remove(res) assert not expected def test_wait(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: with c.get_executor(pure=False) as e: N = 10 fs = [e.submit(slowinc, i, delay=0.05) for i in range(N)] res = wait(fs, timeout=0.01) assert len(res.not_done) > 0 res = wait(fs) assert len(res.not_done) == 0 assert res.done == set(fs) fs = [e.submit(slowinc, i, delay=0.05) for i in range(N)] res = wait(fs, return_when=FIRST_COMPLETED) assert len(res.not_done) > 0 assert len(res.done) >= 1 res = wait(fs) assert len(res.not_done) == 0 assert res.done == set(fs) fs = [e.submit(slowinc, i, delay=0.05) for i in range(N)] fs += [e.submit(throws, None)] fs += [e.submit(slowdec, i, delay=0.05) for i in range(N)] res = wait(fs, return_when=FIRST_EXCEPTION) assert len(res.not_done) > 0 assert len(res.done) >= N - 2 # likely, unless tasks get reordered errors = [] for fs in res.done: try: fs.result() except RuntimeError as e: errors.append(e) assert len(errors) == 1 assert "hello" in str(errors[0]) def test_cancellation(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: with c.get_executor(pure=False) as e: fut = e.submit(time.sleep, 2.0) start = time.time() while number_of_processing_tasks(c) == 0: assert time.time() < start + 1 time.sleep(0.01) assert not fut.done() fut.cancel() assert fut.cancelled() start = time.time() while number_of_processing_tasks(c) != 0: assert time.time() < start + 1 time.sleep(0.01) with pytest.raises(CancelledError): fut.result() # With wait() with c.get_executor(pure=False) as e: N = 10 fs = [e.submit(slowinc, i, delay=0.02) for i in range(N)] fs[3].cancel() res = wait(fs, return_when=FIRST_COMPLETED) assert len(res.not_done) > 0 assert len(res.done) >= 1 assert fs[3] in res.done assert fs[3].cancelled() # With as_completed() with c.get_executor(pure=False) as e: N = 10 fs = [e.submit(slowinc, i, delay=0.02) for i in range(N)] fs[3].cancel() fs[8].cancel() n_cancelled = sum(f.cancelled() for f in as_completed(fs)) assert n_cancelled == 2 def test_map(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: with c.get_executor() as e: N = 10 it = e.map(inc, range(N)) expected = set(range(1, N + 1)) for x in it: expected.remove(x) assert not expected with c.get_executor(pure=False) as e: N = 10 it = e.map(slowinc, range(N), [0.1] * N, timeout=0.4) results = [] with pytest.raises(TimeoutError): for x in it: results.append(x) assert 2 <= len(results) < 7 with c.get_executor(pure=False) as e: N = 10 # Not consuming the iterator will cancel remaining tasks it = e.map(slowinc, range(N), [0.1] * N) for x in take(2, it): pass # Some tasks still processing assert number_of_processing_tasks(c) > 0 # Garbage collect the iterator => remaining tasks are cancelled del it assert number_of_processing_tasks(c) == 0 def get_random(): return random.random() def test_pure(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: N = 10 with c.get_executor() as e: fs = [e.submit(get_random) for i in range(N)] res = [fut.result() for fut in as_completed(fs)] assert len(set(res)) < len(res) with c.get_executor(pure=False) as e: fs = [e.submit(get_random) for i in range(N)] res = [fut.result() for fut in as_completed(fs)] assert len(set(res)) == len(res) def test_workers(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: N = 10 with c.get_executor(workers=[b['address']]) as e: fs = [e.submit(slowinc, i) for i in range(N)] wait(fs) has_what = c.has_what() assert not has_what.get(a['address']) assert len(has_what[b['address']]) == N def test_unsupported_arguments(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: with pytest.raises(TypeError) as excinfo: c.get_executor(workers=[b['address']], foo=1, bar=2) assert ("unsupported arguments to ClientExecutor: ['bar', 'foo']" in str(excinfo.value)) def test_shutdown(loop): with cluster(active_rpc_timeout=10) as (s, [a, b]): with Client(s['address'], loop=loop) as c: # shutdown(wait=True) waits for pending tasks to finish e = c.get_executor() fut = e.submit(time.sleep, 1.0) t1 = time.time() e.shutdown() dt = time.time() - t1 assert 0.5 <= dt <= 2.0 time.sleep(0.1) # wait for future outcome to propagate assert fut.done() fut.result() # doesn't raise with pytest.raises(RuntimeError): e.submit(time.sleep, 1.0) # shutdown(wait=False) cancels pending tasks e = c.get_executor() fut = e.submit(time.sleep, 2.0) t1 = time.time() e.shutdown(wait=False) dt = time.time() - t1 assert dt < 0.5 time.sleep(0.1) # wait for future outcome to propagate assert fut.cancelled() with pytest.raises(RuntimeError): e.submit(time.sleep, 1.0) distributed-1.20.2/distributed/tests/test_collections.py000066400000000000000000000141401321233345200235270ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import pytest pytest.importorskip('numpy') pytest.importorskip('pandas') import dask import dask.dataframe as dd import dask.bag as db from distributed import Client from distributed.client import wait from distributed.utils_test import cluster, gen_cluster from distributed.utils_test import loop # flake8: noqa import numpy as np import pandas as pd import pandas.util.testing as tm dfs = [pd.DataFrame({'x': [1, 2, 3]}, index=[0, 10, 20]), pd.DataFrame({'x': [4, 5, 6]}, index=[30, 40, 50]), pd.DataFrame({'x': [7, 8, 9]}, index=[60, 70, 80])] def assert_equal(a, b): assert type(a) == type(b) if isinstance(a, pd.DataFrame): tm.assert_frame_equal(a, b) elif isinstance(a, pd.Series): tm.assert_series_equal(a, b) elif isinstance(a, pd.Index): tm.assert_index_equal(a, b) else: assert a == b @gen_cluster(timeout=240, client=True) def test_dataframes(c, s, a, b): df = pd.DataFrame({'x': np.random.random(1000), 'y': np.random.random(1000)}, index=np.arange(1000)) ldf = dd.from_pandas(df, npartitions=10) rdf = c.persist(ldf) assert rdf.divisions == ldf.divisions remote = c.compute(rdf) result = yield remote tm.assert_frame_equal(result, ldf.compute(get=dask.get)) exprs = [lambda df: df.x.mean(), lambda df: df.y.std(), lambda df: df.assign(z=df.x + df.y).drop_duplicates(), lambda df: df.index, lambda df: df.x, lambda df: df.x.cumsum(), lambda df: df.groupby(['x', 'y']).count(), lambda df: df.loc[50:75]] for f in exprs: local = f(ldf).compute(get=dask.get) remote = c.compute(f(rdf)) remote = yield remote assert_equal(local, remote) @gen_cluster(client=True) def test__dask_array_collections(c, s, a, b): import dask.array as da x_dsk = {('x', i, j): np.random.random((3, 3)) for i in range(3) for j in range(2)} y_dsk = {('y', i, j): np.random.random((3, 3)) for i in range(2) for j in range(3)} x_futures = yield c._scatter(x_dsk) y_futures = yield c._scatter(y_dsk) dt = np.random.random(0).dtype x_local = da.Array(x_dsk, 'x', ((3, 3, 3), (3, 3)), dt) y_local = da.Array(y_dsk, 'y', ((3, 3), (3, 3, 3)), dt) x_remote = da.Array(x_futures, 'x', ((3, 3, 3), (3, 3)), dt) y_remote = da.Array(y_futures, 'y', ((3, 3), (3, 3, 3)), dt) exprs = [lambda x, y: x.T + y, lambda x, y: x.mean() + y.mean(), lambda x, y: x.dot(y).std(axis=0), lambda x, y: x - x.mean(axis=1)[:, None]] for expr in exprs: local = expr(x_local, y_local).compute(get=dask.get) remote = c.compute(expr(x_remote, y_remote)) remote = yield remote assert np.all(local == remote) @gen_cluster(client=True) def test_bag_groupby_tasks_default(c, s, a, b): with dask.set_options(get=c.get): b = db.range(100, npartitions=10) b2 = b.groupby(lambda x: x % 13) assert not any('partd' in k[0] for k in b2.dask) @pytest.mark.parametrize('wait', [wait, lambda x: None]) def test_dataframe_set_index_sync(loop, wait): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: with dask.set_options(get=c.get): df = dd.demo.make_timeseries('2000', '2001', {'value': float, 'name': str, 'id': int}, freq='2H', partition_freq='1M', seed=1) df = c.persist(df) wait(df) df2 = df.set_index('name', shuffle='tasks') df2 = c.persist(df2) assert len(df2) def test_loc_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: df = pd.util.testing.makeTimeDataFrame() ddf = dd.from_pandas(df, npartitions=10) ddf.loc['2000-01-17':'2000-01-24'].compute(get=c.get) def test_rolling_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: df = pd.util.testing.makeTimeDataFrame() ddf = dd.from_pandas(df, npartitions=10) dd.rolling_mean(ddf.A, 2).compute(get=c.get) @gen_cluster(client=True) def test_loc(c, s, a, b): df = pd.util.testing.makeTimeDataFrame() ddf = dd.from_pandas(df, npartitions=10) future = c.compute(ddf.loc['2000-01-17':'2000-01-24']) yield future def test_dataframe_groupby_tasks(loop): df = pd.util.testing.makeTimeDataFrame() df['A'] = df.A // 0.1 df['B'] = df.B // 0.1 ddf = dd.from_pandas(df, npartitions=10) with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: with dask.set_options(get=c.get): for ind in [lambda x: 'A', lambda x: x.A]: a = df.groupby(ind(df)).apply(len) b = ddf.groupby(ind(ddf)).apply(len, meta=int) assert_equal(a, b.compute(get=dask.get).sort_index()) assert not any('partd' in k[0] for k in b.dask) a = df.groupby(ind(df)).B.apply(len) b = ddf.groupby(ind(ddf)).B.apply(len, meta=('B', int)) assert_equal(a, b.compute(get=dask.get).sort_index()) assert not any('partd' in k[0] for k in b.dask) with pytest.raises(NotImplementedError): ddf.groupby(ddf[['A', 'B']]).apply(len, meta=int) a = df.groupby(['A', 'B']).apply(len) b = ddf.groupby(['A', 'B']).apply(len, meta=int) assert_equal(a, b.compute(get=dask.get).sort_index()) @gen_cluster(client=True) def test_sparse_arrays(c, s, a, b): sparse = pytest.importorskip('sparse') da = pytest.importorskip('dask.array') x = da.random.random((100, 10), chunks=(10, 10)) x[x < 0.95] = 0 s = x.map_blocks(sparse.COO) future = c.compute(s.sum(axis=0)[:10]) yield future distributed-1.20.2/distributed/tests/test_compatibility.py000066400000000000000000000013161321233345200240630ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from distributed.compatibility import ( gzip_compress, gzip_decompress, finalize) def test_gzip(): b = b'Hello, world!' c = gzip_compress(b) d = gzip_decompress(c) assert b == d def test_finalize(): class C(object): pass l = [] def cb(value): l.append(value) o = C() f = finalize(o, cb, 1) assert f in f._select_for_exit() f.atexit = False assert f not in f._select_for_exit() assert not l del o assert l.pop() == 1 o = C() fin = finalize(o, cb, 2) assert fin.alive fin() assert not fin.alive assert l.pop() == 2 del o assert not l distributed-1.20.2/distributed/tests/test_config.py000066400000000000000000000175541321233345200224720ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import logging import subprocess import sys import tempfile import os import pytest from distributed.utils_test import (captured_handler, captured_logger, new_config, new_config_file) from distributed.config import initialize_logging, set_config, config def dump_logger_list(): root = logging.getLogger() loggers = root.manager.loggerDict print() print("== Loggers (name, level, effective level, propagate) ==") def logger_info(name, logger): return (name, logging.getLevelName(logger.level), logging.getLevelName(logger.getEffectiveLevel()), logger.propagate) infos = [] infos.append(logger_info('', root)) for name, logger in sorted(loggers.items()): if not isinstance(logger, logging.Logger): # Skip 'PlaceHolder' objects continue assert logger.name == name infos.append(logger_info(name, logger)) for info in infos: print("%-40s %-8s %-8s %-5s" % info) print() def test_logging_default(): """ Test default logging configuration. """ d = logging.getLogger('distributed') assert len(d.handlers) == 1 assert isinstance(d.handlers[0], logging.StreamHandler) # Work around Bokeh messing with the root logger level # https://github.com/bokeh/bokeh/issues/5793 root = logging.getLogger('') old_root_level = root.level root.setLevel('WARN') try: dfb = logging.getLogger('distributed.foo.bar') f = logging.getLogger('foo') fb = logging.getLogger('foo.bar') with captured_handler(d.handlers[0]) as distributed_log: with captured_logger(root, level=logging.ERROR) as foreign_log: h = logging.StreamHandler(foreign_log) fmt = '[%(levelname)s in %(name)s] - %(message)s' h.setFormatter(logging.Formatter(fmt)) fb.addHandler(h) fb.propagate = False # For debugging dump_logger_list() d.debug("1: debug") d.info("2: info") dfb.info("3: info") fb.info("4: info") fb.error("5: error") f.info("6: info") f.error("7: error") distributed_log = distributed_log.getvalue().splitlines() foreign_log = foreign_log.getvalue().splitlines() # distributed log is configured at INFO level by default assert distributed_log == [ "distributed - INFO - 2: info", "distributed.foo.bar - INFO - 3: info", ] # foreign logs should be unaffected by distributed's logging # configuration. They get the default ERROR level from logging. assert foreign_log == [ "[ERROR in foo.bar] - 5: error", "7: error", ] finally: root.setLevel(old_root_level) def test_logging_empty_simple(): with new_config({}): test_logging_default() def test_logging_simple(): """ Test simple ("old-style") logging configuration. """ c = { 'logging': { 'distributed.foo': 'info', 'distributed.foo.bar': 'error', } } # Must test using a subprocess to avoid wrecking pre-existing configuration with new_config_file(c): code = """if 1: import logging from distributed.utils_test import captured_handler d = logging.getLogger('distributed') assert len(d.handlers) == 1 assert isinstance(d.handlers[0], logging.StreamHandler) df = logging.getLogger('distributed.foo') dfb = logging.getLogger('distributed.foo.bar') with captured_handler(d.handlers[0]) as distributed_log: df.info("1: info") dfb.warning("2: warning") dfb.error("3: error") distributed_log = distributed_log.getvalue().splitlines() assert distributed_log == [ "distributed.foo - INFO - 1: info", "distributed.foo.bar - ERROR - 3: error", ] """ subprocess.check_call([sys.executable, "-c", code]) def test_logging_extended(): """ Test extended ("new-style") logging configuration. """ c = { 'logging': { 'version': '1', 'formatters': { 'simple': { 'format': '%(levelname)s: %(name)s: %(message)s', }, }, 'handlers': { 'console': { 'class': 'logging.StreamHandler', 'stream': 'ext://sys.stderr', 'formatter': 'simple', }, }, 'loggers': { 'distributed.foo': { 'level': 'INFO', #'handlers': ['console'], }, 'distributed.foo.bar': { 'level': 'ERROR', #'handlers': ['console'], }, }, 'root': { 'level': 'WARNING', 'handlers': ['console'], }, }, } # Must test using a subprocess to avoid wrecking pre-existing configuration with new_config_file(c): code = """if 1: import logging from distributed.utils_test import captured_handler root = logging.getLogger() d = logging.getLogger('distributed') df = logging.getLogger('distributed.foo') dfb = logging.getLogger('distributed.foo.bar') with captured_handler(root.handlers[0]) as root_log: df.info("1: info") dfb.warning("2: warning") dfb.error("3: error") d.info("4: info") d.warning("5: warning") root_log = root_log.getvalue().splitlines() print(root_log) assert root_log == [ "INFO: distributed.foo: 1: info", "ERROR: distributed.foo.bar: 3: error", "WARNING: distributed: 5: warning", ] """ subprocess.check_call([sys.executable, "-c", code]) def test_logging_mutual_exclusive(): """ Ensure that 'logging-file-config' and 'logging' have to be mutual exclusive. """ config = {'logging': {'dask': 'warning'}, 'logging-file-config': '/path/to/config'} with pytest.raises(RuntimeError): initialize_logging(config) def test_logging_file_config(): """ Test `logging-file-config` logging configuration """ logging_config_contents = """ [handlers] keys=console [formatters] keys=simple [loggers] keys=root, foo, foo_bar [handler_console] class=StreamHandler level=INFO formatter=simple args=(sys.stdout,) [formatter_simple] format=%(levelname)s: %(name)s: %(message)s [logger_root] level=WARNING handlers=console [logger_foo] level=INFO handlers=console qualname=foo [logger_foo_bar] level=ERROR handlers=console qualname=foo.bar """ with tempfile.NamedTemporaryFile(mode='w', delete=False) as logging_config: logging_config.write(logging_config_contents) dask_config = {'logging-file-config': logging_config.name} with new_config_file(dask_config): code = """if 1: import logging from distributed import config foo = logging.getLogger('foo') bar = logging.getLogger('foo.bar') assert logging.INFO == foo.getEffectiveLevel() assert logging.ERROR == bar.getEffectiveLevel() """ subprocess.check_call([sys.executable, "-c", code]) os.remove(logging_config.name) def test_set_config(): assert 'foo' not in config with set_config(foo=1): assert config['foo'] == 1 assert 'foo' not in config distributed-1.20.2/distributed/tests/test_core.py000066400000000000000000000366271321233345200221570ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from contextlib import contextmanager import os import socket import weakref from tornado import gen import pytest from distributed.compatibility import finalize from distributed.config import set_config from distributed.core import (pingpong, Server, rpc, connect, send_recv, coerce_to_address, ConnectionPool) from distributed.metrics import time from distributed.protocol import to_serialize from distributed.utils import get_ip, get_ipv6 from distributed.utils_test import ( slow, gen_test, gen_cluster, has_ipv6, assert_can_connect, assert_cannot_connect, assert_can_connect_from_everywhere_4, assert_can_connect_from_everywhere_4_6, assert_can_connect_from_everywhere_6, assert_can_connect_locally_4, assert_can_connect_locally_6, tls_security, captured_logger) from distributed.utils_test import loop # flake8: noqa EXTERNAL_IP4 = get_ip() if has_ipv6(): EXTERNAL_IP6 = get_ipv6() def echo(comm, x): return x class CountedObject(object): """ A class which counts the number of live instances. """ n_instances = 0 # Use __new__, as __init__ can be bypassed by pickle. def __new__(cls): cls.n_instances += 1 obj = object.__new__(cls) finalize(obj, cls._finalize) return obj @classmethod def _finalize(cls, *args): cls.n_instances -= 1 def echo_serialize(comm, x): return {'result': to_serialize(x)} def test_server(loop): """ Simple Server test. """ @gen.coroutine def f(): server = Server({'ping': pingpong}) with pytest.raises(ValueError): server.port server.listen(8881) assert server.port == 8881 assert server.address == ('tcp://%s:8881' % get_ip()) for addr in ('127.0.0.1:8881', 'tcp://127.0.0.1:8881', server.address): comm = yield connect(addr) n = yield comm.write({'op': 'ping'}) assert isinstance(n, int) assert 4 <= n <= 1000 response = yield comm.read() assert response == b'pong' yield comm.write({'op': 'ping', 'close': True}) response = yield comm.read() assert response == b'pong' yield comm.close() server.stop() loop.run_sync(f) class MyServer(Server): default_port = 8756 @gen_test() def test_server_listen(): """ Test various Server.listen() arguments and their effect. """ @contextmanager def listen_on(cls, *args, **kwargs): server = cls({}) server.listen(*args, **kwargs) try: yield server finally: server.stop() # Note server.address is the concrete, contactable address with listen_on(Server, 7800) as server: assert server.port == 7800 assert server.address == 'tcp://%s:%d' % (EXTERNAL_IP4, server.port) yield assert_can_connect(server.address) yield assert_can_connect_from_everywhere_4_6(server.port) with listen_on(Server) as server: assert server.port > 0 assert server.address == 'tcp://%s:%d' % (EXTERNAL_IP4, server.port) yield assert_can_connect(server.address) yield assert_can_connect_from_everywhere_4_6(server.port) with listen_on(MyServer) as server: assert server.port == MyServer.default_port assert server.address == 'tcp://%s:%d' % (EXTERNAL_IP4, server.port) yield assert_can_connect(server.address) yield assert_can_connect_from_everywhere_4_6(server.port) with listen_on(Server, ('', 7801)) as server: assert server.port == 7801 assert server.address == 'tcp://%s:%d' % (EXTERNAL_IP4, server.port) yield assert_can_connect(server.address) yield assert_can_connect_from_everywhere_4_6(server.port) with listen_on(Server, 'tcp://:7802') as server: assert server.port == 7802 assert server.address == 'tcp://%s:%d' % (EXTERNAL_IP4, server.port) yield assert_can_connect(server.address) yield assert_can_connect_from_everywhere_4_6(server.port) # Only IPv4 with listen_on(Server, ('0.0.0.0', 7810)) as server: assert server.port == 7810 assert server.address == 'tcp://%s:%d' % (EXTERNAL_IP4, server.port) yield assert_can_connect(server.address) yield assert_can_connect_from_everywhere_4(server.port) with listen_on(Server, ('127.0.0.1', 7811)) as server: assert server.port == 7811 assert server.address == 'tcp://127.0.0.1:%d' % server.port yield assert_can_connect(server.address) yield assert_can_connect_locally_4(server.port) with listen_on(Server, 'tcp://127.0.0.1:7812') as server: assert server.port == 7812 assert server.address == 'tcp://127.0.0.1:%d' % server.port yield assert_can_connect(server.address) yield assert_can_connect_locally_4(server.port) # Only IPv6 if has_ipv6(): with listen_on(Server, ('::', 7813)) as server: assert server.port == 7813 assert server.address == 'tcp://[%s]:%d' % (EXTERNAL_IP6, server.port) yield assert_can_connect(server.address) yield assert_can_connect_from_everywhere_6(server.port) with listen_on(Server, ('::1', 7814)) as server: assert server.port == 7814 assert server.address == 'tcp://[::1]:%d' % server.port yield assert_can_connect(server.address) yield assert_can_connect_locally_6(server.port) with listen_on(Server, 'tcp://[::1]:7815') as server: assert server.port == 7815 assert server.address == 'tcp://[::1]:%d' % server.port yield assert_can_connect(server.address) yield assert_can_connect_locally_6(server.port) # TLS sec = tls_security() with listen_on(Server, 'tls://', listen_args=sec.get_listen_args('scheduler')) as server: assert server.address.startswith('tls://') yield assert_can_connect(server.address, connection_args=sec.get_connection_args('client')) # InProc with listen_on(Server, 'inproc://') as server: inproc_addr1 = server.address assert inproc_addr1.startswith('inproc://%s/%d/' % (get_ip(), os.getpid())) yield assert_can_connect(inproc_addr1) with listen_on(Server, 'inproc://') as server2: inproc_addr2 = server2.address assert inproc_addr2.startswith('inproc://%s/%d/' % (get_ip(), os.getpid())) yield assert_can_connect(inproc_addr2) yield assert_can_connect(inproc_addr1) yield assert_cannot_connect(inproc_addr2) @gen.coroutine def check_rpc(listen_addr, rpc_addr=None, listen_args=None, connection_args=None): server = Server({'ping': pingpong}) server.listen(listen_addr, listen_args=listen_args) if rpc_addr is None: rpc_addr = server.address with rpc(rpc_addr, connection_args=connection_args) as remote: response = yield remote.ping() assert response == b'pong' assert remote.comms response = yield remote.ping(close=True) assert response == b'pong' response = yield remote.ping() assert response == b'pong' assert not remote.comms assert remote.status == 'closed' server.stop() @gen_test() def test_rpc_default(): yield check_rpc(8883, '127.0.0.1:8883') yield check_rpc(8883) @gen_test() def test_rpc_tcp(): yield check_rpc('tcp://:8883', 'tcp://127.0.0.1:8883') yield check_rpc('tcp://') @gen_test() def test_rpc_tls(): sec = tls_security() yield check_rpc('tcp://', None, sec.get_listen_args('scheduler'), sec.get_connection_args('worker')) @gen_test() def test_rpc_inproc(): yield check_rpc('inproc://', None) def test_rpc_inputs(): L = [rpc('127.0.0.1:8884'), rpc(('127.0.0.1', 8884)), rpc('tcp://127.0.0.1:8884'), ] assert all(r.address == 'tcp://127.0.0.1:8884' for r in L), L for r in L: r.close_rpc() @gen.coroutine def check_rpc_message_lifetime(*listen_args): # Issue #956: rpc arguments and result shouldn't be kept alive longer # than necessary server = Server({'echo': echo_serialize}) server.listen(*listen_args) # Sanity check obj = CountedObject() assert CountedObject.n_instances == 1 del obj assert CountedObject.n_instances == 0 with rpc(server.address) as remote: obj = CountedObject() res = yield remote.echo(x=to_serialize(obj)) assert isinstance(res['result'], CountedObject) # Make sure resource cleanup code in coroutines runs yield gen.sleep(0.05) w1 = weakref.ref(obj) w2 = weakref.ref(res['result']) del obj, res assert w1() is None assert w2() is None # If additional instances were created, they were deleted as well assert CountedObject.n_instances == 0 @gen_test() def test_rpc_message_lifetime_default(): yield check_rpc_message_lifetime() @gen_test() def test_rpc_message_lifetime_tcp(): yield check_rpc_message_lifetime('tcp://') @gen_test() def test_rpc_message_lifetime_inproc(): yield check_rpc_message_lifetime('inproc://') @gen.coroutine def check_rpc_with_many_connections(listen_arg): @gen.coroutine def g(): for i in range(10): yield remote.ping() server = Server({'ping': pingpong}) server.listen(listen_arg) remote = rpc(server.address) yield [g() for i in range(10)] server.stop() remote.close_comms() assert all(comm.closed() for comm in remote.comms) @gen_test() def test_rpc_with_many_connections_tcp(): yield check_rpc_with_many_connections('tcp://') @gen_test() def test_rpc_with_many_connections_inproc(): yield check_rpc_with_many_connections('inproc://') @gen.coroutine def check_large_packets(listen_arg): """ tornado has a 100MB cap by default """ server = Server({'echo': echo}) server.listen(listen_arg) data = b'0' * int(200e6) # slightly more than 100MB conn = rpc(server.address) result = yield conn.echo(x=data) assert result == data d = {'x': data} result = yield conn.echo(x=d) assert result == d conn.close_comms() server.stop() @slow @gen_test() def test_large_packets_tcp(): yield check_large_packets('tcp://') @gen_test() def test_large_packets_inproc(): yield check_large_packets('inproc://') @gen.coroutine def check_identity(listen_arg): server = Server({}) server.listen(listen_arg) with rpc(server.address) as remote: a = yield remote.identity() b = yield remote.identity() assert a['type'] == 'Server' assert a['id'] == b['id'] server.stop() @gen_test() def test_identity_tcp(): yield check_identity('tcp://') @gen_test() def test_identity_inproc(): yield check_identity('inproc://') def test_ports(loop): port = 9876 server = Server({}, io_loop=loop) server.listen(port) try: assert server.port == port with pytest.raises((OSError, socket.error)): server2 = Server({}, io_loop=loop) server2.listen(port) finally: server.stop() try: server3 = Server({}, io_loop=loop) server3.listen(0) assert isinstance(server3.port, int) assert server3.port > 1024 finally: server3.stop() def stream_div(stream=None, x=None, y=None): return x / y @gen_test() def test_errors(): server = Server({'div': stream_div}) server.listen(0) with rpc(('127.0.0.1', server.port)) as r: with pytest.raises(ZeroDivisionError): yield r.div(x=1, y=0) @gen_test() def test_connect_raises(): with pytest.raises((gen.TimeoutError, IOError)): yield connect('127.0.0.1:58259', timeout=0.01) @gen_test() def test_send_recv_args(): server = Server({'echo': echo}) server.listen(0) comm = yield connect(server.address) result = yield send_recv(comm, op='echo', x=b'1') assert result == b'1' assert not comm.closed() result = yield send_recv(comm, op='echo', x=b'2', reply=False) assert result is None assert not comm.closed() result = yield send_recv(comm, op='echo', x=b'3', close=True) assert result == b'3' assert comm.closed() server.stop() def test_coerce_to_address(): for arg in ['127.0.0.1:8786', ('127.0.0.1', 8786), ('127.0.0.1', '8786')]: assert coerce_to_address(arg) == 'tcp://127.0.0.1:8786' @gen_test() def test_connection_pool(): @gen.coroutine def ping(comm, delay=0.1): yield gen.sleep(delay) raise gen.Return('pong') servers = [Server({'ping': ping}) for i in range(10)] for server in servers: server.listen(0) rpc = ConnectionPool(limit=5) # Reuse connections yield [rpc(ip='127.0.0.1', port=s.port).ping() for s in servers[:5]] yield [rpc(s.address).ping() for s in servers[:5]] yield [rpc('127.0.0.1:%d' % s.port).ping() for s in servers[:5]] yield [rpc(ip='127.0.0.1', port=s.port).ping() for s in servers[:5]] assert sum(map(len, rpc.available.values())) == 5 assert sum(map(len, rpc.occupied.values())) == 0 assert rpc.active == 0 assert rpc.open == 5 # Clear out connections to make room for more yield [rpc(ip='127.0.0.1', port=s.port).ping() for s in servers[5:]] assert rpc.active == 0 assert rpc.open == 5 s = servers[0] yield [rpc(ip='127.0.0.1', port=s.port).ping(delay=0.1) for i in range(3)] assert len(rpc.available['tcp://127.0.0.1:%d' % s.port]) == 3 # Explicitly clear out connections rpc.collect() start = time() while any(rpc.available.values()): yield gen.sleep(0.01) assert time() < start + 2 rpc.close() @gen_test() def test_connection_pool_tls(): """ Make sure connection args are supported. """ sec = tls_security() connection_args = sec.get_connection_args('client') listen_args = sec.get_listen_args('scheduler') @gen.coroutine def ping(comm, delay=0.01): yield gen.sleep(delay) raise gen.Return('pong') servers = [Server({'ping': ping}) for i in range(10)] for server in servers: server.listen('tls://', listen_args=listen_args) rpc = ConnectionPool(limit=5, connection_args=connection_args) yield [rpc(s.address).ping() for s in servers[:5]] yield [rpc(s.address).ping() for s in servers[::2]] yield [rpc(s.address).ping() for s in servers] assert rpc.active == 0 rpc.close() @gen_test() def test_counters(): server = Server({'div': stream_div}) server.listen('tcp://') with rpc(server.address) as r: for i in range(2): yield r.identity() with pytest.raises(ZeroDivisionError): yield r.div(x=1, y=0) c = server.counters assert c['op'].components[0] == {'identity': 2, 'div': 1} @gen_cluster() def test_ticks(s, a, b): pytest.importorskip('crick') yield gen.sleep(0.1) c = s.digests['tick-duration'] assert c.size() assert 0.01 < c.components[0].quantile(0.5) < 0.5 @gen_cluster() def test_tick_logging(s, a, b): pytest.importorskip('crick') with set_config(**{'tick-maximum-delay': 10}): with captured_logger('distributed.core') as sio: yield gen.sleep(0.1) text = sio.getvalue() assert "unresponsive" in text distributed-1.20.2/distributed/tests/test_counter.py000066400000000000000000000021521321233345200226700ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import pytest from distributed.counter import Counter from distributed.utils_test import loop # flake8: noqa try: from distributed.counter import Digest except ImportError: Digest = None @pytest.mark.parametrize('CD,size', [(Counter, lambda d: sum(d.values())), pytest.mark.skipif(not Digest, reason="no crick library")( (Digest, lambda x: x.size()))]) def test_digest(loop, CD, size): c = CD(loop=loop) c.add(1) c.add(2) assert size(c.components[0]) == 2 c.shift() assert 0 < size(c.components[0]) < 2 assert 0 < size(c.components[1]) < 1 assert sum(size(d) for d in c.components) == 2 for i in range(len(c.components) - 1): assert size(c.components[i]) >= size(c.components[i + 1]) c.add(3) assert sum(size(d) for d in c.components) == c.size() def test_counter(loop): c = Counter(loop=loop) c.add(1) for i in range(5): c.shift() assert abs(sum(cc[1] for cc in c.components) - 1) < 1e-13 distributed-1.20.2/distributed/tests/test_diskutils.py000066400000000000000000000172241321233345200232320ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import functools import gc import os import shutil import subprocess import sys from time import sleep import mock from distributed.compatibility import Empty from distributed.diskutils import WorkSpace from distributed.metrics import time from distributed.utils import mp_context from distributed.utils_test import captured_logger, slow, new_config def assert_directory_contents(dir_path, expected): expected = [os.path.join(dir_path, p) for p in expected] actual = [os.path.join(dir_path, p) for p in os.listdir(dir_path) if p not in ('global.lock', 'purge.lock')] assert sorted(actual) == sorted(expected) def test_workdir_simple(tmpdir): # Test nominal operation of WorkSpace and WorkDirs base_dir = str(tmpdir) assert_contents = functools.partial(assert_directory_contents, base_dir) ws = WorkSpace(base_dir) assert_contents([]) a = ws.new_work_dir(name='aa') assert_contents(['aa', 'aa.dirlock']) b = ws.new_work_dir(name='bb') assert_contents(['aa', 'aa.dirlock', 'bb', 'bb.dirlock']) ws._purge_leftovers() assert_contents(['aa', 'aa.dirlock', 'bb', 'bb.dirlock']) a.release() assert_contents(['bb', 'bb.dirlock']) del b gc.collect() assert_contents([]) # Generated temporary name with a prefix a = ws.new_work_dir(prefix='foo-') b = ws.new_work_dir(prefix='bar-') c = ws.new_work_dir(prefix='bar-') assert_contents({a.dir_path, a._lock_path, b.dir_path, b._lock_path, c.dir_path, c._lock_path}) assert os.path.basename(a.dir_path).startswith('foo-') assert os.path.basename(b.dir_path).startswith('bar-') assert os.path.basename(c.dir_path).startswith('bar-') assert b.dir_path != c.dir_path def test_two_workspaces_in_same_directory(tmpdir): # If handling the same directory with two WorkSpace instances, # things should work ok too base_dir = str(tmpdir) assert_contents = functools.partial(assert_directory_contents, base_dir) ws = WorkSpace(base_dir) assert_contents([]) a = ws.new_work_dir(name='aa') assert_contents(['aa', 'aa.dirlock']) ws2 = WorkSpace(base_dir) ws2._purge_leftovers() assert_contents(['aa', 'aa.dirlock']) b = ws.new_work_dir(name='bb') assert_contents(['aa', 'aa.dirlock', 'bb', 'bb.dirlock']) del ws del b gc.collect() assert_contents(['aa', 'aa.dirlock']) del a gc.collect() assert_contents([]) def test_workspace_process_crash(tmpdir): # WorkSpace should be able to clean up stale contents left by # crashed process base_dir = str(tmpdir) assert_contents = functools.partial(assert_directory_contents, base_dir) ws = WorkSpace(base_dir) code = """if 1: import signal import sys import time from distributed.diskutils import WorkSpace ws = WorkSpace(%(base_dir)r) a = ws.new_work_dir(name='aa') b = ws.new_work_dir(prefix='foo-') print((a.dir_path, b.dir_path)) sys.stdout.flush() time.sleep(100) """ % dict(base_dir=base_dir) p = subprocess.Popen([sys.executable, '-c', code], stdin=subprocess.PIPE, stdout=subprocess.PIPE, universal_newlines=True) line = p.stdout.readline() assert p.poll() is None a_path, b_path = eval(line) assert_contents([a_path, a_path + '.dirlock', b_path, b_path + '.dirlock']) # The child process holds a lock so the work dirs shouldn't be removed ws._purge_leftovers() assert_contents([a_path, a_path + '.dirlock', b_path, b_path + '.dirlock']) # Kill the process so it's unable to clear the work dirs itself p.kill() assert p.wait() # process returned with non-zero code assert_contents([a_path, a_path + '.dirlock', b_path, b_path + '.dirlock']) with captured_logger('distributed.diskutils', 'WARNING', propagate=False) as sio: ws._purge_leftovers() assert_contents([]) # One log line per purged directory lines = sio.getvalue().splitlines() assert len(lines) == 2 for p in (a_path, b_path): assert any(repr(p) in line for line in lines) def test_workspace_rmtree_failure(tmpdir): base_dir = str(tmpdir) ws = WorkSpace(base_dir) a = ws.new_work_dir(name='aa') shutil.rmtree(a.dir_path) with captured_logger('distributed.diskutils', 'ERROR', propagate=False) as sio: a.release() lines = sio.getvalue().splitlines() # shutil.rmtree() may call its onerror callback several times assert lines for line in lines: assert line.startswith("Failed to remove %r" % (a.dir_path,)) def test_locking_disabled(tmpdir): base_dir = str(tmpdir) with new_config({'use-file-locking': False}): with mock.patch('distributed.diskutils.locket.lock_file') as lock_file: assert_contents = functools.partial(assert_directory_contents, base_dir) ws = WorkSpace(base_dir) assert_contents([]) a = ws.new_work_dir(name='aa') assert_contents(['aa']) b = ws.new_work_dir(name='bb') assert_contents(['aa', 'bb']) ws._purge_leftovers() assert_contents(['aa', 'bb']) a.release() assert_contents(['bb']) del b gc.collect() assert_contents([]) lock_file.assert_not_called() def _workspace_concurrency(base_dir, purged_q, err_q, stop_evt): ws = WorkSpace(base_dir) n_purged = 0 with captured_logger('distributed.diskutils', 'ERROR') as sio: while not stop_evt.is_set(): # Add a bunch of locks, and simulate forgetting them try: purged = ws._purge_leftovers() except Exception as e: err_q.put(e) else: n_purged += len(purged) lines = sio.getvalue().splitlines() if lines: try: raise AssertionError("got %d logs, see stderr" % (len(lines,))) except Exception as e: err_q.put(e) purged_q.put(n_purged) def _test_workspace_concurrency(tmpdir, timeout, max_procs): base_dir = str(tmpdir) err_q = mp_context.Queue() purged_q = mp_context.Queue() stop_evt = mp_context.Event() ws = WorkSpace(base_dir) # Make sure purging only happens in the child processes ws._purge_leftovers = lambda: None NPROCS = 2 if sys.platform == 'win32' else max_procs processes = [mp_context.Process(target=_workspace_concurrency, args=(base_dir, purged_q, err_q, stop_evt)) for i in range(NPROCS)] for p in processes: p.start() n_created = 0 n_purged = 0 try: t1 = time() while time() - t1 < timeout: # Add a bunch of locks, and simulate forgetting them for i in range(50): d = ws.new_work_dir(prefix='workspace-concurrency-') d._finalizer.detach() n_created += 1 sleep(1e-2) finally: stop_evt.set() for p in processes: p.join() # Any errors? try: err = err_q.get_nowait() except Empty: pass else: raise err try: while True: n_purged += purged_q.get_nowait() except Empty: pass return n_created, n_purged def test_workspace_concurrency(tmpdir): _test_workspace_concurrency(tmpdir, 2.0, 6) @slow def test_workspace_concurrency_intense(tmpdir): n_purged, n_created = _test_workspace_concurrency(tmpdir, 8.0, 16) assert n_purged >= n_created / 10 > 100 distributed-1.20.2/distributed/tests/test_ipython.py000066400000000000000000000144141321233345200227070ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import mock import pytest from toolz import first import tornado from distributed import Client from distributed.utils_test import cluster, mock_ipython from distributed.utils_test import loop, zmq_ctx # flake8: noqa def need_functional_ipython(func): try: import ipykernel import jupyter_client except ImportError: return pytest.mark.skip("need ipykernel and jupyter_client installed")(func) if tornado.version_info >= (5,): # https://github.com/ipython/ipykernel/issues/277 return pytest.mark.skip("IPython kernel broken with Tornado 5")(func) else: return func @pytest.mark.ipython @need_functional_ipython def test_start_ipython_workers(loop, zmq_ctx): from jupyter_client import BlockingKernelClient with cluster(1) as (s, [a]): with Client(s['address'], loop=loop) as e: info_dict = e.start_ipython_workers() info = first(info_dict.values()) key = info.pop('key') kc = BlockingKernelClient(**info) kc.session.key = key kc.start_channels() kc.wait_for_ready(timeout=10) msg_id = kc.execute("worker") reply = kc.get_shell_msg(timeout=10) assert reply['parent_header']['msg_id'] == msg_id assert reply['content']['status'] == 'ok' kc.stop_channels() @pytest.mark.ipython @need_functional_ipython def test_start_ipython_scheduler(loop, zmq_ctx): from jupyter_client import BlockingKernelClient with cluster(1) as (s, [a]): with Client(s['address'], loop=loop) as e: info = e.start_ipython_scheduler() key = info.pop('key') kc = BlockingKernelClient(**info) kc.session.key = key kc.start_channels() msg_id = kc.execute("scheduler") reply = kc.get_shell_msg(timeout=10) kc.stop_channels() @pytest.mark.ipython @need_functional_ipython def test_start_ipython_scheduler_magic(loop, zmq_ctx): with cluster(1) as (s, [a]): with Client(s['address'], loop=loop) as e, mock_ipython() as ip: info = e.start_ipython_scheduler() expected = [ {'magic_kind': 'line', 'magic_name': 'scheduler'}, {'magic_kind': 'cell', 'magic_name': 'scheduler'}, ] call_kwargs_list = [kwargs for (args, kwargs) in ip.register_magic_function.call_args_list] assert call_kwargs_list == expected magic = ip.register_magic_function.call_args_list[0][0][0] magic(line="", cell="scheduler") @pytest.mark.ipython @need_functional_ipython def test_start_ipython_workers_magic(loop, zmq_ctx): with cluster(2) as (s, [a, b]): with Client(s['address'], loop=loop) as e, mock_ipython() as ip: workers = list(e.ncores())[:2] names = ['magic%i' % i for i in range(len(workers))] info_dict = e.start_ipython_workers(workers, magic_names=names) expected = [ {'magic_kind': 'line', 'magic_name': 'remote'}, {'magic_kind': 'cell', 'magic_name': 'remote'}, {'magic_kind': 'line', 'magic_name': 'magic0'}, {'magic_kind': 'cell', 'magic_name': 'magic0'}, {'magic_kind': 'line', 'magic_name': 'magic1'}, {'magic_kind': 'cell', 'magic_name': 'magic1'}, ] call_kwargs_list = [kwargs for (args, kwargs) in ip.register_magic_function.call_args_list] assert call_kwargs_list == expected assert ip.register_magic_function.call_count == 6 magics = [args[0][0] for args in ip.register_magic_function.call_args_list[2:]] magics[-1](line="", cell="worker") [m.client.stop_channels() for m in magics] @pytest.mark.ipython @need_functional_ipython def test_start_ipython_workers_magic_asterix(loop, zmq_ctx): with cluster(2) as (s, [a, b]): with Client(s['address'], loop=loop) as e, mock_ipython() as ip: workers = list(e.ncores())[:2] info_dict = e.start_ipython_workers(workers, magic_names='magic_*') expected = [ {'magic_kind': 'line', 'magic_name': 'remote'}, {'magic_kind': 'cell', 'magic_name': 'remote'}, {'magic_kind': 'line', 'magic_name': 'magic_0'}, {'magic_kind': 'cell', 'magic_name': 'magic_0'}, {'magic_kind': 'line', 'magic_name': 'magic_1'}, {'magic_kind': 'cell', 'magic_name': 'magic_1'}, ] call_kwargs_list = [kwargs for (args, kwargs) in ip.register_magic_function.call_args_list] assert call_kwargs_list == expected assert ip.register_magic_function.call_count == 6 magics = [args[0][0] for args in ip.register_magic_function.call_args_list[2:]] magics[-1](line="", cell="worker") [m.client.stop_channels() for m in magics] @pytest.mark.ipython @need_functional_ipython def test_start_ipython_remote(loop, zmq_ctx): from distributed._ipython_utils import remote_magic with cluster(1) as (s, [a]): with Client(s['address'], loop=loop) as e, mock_ipython() as ip: worker = first(e.ncores()) ip.user_ns['info'] = e.start_ipython_workers(worker)[worker] remote_magic('info 1') # line magic remote_magic('info', 'worker') # cell magic expected = [ ((remote_magic,), {'magic_kind': 'line', 'magic_name': 'remote'}), ((remote_magic,), {'magic_kind': 'cell', 'magic_name': 'remote'}), ] assert ip.register_magic_function.call_args_list == expected assert ip.register_magic_function.call_count == 2 @pytest.mark.ipython @need_functional_ipython def test_start_ipython_qtconsole(loop): Popen = mock.Mock() with cluster() as (s, [a, b]): with mock.patch('distributed._ipython_utils.Popen', Popen), Client(s['address'], loop=loop) as e: worker = first(e.ncores()) e.start_ipython_workers(worker, qtconsole=True) e.start_ipython_workers(worker, qtconsole=True, qtconsole_args=['--debug']) assert Popen.call_count == 2 (cmd,), kwargs = Popen.call_args_list[0] assert cmd[:3] == ['jupyter', 'qtconsole', '--existing'] (cmd,), kwargs = Popen.call_args_list[1] assert cmd[-1:] == ['--debug'] distributed-1.20.2/distributed/tests/test_joblib.py000066400000000000000000000102161321233345200224520ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import pytest from random import random from time import sleep from distributed.utils_test import cluster, inc from distributed.utils_test import loop # flake8: noqa distributed_joblib = pytest.importorskip('distributed.joblib') joblibs = [distributed_joblib.joblib, distributed_joblib.sk_joblib] joblib_funcname = distributed_joblib.joblib_funcname def slow_raise_value_error(condition, duration=0.05): sleep(duration) if condition: raise ValueError("condition evaluated to True") @pytest.mark.skip(reason="intermittent blocking failures") @pytest.mark.parametrize('joblib', joblibs) def test_simple(loop, joblib): if joblib is None: pytest.skip() Parallel = joblib.Parallel delayed = joblib.delayed with cluster() as (s, [a, b]): with joblib.parallel_backend('dask.distributed', loop=loop, scheduler_host=s['address']) as (ba, _): seq = Parallel()(delayed(inc)(i) for i in range(10)) assert seq == [inc(i) for i in range(10)] with pytest.raises(ValueError): Parallel()(delayed(slow_raise_value_error)(i == 3) for i in range(10)) seq = Parallel()(delayed(inc)(i) for i in range(10)) assert seq == [inc(i) for i in range(10)] ba.client.close() def random2(): return random() @pytest.mark.parametrize('joblib', joblibs) def test_dont_assume_function_purity(loop, joblib): if joblib is None: pytest.skip() Parallel = joblib.Parallel delayed = joblib.delayed with cluster() as (s, [a, b]): with joblib.parallel_backend('dask.distributed', loop=loop, scheduler_host=s['address']) as (ba, _): x, y = Parallel()(delayed(random2)() for i in range(2)) assert x != y ba.client.close() @pytest.mark.parametrize('joblib', joblibs) def test_joblib_funcname(joblib): if joblib is None: pytest.skip() BatchedCalls = joblib.parallel.BatchedCalls func = BatchedCalls([(random2,), (random2,)]) assert joblib_funcname(func) == 'random2' assert joblib_funcname(random2) == 'random2' @pytest.mark.parametrize('joblib', joblibs) def test_joblib_backend_subclass(joblib): if joblib is None: pytest.skip() assert issubclass(distributed_joblib.DaskDistributedBackend, joblib.parallel.ParallelBackendBase) def add5(a, b, c, d=0, e=0): return a + b + c + d + e class CountSerialized(object): def __init__(self, x): self.x = x self.count = 0 def __add__(self, other): return self.x + getattr(other, 'x', other) __radd__ = __add__ def __reduce__(self): self.count += 1 return (CountSerialized, (self.x,)) @pytest.mark.parametrize('joblib', joblibs) def test_joblib_scatter(loop, joblib): if joblib is None: pytest.skip() Parallel = joblib.Parallel delayed = joblib.delayed x = CountSerialized(1) y = CountSerialized(2) z = CountSerialized(3) with cluster() as (s, [a, b]): with joblib.parallel_backend('dask.distributed', loop=loop, scheduler_host=s['address'], scatter=[x, y]) as (ba, _): f = delayed(add5) tasks = [f(x, y, z, d=4, e=5), f(x, z, y, d=5, e=4), f(y, x, z, d=x, e=5), f(z, z, x, d=z, e=y)] sols = [func(*args, **kwargs) for func, args, kwargs in tasks] results = Parallel()(tasks) ba.client.close() # Scatter must take a list/tuple with pytest.raises(TypeError): with joblib.parallel_backend('dask.distributed', loop=loop, scheduler_host=s['address'], scatter=1): pass for l, r in zip(sols, results): assert l == r # Scattered variables only serialized once assert x.count == 1 assert y.count == 1 assert z.count == 4 distributed-1.20.2/distributed/tests/test_locks.py000066400000000000000000000051321321233345200223250ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from time import sleep import sys import pytest from tornado import gen from distributed import Client, Lock, wait, get_client from distributed.metrics import time from distributed.utils_test import (gen_cluster, inc, cluster, slow, div) from distributed.utils_test import loop # flake8: noqa @gen_cluster(client=True, ncores=[('127.0.0.1', 8)] * 2) def test_lock(c, s, a, b): c.set_metadata('locked', False) def f(x): client = get_client() with Lock('x') as lock: assert client.get_metadata('locked') == False client.set_metadata('locked', True) sleep(0.05) assert client.get_metadata('locked') == True client.set_metadata('locked', False) futures = c.map(f, range(20)) results = yield futures assert not s.extensions['locks'].events assert not s.extensions['locks'].ids @gen_cluster(client=True) def test_timeout(c, s, a, b): locks = s.extensions['locks'] lock = Lock('x') result = yield lock.acquire() assert result is True assert locks.ids['x'] == lock.id lock2 = Lock('x') assert lock.id != lock2.id start = time() result = yield lock2.acquire(timeout=0.1) stop = time() assert stop - start < 0.3 assert result is False assert locks.ids['x'] == lock.id assert not locks.events['x'] yield lock.release() @gen_cluster(client=True) def test_acquires_with_zero_timeout(c, s, a, b): lock = Lock('x') yield lock.acquire(timeout=0) assert lock.locked() yield lock.release() yield lock.acquire(timeout=1) yield lock.release() yield lock.acquire(timeout=1) yield lock.release() def test_timeout_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: with Lock('x') as lock: assert Lock('x').acquire(timeout=0.1) is False @gen_cluster(client=True) def test_errors(c, s, a, b): lock = Lock('x') with pytest.raises(ValueError): yield lock.release() def test_lock_sync(loop): def f(x): with Lock('x') as lock: client = get_client() assert client.get_metadata('locked') == False client.set_metadata('locked', True) sleep(0.05) assert client.get_metadata('locked') == True client.set_metadata('locked', False) with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: c.set_metadata('locked', False) futures = c.map(f, range(10)) c.gather(futures) distributed-1.20.2/distributed/tests/test_metrics.py000066400000000000000000000011241321233345200226550ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import time from distributed import metrics def test_wall_clock(): for i in range(3): time.sleep(0.01) t = time.time() samples = [metrics.time() for j in range(50)] # Resolution deltas = [samples[j + 1] - samples[j] for j in range(len(samples) - 1)] assert min(deltas) >= 0.0, deltas assert max(deltas) <= 1.0, deltas assert any(lambda d: 0.0 < d < 0.0001 for d in deltas), deltas # Close to time.time() assert t - 0.5 < samples[0] < t + 0.5 distributed-1.20.2/distributed/tests/test_nanny.py000066400000000000000000000171001321233345200223330ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import gc import logging import os import random import sys import numpy as np import pytest from toolz import valmap, first from tornado import gen from distributed import Nanny, rpc, Scheduler from distributed.core import CommClosedError from distributed.metrics import time from distributed.protocol.pickle import dumps from distributed.utils import ignoring, tmpfile from distributed.utils_test import (gen_cluster, gen_test, slow, inc, captured_logger) @gen_cluster(ncores=[]) def test_nanny(s): n = Nanny(s.ip, s.port, ncores=2, loop=s.loop) yield n._start(0) with rpc(n.address) as nn: assert n.is_alive() assert s.ncores[n.worker_address] == 2 assert s.worker_info[n.worker_address]['services']['nanny'] > 1024 yield nn.kill() assert not n.is_alive() assert n.worker_address not in s.ncores assert n.worker_address not in s.worker_info yield nn.kill() assert not n.is_alive() assert n.worker_address not in s.ncores assert n.worker_address not in s.worker_info yield nn.instantiate() assert n.is_alive() assert s.ncores[n.worker_address] == 2 assert s.worker_info[n.worker_address]['services']['nanny'] > 1024 yield nn.terminate() assert not n.is_alive() yield n._close() @gen_cluster(ncores=[]) def test_many_kills(s): n = Nanny(s.address, ncores=2, loop=s.loop) yield n._start(0) assert n.is_alive() yield [n.kill() for i in range(5)] yield [n.kill() for i in range(5)] yield n._close() @gen_cluster(Worker=Nanny) def test_str(s, a, b): assert a.worker_address in str(a) assert a.worker_address in repr(a) assert str(a.ncores) in str(a) assert str(a.ncores) in repr(a) @gen_cluster(ncores=[], timeout=20, client=True) def test_nanny_process_failure(c, s): n = Nanny(s.ip, s.port, ncores=2, loop=s.loop) yield n._start() first_dir = n.worker_dir assert os.path.exists(first_dir) original_address = n.worker_address ww = rpc(n.worker_address) yield ww.update_data(data=valmap(dumps, {'x': 1, 'y': 2})) pid = n.pid assert pid is not None with ignoring(CommClosedError): yield c._run(os._exit, 0, workers=[n.worker_address]) start = time() while n.pid == pid: # wait while process dies and comes back yield gen.sleep(0.01) assert time() - start < 5 start = time() while not n.is_alive(): # wait while process comes back yield gen.sleep(0.01) assert time() - start < 5 # assert n.worker_address != original_address # most likely start = time() while n.worker_address not in s.ncores or n.worker_dir is None: yield gen.sleep(0.01) assert time() - start < 5 second_dir = n.worker_dir yield n._close() assert not os.path.exists(second_dir) assert not os.path.exists(first_dir) assert first_dir != n.worker_dir ww.close_rpc() s.stop() def test_nanny_no_port(): _ = str(Nanny('127.0.0.1', 8786)) @gen_cluster(ncores=[]) def test_run(s): pytest.importorskip('psutil') n = Nanny(s.ip, s.port, ncores=2, loop=s.loop) yield n._start() with rpc(n.address) as nn: response = yield nn.run(function=dumps(lambda: 1)) assert response['status'] == 'OK' assert response['result'] == 1 yield n._close() @slow @gen_cluster(Worker=Nanny, ncores=[('127.0.0.1', 1)], worker_kwargs={'reconnect': False}) def test_close_on_disconnect(s, w): yield s.close() start = time() while w.status != 'closed': yield gen.sleep(0.05) assert time() < start + 9 @slow @gen_cluster(client=False, ncores=[]) def test_nanny_death_timeout(s): yield s.close() w = Nanny(s.address, death_timeout=1) yield w._start() yield gen.sleep(3) assert w.status == 'closed' @gen_cluster(client=True, Worker=Nanny) def test_random_seed(c, s, a, b): @gen.coroutine def check_func(func): x = c.submit(func, 0, 2**31, pure=False, workers=a.worker_address) y = c.submit(func, 0, 2**31, pure=False, workers=b.worker_address) assert x.key != y.key x = yield x y = yield y assert x != y yield check_func(lambda a, b: random.randint(a, b)) yield check_func(lambda a, b: np.random.randint(a, b)) @pytest.mark.skipif(sys.platform.startswith('win'), reason="num_fds not supported on windows") @gen_cluster(client=False, ncores=[]) def test_num_fds(s): psutil = pytest.importorskip('psutil') proc = psutil.Process() # Warm up w = Nanny(s.address) yield w._start() yield w._close() del w gc.collect() before = proc.num_fds() for i in range(3): w = Nanny(s.address) yield w._start() yield gen.sleep(0.1) yield w._close() start = time() while proc.num_fds() > before: print("fds:", before, proc.num_fds()) yield gen.sleep(0.1) assert time() < start + 10 @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster(client=True, ncores=[]) def test_worker_uses_same_host_as_nanny(c, s): for host in ['tcp://0.0.0.0', 'tcp://127.0.0.2']: n = Nanny(s.address) yield n._start(host) def func(dask_worker): return dask_worker.listener.listen_address result = yield c.run(func) assert host in first(result.values()) yield n._close() @gen_test() def test_scheduler_file(): with tmpfile() as fn: s = Scheduler(scheduler_file=fn) s.start(8008) w = Nanny(scheduler_file=fn) yield w._start() assert s.workers == {w.worker_address} yield w._close() s.stop() @gen_cluster(client=True, Worker=Nanny, ncores=[('127.0.0.1', 2)]) def test_nanny_timeout(c, s, a): x = yield c.scatter(123) with captured_logger(logging.getLogger('distributed.nanny'), level=logging.ERROR) as logger: response = yield a.restart(timeout=0.1) out = logger.getvalue() assert 'timed out' in out.lower() start = time() while x.status != 'cancelled': yield gen.sleep(0.1) assert time() < start + 7 @gen_cluster(ncores=[('127.0.0.1', 1)], client=True, Worker=Nanny, worker_kwargs={'memory_limit': 1e8}, timeout=20) def test_nanny_terminate(c, s, a): from time import sleep def leak(): L = [] while True: L.append(b'0' * 5000000) sleep(0.01) proc = a.process.pid with captured_logger(logging.getLogger('distributed.nanny')) as logger: future = c.submit(leak) start = time() while a.process.pid == proc: yield gen.sleep(0.1) assert time() < start + 10 out = logger.getvalue() assert 'restart' in out.lower() assert 'memory' in out.lower() @gen_cluster(ncores=[], client=True) def test_avoid_memory_monitor_if_zero_limit(c, s): nanny = Nanny(s.address, loop=s.loop, memory_limit=0) yield nanny._start() typ = yield c.run(lambda dask_worker: type(dask_worker.data)) assert typ == {nanny.worker_address: dict} pcs = yield c.run(lambda dask_worker: list(dask_worker.periodic_callbacks)) assert 'memory' not in pcs assert 'memory' not in nanny.periodic_callbacks future = c.submit(inc, 1) assert (yield future) == 2 yield gen.sleep(0.02) yield c.submit(inc, 2) # worker doesn't pause yield nanny._close() distributed-1.20.2/distributed/tests/test_preload.py000066400000000000000000000032211321233345200226350ustar00rootroot00000000000000import os import shutil import sys import tempfile from distributed import Client from distributed.utils_test import cluster from distributed.utils_test import loop # flake8: noqa PRELOAD_TEXT = """ _worker_info = {} def dask_setup(worker): _worker_info['address'] = worker.address def get_worker_address(): return _worker_info['address'] """ def test_worker_preload_file(loop): def check_worker(): import worker_info return worker_info.get_worker_address() tmpdir = tempfile.mkdtemp() try: path = os.path.join(tmpdir, 'worker_info.py') with open(path, 'w') as f: f.write(PRELOAD_TEXT) with cluster(worker_kwargs={'preload': [path]}) as (s, workers), \ Client(s['address'], loop=loop) as c: assert c.run(check_worker) == { worker['address']: worker['address'] for worker in workers } finally: shutil.rmtree(tmpdir) def test_worker_preload_module(loop): def check_worker(): import worker_info return worker_info.get_worker_address() tmpdir = tempfile.mkdtemp() sys.path.insert(0, tmpdir) try: path = os.path.join(tmpdir, 'worker_info.py') with open(path, 'w') as f: f.write(PRELOAD_TEXT) with cluster(worker_kwargs={'preload': ['worker_info']}) \ as (s, workers), Client(s['address'], loop=loop) as c: assert c.run(check_worker) == { worker['address']: worker['address'] for worker in workers } finally: sys.path.remove(tmpdir) shutil.rmtree(tmpdir) distributed-1.20.2/distributed/tests/test_profile.py000066400000000000000000000060561321233345200226600ustar00rootroot00000000000000import sys import time from toolz import first from threading import Thread from distributed.profile import (process, merge, create, call_stack, identifier) from distributed.compatibility import get_thread_identity def test_basic(): def test_g(): time.sleep(0.01) def test_h(): time.sleep(0.02) def test_f(): for i in range(100): test_g() test_h() thread = Thread(target=test_f) thread.daemon = True thread.start() state = create() for i in range(100): time.sleep(0.02) frame = sys._current_frames()[thread.ident] process(frame, None, state) assert state['count'] == 100 d = state while len(d['children']) == 1: d = first(d['children'].values()) assert d['count'] == 100 assert 'test_f' in str(d['description']) g = [c for c in d['children'].values() if 'test_g' in str(c['description'])][0] h = [c for c in d['children'].values() if 'test_h' in str(c['description'])][0] assert g['count'] < h['count'] assert g['count'] + h['count'] == 100 def test_merge(): a1 = { 'count': 5, 'identifier': 'root', 'description': 'a', 'children': { 'b': {'count': 3, 'description': 'b-func', 'identifier': 'b', 'children': {}}, 'c': {'count': 2, 'description': 'c-func', 'identifier': 'c', 'children': {}}}} a2 = { 'count': 4, 'description': 'a', 'identifier': 'root', 'children': { 'd': {'count': 2, 'description': 'd-func', 'children': {}, 'identifier': 'd'}, 'c': {'count': 2, 'description': 'c-func', 'children': {}, 'identifier': 'c'}}} expected = { 'count': 9, 'identifier': 'root', 'description': 'a', 'children': { 'b': {'count': 3, 'description': 'b-func', 'identifier': 'b', 'children': {}}, 'd': {'count': 2, 'description': 'd-func', 'identifier': 'd', 'children': {}}, 'c': {'count': 4, 'description': 'c-func', 'identifier': 'c', 'children': {}}}} assert merge(a1, a2) == expected def test_merge_empty(): assert merge() == create() assert merge(create()) == create() assert merge(create(), create()) == create() def test_call_stack(): frame = sys._current_frames()[get_thread_identity()] L = call_stack(frame) assert isinstance(L, list) assert all(isinstance(s, str) for s in L) assert 'test_call_stack' in str(L[-1]) def test_identifier(): frame = sys._current_frames()[get_thread_identity()] assert identifier(frame) == identifier(frame) assert identifier(None) == identifier(None) distributed-1.20.2/distributed/tests/test_publish.py000066400000000000000000000141701321233345200226620ustar00rootroot00000000000000 import pytest from dask import delayed from distributed import Client from distributed.client import futures_of from distributed.metrics import time from distributed.utils_test import gen_cluster, inc, cluster from distributed.utils_test import loop # flake8: noqa from tornado import gen @gen_cluster(client=False) def test_publish_simple(s, a, b): c = yield Client((s.ip, s.port), asynchronous=True) f = yield Client((s.ip, s.port), asynchronous=True) data = yield c.scatter(range(3)) out = yield c.publish_dataset(data=data) assert 'data' in s.extensions['publish'].datasets with pytest.raises(KeyError) as exc_info: out = yield c.publish_dataset(data=data) assert "exists" in str(exc_info.value) assert "data" in str(exc_info.value) result = yield c.scheduler.publish_list() assert result == ['data'] result = yield f.scheduler.publish_list() assert result == ['data'] yield c.close() yield f.close() @gen_cluster(client=False) def test_publish_roundtrip(s, a, b): c = yield Client((s.ip, s.port), asynchronous=True) f = yield Client((s.ip, s.port), asynchronous=True) data = yield c.scatter([0, 1, 2]) yield c.publish_dataset(data=data) assert 'published-data' in s.who_wants[data[0].key] result = yield f.get_dataset(name='data') assert len(result) == len(data) out = yield f.gather(result) assert out == [0, 1, 2] with pytest.raises(KeyError) as exc_info: result = yield f.get_dataset(name='nonexistent') assert "not found" in str(exc_info.value) assert "nonexistent" in str(exc_info.value) yield c.close() yield f.close() @gen_cluster(client=True) def test_unpublish(c, s, a, b): data = yield c.scatter([0, 1, 2]) yield c.publish_dataset(data=data) key = data[0].key del data yield c.scheduler.publish_delete(name='data') assert 'data' not in s.extensions['publish'].datasets start = time() while key in s.who_wants: yield gen.sleep(0.01) assert time() < start + 5 with pytest.raises(KeyError) as exc_info: result = yield c.get_dataset(name='data') assert "not found" in str(exc_info.value) assert "data" in str(exc_info.value) def test_unpublish_sync(loop): with cluster() as (s, [a, b]): with Client(s['address']) as c: data = c.scatter([0, 1, 2]) c.publish_dataset(data=data) c.unpublish_dataset(name='data') with pytest.raises(KeyError) as exc_info: result = c.get_dataset(name='data') assert "not found" in str(exc_info.value) assert "data" in str(exc_info.value) @gen_cluster(client=True) def test_publish_multiple_datasets(c, s, a, b): x = delayed(inc)(1) y = delayed(inc)(2) yield c.publish_dataset(x=x, y=y) datasets = yield c.scheduler.publish_list() assert set(datasets) == {'x', 'y'} def test_unpublish_multiple_datasets_sync(loop): with cluster() as (s, [a, b]): with Client(s['address']) as c: x = delayed(inc)(1) y = delayed(inc)(2) c.publish_dataset(x=x, y=y) c.unpublish_dataset(name='x') with pytest.raises(KeyError) as exc_info: result = c.get_dataset(name='x') datasets = c.list_datasets() assert set(datasets) == {'y'} assert "not found" in str(exc_info.value) assert "x" in str(exc_info.value) c.unpublish_dataset(name='y') with pytest.raises(KeyError) as exc_info: result = c.get_dataset(name='y') assert "not found" in str(exc_info.value) assert "y" in str(exc_info.value) @gen_cluster(client=False) def test_publish_bag(s, a, b): db = pytest.importorskip('dask.bag') c = yield Client((s.ip, s.port), asynchronous=True) f = yield Client((s.ip, s.port), asynchronous=True) bag = db.from_sequence([0, 1, 2]) bagp = c.persist(bag) assert len(futures_of(bagp)) == 3 keys = {f.key for f in futures_of(bagp)} assert keys == set(bag.dask) yield c.publish_dataset(data=bagp) # check that serialization didn't affect original bag's dask assert len(futures_of(bagp)) == 3 result = yield f.get_dataset('data') assert set(result.dask.keys()) == set(bagp.dask.keys()) assert {f.key for f in result.dask.values()} == {f.key for f in bagp.dask.values()} out = yield f.compute(result) assert out == [0, 1, 2] yield c.close() yield f.close() def test_datasets_setitem(loop): with cluster() as (s, _): with Client(s['address'], loop=loop) as client: key, value = 'key', 'value' client.datasets[key] = value assert client.get_dataset('key') == value def test_datasets_getitem(loop): with cluster() as (s, _): with Client(s['address'], loop=loop) as client: key, value = 'key', 'value' client.publish_dataset(key=value) assert client.datasets[key] == value def test_datasets_delitem(loop): with cluster() as (s, _): with Client(s['address'], loop=loop) as client: key, value = 'key', 'value' client.publish_dataset(key=value) del client.datasets[key] assert key not in client.list_datasets() def test_datasets_keys(loop): with cluster() as (s, _): with Client(s['address'], loop=loop) as client: client.publish_dataset(**{str(n): n for n in range(10)}) keys = list(client.datasets.keys()) assert keys == [str(n) for n in range(10)] def test_datasets_contains(loop): with cluster() as (s, _): with Client(s['address'], loop=loop) as client: key, value = 'key', 'value' client.publish_dataset(key=value) assert key in client.datasets def test_datasets_iter(loop): with cluster() as (s, _): with Client(s['address'], loop=loop) as client: keys = [n for n in range(10)] client.publish_dataset(**{str(key): key for key in keys}) for n, key in enumerate(client.datasets): assert key == str(n) distributed-1.20.2/distributed/tests/test_queues.py000066400000000000000000000131101321233345200225140ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from datetime import timedelta from time import sleep import sys import pytest from tornado import gen from distributed import Client, Queue, Nanny, worker_client, wait from distributed.metrics import time from distributed.utils_test import (gen_cluster, inc, cluster, slow, div) from distributed.utils_test import loop # flake8: noqa @gen_cluster(client=True) def test_queue(c, s, a, b): x = yield Queue('x') y = yield Queue('y') xx = yield Queue('x') assert x.client is c future = c.submit(inc, 1) yield x.put(future) yield y.put(future) future2 = yield xx.get() assert future.key == future2.key with pytest.raises(gen.TimeoutError): yield x.get(timeout=0.1) del future, future2 yield gen.sleep(0.1) assert s.task_state # future still present in y's queue yield y.get() # burn future start = time() while s.task_state: yield gen.sleep(0.01) assert time() < start + 5 @gen_cluster(client=True) def test_queue_with_data(c, s, a, b): x = yield Queue('x') xx = yield Queue('x') assert x.client is c yield x.put([1, 'hello']) data = yield xx.get() assert data == [1, 'hello'] with pytest.raises(gen.TimeoutError): yield x.get(timeout=0.1) def test_sync(loop): with cluster() as (s, [a, b]): with Client(s['address']) as c: future = c.submit(lambda x: x + 1, 10) x = Queue('x') xx = Queue('x') x.put(future) assert x.qsize() == 1 assert xx.qsize() == 1 future2 = xx.get() assert future2.result() == 11 @gen_cluster() def test_hold_futures(s, a, b): c1 = yield Client(s.address, asynchronous=True) future = c1.submit(lambda x: x + 1, 10) q1 = yield Queue('q') yield q1.put(future) del q1 yield c1.close() yield gen.sleep(0.1) c2 = yield Client(s.address, asynchronous=True) q2 = yield Queue('q') future2 = yield q2.get() result = yield future2 assert result == 11 yield c2.close() @pytest.mark.skip(reason='getting same client from main thread') @gen_cluster(client=True) def test_picklability(c, s, a, b): q = Queue() def f(x): q.put(x + 1) yield c.submit(f, 10) result = yield q.get() assert result == 11 def test_picklability_sync(loop): with cluster() as (s, [a, b]): with Client(s['address']) as c: q = Queue() def f(x): q.put(x + 1) c.submit(f, 10).result() assert q.get() == 11 @pytest.mark.skipif(sys.version_info[0] == 2, reason='Multi-client issues') @slow @gen_cluster(client=True, ncores=[('127.0.0.1', 2)] * 5, Worker=Nanny, timeout=None) def test_race(c, s, *workers): def f(i): with worker_client() as c: q = Queue('x', client=c) for _ in range(100): future = q.get() x = future.result() y = c.submit(inc, x) q.put(y) sleep(0.01) result = q.get().result() return result q = Queue('x', client=c) L = yield c.scatter(range(5)) for future in L: yield q.put(future) futures = c.map(f, range(5)) results = yield c.gather(futures) assert all(r > 80 for r in results) qsize = yield q.qsize() assert not qsize @gen_cluster(client=True) def test_same_futures(c, s, a, b): q = Queue('x') future = yield c.scatter(123) for i in range(5): yield q.put(future) assert s.wants_what['queue-x'] == {future.key} for i in range(4): future2 = yield q.get() assert s.wants_what['queue-x'] == {future.key} yield gen.sleep(0.05) assert s.wants_what['queue-x'] == {future.key} yield q.get() start = time() while s.wants_what['queue-x']: yield gen.sleep(0.01) assert time() - start < 2 @gen_cluster(client=True) def test_get_many(c, s, a, b): x = yield Queue('x') xx = yield Queue('x') yield x.put(1) yield x.put(2) yield x.put(3) data = yield xx.get(batch=True) assert data == [1, 2, 3] yield x.put(1) yield x.put(2) yield x.put(3) data = yield xx.get(batch=2) assert data == [1, 2] with pytest.raises(gen.TimeoutError): data = yield gen.with_timeout(timedelta(seconds=0.100), xx.get(batch=2)) @gen_cluster(client=True) def test_Future_knows_status_immediately(c, s, a, b): x = yield c.scatter(123) q = yield Queue('q') yield q.put(x) c2 = yield Client(s.address, asynchronous=True) q2 = yield Queue('q', client=c2) future = yield q2.get() assert future.status == 'finished' x = c.submit(div, 1, 0) yield wait(x) yield q.put(x) future2 = yield q2.get() assert future2.status == 'error' with pytest.raises(Exception): yield future2 start = time() while True: # we learn about the true error eventually try: yield future2 except ZeroDivisionError: break except Exception: assert time() < start + 5 yield gen.sleep(0.05) yield c2.close() @gen_cluster(client=True) def test_erred_future(c, s, a, b): future = c.submit(div, 1, 0) q = Queue() yield q.put(future) yield gen.sleep(0.1) future2 = yield q.get() with pytest.raises(ZeroDivisionError): yield future2.result() exc = yield future2.exception() assert isinstance(exc, ZeroDivisionError) distributed-1.20.2/distributed/tests/test_resources.py000066400000000000000000000204431321233345200232260ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from time import time from dask import delayed import pytest from tornado import gen from distributed import Worker from distributed.client import wait from distributed.utils import tokey from distributed.utils_test import (inc, gen_cluster, slowinc, slowadd) from distributed.utils_test import loop # flake8: noqa @gen_cluster(client=True, ncores=[]) def test_resources(c, s): assert not s.worker_resources assert not s.resources a = Worker(s.ip, s.port, loop=s.loop, resources={'GPU': 2}) b = Worker(s.ip, s.port, loop=s.loop, resources={'GPU': 1, 'DB': 1}) yield [a._start(), b._start()] assert s.resources == {'GPU': {a.address: 2, b.address: 1}, 'DB': {b.address: 1}} assert s.worker_resources == {a.address: {'GPU': 2}, b.address: {'GPU': 1, 'DB': 1}} yield b._close() assert s.resources == {'GPU': {a.address: 2}, 'DB': {}} assert s.worker_resources == {a.address: {'GPU': 2}} yield a._close() @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 5}}), ('127.0.0.1', 1, {'resources': {'A': 1, 'B': 1}})]) def test_resource_submit(c, s, a, b): x = c.submit(inc, 1, resources={'A': 3}) y = c.submit(inc, 2, resources={'B': 1}) z = c.submit(inc, 3, resources={'C': 2}) yield wait(x) assert x.key in a.data yield wait(y) assert y.key in b.data assert z.key in s.unrunnable d = Worker(s.ip, s.port, loop=s.loop, resources={'C': 10}) yield d._start() yield wait(z) assert z.key in d.data yield d._close() @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 1}}), ('127.0.0.1', 1, {'resources': {'B': 1}})]) def test_submit_many_non_overlapping(c, s, a, b): futures = [c.submit(inc, i, resources={'A': 1}) for i in range(5)] yield wait(futures) assert len(a.data) == 5 assert len(b.data) == 0 @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 1}}), ('127.0.0.1', 1, {'resources': {'B': 1}})]) def test_move(c, s, a, b): [x] = yield c._scatter([1], workers=b.address) future = c.submit(inc, x, resources={'A': 1}) yield wait(future) assert a.data[future.key] == 2 @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 1}}), ('127.0.0.1', 1, {'resources': {'B': 1}})]) def test_dont_work_steal(c, s, a, b): [x] = yield c._scatter([1], workers=a.address) futures = [c.submit(slowadd, x, i, resources={'A': 1}, delay=0.05) for i in range(10)] yield wait(futures) assert all(f.key in a.data for f in futures) @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 1}}), ('127.0.0.1', 1, {'resources': {'B': 1}})]) def test_map(c, s, a, b): futures = c.map(inc, range(10), resources={'B': 1}) yield wait(futures) assert set(b.data) == {f.key for f in futures} assert not a.data @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 1}}), ('127.0.0.1', 1, {'resources': {'B': 1}})]) def test_persist(c, s, a, b): x = delayed(inc)(1) y = delayed(inc)(x) xx, yy = c.persist([x, y], resources={x: {'A': 1}, y: {'B': 1}}) yield wait([xx, yy]) assert x.key in a.data assert y.key in b.data @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 1}}), ('127.0.0.1', 1, {'resources': {'B': 11}})]) def test_compute(c, s, a, b): x = delayed(inc)(1) y = delayed(inc)(x) yy = c.compute(y, resources={x: {'A': 1}, y: {'B': 1}}) yield wait(yy) assert b.data xs = [delayed(inc)(i) for i in range(10, 20)] xxs = c.compute(xs, resources={'B': 1}) yield wait(xxs) assert len(b.data) > 10 @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 1}}), ('127.0.0.1', 1, {'resources': {'B': 1}})]) def test_get(c, s, a, b): dsk = {'x': (inc, 1), 'y': (inc, 'x')} result = yield c.get(dsk, 'y', resources={'y': {'A': 1}}, sync=False) assert result == 3 @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 1}}), ('127.0.0.1', 1, {'resources': {'B': 1}})]) def test_persist_tuple(c, s, a, b): x = delayed(inc)(1) y = delayed(inc)(x) xx, yy = c.persist([x, y], resources={(x, y): {'A': 1}}) yield wait([xx, yy]) assert x.key in a.data assert y.key in a.data assert not b.data @gen_cluster(client=True, ncores=[('127.0.0.1', 4, {'resources': {'A': 2}}), ('127.0.0.1', 4, {'resources': {'A': 1}})]) def test_submit_many_non_overlapping(c, s, a, b): futures = c.map(slowinc, range(100), resources={'A': 1}, delay=0.02) while len(a.data) + len(b.data) < 100: yield gen.sleep(0.01) assert len(a.executing) <= 2 assert len(b.executing) <= 1 yield wait(futures) assert a.total_resources == a.available_resources assert b.total_resources == b.available_resources @gen_cluster(client=True, ncores=[('127.0.0.1', 4, {'resources': {'A': 2, 'B': 1}})]) def test_minimum_resource(c, s, a): futures = c.map(slowinc, range(30), resources={'A': 1, 'B': 1}, delay=0.02) while len(a.data) < 30: yield gen.sleep(0.01) assert len(a.executing) <= 1 yield wait(futures) assert a.total_resources == a.available_resources @gen_cluster(client=True, ncores=[('127.0.0.1', 2, {'resources': {'A': 1}})]) def test_prefer_constrained(c, s, a): futures = c.map(slowinc, range(1000), delay=0.1) constrained = c.map(inc, range(10), resources={'A': 1}) import traceback, sys start = time() yield wait(constrained) end = time() assert end - start < 4 has_what = dict(s.has_what) processing = dict(s.processing) assert len(has_what) < len(constrained) + 2 # at most two slowinc's finished assert s.processing[a.address] @pytest.mark.xfail(reason="") @gen_cluster(client=True, ncores=[('127.0.0.1', 2, {'resources': {'A': 1}}), ('127.0.0.1', 2, {'resources': {'A': 1}})]) def test_balance_resources(c, s, a, b): futures = c.map(slowinc, range(100), delay=0.1, workers=a.address) constrained = c.map(inc, range(2), resources={'A': 1}) yield wait(constrained) assert any(f.key in a.data for f in constrained) # share assert any(f.key in b.data for f in constrained) @gen_cluster(client=True, ncores=[('127.0.0.1', 2)]) def test_set_resources(c, s, a): yield a.set_resources(A=2) assert a.total_resources['A'] == 2 assert a.available_resources['A'] == 2 assert s.worker_resources[a.address] == {'A': 2} future = c.submit(slowinc, 1, delay=1, resources={'A': 1}) while a.available_resources['A'] == 2: yield gen.sleep(0.01) yield a.set_resources(A=3) assert a.total_resources['A'] == 3 assert a.available_resources['A'] == 2 assert s.worker_resources[a.address] == {'A': 3} @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 1}}), ('127.0.0.1', 1, {'resources': {'B': 1}})]) def test_persist_collections(c, s, a, b): da = pytest.importorskip('dask.array') x = da.arange(10, chunks=(5,)) y = x.map_blocks(lambda x: x + 1) z = y.map_blocks(lambda x: 2 * x) w = z.sum() ww, yy = c.persist([w, y], resources={tuple(y.__dask_keys__()): {'A': 1}}) yield wait([ww, yy]) assert all(tokey(key) in a.data for key in y.__dask_keys__()) @pytest.mark.xfail(reason="Should protect resource keys from optimization") @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 1}}), ('127.0.0.1', 1, {'resources': {'B': 1}})]) def test_dont_optimize_out(c, s, a, b): da = pytest.importorskip('dask.array') x = da.arange(10, chunks=(5,)) y = x.map_blocks(lambda x: x + 1) z = y.map_blocks(lambda x: 2 * x) w = z.sum() yield c.compute(w, resources={tuple(y.__dask_keys__()): {'A': 1}},) for key in map(tokey, y.__dask_keys__()): assert 'executing' in str(a.story(key)) distributed-1.20.2/distributed/tests/test_scheduler.py000066400000000000000000001040411321233345200231670ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import cloudpickle from collections import defaultdict, deque from datetime import timedelta import json from operator import add, mul import sys from dask import delayed from toolz import merge, concat, valmap, first, frequencies, pluck from tornado import gen import pytest from distributed import Nanny, Worker, Client, wait, fire_and_forget from distributed.core import connect, rpc, CommClosedError from distributed.scheduler import validate_state, Scheduler, BANDWIDTH from distributed.client import wait from distributed.metrics import time from distributed.protocol.pickle import dumps from distributed.worker import dumps_function, dumps_task from distributed.utils import tmpfile from distributed.utils_test import (inc, dec, gen_cluster, gen_test, readone, slowinc, slowadd, slowdec, cluster, div, varying, slow) from distributed.utils_test import loop, nodebug # flake8: noqa from dask.compatibility import apply alice = 'alice:1234' bob = 'bob:1234' occupancy = defaultdict(lambda: 0) @gen_cluster() def test_administration(s, a, b): assert isinstance(s.address, str) assert s.address in str(s) assert str(sum(s.ncores.values())) in repr(s) assert str(len(s.ncores)) in repr(s) @gen_cluster(client=True, ncores=[('127.0.0.1', 1)]) def test_respect_data_in_memory(c, s, a): x = delayed(inc)(1) y = delayed(inc)(x) f = c.persist(y) yield wait([f]) assert s.released == {x.key} assert s.who_has == {y.key: {a.address}} z = delayed(add)(x, y) f2 = c.persist(z) while f2.key not in s.who_has: assert y.key in s.who_has yield gen.sleep(0.0001) @gen_cluster(client=True) def test_recompute_released_results(c, s, a, b): x = delayed(inc)(1) y = delayed(inc)(x) yy = c.persist(y) yield wait(yy) while x.key in s.who_has or x.key in a.data or x.key in b.data: # let x go away yield gen.sleep(0.01) z = delayed(dec)(x) zz = c.compute(z) result = yield zz assert result == 1 @gen_cluster(client=True) def test_decide_worker_with_many_independent_leaves(c, s, a, b): xs = yield [c._scatter(list(range(0, 100, 2)), workers=a.address), c._scatter(list(range(1, 100, 2)), workers=b.address)] xs = list(concat(zip(*xs))) ys = [delayed(inc)(x) for x in xs] y2s = c.persist(ys) yield wait(y2s) nhits = (sum(y.key in a.data for y in y2s[::2]) + sum(y.key in b.data for y in y2s[1::2])) assert nhits > 80 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_decide_worker_with_restrictions(client, s, a, b, c): x = client.submit(inc, 1, workers=[a.address, b.address]) yield wait(x) assert x.key in a.data or x.key in b.data @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_move_data_over_break_restrictions(client, s, a, b, c): [x] = yield client._scatter([1], workers=b.address) y = client.submit(inc, x, workers=[a.address, b.address]) yield wait(y) assert y.key in a.data or y.key in b.data @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_balance_with_restrictions(client, s, a, b, c): [x], [y] = yield [client._scatter([[1, 2, 3]], workers=a.address), client._scatter([1], workers=c.address)] z = client.submit(inc, 1, workers=[a.address, c.address]) yield wait(z) assert s.who_has[z.key] == {c.address} @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_no_valid_workers(client, s, a, b, c): x = client.submit(inc, 1, workers='127.0.0.5:9999') while not s.tasks: yield gen.sleep(0.01) assert x.key in s.unrunnable with pytest.raises(gen.TimeoutError): yield gen.with_timeout(timedelta(milliseconds=50), x) @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_no_valid_workers_loose_restrictions(client, s, a, b, c): x = client.submit(inc, 1, workers='127.0.0.5:9999', allow_other_workers=True) result = yield x assert result == 2 @gen_cluster(client=True, ncores=[]) def test_no_workers(client, s): x = client.submit(inc, 1) while not s.tasks: yield gen.sleep(0.01) assert x.key in s.unrunnable with pytest.raises(gen.TimeoutError): yield gen.with_timeout(timedelta(milliseconds=50), x) @gen_cluster(ncores=[]) def test_retire_workers_empty(s): yield s.retire_workers(workers=[]) @pytest.mark.skip def test_validate_state(): dsk = {'x': 1, 'y': (inc, 'x')} dependencies = {'x': set(), 'y': {'x'}} waiting = {'y': {'x'}} ready = deque(['x']) dependents = {'x': {'y'}, 'y': set()} waiting_data = {'x': {'y'}} who_has = dict() stacks = {alice: [], bob: []} processing = {alice: dict(), bob: dict()} finished_results = set() released = set() in_play = {'x', 'y'} who_wants = {'y': {'client'}} wants_what = {'client': {'y'}} erred = {} validate_state(**locals()) who_has['x'] = {alice} with pytest.raises(Exception): validate_state(**locals()) ready.remove('x') with pytest.raises(Exception): validate_state(**locals()) waiting['y'].remove('x') with pytest.raises(Exception): validate_state(**locals()) del waiting['y'] ready.appendleft('y') validate_state(**locals()) stacks[alice].append('y') with pytest.raises(Exception): validate_state(**locals()) ready.remove('y') validate_state(**locals()) stacks[alice].pop() with pytest.raises(Exception): validate_state(**locals()) processing[alice]['y'] = 1 validate_state(**locals()) del processing[alice]['y'] with pytest.raises(Exception): validate_state(**locals()) who_has['y'] = {alice} with pytest.raises(Exception): validate_state(**locals()) finished_results.add('y') with pytest.raises(Exception): validate_state(**locals()) waiting_data.pop('x') who_has.pop('x') released.add('x') validate_state(**locals()) @gen_cluster() def test_server(s, a, b): comm = yield connect(s.address) yield comm.write({'op': 'register-client', 'client': 'ident'}) yield comm.write({'op': 'update-graph', 'tasks': {'x': dumps_task((inc, 1)), 'y': dumps_task((inc, 'x'))}, 'dependencies': {'x': [], 'y': ['x']}, 'keys': ['y'], 'client': 'ident'}) while True: msg = yield readone(comm) if msg['op'] == 'key-in-memory' and msg['key'] == 'y': break yield comm.write({'op': 'close-stream'}) msg = yield readone(comm) assert msg == {'op': 'stream-closed'} with pytest.raises(CommClosedError): yield readone(comm) yield comm.close() @gen_cluster() def test_remove_client(s, a, b): s.update_graph(tasks={'x': dumps_task((inc, 1)), 'y': dumps_task((inc, 'x'))}, dependencies={'x': [], 'y': ['x']}, keys=['y'], client='ident') assert s.tasks assert s.dependencies s.remove_client(client='ident') assert not s.tasks assert not s.dependencies @gen_cluster() def test_server_listens_to_other_ops(s, a, b): with rpc(s.address) as r: ident = yield r.identity() assert ident['type'] == 'Scheduler' assert ident['id'].lower().startswith('scheduler') @gen_cluster() def test_remove_worker_from_scheduler(s, a, b): dsk = {('x-%d' % i): (inc, i) for i in range(20)} s.update_graph(tasks=valmap(dumps_task, dsk), keys=list(dsk), dependencies={k: set() for k in dsk}) assert a.address in s.worker_comms s.remove_worker(address=a.address) assert a.address not in s.ncores assert len(s.processing[b.address]) == len(dsk) # b owns everything s.validate_state() @gen_cluster() def test_add_worker(s, a, b): w = Worker(s.ip, s.port, ncores=3) w.data['x-5'] = 6 w.data['y'] = 1 yield w._start(0) dsk = {('x-%d' % i): (inc, i) for i in range(10)} s.update_graph(tasks=valmap(dumps_task, dsk), keys=list(dsk), client='client', dependencies={k: set() for k in dsk}) s.add_worker(address=w.address, keys=list(w.data), ncores=w.ncores, services=s.services) s.validate_state() assert w.ip in s.host_info assert s.host_info[w.ip]['addresses'] == {a.address, b.address, w.address} yield w._close() @gen_cluster() def test_feed(s, a, b): def func(scheduler): return dumps(scheduler.processing) comm = yield connect(s.address) yield comm.write({'op': 'feed', 'function': dumps(func), 'interval': 0.01}) for i in range(5): response = yield comm.read() expected = s.processing assert cloudpickle.loads(response) == expected yield comm.close() @gen_cluster() def test_feed_setup_teardown(s, a, b): def setup(scheduler): return 1 def func(scheduler, state): assert state == 1 return 'OK' def teardown(scheduler, state): scheduler.flag = 'done' comm = yield connect(s.address) yield comm.write({'op': 'feed', 'function': dumps(func), 'setup': dumps(setup), 'teardown': dumps(teardown), 'interval': 0.01}) for i in range(5): response = yield comm.read() assert response == 'OK' yield comm.close() start = time() while not hasattr(s, 'flag'): yield gen.sleep(0.01) assert time() - start < 5 @gen_cluster() def test_feed_large_bytestring(s, a, b): np = pytest.importorskip('numpy') x = np.ones(10000000) def func(scheduler): y = x return True comm = yield connect(s.address) yield comm.write({'op': 'feed', 'function': dumps(func), 'interval': 0.05}) for i in range(5): response = yield comm.read() assert response is True yield comm.close() @gen_test(timeout=None) def test_scheduler_as_center(): s = Scheduler(validate=True) done = s.start(0) a = Worker(s.address, ncores=1) a.data.update({'x': 1, 'y': 2}) b = Worker(s.address, ncores=2) b.data.update({'y': 2, 'z': 3}) c = Worker(s.address, ncores=3) yield [w._start(0) for w in [a, b, c]] assert s.ncores == {w.address: w.ncores for w in [a, b, c]} assert not s.who_has s.update_graph(tasks={'a': dumps_task((inc, 1))}, keys=['a'], dependencies={'a': []}) start = time() while 'a' not in s.who_has: assert time() - start < 5 yield gen.sleep(0.01) assert 'a' in a.data or 'a' in b.data or 'a' in c.data yield [w._close() for w in [a, b, c]] assert s.ncores == {} assert s.who_has == {} yield s.close() @gen_cluster(client=True) def test_delete_data(c, s, a, b): d = yield c._scatter({'x': 1, 'y': 2, 'z': 3}) assert set(s.who_has) == {'x', 'y', 'z'} assert set(a.data) | set(b.data) == {'x', 'y', 'z'} assert merge(a.data, b.data) == {'x': 1, 'y': 2, 'z': 3} del d['x'] del d['y'] start = time() while set(a.data) | set(b.data) != {'z'}: yield gen.sleep(0.01) assert time() < start + 5 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)]) def test_delete(c, s, a): x = c.submit(inc, 1) yield x assert x.key in a.data yield c._cancel(x) start = time() while x.key in a.data: yield gen.sleep(0.01) assert time() < start + 5 @gen_cluster() def test_filtered_communication(s, a, b): c = yield connect(s.address) f = yield connect(s.address) yield c.write({'op': 'register-client', 'client': 'c'}) yield f.write({'op': 'register-client', 'client': 'f'}) yield c.read() yield f.read() assert set(s.comms) == {'c', 'f'} yield c.write({'op': 'update-graph', 'tasks': {'x': dumps_task((inc, 1)), 'y': dumps_task((inc, 'x'))}, 'dependencies': {'x': [], 'y': ['x']}, 'client': 'c', 'keys': ['y']}) yield f.write({'op': 'update-graph', 'tasks': {'x': dumps_task((inc, 1)), 'z': dumps_task((add, 'x', 10))}, 'dependencies': {'x': [], 'z': ['x']}, 'client': 'f', 'keys': ['z']}) msg, = yield c.read() assert msg['op'] == 'key-in-memory' assert msg['key'] == 'y' msg, = yield f.read() assert msg['op'] == 'key-in-memory' assert msg['key'] == 'z' def test_dumps_function(): a = dumps_function(inc) assert cloudpickle.loads(a)(10) == 11 b = dumps_function(inc) assert a is b c = dumps_function(dec) assert a != c def test_dumps_task(): d = dumps_task((inc, 1)) assert set(d) == {'function', 'args'} f = lambda x, y=2: x + y d = dumps_task((apply, f, (1,), {'y': 10})) assert cloudpickle.loads(d['function'])(1, 2) == 3 assert cloudpickle.loads(d['args']) == (1,) assert cloudpickle.loads(d['kwargs']) == {'y': 10} d = dumps_task((apply, f, (1,))) assert cloudpickle.loads(d['function'])(1, 2) == 3 assert cloudpickle.loads(d['args']) == (1,) assert set(d) == {'function', 'args'} @gen_cluster() def test_ready_remove_worker(s, a, b): s.update_graph(tasks={'x-%d' % i: dumps_task((inc, i)) for i in range(20)}, keys=['x-%d' % i for i in range(20)], client='client', dependencies={'x-%d' % i: [] for i in range(20)}) assert all(len(s.processing[w]) >= s.ncores[w] for w in s.ncores) s.remove_worker(address=a.address) for collection in [s.ncores, s.processing]: assert set(collection) == {b.address} assert all(len(s.processing[w]) >= s.ncores[w] for w in s.ncores) assert set(s.processing) == {b.address} @gen_cluster(client=True, Worker=Nanny) def test_restart(c, s, a, b): futures = c.map(inc, range(20)) yield wait(futures) yield s.restart() for c in [s.processing, s.ncores, s.occupancy]: assert len(c) == 2 for c in [s.processing, s.occupancy]: assert not any(v for v in c.values()) assert not s.tasks assert not s.dependencies @gen_cluster() def test_broadcast(s, a, b): result = yield s.broadcast(msg={'op': 'ping'}) assert result == {a.address: b'pong', b.address: b'pong'} result = yield s.broadcast(msg={'op': 'ping'}, workers=[a.address]) assert result == {a.address: b'pong'} result = yield s.broadcast(msg={'op': 'ping'}, hosts=[a.ip]) assert result == {a.address: b'pong', b.address: b'pong'} @gen_cluster(Worker=Nanny) def test_broadcast_nanny(s, a, b): result1 = yield s.broadcast(msg={'op': 'identity'}, nanny=True) assert all(d['type'] == 'Nanny' for d in result1.values()) result2 = yield s.broadcast(msg={'op': 'identity'}, workers=[a.worker_address], nanny=True) assert len(result2) == 1 assert first(result2.values())['id'] == a.id result3 = yield s.broadcast(msg={'op': 'identity'}, hosts=[a.ip], nanny=True) assert result1 == result3 @gen_test() def test_worker_name(): s = Scheduler(validate=True) s.start(0) w = Worker(s.ip, s.port, name='alice') yield w._start() assert s.worker_info[w.address]['name'] == 'alice' assert s.aliases['alice'] == w.address with pytest.raises(ValueError): w = Worker(s.ip, s.port, name='alice') yield w._start() yield s.close() yield w._close() @gen_test() def test_coerce_address(): s = Scheduler(validate=True) s.start(0) print("scheduler:", s.address, s.listen_address) a = Worker(s.ip, s.port, name='alice') b = Worker(s.ip, s.port, name=123) c = Worker('127.0.0.1', s.port, name='charlie') yield [a._start(), b._start(), c._start()] assert s.coerce_address('127.0.0.1:8000') == 'tcp://127.0.0.1:8000' assert s.coerce_address('[::1]:8000') == 'tcp://[::1]:8000' assert s.coerce_address('tcp://127.0.0.1:8000') == 'tcp://127.0.0.1:8000' assert s.coerce_address('tcp://[::1]:8000') == 'tcp://[::1]:8000' assert s.coerce_address('localhost:8000') in ('tcp://127.0.0.1:8000', 'tcp://[::1]:8000') assert s.coerce_address(u'localhost:8000') in ('tcp://127.0.0.1:8000', 'tcp://[::1]:8000') assert s.coerce_address(a.address) == a.address # Aliases assert s.coerce_address('alice') == a.address assert s.coerce_address(123) == b.address assert s.coerce_address('charlie') == c.address assert s.coerce_hostname('127.0.0.1') == '127.0.0.1' assert s.coerce_hostname('alice') == a.ip assert s.coerce_hostname(123) == b.ip assert s.coerce_hostname('charlie') == c.ip assert s.coerce_hostname('jimmy') == 'jimmy' assert s.coerce_address('zzzt:8000', resolve=False) == 'tcp://zzzt:8000' yield s.close() yield [w._close() for w in [a, b, c]] @pytest.mark.skipif(sys.platform.startswith('win'), reason="file descriptors not really a thing") @gen_cluster(ncores=[]) def test_file_descriptors_dont_leak(s): psutil = pytest.importorskip('psutil') proc = psutil.Process() before = proc.num_fds() w = Worker(s.ip, s.port) yield w._start(0) yield w._close() during = proc.num_fds() start = time() while proc.num_fds() > before: yield gen.sleep(0.01) assert time() < start + 5 @gen_cluster() def test_update_graph_culls(s, a, b): s.update_graph(tasks={'x': dumps_task((inc, 1)), 'y': dumps_task((inc, 'x')), 'z': dumps_task((inc, 2))}, keys=['y'], dependencies={'y': 'x', 'x': [], 'z': []}, client='client') assert 'z' not in s.tasks assert 'z' not in s.dependencies @gen_cluster(ncores=[]) def test_add_worker_is_idempotent(s): s.add_worker(address=alice, ncores=1, resolve_address=False) ncores = s.ncores.copy() s.add_worker(address=alice, resolve_address=False) assert s.ncores == s.ncores def test_io_loop(loop): s = Scheduler(loop=loop, validate=True) assert s.io_loop is loop @gen_cluster(client=True) def test_story(c, s, a, b): x = delayed(inc)(1) y = delayed(inc)(x) f = c.persist(y) yield wait([f]) assert s.transition_log story = s.story(x.key) assert all(line in s.transition_log for line in story) assert len(story) < len(s.transition_log) assert all(x.key == line[0] or x.key in line[-2] for line in story) assert len(s.story(x.key, y.key)) > len(story) @gen_cluster(ncores=[], client=True) def test_scatter_no_workers(c, s): with pytest.raises(gen.TimeoutError): yield s.scatter(data={'x': 1}, client='alice', timeout=0.1) w = Worker(s.ip, s.port, ncores=3) yield [c._scatter(data={'x': 1}), w._start()] assert w.data['x'] == 1 yield w._close() @gen_cluster(ncores=[]) def test_scheduler_sees_memory_limits(s): w = Worker(s.ip, s.port, ncores=3, memory_limit=12345) yield w._start(0) assert s.worker_info[w.address]['memory_limit'] == 12345 yield w._close() @gen_cluster(client=True, timeout=1000) def test_retire_workers(c, s, a, b): [x] = yield c._scatter([1], workers=a.address) [y] = yield c._scatter([list(range(1000))], workers=b.address) assert s.workers_to_close() == [a.address] workers = yield s.retire_workers() assert workers == [a.address] assert list(s.ncores) == [b.address] assert s.workers_to_close() == [] assert s.has_what[b.address] == {x.key, y.key} workers = yield s.retire_workers() assert not workers @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 4) def test_workers_to_close(cl, s, *workers): s.task_duration['a'] = 4 s.task_duration['b'] = 4 s.task_duration['c'] = 1 cl.map(slowinc, [1, 1, 1], key=['a-4','b-4','c-1']) while len(s.rprocessing) < 3: yield gen.sleep(0.001) wtc = s.workers_to_close() assert all(not s.processing[w] for w in wtc) assert len(wtc) == 1 @gen_cluster(client=True) def test_retire_workers_no_suspicious_tasks(c, s, a, b): future = c.submit(slowinc, 100, delay=0.5, workers=a.address, allow_other_workers=True) yield gen.sleep(0.2) yield s.retire_workers(workers=[a.address]) assert all(v==0 for v in s.suspicious_tasks.values()) @slow @pytest.mark.skipif(sys.platform.startswith('win'), reason="file descriptors not really a thing") @gen_cluster(client=True, ncores=[], timeout=240) def test_file_descriptors(c, s): psutil = pytest.importorskip('psutil') da = pytest.importorskip('dask.array') proc = psutil.Process() num_fds_1 = proc.num_fds() N = 20 nannies = [Nanny(s.ip, s.port, loop=s.loop) for i in range(N)] yield [n._start() for n in nannies] while len(s.ncores) < N: yield gen.sleep(0.1) num_fds_2 = proc.num_fds() yield gen.sleep(0.2) num_fds_3 = proc.num_fds() assert num_fds_3 == num_fds_2 x = da.random.random(size=(1000, 1000), chunks=(25, 25)) x = c.persist(x) yield wait(x) num_fds_4 = proc.num_fds() assert num_fds_4 < num_fds_3 + N y = c.persist(x + x.T) yield wait(y) num_fds_5 = proc.num_fds() assert num_fds_5 < num_fds_4 + N yield gen.sleep(1) num_fds_6 = proc.num_fds() assert num_fds_6 < num_fds_5 + N yield [n._close() for n in nannies] @nodebug @gen_cluster(client=True) def test_learn_occupancy(c, s, a, b): futures = c.map(slowinc, range(1000), delay=0.01) while not any(s.who_has): yield gen.sleep(0.01) assert 1 < s.total_occupancy < 40 for w in [a, b]: assert 1 < s.occupancy[w.address] < 20 @nodebug @gen_cluster(client=True) def test_learn_occupancy_2(c, s, a, b): future = c.map(slowinc, range(1000), delay=0.1) while not any(s.who_has): yield gen.sleep(0.01) assert 50 < s.total_occupancy < 200 @gen_cluster(client=True) def test_occupancy_cleardown(c, s, a, b): s.validate = False # Inject excess values in s.occupancy s.occupancy[a.address] = 2 s.total_occupancy += 2 futures = c.map(slowinc, range(100), delay=0.01) yield wait(futures) # Verify that occupancy values have been zeroed out assert abs(s.total_occupancy) < 0.01 assert all(v == 0 for v in s.occupancy.values()) @nodebug @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 30) def test_balance_many_workers(c, s, *workers): futures = c.map(slowinc, range(20), delay=0.2) yield wait(futures) assert set(map(len, s.has_what.values())) == {0, 1} @nodebug @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 30) def test_balance_many_workers_2(c, s, *workers): s.extensions['stealing']._pc.callback_time = 100000000 futures = c.map(slowinc, range(90), delay=0.2) yield wait(futures) assert set(map(len, s.has_what.values())) == {3} @gen_cluster(client=True) def test_learn_occupancy_multiple_workers(c, s, a, b): x = c.submit(slowinc, 1, delay=0.2, workers=a.address) yield gen.sleep(0.05) futures = c.map(slowinc, range(100), delay=0.2) yield wait(x) assert not any(v == 0.5 for vv in s.processing.values() for v in vv) s.validate_state() @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) z = c.submit(slowadd, x, y, delay=1) while z.key not in s.rprocessing: yield gen.sleep(0.01) try: assert s.processing[b.address][z.key] > 1 except Exception: print("processing:", s.processing) print("rprocessing:", s.rprocessing) print("task_duration:", s.task_duration) print("nbytes:", s.nbytes) raise yield wait(z) del z @gen_cluster(client=True) def test_worker_arrives_with_processing_data(c, s, a, b): x = delayed(slowinc)(1, delay=0.4) y = delayed(slowinc)(x, delay=0.4) z = delayed(slowinc)(y, delay=0.4) yy, zz = c.persist([y, z]) while not s.processing: yield gen.sleep(0.01) w = Worker(s.ip, s.port, ncores=1) w.put_key_in_memory(y.key, 3) yield w._start() start = time() while len(s.workers) < 3: yield gen.sleep(0.01) assert s.task_state[y.key] == 'memory' assert s.task_state[x.key] == 'released' assert s.task_state[z.key] == 'processing' yield w._close() @gen_cluster(client=True, ncores=[('127.0.0.1', 1)]) def test_worker_breaks_and_returns(c, s, a): future = c.submit(slowinc, 1, delay=0.1) for i in range(10): future = c.submit(slowinc, future, delay=0.1) yield wait(future) a.batched_stream.comm.close() yield gen.sleep(0.1) start = time() yield wait(future) end = time() assert end - start < 1 assert frequencies(s.task_state.values()) == {'memory': 1, 'released': 10} @gen_cluster(client=True, ncores=[]) def test_no_workers_to_memory(c, s): x = delayed(slowinc)(1, delay=0.4) y = delayed(slowinc)(x, delay=0.4) z = delayed(slowinc)(y, delay=0.4) yy, zz = c.persist([y, z]) while not s.task_state: yield gen.sleep(0.01) w = Worker(s.ip, s.port, ncores=1) w.put_key_in_memory(y.key, 3) yield w._start() start = time() while not s.workers: yield gen.sleep(0.01) assert s.task_state[y.key] == 'memory' assert s.task_state[x.key] == 'released' assert s.task_state[z.key] == 'processing' yield w._close() @gen_cluster(client=True) def test_no_worker_to_memory_restrictions(c, s, a, b): x = delayed(slowinc)(1, delay=0.4) y = delayed(slowinc)(x, delay=0.4) z = delayed(slowinc)(y, delay=0.4) yy, zz = c.persist([y, z], workers={(x, y, z): 'alice'}) while not s.task_state: yield gen.sleep(0.01) w = Worker(s.ip, s.port, ncores=1, name='alice') w.put_key_in_memory(y.key, 3) yield w._start() while len(s.workers) < 3: yield gen.sleep(0.01) yield gen.sleep(0.3) assert s.task_state[y.key] == 'memory' assert s.task_state[x.key] == 'released' assert s.task_state[z.key] == 'processing' yield w._close() def test_run_on_scheduler_sync(loop): def f(dask_scheduler=None): return dask_scheduler.address with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: address = c.run_on_scheduler(f) assert address == s['address'] with pytest.raises(ZeroDivisionError): c.run_on_scheduler(div, 1, 0) @gen_cluster(client=True) def test_run_on_scheduler(c, s, a, b): def f(dask_scheduler=None): return dask_scheduler.address response = yield c._run_on_scheduler(f) assert response == s.address @gen_cluster(client=True) def test_close_worker(c, s, a, b): assert len(s.workers) == 2 yield s.close_worker(worker=a.address) assert len(s.workers) == 1 assert a.address not in s.workers yield gen.sleep(0.5) assert len(s.workers) == 1 @slow @gen_cluster(client=True, Worker=Nanny, timeout=20) def test_close_nanny(c, s, a, b): assert len(s.workers) == 2 assert a.process.is_alive() a_worker_address = a.worker_address start = time() yield s.close_worker(worker=a_worker_address) assert len(s.workers) == 1 assert a_worker_address not in s.workers assert not a.is_alive() assert a.pid is None for i in range(10): yield gen.sleep(0.1) assert len(s.workers) == 1 assert not a.is_alive() assert a.pid is None while a.status != 'closed': yield gen.sleep(0.05) assert time() < start + 10 @gen_cluster(client=True, timeout=20) def test_retire_workers_close(c, s, a, b): yield s.retire_workers(close=True) assert not s.workers @gen_cluster(client=True, timeout=20, Worker=Nanny) def test_retire_nannies_close(c, s, a, b): nannies = [a, b] yield s.retire_workers(close=True, remove=True) assert not s.workers start = time() while any(n.status != 'closed' for n in nannies): yield gen.sleep(0.05) assert time() < start + 10 assert not any(n.is_alive() for n in nannies) assert not s.workers @gen_cluster(client=True, ncores=[('127.0.0.1', 2)]) def test_fifo_submission(c, s, w): futures = [] for i in range(20): future = c.submit(slowinc, i, delay=0.1, key='inc-%02d' % i) futures.append(future) yield gen.sleep(0.01) yield wait(futures[-1]) assert futures[10].status == 'finished' @gen_test() def test_scheduler_file(): with tmpfile() as fn: s = Scheduler(scheduler_file=fn) s.start(0) with open(fn) as f: data = json.load(f) assert data['address'] == s.address c = yield Client(scheduler_file=fn, loop=s.loop, asynchronous=True) yield s.close() @slow @gen_cluster(client=True, ncores=[]) def test_non_existent_worker(c, s): s.add_worker(address='127.0.0.1:5738', ncores=2, nbytes={}, host_info={}) futures = c.map(inc, range(10)) yield gen.sleep(4) assert not s.workers assert all(v == 'no-worker' for v in s.task_state.values()) @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_correct_bad_time_estimate(c, s, *workers): future = c.submit(slowinc, 1, delay=0) yield wait(future) futures = [c.submit(slowinc, future, delay=0.1, pure=False) for i in range(20)] yield gen.sleep(0.5) yield wait(futures) assert all(w.data for w in workers) @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_test(timeout=None) def test_service_hosts_match_scheduler(): pytest.importorskip('bokeh') from distributed.bokeh.scheduler import BokehScheduler services = {('bokeh', 0): BokehScheduler} s = Scheduler(services=services) yield s.start('tcp://0.0.0.0') sock = first(s.services['bokeh'].server._http._sockets.values()) assert sock.getsockname()[0] in ('::', '0.0.0.0') yield s.close() for host in ['tcp://127.0.0.2', 'tcp://127.0.0.2:38275']: s = Scheduler(services=services) yield s.start(host) sock = first(s.services['bokeh'].server._http._sockets.values()) assert sock.getsockname()[0] == '127.0.0.2' yield s.close() @gen_cluster(client=True, worker_kwargs={'profile_cycle_interval': 100}) def test_profile_metadata(c, s, a, b): start = time() - 1 futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) yield wait(futures) yield gen.sleep(0.200) meta = yield s.get_profile_metadata(profile_cycle_interval=0.100) now = time() + 1 assert meta assert all(start < t < now for t, count in meta['counts']) assert all(0 <= count < 30 for t, count in meta['counts'][:4]) assert not meta['counts'][-1][1] @gen_cluster(client=True, worker_kwargs={'profile_cycle_interval': 100}) def test_profile_metadata_keys(c, s, a, b): start = time() - 1 x = c.map(slowinc, range(10), delay=0.05) y = c.map(slowdec, range(10), delay=0.05) yield wait(x + y) meta = yield s.get_profile_metadata(profile_cycle_interval=0.100) assert set(meta['keys']) == {'slowinc', 'slowdec'} assert len(meta['counts']) == len(meta['keys']['slowinc']) @gen_cluster(client=True) def test_cancel_fire_and_forget(c, s, a, b): x = delayed(slowinc)(1, delay=0.05) y = delayed(slowinc)(x, delay=0.05) z = delayed(slowinc)(y, delay=0.05) w = delayed(slowinc)(z, delay=0.05) future = c.compute(w) fire_and_forget(future) yield gen.sleep(0.05) yield future.cancel(force=True) assert future.status == 'cancelled' assert not s.task_state @gen_cluster(client=True, Worker=Nanny) def test_log_tasks_during_restart(c, s, a, b): future = c.submit(sys.exit, 0) yield wait(future) assert 'exit' in str(s.events) @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_reschedule(c, s, a, b): yield c.submit(slowinc, -1, delay=0.1) # learn cost x = c.map(slowinc, range(4), delay=0.1) # add much more work onto worker a futures = c.map(slowinc, range(10, 20), delay=0.1, workers=a.address) while len(s.task_state) < len(x) + len(futures): yield gen.sleep(0.001) for future in x: s.reschedule(key=future.key) # Worker b gets more of the original tasks yield wait(x) assert sum(future.key in b.data for future in x) >= 3 assert sum(future.key in a.data for future in x) <= 1 @gen_cluster(client=True) def test_get_task_status(c, s, a, b): future = c.submit(inc, 1) yield wait(future) result = yield a.scheduler.get_task_status(keys=[future.key]) assert result == {future.key: 'memory'} def test_deque_handler(): from distributed.scheduler import logger s = Scheduler() deque_handler = s._deque_handler logger.info('foo123') assert len(deque_handler.deque) >= 1 msg = deque_handler.deque[-1] assert 'distributed.scheduler' in deque_handler.format(msg) assert any(msg.msg == 'foo123' for msg in deque_handler.deque) @gen_cluster(client=True) def test_retries(c, s, a, b): args = [ZeroDivisionError("one"), ZeroDivisionError("two"), 42] future = c.submit(varying(args), retries=3) result = yield future assert result == 42 assert s.retries[future.key] == 1 assert future.key not in s.exceptions future = c.submit(varying(args), retries=2, pure=False) result = yield future assert result == 42 assert s.retries[future.key] == 0 assert future.key not in s.exceptions future = c.submit(varying(args), retries=1, pure=False) with pytest.raises(ZeroDivisionError) as exc_info: res = yield future exc_info.match("two") future = c.submit(varying(args), retries=0, pure=False) with pytest.raises(ZeroDivisionError) as exc_info: res = yield future exc_info.match("one") distributed-1.20.2/distributed/tests/test_security.py000066400000000000000000000241401321233345200230610ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from contextlib import contextmanager import sys try: import ssl except ImportError: ssl = None import pytest from tornado import gen from distributed.comm import connect, listen from distributed.security import Security from distributed.utils_test import new_config, get_cert, gen_test ca_file = get_cert('tls-ca-cert.pem') cert1 = get_cert('tls-cert.pem') key1 = get_cert('tls-key.pem') keycert1 = get_cert('tls-key-cert.pem') # Note this cipher uses RSA auth as this matches our test certs FORCED_CIPHER = 'ECDHE-RSA-AES128-GCM-SHA256' def test_defaults(): with new_config({}): sec = Security() assert sec.require_encryption in (None, False) assert sec.tls_ca_file is None assert sec.tls_ciphers is None assert sec.tls_client_key is None assert sec.tls_client_cert is None assert sec.tls_scheduler_key is None assert sec.tls_scheduler_cert is None assert sec.tls_worker_key is None assert sec.tls_worker_cert is None def test_attribute_error(): sec = Security() assert hasattr(sec, 'tls_ca_file') with pytest.raises(AttributeError): sec.tls_foobar with pytest.raises(AttributeError): sec.tls_foobar = "" def test_from_config(): c = { 'tls': { 'ca-file': 'ca.pem', 'scheduler': { 'key': 'skey.pem', 'cert': 'scert.pem', }, 'worker': { 'cert': 'wcert.pem', }, 'ciphers': FORCED_CIPHER, }, 'require-encryption': True, } with new_config(c): sec = Security() assert sec.require_encryption is True assert sec.tls_ca_file == 'ca.pem' assert sec.tls_ciphers == FORCED_CIPHER assert sec.tls_client_key is None assert sec.tls_client_cert is None assert sec.tls_scheduler_key == 'skey.pem' assert sec.tls_scheduler_cert == 'scert.pem' assert sec.tls_worker_key is None assert sec.tls_worker_cert == 'wcert.pem' def test_kwargs(): c = { 'tls': { 'ca-file': 'ca.pem', 'scheduler': { 'key': 'skey.pem', 'cert': 'scert.pem', }, }, } with new_config(c): sec = Security(tls_scheduler_cert='newcert.pem', require_encryption=True, tls_ca_file=None) assert sec.require_encryption is True # None value didn't override default assert sec.tls_ca_file == 'ca.pem' assert sec.tls_ciphers is None assert sec.tls_client_key is None assert sec.tls_client_cert is None assert sec.tls_scheduler_key == 'skey.pem' assert sec.tls_scheduler_cert == 'newcert.pem' assert sec.tls_worker_key is None assert sec.tls_worker_cert is None def test_repr(): with new_config({}): sec = Security(tls_ca_file='ca.pem', tls_scheduler_cert='scert.pem') assert repr(sec) == "Security(tls_ca_file='ca.pem', tls_scheduler_cert='scert.pem')" def test_tls_config_for_role(): c = { 'tls': { 'ca-file': 'ca.pem', 'scheduler': { 'key': 'skey.pem', 'cert': 'scert.pem', }, 'worker': { 'cert': 'wcert.pem', }, 'ciphers': FORCED_CIPHER, }, } with new_config(c): sec = Security() t = sec.get_tls_config_for_role('scheduler') assert t == { 'ca_file': 'ca.pem', 'key': 'skey.pem', 'cert': 'scert.pem', 'ciphers': FORCED_CIPHER, } t = sec.get_tls_config_for_role('worker') assert t == { 'ca_file': 'ca.pem', 'key': None, 'cert': 'wcert.pem', 'ciphers': FORCED_CIPHER, } t = sec.get_tls_config_for_role('client') assert t == { 'ca_file': 'ca.pem', 'key': None, 'cert': None, 'ciphers': FORCED_CIPHER, } with pytest.raises(ValueError): sec.get_tls_config_for_role('supervisor') def test_connection_args(): def basic_checks(ctx): assert ctx.verify_mode == ssl.CERT_REQUIRED assert ctx.check_hostname is False def many_ciphers(ctx): if sys.version_info >= (3, 6): assert len(ctx.get_ciphers()) > 2 # Most likely c = { 'tls': { 'ca-file': ca_file, 'scheduler': { 'key': key1, 'cert': cert1, }, 'worker': { 'cert': keycert1, }, }, } with new_config(c): sec = Security() d = sec.get_connection_args('scheduler') assert not d['require_encryption'] ctx = d['ssl_context'] basic_checks(ctx) many_ciphers(ctx) d = sec.get_connection_args('worker') ctx = d['ssl_context'] basic_checks(ctx) many_ciphers(ctx) # No cert defined => no TLS d = sec.get_connection_args('client') assert d.get('ssl_context') is None # With more settings c['tls']['ciphers'] = FORCED_CIPHER c['require-encryption'] = True with new_config(c): sec = Security() d = sec.get_listen_args('scheduler') assert d['require_encryption'] ctx = d['ssl_context'] basic_checks(ctx) if sys.version_info >= (3, 6): assert len(ctx.get_ciphers()) == 1 def test_listen_args(): def basic_checks(ctx): assert ctx.verify_mode == ssl.CERT_REQUIRED assert ctx.check_hostname is False def many_ciphers(ctx): if sys.version_info >= (3, 6): assert len(ctx.get_ciphers()) > 2 # Most likely c = { 'tls': { 'ca-file': ca_file, 'scheduler': { 'key': key1, 'cert': cert1, }, 'worker': { 'cert': keycert1, }, }, } with new_config(c): sec = Security() d = sec.get_listen_args('scheduler') assert not d['require_encryption'] ctx = d['ssl_context'] basic_checks(ctx) many_ciphers(ctx) d = sec.get_listen_args('worker') ctx = d['ssl_context'] basic_checks(ctx) many_ciphers(ctx) # No cert defined => no TLS d = sec.get_listen_args('client') assert d.get('ssl_context') is None # With more settings c['tls']['ciphers'] = FORCED_CIPHER c['require-encryption'] = True with new_config(c): sec = Security() d = sec.get_listen_args('scheduler') assert d['require_encryption'] ctx = d['ssl_context'] basic_checks(ctx) if sys.version_info >= (3, 6): assert len(ctx.get_ciphers()) == 1 @gen_test() def test_tls_listen_connect(): """ Functional test for TLS connection args. """ @gen.coroutine def handle_comm(comm): peer_addr = comm.peer_address assert peer_addr.startswith('tls://') yield comm.write('hello') yield comm.close() c = { 'tls': { 'ca-file': ca_file, 'scheduler': { 'key': key1, 'cert': cert1, }, 'worker': { 'cert': keycert1, }, }, } with new_config(c): sec = Security() c['tls']['ciphers'] = FORCED_CIPHER with new_config(c): forced_cipher_sec = Security() with listen('tls://', handle_comm, connection_args=sec.get_listen_args('scheduler')) as listener: comm = yield connect(listener.contact_address, connection_args=sec.get_connection_args('worker')) msg = yield comm.read() assert msg == 'hello' comm.abort() # No SSL context for client with pytest.raises(TypeError): yield connect(listener.contact_address, connection_args=sec.get_connection_args('client')) # Check forced cipher comm = yield connect(listener.contact_address, connection_args=forced_cipher_sec.get_connection_args('worker')) cipher, _, _, = comm.extra_info['cipher'] assert cipher == FORCED_CIPHER comm.abort() @gen_test() def test_require_encryption(): """ Functional test for "require_encryption" setting. """ @gen.coroutine def handle_comm(comm): comm.abort() c = { 'tls': { 'ca-file': ca_file, 'scheduler': { 'key': key1, 'cert': cert1, }, 'worker': { 'cert': keycert1, }, }, } with new_config(c): sec = Security() c['require-encryption'] = True with new_config(c): sec2 = Security() for listen_addr in ['inproc://', 'tls://']: with listen(listen_addr, handle_comm, connection_args=sec.get_listen_args('scheduler')) as listener: comm = yield connect(listener.contact_address, connection_args=sec2.get_connection_args('worker')) comm.abort() with listen(listen_addr, handle_comm, connection_args=sec2.get_listen_args('scheduler')) as listener: comm = yield connect(listener.contact_address, connection_args=sec2.get_connection_args('worker')) comm.abort() @contextmanager def check_encryption_error(): with pytest.raises(RuntimeError) as excinfo: yield assert "encryption required" in str(excinfo.value) for listen_addr in ['tcp://']: with listen(listen_addr, handle_comm, connection_args=sec.get_listen_args('scheduler')) as listener: comm = yield connect(listener.contact_address, connection_args=sec.get_connection_args('worker')) comm.abort() with pytest.raises(RuntimeError): yield connect(listener.contact_address, connection_args=sec2.get_connection_args('worker')) with pytest.raises(RuntimeError): listen(listen_addr, handle_comm, connection_args=sec2.get_listen_args('scheduler')) distributed-1.20.2/distributed/tests/test_sizeof.py000066400000000000000000000045631321233345200225200ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import sys import pytest from distributed.sizeof import sizeof, getsizeof def test_base(): assert sizeof(1) == getsizeof(1) def test_containers(): assert sizeof([1, 2, [3]]) > (getsizeof(3) * 3 + getsizeof([])) def test_numpy(): np = pytest.importorskip('numpy') assert 8000 <= sizeof(np.empty(1000, dtype='f8')) <= 9000 dt = np.dtype('f8') assert sizeof(dt) == sys.getsizeof(dt) def test_pandas(): pd = pytest.importorskip('pandas') df = pd.DataFrame({'x': [1, 2, 3], 'y': ['a' * 100, 'b' * 100, 'c' * 100]}, index=[10, 20, 30]) assert sizeof(df) >= sizeof(df.x) + sizeof(df.y) - sizeof(df.index) assert sizeof(df.x) >= sizeof(df.index) if pd.__version__ >= '0.17.1': assert sizeof(df.y) >= 100 * 3 assert sizeof(df.index) >= 20 assert isinstance(sizeof(df), int) assert isinstance(sizeof(df.x), int) assert isinstance(sizeof(df.index), int) def test_pandas_repeated_column(): pd = pytest.importorskip('pandas') df = pd.DataFrame({'x': [1, 2, 3]}) assert sizeof(df[['x', 'x', 'x']]) > sizeof(df) def test_sparse_matrix(): sparse = pytest.importorskip('scipy.sparse') sp = sparse.eye(10) # These are the 32-bit Python 2.7 values. assert sizeof(sp.todia()) >= 152 assert sizeof(sp.tobsr()) >= 232 assert sizeof(sp.tocoo()) >= 240 assert sizeof(sp.tocsc()) >= 232 assert sizeof(sp.tocsr()) >= 232 assert sizeof(sp.todok()) >= 192 assert sizeof(sp.tolil()) >= 204 def test_serires_object_dtype(): pd = pytest.importorskip('pandas') s = pd.Series(['a'] * 1000) assert sizeof('a') * 1000 < sizeof(s) < 2 * sizeof('a') * 1000 s = pd.Series(['a' * 1000] * 1000) assert sizeof(s) > 1000000 def test_dataframe_object_dtype(): pd = pytest.importorskip('pandas') df = pd.DataFrame({'x': ['a'] * 1000}) assert sizeof('a') * 1000 < sizeof(df) < 2 * sizeof('a') * 1000 s = pd.Series(['a' * 1000] * 1000) assert sizeof(s) > 1000000 def test_empty(): pd = pytest.importorskip('pandas') df = pd.DataFrame({'x': [1, 2, 3], 'y': ['a' * 100, 'b' * 100, 'c' * 100]}, index=[10, 20, 30]) empty = df.head(0) assert sizeof(empty) > 0 assert sizeof(empty.x) > 0 assert sizeof(empty.y) > 0 assert sizeof(empty.index) > 0 distributed-1.20.2/distributed/tests/test_steal.py000066400000000000000000000434561321233345200223350ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import itertools from operator import mul import random import sys from time import sleep import weakref import pytest from toolz import sliding_window, concat from tornado import gen 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.utils_test import (slowinc, slowadd, inc, gen_cluster, slowidentity) from distributed.utils_test import (loop, nodebug_setup_module, nodebug_teardown_module) # flake8: noqa from distributed.worker import TOTAL_MEMORY import pytest # Most tests here are timing-dependent setup_module = nodebug_setup_module teardown_module = nodebug_teardown_module @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster(client=True, ncores=[('127.0.0.1', 2), ('127.0.0.2', 2)], timeout=20) def test_work_stealing(c, s, a, b): [x] = yield c._scatter([1], workers=a.address) futures = c.map(slowadd, range(50), [x] * 50) yield wait(futures) assert len(a.data) > 10 assert len(b.data) > 10 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_dont_steal_expensive_data_fast_computation(c, s, a, b): np = pytest.importorskip('numpy') x = c.submit(np.arange, 1000000, workers=a.address) yield wait([x]) future = c.submit(np.sum, [1], workers=a.address) # learn that sum is fast yield wait([future]) cheap = [c.submit(np.sum, x, pure=False, workers=a.address, allow_other_workers=True) for i in range(10)] yield wait(cheap) assert len(s.who_has[x.key]) == 1 assert len(b.data) == 0 assert len(a.data) == 12 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_steal_cheap_data_slow_computation(c, s, a, b): x = c.submit(slowinc, 100, delay=0.1) # learn that slowinc is slow yield wait(x) futures = c.map(slowinc, range(10), delay=0.1, workers=a.address, allow_other_workers=True) yield wait(futures) assert abs(len(a.data) - len(b.data)) <= 5 @pytest.mark.avoid_travis @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_steal_expensive_data_slow_computation(c, s, a, b): np = pytest.importorskip('numpy') x = c.submit(slowinc, 100, delay=0.2, workers=a.address) yield wait(x) # learn that slowinc is slow x = c.submit(np.arange, 1000000, workers=a.address) # put expensive data yield wait(x) slow = [c.submit(slowinc, x, delay=0.1, pure=False) for i in range(20)] yield wait(slow) assert len(s.who_has[x.key]) > 1 assert b.data # not empty @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10) def test_worksteal_many_thieves(c, s, *workers): x = c.submit(slowinc, -1, delay=0.1) yield x xs = c.map(slowinc, [x] * 100, pure=False, delay=0.1) yield wait(xs) for w, keys in s.has_what.items(): assert 2 < len(keys) < 30 assert len(s.who_has[x.key]) > 1 assert sum(map(len, s.has_what.values())) < 150 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_dont_steal_unknown_functions(c, s, a, b): futures = c.map(inc, [1, 2], workers=a.address, allow_other_workers=True) yield wait(futures) assert len(a.data) == 2 assert len(b.data) == 0 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_eventually_steal_unknown_functions(c, s, a, b): futures = c.map(slowinc, range(10), delay=0.1, workers=a.address, allow_other_workers=True) yield wait(futures) assert len(a.data) >= 3 assert len(b.data) >= 3 @pytest.mark.xfail(reason='') @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_steal_related_tasks(e, s, a, b, c): futures = e.map(slowinc, range(20), delay=0.05, workers=a.address, allow_other_workers=True) yield wait(futures) nearby = 0 for f1, f2 in sliding_window(2, futures): if s.who_has[f1.key] == s.who_has[f2.key]: nearby += 1 assert nearby > 10 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10, timeout=1000) def test_dont_steal_fast_tasks(c, s, *workers): np = pytest.importorskip('numpy') x = c.submit(np.random.random, 10000000, workers=workers[0].address) def do_nothing(x, y=None): pass yield wait(c.submit(do_nothing, 1)) futures = c.map(do_nothing, range(1000), y=x) yield wait(futures) assert len(s.who_has[x.key]) == 1 assert len(s.has_what[workers[0].address]) == 1001 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)], timeout=20) def test_new_worker_steals(c, s, a): yield wait(c.submit(slowinc, 1, delay=0.01)) futures = c.map(slowinc, range(100), delay=0.05) total = c.submit(sum, futures) while len(a.task_state) < 10: yield gen.sleep(0.01) b = Worker(s.ip, s.port, loop=s.loop, ncores=1, memory_limit=TOTAL_MEMORY) yield b._start() result = yield total assert result == sum(map(inc, range(100))) for w in [a, b]: assert all(isinstance(v, int) for v in w.data.values()) assert b.data yield b._close() @gen_cluster(client=True, timeout=20) def test_work_steal_no_kwargs(c, s, a, b): yield wait(c.submit(slowinc, 1, delay=0.05)) futures = c.map(slowinc, range(100), workers=a.address, allow_other_workers=True, delay=0.05) yield wait(futures) assert 20 < len(a.data) < 80 assert 20 < len(b.data) < 80 total = c.submit(sum, futures) result = yield total assert result == sum(map(inc, range(100))) @gen_cluster(client=True, ncores=[('127.0.0.1', 1), ('127.0.0.1', 2)]) def test_dont_steal_worker_restrictions(c, s, a, b): future = c.submit(slowinc, 1, delay=0.10, workers=a.address) yield future futures = c.map(slowinc, range(100), delay=0.1, workers=a.address) while len(a.task_state) + len(b.task_state) < 100: yield gen.sleep(0.01) assert len(a.task_state) == 100 assert len(b.task_state) == 0 result = s.extensions['stealing'].balance() yield gen.sleep(0.1) assert len(a.task_state) == 100 assert len(b.task_state) == 0 @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster(client=True, ncores=[('127.0.0.1', 1), ('127.0.0.2', 1)]) def test_dont_steal_host_restrictions(c, s, a, b): future = c.submit(slowinc, 1, delay=0.10, workers=a.address) yield future futures = c.map(slowinc, range(100), delay=0.1, workers='127.0.0.1') while len(a.task_state) < 10: yield gen.sleep(0.01) assert len(a.task_state) == 100 assert len(b.task_state) == 0 result = s.extensions['stealing'].balance() yield gen.sleep(0.1) assert len(a.task_state) == 100 assert len(b.task_state) == 0 @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 2}}), ('127.0.0.1', 1)]) def test_dont_steal_resource_restrictions(c, s, a, b): future = c.submit(slowinc, 1, delay=0.10, workers=a.address) yield future futures = c.map(slowinc, range(100), delay=0.1, resources={'A': 1}) while len(a.task_state) < 10: yield gen.sleep(0.01) assert len(a.task_state) == 100 assert len(b.task_state) == 0 result = s.extensions['stealing'].balance() yield gen.sleep(0.1) assert len(a.task_state) == 100 assert len(b.task_state) == 0 @pytest.mark.xfail(reason='no stealing of resources') @gen_cluster(client=True, ncores=[('127.0.0.1', 1, {'resources': {'A': 2}})], timeout=3) def test_steal_resource_restrictions(c, s, a): future = c.submit(slowinc, 1, delay=0.10, workers=a.address) yield future futures = c.map(slowinc, range(100), delay=0.2, resources={'A': 1}) while len(a.task_state) < 101: yield gen.sleep(0.01) assert len(a.task_state) == 101 b = Worker(s.ip, s.port, loop=s.loop, ncores=1, resources={'A': 4}) yield b._start() start = time() while not b.task_state or len(a.task_state) == 101: yield gen.sleep(0.01) assert time() < start + 3 assert len(b.task_state) > 0 assert len(a.task_state) < 101 yield b._close() @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 5, timeout=20) def test_balance_without_dependencies(c, s, *workers): s.extensions['stealing']._pc.callback_time = 20 def slow(x): y = random.random() * 0.1 sleep(y) return y futures = c.map(slow, range(100)) yield wait(futures) durations = [sum(w.data.values()) for w in workers] assert max(durations) / min(durations) < 3 @gen_cluster(client=True, ncores=[('127.0.0.1', 4)] * 2) def test_dont_steal_executing_tasks(c, s, a, b): futures = c.map(slowinc, range(4), delay=0.1, workers=a.address, allow_other_workers=True) yield wait(futures) assert len(a.data) == 4 assert len(b.data) == 0 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10) def test_dont_steal_few_saturated_tasks_many_workers(c, s, a, *rest): s.extensions['stealing']._pc.callback_time = 20 x = c.submit(mul, b'0', 100000000, workers=a.address) # 100 MB yield wait(x) s.task_duration['slowidentity'] = 0.2 futures = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(2)] yield wait(futures) assert len(a.data) == 3 assert not any(w.task_state for w in rest) @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10, worker_kwargs={'memory_limit': TOTAL_MEMORY}) def test_steal_when_more_tasks(c, s, a, *rest): s.extensions['stealing']._pc.callback_time = 20 x = c.submit(mul, b'0', 50000000, workers=a.address) # 50 MB yield wait(x) s.task_duration['slowidentity'] = 0.2 futures = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(20)] yield gen.sleep(0.1) assert any(w.task_state for w in rest) @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10) def test_steal_more_attractive_tasks(c, s, a, *rest): def slow2(x): sleep(1) return x s.extensions['stealing']._pc.callback_time = 20 x = c.submit(mul, b'0', 100000000, workers=a.address) # 100 MB yield wait(x) s.task_duration['slowidentity'] = 0.2 s.task_duration['slow2'] = 1 futures = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(10)] future = c.submit(slow2, x) while not any(w.task_state for w in rest): yield gen.sleep(0.01) # good future moves first assert any(future.key in w.task_state for w in rest) def func(x): sleep(1) def assert_balanced(inp, expected, c, s, *workers): steal = s.extensions['stealing'] steal._pc.stop() counter = itertools.count() B = BANDWIDTH tasks = list(concat(inp)) data_seq = itertools.count() futures = [] for w, ts in zip(workers, inp): for t in sorted(ts, reverse=True): if t: [dat] = yield c._scatter([next(data_seq)], workers=w.address) s.nbytes[dat.key] = BANDWIDTH * t else: dat = 123 s.task_duration[str(int(t))] = 1 f = c.submit(func, dat, key='%d-%d' % (int(t), next(counter)), workers=w.address, allow_other_workers=True, pure=False) futures.append(f) while len(s.rprocessing) < len(futures): yield gen.sleep(0.001) for i in range(10): steal.balance() while steal.in_flight: yield gen.sleep(0.001) result = [sorted([int(key_split(k)) for k in s.processing[w.address]], reverse=True) for w in workers] result2 = sorted(result, reverse=True) expected2 = sorted(expected, reverse=True) if config.get('pdb-on-err'): if result2 != expected2: import pdb pdb.set_trace() if result2 == expected2: return raise Exception('Expected: {}; got: {}'.format(str(expected2), str(result2))) @pytest.mark.parametrize('inp,expected', [ ([[1], []], # don't move unnecessarily [[1], []]), ([[0, 0], []], # balance [[0], [0]]), ([[0.1, 0.1], []], # balance even if results in even [[0], [0]]), ([[0, 0, 0], []], # don't over balance [[0, 0], [0]]), ([[0, 0], [0, 0, 0], []], # move from larger [[0, 0], [0, 0], [0]]), ([[0, 0, 0], [0], []], # move to smaller [[0, 0], [0], [0]]), ([[0, 1], []], # choose easier first [[1], [0]]), ([[0, 0, 0, 0], [], []], # spread evenly [[0, 0], [0], [0]]), ([[1, 0, 2, 0], [], []], # move easier [[2, 1], [0], [0]]), ([[1, 1, 1], []], # be willing to move costly items [[1, 1], [1]]), ([[1, 1, 1, 1], []], # but don't move too many [[1, 1, 1], [1]]), ([[0, 0], [0, 0], [0, 0], []], # no one clearly saturated [[0, 0], [0, 0], [0], [0]]), ([[4, 2, 2, 2, 2, 1, 1], [4, 2, 1, 1], [], [], []], [[4, 2, 2, 2, 2], [4, 2, 1], [1], [1], [1]]), pytest.mark.xfail(([[1, 1, 1, 1, 1, 1, 1], [1, 1], [1, 1], [1, 1], []], [[1, 1, 1, 1, 1], [1, 1], [1, 1], [1, 1], [1, 1]]), reason="Some uncertainty based on executing stolen task") ]) def test_balance(inp, expected): test = lambda *args, **kwargs: assert_balanced(inp, expected, *args, **kwargs) test = gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * len(inp))(test) test() @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2, Worker=Nanny, timeout=20) def test_restart(c, s, a, b): futures = c.map(slowinc, range(100), delay=0.1, workers=a.address, allow_other_workers=True) while not s.processing[b.worker_address]: yield gen.sleep(0.01) steal = s.extensions['stealing'] assert any(st for st in steal.stealable_all) assert any(x for L in steal.stealable.values() for x in L) yield c.restart(timeout=10) assert not any(x for x in steal.stealable_all) assert not any(x for L in steal.stealable.values() for x in L) @gen_cluster(client=True) 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) futures = [c.submit(slowadd, x, y, delay=1, pure=False, workers=a.address, allow_other_workers=True) for i in range(10)] while not any(f.key in s.rprocessing for f in futures): yield gen.sleep(0.01) steal.balance() while steal.in_flight: yield gen.sleep(0.001) assert s.processing[b.address] @gen_cluster(client=True) def test_steal_twice(c, s, a, b): x = c.submit(inc, 1, workers=a.address) yield wait(x) futures = [c.submit(slowadd, x, i, delay=0.2) for i in range(100)] while len(s.task_state) < 100: # tasks are all allocated yield gen.sleep(0.01) workers = [Worker(s.ip, s.port, loop=s.loop) for _ in range(20)] yield [w._start() for w in workers] # army of new workers arrives to help yield wait(futures) has_what = dict(s.has_what) # take snapshot empty_workers = [w for w, keys in has_what.items() if not len(keys)] if len(empty_workers) > 2: pytest.fail("Too many workers without keys (%d out of %d)" % (len(empty_workers), len(has_what))) assert max(map(len, has_what.values())) < 30 yield c._close() yield [w._close() for w in workers] @gen_cluster(client=True) def test_dont_steal_executing_tasks(c, s, a, b): steal = s.extensions['stealing'] future = c.submit(slowinc, 1, delay=0.5, workers=a.address) while not a.executing: yield gen.sleep(0.01) steal.move_task_request(future.key, a.address, b.address) yield gen.sleep(0.1) assert future.key in a.executing assert not b.executing @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_dont_steal_long_running_tasks(c, s, a, b): def long(delay): with worker_client() as c: sleep(delay) yield c.submit(long, 0.1) # learn duration yield c.submit(inc, 1) # learn duration long_tasks = c.map(long, [0.5, 0.6], workers=a.address, allow_other_workers=True) while sum(map(len, s.processing.values())) < 2: # let them start yield gen.sleep(0.01) start = time() while any(t.key in s.extensions['stealing'].key_stealable for t in long_tasks): yield gen.sleep(0.01) assert time() < start + 1 na = len(a.executing) nb = len(b.executing) incs = c.map(inc, range(100), workers=a.address, allow_other_workers=True) yield gen.sleep(0.2) assert sum(1 for k in s.processing[b.address] if k.startswith('long')) <= nb yield wait(long_tasks) @gen_cluster(client=True, ncores=[('127.0.0.1', 5)] * 2) def test_cleanup_repeated_tasks(c, s, a, b): class Foo(object): pass s.extensions['stealing']._pc.callback_time = 20 yield c.submit(slowidentity, -1, delay=0.1) objects = [c.submit(Foo, pure=False, workers=a.address) for _ in range(50)] x = c.map(slowidentity, objects, workers=a.address, allow_other_workers=True, delay=0.05) del objects yield wait(x) assert a.data and b.data assert len(a.data) + len(b.data) > 10 ws = weakref.WeakSet() ws.update(a.data.values()) ws.update(b.data.values()) del x start = time() while a.data or b.data: yield gen.sleep(0.01) assert time() < start + 1 assert not s.who_has assert not any(s.has_what.values()) assert not list(ws) distributed-1.20.2/distributed/tests/test_stress.py000066400000000000000000000204411321233345200225350ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from concurrent.futures import CancelledError from datetime import timedelta from operator import add import random import sys from time import sleep from dask import delayed import pytest from toolz import concat, sliding_window from distributed import Client, wait, Nanny from distributed.config import config from distributed.metrics import time from distributed.utils import All from distributed.utils_test import (gen_cluster, cluster, inc, slowinc, slowadd, slow, slowsum, bump_rlimit) from distributed.utils_test import (loop, nodebug_setup_module, nodebug_teardown_module) # flake8: noqa from distributed.client import wait from tornado import gen # All tests here are slow in some way setup_module = nodebug_setup_module teardown_module = nodebug_teardown_module @gen_cluster(client=True) def test_stress_1(c, s, a, b): n = 2**6 seq = c.map(inc, range(n)) while len(seq) > 1: yield gen.sleep(0.1) seq = [c.submit(add, seq[i], seq[i + 1]) for i in range(0, len(seq), 2)] result = yield seq[0] assert result == sum(map(inc, range(n))) @pytest.mark.parametrize(('func', 'n'), [(slowinc, 100), (inc, 1000)]) def test_stress_gc(loop, func, n): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(func, 1) for i in range(n): x = c.submit(func, x) assert x.result() == n + 2 @pytest.mark.skipif(sys.platform.startswith('win'), reason="test can leave dangling RPC objects") @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 8, timeout=None) def test_cancel_stress(c, s, *workers): da = pytest.importorskip('dask.array') x = da.random.random((50, 50), chunks=(2, 2)) x = c.persist(x) yield wait([x]) y = (x.sum(axis=0) + x.sum(axis=1) + 1).std() n_todo = len(y.dask) - len(x.dask) for i in range(5): f = c.compute(y) while len(s.waiting) > (random.random() + 1) * 0.5 * n_todo: yield gen.sleep(0.01) yield c._cancel(f) def test_cancel_stress_sync(loop): da = pytest.importorskip('dask.array') x = da.random.random((50, 50), chunks=(2, 2)) with cluster(active_rpc_timeout=10) as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.persist(x) y = (x.sum(axis=0) + x.sum(axis=1) + 1).std() wait(x) for i in range(5): f = c.compute(y) sleep(random.random()) c.cancel(f) @gen_cluster(ncores=[], client=True, timeout=None) def test_stress_creation_and_deletion(c, s): # Assertions are handled by the validate mechanism in the scheduler s.allowed_failures = 100000 da = pytest.importorskip('dask.array') x = da.random.random(size=(2000, 2000), chunks=(100, 100)) y = (x + 1).T + (x * 2) - x.mean(axis=1) z = c.persist(y) @gen.coroutine def create_and_destroy_worker(delay): start = time() while time() < start + 5: n = Nanny(s.address, ncores=2, loop=s.loop) n.start(0) yield gen.sleep(delay) yield n._close() print("Killed nanny") yield gen.with_timeout(timedelta(minutes=1), All([create_and_destroy_worker(0.1 * i) for i in range(20)])) @gen_cluster(ncores=[('127.0.0.1', 1)] * 10, client=True, timeout=60) def test_stress_scatter_death(c, s, *workers): import random s.allowed_failures = 1000 np = pytest.importorskip('numpy') L = yield c.scatter([np.random.random(10000) for i in range(len(workers))]) yield c._replicate(L, n=2) adds = [delayed(slowadd, pure=True)(random.choice(L), random.choice(L), delay=0.05) for i in range(50)] adds = [delayed(slowadd, pure=True)(a, b, delay=0.02) for a, b in sliding_window(2, adds)] futures = c.compute(adds) alive = list(workers) from distributed.scheduler import logger for i in range(7): yield gen.sleep(0.1) try: s.validate_state() except Exception as c: logger.exception(c) if config.get('log-on-err'): import pdb pdb.set_trace() else: raise w = random.choice(alive) yield w._close() alive.remove(w) try: yield gen.with_timeout(timedelta(seconds=25), c._gather(futures)) except gen.TimeoutError: ws = {w.address: w for w in workers if w.status != 'closed'} print(s.processing) print(ws) print(futures) try: worker = [w for w in ws.values() if w.waiting_for_data][0] except Exception: pass if config.get('log-on-err'): import pdb pdb.set_trace() else: raise except CancelledError: pass def vsum(*args): return sum(args) @pytest.mark.avoid_travis @slow @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 80, timeout=1000) def test_stress_communication(c, s, *workers): s.validate = False # very slow otherwise da = pytest.importorskip('dask.array') # Test consumes many file descriptors and can hang if the limit is too low resource = pytest.importorskip('resource') bump_rlimit(resource.RLIMIT_NOFILE, 8192) n = 20 xs = [da.random.random((100, 100), chunks=(5, 5)) for i in range(n)] ys = [x + x.T for x in xs] z = da.atop(vsum, 'ij', *concat(zip(ys, ['ij'] * n)), dtype='float64') future = c.compute(z.sum()) result = yield future assert isinstance(result, float) @pytest.mark.skip @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10, timeout=60) def test_stress_steal(c, s, *workers): s.validate = False for w in workers: w.validate = False dinc = delayed(slowinc) L = [delayed(slowinc)(i, delay=0.005) for i in range(100)] for i in range(5): L = [delayed(slowsum)(part, delay=0.005) for part in sliding_window(5, L)] total = delayed(sum)(L) future = c.compute(total) while future.status != 'finished': yield gen.sleep(0.1) for i in range(3): a = random.choice(workers) b = random.choice(workers) if a is not b: s.work_steal(a.address, b.address, 0.5) if not s.processing: break @slow @gen_cluster(ncores=[('127.0.0.1', 1)] * 10, client=True, timeout=120) def test_close_connections(c, s, *workers): da = pytest.importorskip('dask.array') x = da.random.random(size=(1000, 1000), chunks=(1000, 1)) for i in range(3): x = x.rechunk((1, 1000)) x = x.rechunk((1000, 1)) future = c.compute(x.sum()) while any(s.processing.values()): yield gen.sleep(0.5) worker = random.choice(list(workers)) for comm in worker._comms: comm.abort() # print(frequencies(s.task_state.values())) # for w in workers: # print(w) yield wait(future) @pytest.mark.xfail(reason="IOStream._handle_write blocks on large write_buffer" " https://github.com/tornadoweb/tornado/issues/2110") @gen_cluster(client=True, timeout=20, ncores=[('127.0.0.1', 1)]) def test_no_delay_during_large_transfer(c, s, w): pytest.importorskip('crick') np = pytest.importorskip('numpy') x = np.random.random(100000000) # Reset digests from distributed.counter import Digest from collections import defaultdict from functools import partial from dask.diagnostics import ResourceProfiler for server in [s, w]: server.digests = defaultdict(partial(Digest, loop=server.io_loop)) server._last_tick = time() with ResourceProfiler(dt=0.01) as rprof: future = yield c.scatter(x, direct=True, hash=False) yield gen.sleep(0.5) rprof.close() for server in [s, w]: assert server.digests['tick-duration'].components[0].max() < 0.5 nbytes = np.array([t.mem for t in rprof.results]) nbytes -= nbytes[0] assert nbytes.max() < (x.nbytes * 2) / 1e6 assert nbytes[-1] < (x.nbytes * 1.2) / 1e6 distributed-1.20.2/distributed/tests/test_submit_cli.py000066400000000000000000000035501321233345200233460ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from mock import Mock from tornado import gen from tornado.ioloop import IOLoop from distributed.submit import RemoteClient, _submit, _remote from distributed.utils_test import (valid_python_script, invalid_python_script, loop) # flake8: noqa def test_dask_submit_cli_writes_result_to_stdout(loop, tmpdir, valid_python_script): @gen.coroutine def test(): remote_client = RemoteClient(ip='127.0.0.1', local_dir=str(tmpdir)) yield remote_client._start() out, err = yield _submit('127.0.0.1:{0}'.format(remote_client.port), str(valid_python_script)) assert b'hello world!' in out yield remote_client._close() loop.run_sync(test, timeout=5) def test_dask_submit_cli_writes_traceback_to_stdout(loop, tmpdir, invalid_python_script): @gen.coroutine def test(): remote_client = RemoteClient(ip='127.0.0.1', local_dir=str(tmpdir)) yield remote_client._start() out, err = yield _submit('127.0.0.1:{0}'.format(remote_client.port), str(invalid_python_script)) assert b'Traceback' in err yield remote_client._close() loop.run_sync(test, timeout=5) def test_cli_runs_remote_client(): mock_remote_client = Mock(spec=RemoteClient) mock_ioloop = Mock(spec=IOLoop.current()) _remote('127.0.0.1:8799', 8788, loop=mock_ioloop, client=mock_remote_client) mock_remote_client.assert_called_once_with(ip='127.0.0.1', loop=mock_ioloop) mock_remote_client().start.assert_called_once_with(port=8799) assert mock_ioloop.start.called assert mock_ioloop.close.called assert mock_remote_client().stop.called distributed-1.20.2/distributed/tests/test_submit_remote_client.py000066400000000000000000000036441321233345200254340ustar00rootroot00000000000000import os from tornado import gen from distributed import rpc from distributed.submit import RemoteClient from distributed.utils_test import (loop, valid_python_script, invalid_python_script) # flake8: noqa def test_remote_client_uploads_a_file(loop, tmpdir): @gen.coroutine def test(): remote_client = RemoteClient(ip='127.0.0.1', local_dir=str(tmpdir)) yield remote_client._start(0) remote_process = rpc(remote_client.address) upload = yield remote_process.upload_file(filename='script.py', file_payload='x=1') assert upload == {'status': 'OK', 'nbytes': 3} assert tmpdir.join('script.py').read() == "x=1" yield remote_client._close() loop.run_sync(test, timeout=5) def test_remote_client_execution_outputs_to_stdout(loop, tmpdir): @gen.coroutine def test(): remote_client = RemoteClient(ip='127.0.0.1', local_dir=str(tmpdir)) yield remote_client._start(0) rr = rpc(remote_client.address) yield rr.upload_file(filename='script.py', file_payload='print("hello world!")') message = yield rr.execute(filename='script.py') assert message['stdout'] == b'hello world!' + os.linesep.encode() assert message['returncode'] == 0 yield remote_client._close() loop.run_sync(test, timeout=5) def test_remote_client_execution_outputs_stderr(loop, tmpdir, invalid_python_script): @gen.coroutine def test(): remote_client = RemoteClient(ip='127.0.0.1', local_dir=str(tmpdir)) yield remote_client._start(0) rr = rpc(remote_client.address) yield rr.upload_file(filename='script.py', file_payload='a+1') message = yield rr.execute(filename='script.py') assert b'\'a\' is not defined' in message['stderr'] assert message['returncode'] == 1 yield remote_client._close() loop.run_sync(test, timeout=5) distributed-1.20.2/distributed/tests/test_system_monitor.py000066400000000000000000000024171321233345200243100ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from time import sleep from distributed.system_monitor import SystemMonitor def test_SystemMonitor(): sm = SystemMonitor() a = sm.update() sleep(0.01) b = sm.update() assert sm.cpu assert sm.memory assert set(a) == set(b) assert all(rb >= 0 for rb in sm.read_bytes) assert all(wb >= 0 for wb in sm.write_bytes) assert all(len(q) == 3 for q in sm.quantities.values()) assert 'cpu' in repr(sm) def test_count(): sm = SystemMonitor(n=5) assert sm.count == 1 sm.update() assert sm.count == 2 for i in range(10): sm.update() assert sm.count == 12 for v in sm.quantities.values(): assert len(v) == 5 def test_range_query(): sm = SystemMonitor(n=5) assert all(len(v) == 1 for v in sm.range_query(0).values()) assert all(len(v) == 0 for v in sm.range_query(123).values()) sm.update() sm.update() sm.update() assert all(len(v) == 4 for v in sm.range_query(0).values()) assert all(len(v) == 3 for v in sm.range_query(1).values()) for i in range(10): sm.update() assert all(len(v) == 4 for v in sm.range_query(10).values()) assert all(len(v) == 5 for v in sm.range_query(0).values()) distributed-1.20.2/distributed/tests/test_threadpoolexecutor.py000066400000000000000000000047401321233345200251360ustar00rootroot00000000000000from time import sleep import threading from distributed.metrics import time from distributed.threadpoolexecutor import ThreadPoolExecutor, secede, rejoin def test_tpe(): with ThreadPoolExecutor(2) as e: list(e.map(sleep, [0.01] * 4)) threads = e._threads.copy() assert len(threads) == 2 def f(): secede() return 1 assert e.submit(f).result() == 1 list(e.map(sleep, [0.01] * 4)) assert len(threads | e._threads) == 3 start = time() while all(t.is_alive() for t in threads): sleep(0.01) assert time() < start + 1 def test_shutdown_timeout(): e = ThreadPoolExecutor(1) futures = [e.submit(sleep, 0.1 * i) for i in range(1, 3, 1)] sleep(0.01) start = time() e.shutdown() end = time() assert end - start > 0.1 def test_shutdown_timeout_raises(): e = ThreadPoolExecutor(1) futures = [e.submit(sleep, 0.1 * i) for i in range(1, 3, 1)] sleep(0.05) start = time() e.shutdown(timeout=0.1) end = time() assert end - start > 0.05 def test_secede_rejoin_busy(): with ThreadPoolExecutor(2) as e: def f(): assert threading.current_thread() in e._threads secede() sleep(0.1) assert threading.current_thread() not in e._threads rejoin() assert len(e._threads) == 2 assert threading.current_thread() in e._threads return threading.current_thread() future = e.submit(f) L = [e.submit(sleep, 0.2) for i in range(10)] start = time() special_thread = future.result() stop = time() assert 0.1 < stop - start < 0.3 assert len(e._threads) == 2 assert special_thread in e._threads def f(): sleep(0.01) return threading.current_thread() futures = [e.submit(f) for _ in range(10)] assert special_thread in {future.result() for future in futures} def test_secede_rejoin_quiet(): with ThreadPoolExecutor(2) as e: def f(): assert threading.current_thread() in e._threads secede() sleep(0.1) assert threading.current_thread() not in e._threads rejoin() assert len(e._threads) == 2 assert threading.current_thread() in e._threads return threading.current_thread() future = e.submit(f) result = future.result() distributed-1.20.2/distributed/tests/test_tls_functional.py000066400000000000000000000102101321233345200242270ustar00rootroot00000000000000""" Various functional tests for TLS networking. Most are taken from other test files and adapted. """ from __future__ import print_function, division, absolute_import from tornado import gen from distributed import Nanny, worker_client, Queue from distributed.client import wait from distributed.utils_test import (gen_cluster, tls_only_security, inc, double, slowinc, slowadd) def gen_tls_cluster(**kwargs): kwargs.setdefault('ncores', [('tls://127.0.0.1', 1), ('tls://127.0.0.1', 2)]) return gen_cluster(scheduler='tls://127.0.0.1', security=tls_only_security(), **kwargs) @gen_tls_cluster(client=True) def test_Queue(c, s, a, b): assert s.address.startswith('tls://') x = Queue('x') y = Queue('y') size = yield x.qsize() assert size == 0 future = c.submit(inc, 1) yield x.put(future) future2 = yield x.get() assert future.key == future2.key @gen_tls_cluster(client=True, timeout=None) def test_client_submit(c, s, a, b): assert s.address.startswith('tls://') x = c.submit(inc, 10) result = yield x assert result == 11 yy = [c.submit(slowinc, i) for i in range(10)] results = [] for y in yy: results.append((yield y)) assert results == list(range(1, 11)) @gen_tls_cluster(client=True) def test_gather(c, s, a, b): assert s.address.startswith('tls://') x = c.submit(inc, 10) y = c.submit(inc, x) result = yield c._gather(x) assert result == 11 result = yield c._gather([x]) assert result == [11] result = yield c._gather({'x': x, 'y': [y]}) assert result == {'x': 11, 'y': [12]} @gen_tls_cluster(client=True) def test_scatter(c, s, a, b): assert s.address.startswith('tls://') d = yield c._scatter({'y': 20}) assert s.who_has['y'] assert s.nbytes['y'] > 0 yy = yield c._gather([d['y']]) assert yy == [20] @gen_tls_cluster(client=True, Worker=Nanny) def test_nanny(c, s, a, b): assert s.address.startswith('tls://') for n in [a, b]: assert isinstance(n, Nanny) assert n.address.startswith('tls://') assert n.worker_address.startswith('tls://') assert s.ncores == {n.worker_address: n.ncores for n in [a, b]} x = c.submit(inc, 10) result = yield x assert result == 11 @gen_tls_cluster(client=True) def test_rebalance(c, s, a, b): x, y = yield c._scatter([1, 2], workers=[a.address]) assert len(a.data) == 2 assert len(b.data) == 0 yield c._rebalance() assert len(a.data) == 1 assert len(b.data) == 1 @gen_tls_cluster(client=True, ncores=[('tls://127.0.0.1', 2)] * 2) def test_work_stealing(c, s, a, b): [x] = yield c._scatter([1], workers=a.address) futures = c.map(slowadd, range(50), [x] * 50, delay=0.1) yield gen.sleep(0.1) yield wait(futures) assert len(a.data) > 10 assert len(b.data) > 10 @gen_tls_cluster(client=True) def test_worker_client(c, s, a, b): def func(x): with worker_client() as c: x = c.submit(inc, x) y = c.submit(double, x) result = x.result() + y.result() return result x, y = c.map(func, [10, 20]) xx, yy = yield c._gather([x, y]) assert xx == 10 + 1 + (10 + 1) * 2 assert yy == 20 + 1 + (20 + 1) * 2 @gen_tls_cluster(client=True, ncores=[('tls://127.0.0.1', 1)] * 2) def test_worker_client_gather(c, s, a, b): a_address = a.address b_address = b.address assert a_address.startswith('tls://') assert b_address.startswith('tls://') assert a_address != b_address def func(): with worker_client() as ee: x = ee.submit(inc, 1, workers=a_address) y = ee.submit(inc, 2, workers=b_address) xx, yy = ee.gather([x, y]) return xx, yy future = c.submit(func) result = yield future assert result == (2, 3) @gen_tls_cluster(client=True) def test_worker_client_executor(c, s, a, b): def mysum(): with worker_client() as c: with c.get_executor() as e: return sum(e.map(double, range(30))) future = c.submit(mysum) result = yield future assert result == 30 * 29 distributed-1.20.2/distributed/tests/test_utils.py000066400000000000000000000312351321233345200223550ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import Iterator from functools import partial import io import socket import sys from time import sleep from threading import Thread import threading import traceback import numpy as np import pytest from tornado import gen from tornado.ioloop import IOLoop from tornado.locks import Event import dask from distributed.compatibility import Queue, Empty, isqueue, PY2 from distributed.metrics import time from distributed.utils import (All, sync, is_kernel, ensure_ip, str_graph, truncate_exception, get_traceback, queue_to_iterator, iterator_to_queue, _maybe_complex, read_block, seek_delimiter, funcname, ensure_bytes, open_port, get_ip_interface, nbytes, set_thread_state, thread_state, LoopRunner, parse_bytes) from distributed.utils_test import loop, loop_in_thread # flake8: noqa from distributed.utils_test import div, has_ipv6, inc, throws, gen_test def test_All(loop): @gen.coroutine def throws(): 1 / 0 @gen.coroutine def slow(): yield gen.sleep(10) @gen.coroutine def inc(x): raise gen.Return(x + 1) @gen.coroutine def f(): results = yield All(*[inc(i) for i in range(10)]) assert results == list(range(1, 11)) start = time() for tasks in [[throws(), slow()], [slow(), throws()]]: try: yield All(tasks) assert False except ZeroDivisionError: pass end = time() assert end - start < 10 loop.run_sync(f) def test_sync(loop_in_thread): loop = loop_in_thread result = sync(loop, inc, 1) assert result == 2 def test_sync_error(loop_in_thread): loop = loop_in_thread try: result = sync(loop, throws, 1) except Exception as exc: f = exc assert 'hello' in str(exc) tb = get_traceback() L = traceback.format_tb(tb) assert any('throws' in line for line in L) def function1(x): return function2(x) def function2(x): return throws(x) try: result = sync(loop, function1, 1) except Exception as exc: assert 'hello' in str(exc) tb = get_traceback() L = traceback.format_tb(tb) assert any('function1' in line for line in L) assert any('function2' in line for line in L) def test_sync_timeout(loop_in_thread): loop = loop_in_thread with pytest.raises(gen.TimeoutError): sync(loop_in_thread, gen.sleep, 0.5, callback_timeout=0.05) def test_sync_closed_loop(): loop = IOLoop.current() loop.close() IOLoop.clear_current() IOLoop.clear_instance() with pytest.raises(RuntimeError) as exc_info: sync(loop, inc, 1) exc_info.match("IOLoop is clos(ed|ing)") def test_is_kernel(): pytest.importorskip('IPython') assert is_kernel() is False #@pytest.mark.leaking('fds') #def test_zzz_leaks(l=[]): #import os, subprocess #l.append(b"x" * (17 * 1024**2)) #os.open(__file__, os.O_RDONLY) #subprocess.Popen('sleep 100', shell=True, stdin=subprocess.DEVNULL) def test_ensure_ip(): assert ensure_ip('localhost') in ('127.0.0.1', '::1') assert ensure_ip('123.123.123.123') == '123.123.123.123' assert ensure_ip('8.8.8.8') == '8.8.8.8' if has_ipv6(): assert ensure_ip('2001:4860:4860::8888') == '2001:4860:4860::8888' assert ensure_ip('::1') == '::1' def test_get_ip_interface(): if sys.platform == 'darwin': assert get_ip_interface('lo0') == '127.0.0.1' elif sys.platform.startswith('linux'): assert get_ip_interface('lo') == '127.0.0.1' else: pytest.skip("test needs to be enhanced for platform %r" % (sys.platform,)) with pytest.raises(KeyError): get_ip_interface('__non-existent-interface') def test_truncate_exception(): e = ValueError('a' * 1000) assert len(str(e)) >= 1000 f = truncate_exception(e, 100) assert type(f) == type(e) assert len(str(f)) < 200 assert 'aaaa' in str(f) e = ValueError('a') assert truncate_exception(e) is e def test_get_traceback(): def a(x): return div(x, 0) def b(x): return a(x) def c(x): return b(x) try: c(1) except Exception as e: tb = get_traceback() 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 dsk = {('x', 1): (inc, 1)} assert str_graph(dsk) == {str(('x', 1)): (inc, 1)} dsk = {('x', 1): (inc, 1), ('x', 2): (inc, ('x', 1))} assert str_graph(dsk) == {str(('x', 1)): (inc, 1), str(('x', 2)): (inc, str(('x', 1)))} dsks = [{'x': 1}, {('x', 1): (inc, 1), ('x', 2): (inc, ('x', 1))}, {('x', 1): (sum, [1, 2, 3]), ('x', 2): (sum, [('x', 1), ('x', 1)])}] for dsk in dsks: sdsk = str_graph(dsk) keys = list(dsk) skeys = [str(k) for k in keys] assert all(isinstance(k, str) for k in sdsk) assert dask.get(dsk, keys) == dask.get(sdsk, skeys) def test_maybe_complex(): assert not _maybe_complex(1) assert not _maybe_complex('x') assert _maybe_complex((inc, 1)) assert _maybe_complex([(inc, 1)]) assert _maybe_complex([(inc, 1)]) assert _maybe_complex({'x': (inc, 1)}) def test_read_block(): delimiter = b'\n' data = delimiter.join([b'123', b'456', b'789']) f = io.BytesIO(data) assert read_block(f, 1, 2) == b'23' assert read_block(f, 0, 1, delimiter=b'\n') == b'123\n' assert read_block(f, 0, 2, delimiter=b'\n') == b'123\n' assert read_block(f, 0, 3, delimiter=b'\n') == b'123\n' assert read_block(f, 0, 5, delimiter=b'\n') == b'123\n456\n' assert read_block(f, 0, 8, delimiter=b'\n') == b'123\n456\n789' assert read_block(f, 0, 100, delimiter=b'\n') == b'123\n456\n789' assert read_block(f, 1, 1, delimiter=b'\n') == b'' assert read_block(f, 1, 5, delimiter=b'\n') == b'456\n' assert read_block(f, 1, 8, delimiter=b'\n') == b'456\n789' for ols in [[(0, 3), (3, 3), (6, 3), (9, 2)], [(0, 4), (4, 4), (8, 4)]]: out = [read_block(f, o, l, b'\n') for o, l in ols] assert b"".join(filter(None, out)) == data def test_seek_delimiter_endline(): f = io.BytesIO(b'123\n456\n789') # if at zero, stay at zero seek_delimiter(f, b'\n', 5) assert f.tell() == 0 # choose the first block for bs in [1, 5, 100]: f.seek(1) seek_delimiter(f, b'\n', blocksize=bs) assert f.tell() == 4 # handle long delimiters well, even with short blocksizes f = io.BytesIO(b'123abc456abc789') for bs in [1, 2, 3, 4, 5, 6, 10]: f.seek(1) seek_delimiter(f, b'abc', blocksize=bs) assert f.tell() == 6 # End at the end f = io.BytesIO(b'123\n456') f.seek(5) seek_delimiter(f, b'\n', 5) assert f.tell() == 7 def test_funcname(): def f(): pass assert funcname(f) == 'f' assert funcname(partial(f)) == 'f' assert funcname(partial(partial(f))) == 'f' def test_ensure_bytes(): data = [b'1', '1', memoryview(b'1'), bytearray(b'1')] if PY2: data.append(buffer(b'1')) # flake8: noqa for d in data: result = ensure_bytes(d) assert isinstance(result, bytes) assert result == b'1' def test_nbytes(): def check(obj, expected): assert nbytes(obj) == expected assert nbytes(memoryview(obj)) == expected check(b'123', 3) check(bytearray(b'4567'), 4) multi_dim = np.ones(shape=(10, 10)) scalar = np.array(1) check(multi_dim, multi_dim.nbytes) check(scalar, scalar.nbytes) def test_open_port(): port = open_port() s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) s.bind(('', port)) s.close() def test_set_thread_state(): with set_thread_state(x=1): assert thread_state.x == 1 assert not hasattr(thread_state, 'x') def assert_running(loop): """ Raise if the given IOLoop is not running. """ q = Queue() loop.add_callback(q.put, 42) assert q.get(timeout=1) == 42 def assert_not_running(loop): """ Raise if the given IOLoop is running. """ q = Queue() try: loop.add_callback(q.put, 42) except RuntimeError: # On AsyncIOLoop, can't add_callback() after the loop is closed pass else: with pytest.raises(Empty): q.get(timeout=0.02) def test_loop_runner(loop_in_thread): # Implicit loop loop = IOLoop() loop.make_current() runner = LoopRunner() assert runner.loop not in (loop, loop_in_thread) assert not runner.is_started() assert_not_running(runner.loop) runner.start() assert runner.is_started() assert_running(runner.loop) runner.stop() assert not runner.is_started() assert_not_running(runner.loop) # Explicit loop loop = IOLoop() runner = LoopRunner(loop=loop) assert runner.loop is loop assert not runner.is_started() assert_not_running(loop) runner.start() assert runner.is_started() assert_running(loop) runner.stop() assert not runner.is_started() assert_not_running(loop) # Explicit loop, already started runner = LoopRunner(loop=loop_in_thread) assert not runner.is_started() assert_running(loop_in_thread) runner.start() assert runner.is_started() assert_running(loop_in_thread) runner.stop() assert not runner.is_started() assert_running(loop_in_thread) # Implicit loop, asynchronous=True loop = IOLoop() loop.make_current() runner = LoopRunner(asynchronous=True) assert runner.loop is loop assert not runner.is_started() assert_not_running(runner.loop) runner.start() assert runner.is_started() assert_not_running(runner.loop) runner.stop() assert not runner.is_started() assert_not_running(runner.loop) # Explicit loop, asynchronous=True loop = IOLoop() runner = LoopRunner(loop=loop, asynchronous=True) assert runner.loop is loop assert not runner.is_started() assert_not_running(runner.loop) runner.start() assert runner.is_started() assert_not_running(runner.loop) runner.stop() assert not runner.is_started() assert_not_running(runner.loop) def test_two_loop_runners(loop_in_thread): # Loop runners tied to the same loop should cooperate # ABCCBA loop = IOLoop() a = LoopRunner(loop=loop) b = LoopRunner(loop=loop) assert_not_running(loop) a.start() assert_running(loop) c = LoopRunner(loop=loop) b.start() assert_running(loop) c.start() assert_running(loop) c.stop() assert_running(loop) b.stop() assert_running(loop) a.stop() assert_not_running(loop) # ABCABC loop = IOLoop() a = LoopRunner(loop=loop) b = LoopRunner(loop=loop) assert_not_running(loop) a.start() assert_running(loop) b.start() assert_running(loop) c = LoopRunner(loop=loop) c.start() assert_running(loop) a.stop() assert_running(loop) b.stop() assert_running(loop) c.stop() assert_not_running(loop) # Explicit loop, already started a = LoopRunner(loop=loop_in_thread) b = LoopRunner(loop=loop_in_thread) assert_running(loop_in_thread) a.start() assert_running(loop_in_thread) b.start() assert_running(loop_in_thread) a.stop() assert_running(loop_in_thread) b.stop() assert_running(loop_in_thread) @gen_test() def test_loop_runner_gen(): runner = LoopRunner(asynchronous=True) assert runner.loop is IOLoop.current() assert not runner.is_started() yield gen.sleep(0.01) runner.start() assert runner.is_started() yield gen.sleep(0.01) runner.stop() assert not runner.is_started() yield gen.sleep(0.01) def test_parse_bytes(): assert parse_bytes('100') == 100 assert parse_bytes('100 MB') == 100000000 assert parse_bytes('100M') == 100000000 assert parse_bytes('5kB') == 5000 assert parse_bytes('5.4 kB') == 5400 assert parse_bytes('1kiB') == 1024 assert parse_bytes('1Mi') == 2**20 assert parse_bytes('1e6') == 1000000 assert parse_bytes('1e6 kB') == 1000000000 assert parse_bytes('MB') == 1000000 distributed-1.20.2/distributed/tests/test_utils_comm.py000066400000000000000000000014011321233345200233600ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from distributed.core import rpc from distributed.utils_test import gen_cluster from distributed.utils_comm import (pack_data, gather_from_workers) def test_pack_data(): data = {'x': 1} assert pack_data(('x', 'y'), data) == (1, 'y') assert pack_data({'a': 'x', 'b': 'y'}, data) == {'a': 1, 'b': 'y'} assert pack_data({'a': ['x'], 'b': 'y'}, data) == {'a': [1], 'b': 'y'} @gen_cluster(client=True) def test_gather_from_workers_permissive(c, s, a, b): x = yield c.scatter({'x': 1}, workers=a.address) data, missing, bad_workers = yield gather_from_workers( {'x': [a.address], 'y': [b.address]}, rpc=rpc) assert data == {'x': 1} assert list(missing) == ['y'] distributed-1.20.2/distributed/tests/test_utils_test.py000077500000000000000000000067521321233345200234250ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from contextlib import contextmanager import socket import sys import threading from time import sleep import pytest from tornado import gen from distributed import Scheduler, Worker, Client, config from distributed.core import rpc from distributed.metrics import time from distributed.utils_test import (cluster, gen_cluster, inc, gen_test, wait_for_port, new_config, tls_only_security) from distributed.utils_test import loop # flake8: noqa from distributed.utils import get_ip def test_bare_cluster(loop): with cluster(nworkers=10) as (s, _): pass def test_cluster(loop): with cluster() as (s, [a, b]): with rpc(s['address']) as s: ident = loop.run_sync(s.identity) assert ident['type'] == 'Scheduler' assert len(ident['workers']) == 2 @gen_cluster(client=True) def test_gen_cluster(c, s, a, b): assert isinstance(c, Client) assert isinstance(s, Scheduler) for w in [a, b]: assert isinstance(w, Worker) assert s.ncores == {w.address: w.ncores for w in [a, b]} @pytest.mark.skip(reason="This hangs on travis") def test_gen_cluster_cleans_up_client(loop): import dask.context assert not dask.context._globals.get('get') @gen_cluster(client=True) def f(c, s, a, b): assert dask.context._globals.get('get') yield c.submit(inc, 1) f() assert not dask.context._globals.get('get') @gen_cluster(client=False) def test_gen_cluster_without_client(s, a, b): assert isinstance(s, Scheduler) for w in [a, b]: assert isinstance(w, Worker) assert s.ncores == {w.address: w.ncores for w in [a, b]} @gen_cluster(client=True, scheduler='tls://127.0.0.1', ncores=[('tls://127.0.0.1', 1), ('tls://127.0.0.1', 2)], security=tls_only_security()) def test_gen_cluster_tls(e, s, a, b): assert isinstance(e, Client) assert isinstance(s, Scheduler) assert s.address.startswith('tls://') for w in [a, b]: assert isinstance(w, Worker) assert w.address.startswith('tls://') assert s.ncores == {w.address: w.ncores for w in [a, b]} @gen_test() def test_gen_test(): yield gen.sleep(0.01) @contextmanager def _listen(delay=0): serv = socket.socket() serv.bind(("127.0.0.1", 0)) e = threading.Event() def do_listen(): e.set() sleep(delay) serv.listen(5) ret = serv.accept() if ret is not None: cli, _ = ret cli.close() serv.close() t = threading.Thread(target=do_listen) t.daemon = True t.start() try: e.wait() sleep(0.01) yield serv finally: t.join(5.0) def test_wait_for_port(): t1 = time() with pytest.raises(RuntimeError): wait_for_port((get_ip(), 9999), 0.5) t2 = time() assert t2 - t1 >= 0.5 with _listen(0) as s1: t1 = time() wait_for_port(s1.getsockname()) t2 = time() assert t2 - t1 <= 1.0 with _listen(1) as s1: t1 = time() wait_for_port(s1.getsockname()) t2 = time() assert t2 - t1 <= 2.0 def test_new_config(): c = config.copy() with new_config({'xyzzy': 5}): assert config == {'xyzzy': 5} assert config == c assert 'xyzzy' not in config if sys.version_info >= (3, 5): from distributed.tests.py3_test_utils_tst import * # flake8: noqa distributed-1.20.2/distributed/tests/test_variable.py000066400000000000000000000124711321233345200230030ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import random from time import sleep import sys import pytest from tornado import gen from distributed import Client, Variable, worker_client, Nanny, wait from distributed.metrics import time from distributed.utils_test import (gen_cluster, inc, cluster, slow, div) from distributed.utils_test import loop # flake8: noqa @gen_cluster(client=True) def test_variable(c, s, a, b): x = Variable('x') xx = Variable('x') assert x.client is c future = c.submit(inc, 1) yield x.set(future) future2 = yield xx.get() assert future.key == future2.key del future, future2 yield gen.sleep(0.1) assert s.task_state # future still present x.delete() start = time() while s.task_state: yield gen.sleep(0.01) assert time() < start + 5 @gen_cluster(client=True) def test_queue_with_data(c, s, a, b): x = Variable('x') xx = Variable('x') assert x.client is c yield x.set([1, 'hello']) data = yield xx.get() assert data == [1, 'hello'] def test_sync(loop): with cluster() as (s, [a, b]): with Client(s['address']) as c: future = c.submit(lambda x: x + 1, 10) x = Variable('x') xx = Variable('x') x.set(future) future2 = xx.get() assert future2.result() == 11 @gen_cluster() def test_hold_futures(s, a, b): c1 = yield Client(s.address, asynchronous=True) future = c1.submit(lambda x: x + 1, 10) x1 = Variable('x') yield x1.set(future) del x1 yield c1.close() yield gen.sleep(0.1) c2 = yield Client(s.address, asynchronous=True) x2 = Variable('x') future2 = yield x2.get() result = yield future2 assert result == 11 yield c2.close() @gen_cluster(client=True) def test_timeout(c, s, a, b): v = Variable('v') start = time() with pytest.raises(gen.TimeoutError): yield v.get(timeout=0.1) assert 0.05 < time() - start < 2.0 @gen_cluster(client=True) def test_cleanup(c, s, a, b): v = Variable('v') vv = Variable('v') x = c.submit(lambda x: x + 1, 10) y = c.submit(lambda x: x + 1, 20) x_key = x.key yield v.set(x) del x yield gen.sleep(0.1) t_future = xx = vv._get() yield gen.moment v._set(y) future = yield t_future assert future.key == x_key result = yield future assert result == 11 def test_pickleable(loop): with cluster() as (s, [a, b]): with Client(s['address']) as c: v = Variable('v') def f(x): v.set(x + 1) c.submit(f, 10).result() assert v.get() == 11 @gen_cluster(client=True) def test_timeout_get(c, s, a, b): v = Variable('v') tornado_future = v.get() vv = Variable('v') yield vv.set(1) result = yield tornado_future assert result == 1 @pytest.mark.skipif(sys.version_info[0] == 2, reason='Multi-client issues') @slow @gen_cluster(client=True, ncores=[('127.0.0.1', 2)] * 5, Worker=Nanny, timeout=None) def test_race(c, s, *workers): NITERS = 50 def f(i): with worker_client() as c: v = Variable('x', client=c) for _ in range(NITERS): future = v.get() x = future.result() y = c.submit(inc, x) v.set(y) sleep(0.01 * random.random()) result = v.get().result() sleep(0.1) # allow fire-and-forget messages to clear return result v = Variable('x', client=c) x = yield c.scatter(1) yield v.set(x) futures = c.map(f, range(15)) results = yield c.gather(futures) assert all(r > NITERS * 0.8 for r in results) start = time() while len(s.wants_what['variable-x']) != 1: yield gen.sleep(0.01) assert time() - start < 2 @gen_cluster(client=True) def test_Future_knows_status_immediately(c, s, a, b): x = yield c.scatter(123) v = Variable('x') yield v.set(x) c2 = yield Client(s.address, asynchronous=True) v2 = Variable('x', client=c2) future = yield v2.get() assert future.status == 'finished' x = c.submit(div, 1, 0) yield wait(x) yield v.set(x) future2 = yield v2.get() assert future2.status == 'error' with pytest.raises(Exception): yield future2 start = time() while True: # we learn about the true error eventually try: yield future2 except ZeroDivisionError: break except Exception: assert time() < start + 5 yield gen.sleep(0.05) yield c2.close() @gen_cluster(client=True) def test_erred_future(c, s, a, b): future = c.submit(div, 1, 0) var = Variable() yield var.set(future) yield gen.sleep(0.1) future2 = yield var.get() with pytest.raises(ZeroDivisionError): yield future2.result() exc = yield future2.exception() assert isinstance(exc, ZeroDivisionError) def test_future_erred_sync(loop): with cluster() as (s, [a, b]): with Client(s['address']) as c: future = c.submit(div, 1, 0) var = Variable() var.set(future) sleep(0.1) future2 = var.get() with pytest.raises(ZeroDivisionError): future2.result() distributed-1.20.2/distributed/tests/test_worker.py000066400000000000000000000722311321233345200225270ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from concurrent.futures import ThreadPoolExecutor import logging from numbers import Number from operator import add import os import psutil import shutil import sys from time import sleep import traceback from dask import delayed import pytest from toolz import pluck, sliding_window, first import tornado from tornado import gen from tornado.ioloop import TimeoutError from distributed import (Nanny, Client, get_client, wait, default_client, get_worker, Reschedule) from distributed.compatibility import WINDOWS from distributed.core import rpc from distributed.client import wait from distributed.scheduler import Scheduler from distributed.metrics import time from distributed.worker import Worker, error_message, logger, TOTAL_MEMORY from distributed.utils import tmpfile from distributed.utils_test import (inc, mul, gen_cluster, div, dec, slow, slowinc, gen_test, cluster, captured_logger) from distributed.utils_test import loop, nodebug # flake8: noqa def test_worker_ncores(): from distributed.worker import _ncores w = Worker('127.0.0.1', 8019) try: assert w.executor._max_workers == _ncores finally: shutil.rmtree(w.local_dir) @gen_cluster() def test_str(s, a, b): assert a.address in str(a) assert a.address in repr(a) assert str(a.ncores) in str(a) assert str(a.ncores) in repr(a) assert str(len(a.executing)) in repr(a) def test_identity(): w = Worker('127.0.0.1', 8019) ident = w.identity(None) assert 'Worker' in ident['type'] assert ident['scheduler'] == 'tcp://127.0.0.1:8019' assert isinstance(ident['ncores'], int) assert isinstance(ident['memory_limit'], Number) @gen_cluster(client=True) def test_worker_bad_args(c, s, a, b): class NoReprObj(object): """ This object cannot be properly represented as a string. """ def __str__(self): raise ValueError("I have no str representation.") def __repr__(self): raise ValueError("I have no repr representation.") x = c.submit(NoReprObj, workers=a.address) yield wait(x) assert not a.executing assert a.data def bad_func(*args, **kwargs): 1 / 0 class MockLoggingHandler(logging.Handler): """Mock logging handler to check for expected logs.""" def __init__(self, *args, **kwargs): self.reset() logging.Handler.__init__(self, *args, **kwargs) def emit(self, record): self.messages[record.levelname.lower()].append(record.getMessage()) def reset(self): self.messages = { 'debug': [], 'info': [], 'warning': [], 'error': [], 'critical': [], } hdlr = MockLoggingHandler() old_level = logger.level logger.setLevel(logging.DEBUG) logger.addHandler(hdlr) y = c.submit(bad_func, x, k=x, workers=b.address) yield wait(y) assert not b.executing assert y.status == 'error' # Make sure job died because of bad func and not because of bad # argument. with pytest.raises(ZeroDivisionError): yield y if sys.version_info[0] >= 3: tb = yield y._traceback() assert any('1 / 0' in line for line in pluck(3, traceback.extract_tb(tb)) if line) assert "Compute Failed" in hdlr.messages['warning'][0] logger.setLevel(old_level) # Now we check that both workers are still alive. xx = c.submit(add, 1, 2, workers=a.address) yy = c.submit(add, 3, 4, workers=b.address) results = yield c._gather([xx, yy]) assert tuple(results) == (3, 7) @slow @gen_cluster() def dont_test_delete_data_with_missing_worker(c, a, b): bad = '127.0.0.1:9001' # this worker doesn't exist c.who_has['z'].add(bad) c.who_has['z'].add(a.address) c.has_what[bad].add('z') c.has_what[a.address].add('z') a.data['z'] = 5 cc = rpc(ip=c.ip, port=c.port) yield cc.delete_data(keys=['z']) # TODO: this hangs for a while assert 'z' not in a.data assert not c.who_has['z'] assert not c.has_what[bad] assert not c.has_what[a.address] cc.close_rpc() @gen_cluster(client=True) def test_upload_file(c, s, a, b): assert not os.path.exists(os.path.join(a.local_dir, 'foobar.py')) assert not os.path.exists(os.path.join(b.local_dir, 'foobar.py')) assert a.local_dir != b.local_dir aa = rpc(a.address) bb = rpc(b.address) yield [aa.upload_file(filename='foobar.py', data=b'x = 123'), bb.upload_file(filename='foobar.py', data='x = 123')] assert os.path.exists(os.path.join(a.local_dir, 'foobar.py')) assert os.path.exists(os.path.join(b.local_dir, 'foobar.py')) def g(): import foobar return foobar.x future = c.submit(g, workers=a.address) result = yield future assert result == 123 yield a._close() yield b._close() aa.close_rpc() bb.close_rpc() assert not os.path.exists(os.path.join(a.local_dir, 'foobar.py')) @gen_cluster(client=True) def test_upload_egg(c, s, a, b): eggname = 'mytestegg-1.0.0-py3.4.egg' local_file = __file__.replace('test_worker.py', eggname) assert not os.path.exists(os.path.join(a.local_dir, eggname)) assert not os.path.exists(os.path.join(b.local_dir, eggname)) assert a.local_dir != b.local_dir aa = rpc(a.address) bb = rpc(b.address) with open(local_file, 'rb') as f: payload = f.read() yield [aa.upload_file(filename=eggname, data=payload), bb.upload_file(filename=eggname, data=payload)] assert os.path.exists(os.path.join(a.local_dir, eggname)) assert os.path.exists(os.path.join(b.local_dir, eggname)) def g(x): import testegg return testegg.inc(x) future = c.submit(g, 10, workers=a.address) result = yield future assert result == 10 + 1 yield a._close() yield b._close() aa.close_rpc() bb.close_rpc() assert not os.path.exists(os.path.join(a.local_dir, eggname)) @pytest.mark.xfail(reason='Still lose time to network I/O') @gen_cluster(client=True) def test_upload_large_file(c, s, a, b): pytest.importorskip('crick') yield gen.sleep(0.05) with rpc(a.address) as aa: yield aa.upload_file(filename='myfile.dat', data=b'0' * 100000000) yield gen.sleep(0.05) assert a.digests['tick-duration'].components[0].max() < 0.050 @gen_cluster() def test_broadcast(s, a, b): with rpc(s.address) as cc: results = yield cc.broadcast(msg={'op': 'ping'}) assert results == {a.address: b'pong', b.address: b'pong'} @gen_test() def test_worker_with_port_zero(): s = Scheduler() s.start(8007) w = Worker(s.address) yield w._start() assert isinstance(w.port, int) assert w.port > 1024 yield w._close() @slow def test_worker_waits_for_center_to_come_up(loop): @gen.coroutine def f(): w = Worker('127.0.0.1', 8007) yield w._start() try: loop.run_sync(f, timeout=4) except TimeoutError: pass @gen_cluster(client=True, ncores=[('127.0.0.1', 1)]) def test_worker_task_data(c, s, w): x = delayed(2) xx = c.persist(x) yield wait(xx) assert w.data[x.key] == 2 def test_error_message(): class MyException(Exception): def __init__(self, a, b): self.args = (a + b,) def __str__(self): return "MyException(%s)" % self.args msg = error_message(MyException('Hello', 'World!')) assert 'Hello' in str(msg['exception']) @gen_cluster() def test_gather(s, a, b): b.data['x'] = 1 b.data['y'] = 2 with rpc(a.address) as aa: resp = yield aa.gather(who_has={'x': [b.address], 'y': [b.address]}) assert resp['status'] == 'OK' assert a.data['x'] == b.data['x'] 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 @gen_cluster(client=True, ncores=[]) def test_spill_to_disk(c, s): np = pytest.importorskip('numpy') w = Worker(s.address, loop=s.loop, memory_limit=1200 / 0.6, memory_pause_fraction=None, memory_spill_fraction=None) yield w._start() x = c.submit(np.random.randint, 0, 255, size=500, dtype='u1', key='x') yield wait(x) y = c.submit(np.random.randint, 0, 255, size=500, dtype='u1', key='y') yield wait(y) assert set(w.data) == {x.key, y.key} assert set(w.data.fast) == {x.key, y.key} z = c.submit(np.random.randint, 0, 255, size=500, dtype='u1', key='z') yield wait(z) assert set(w.data) == {x.key, y.key, z.key} assert set(w.data.fast) == {y.key, z.key} assert set(w.data.slow) == {x.key} or set(w.data.slow) == {x.key, y.key} yield x assert set(w.data.fast) == {x.key, z.key} assert set(w.data.slow) == {y.key} or set(w.data.slow) == {x.key, y.key} yield w._close() @gen_cluster(client=True) def test_access_key(c, s, a, b): def f(i): from distributed.worker import thread_state return thread_state.key futures = [c.submit(f, i, key='x-%d' % i) for i in range(20)] results = yield c._gather(futures) assert list(results) == ['x-%d' % i for i in range(20)] @gen_cluster(client=True) def test_run_dask_worker(c, s, a, b): def f(dask_worker=None): return dask_worker.id response = yield c._run(f) assert response == {a.address: a.id, b.address: b.id} @gen_cluster(client=True) def test_run_coroutine_dask_worker(c, s, a, b): if sys.version_info < (3,) and tornado.version_info < (4, 5): pytest.skip("test needs Tornado 4.5+ on Python 2.7") @gen.coroutine def f(dask_worker=None): yield gen.sleep(0.001) raise gen.Return(dask_worker.id) response = yield c._run_coroutine(f) assert response == {a.address: a.id, b.address: b.id} @gen_cluster(client=True, ncores=[]) def test_Executor(c, s): with ThreadPoolExecutor(2) as e: w = Worker(s.ip, s.port, executor=e) assert w.executor is e yield w._start() future = c.submit(inc, 1) result = yield future assert result == 2 assert e._threads # had to do some work yield w._close() @pytest.mark.skip(reason="Leaks a large amount of memory") @gen_cluster(client=True, ncores=[('127.0.0.1', 1)], timeout=30) def test_spill_by_default(c, s, w): da = pytest.importorskip('dask.array') x = da.ones(int(TOTAL_MEMORY * 0.7), chunks=10000000, dtype='u1') y = c.persist(x) yield wait(y) assert len(w.data.slow) # something is on disk del x, y @gen_cluster(ncores=[('127.0.0.1', 1)], worker_kwargs={'reconnect': False}) def test_close_on_disconnect(s, w): yield s.close() start = time() while w.status != 'closed': yield gen.sleep(0.01) assert time() < start + 5 def test_memory_limit_auto(): a = Worker('127.0.0.1', 8099, ncores=1) b = Worker('127.0.0.1', 8099, ncores=2) c = Worker('127.0.0.1', 8099, ncores=100) d = Worker('127.0.0.1', 8099, ncores=200) assert isinstance(a.memory_limit, Number) assert isinstance(b.memory_limit, Number) assert a.memory_limit < b.memory_limit assert c.memory_limit == d.memory_limit @gen_cluster(client=True) def test_inter_worker_communication(c, s, a, b): [x, y] = yield c._scatter([1, 2], workers=a.address) future = c.submit(add, x, y, workers=b.address) result = yield future assert result == 3 @gen_cluster(client=True) def test_clean(c, s, a, b): x = c.submit(inc, 1, workers=a.address) y = c.submit(inc, x, workers=b.address) yield y collections = [a.tasks, a.task_state, a.startstops, a.data, a.nbytes, a.durations, a.priorities, a.types, a.threads] for c in collections: assert c x.release() y.release() while x.key in a.task_state: yield gen.sleep(0.01) for c in collections: assert not c @pytest.mark.skipif(sys.version_info[:2] == (3, 4), reason="mul bytes fails") @gen_cluster(client=True) def test_message_breakup(c, s, a, b): n = 100000 a.target_message_size = 10 * n b.target_message_size = 10 * n xs = [c.submit(mul, b'%d' % i, n, workers=a.address) for i in range(30)] y = c.submit(lambda *args: None, xs, workers=b.address) yield y assert 2 <= len(b.incoming_transfer_log) <= 20 assert 2 <= len(a.outgoing_transfer_log) <= 20 assert all(msg['who'] == b.address for msg in a.outgoing_transfer_log) assert all(msg['who'] == a.address for msg in a.incoming_transfer_log) @gen_cluster(client=True) def test_types(c, s, a, b): assert not a.types assert not b.types x = c.submit(inc, 1, workers=a.address) yield wait(x) assert a.types[x.key] == int y = c.submit(inc, x, workers=b.address) yield wait(y) assert b.types == {x.key: int, y.key: int} yield c._cancel(y) start = time() while y.key in b.data: yield gen.sleep(0.01) assert time() < start + 5 assert y.key not in b.types @gen_cluster() def test_system_monitor(s, a, b): assert b.monitor b.monitor.update() @gen_cluster(client=True, ncores=[('127.0.0.1', 2, {'resources': {'A': 1}}), ('127.0.0.1', 1)]) def test_restrictions(c, s, a, b): # Resource restrictions x = c.submit(inc, 1, resources={'A': 1}) yield x assert a.resource_restrictions == {x.key: {'A': 1}} yield c._cancel(x) while x.key in a.task_state: yield gen.sleep(0.01) assert a.resource_restrictions == {} @pytest.mark.xfail @gen_cluster(client=True) def test_clean_nbytes(c, s, a, b): L = [delayed(inc)(i) for i in range(10)] for i in range(5): L = [delayed(add)(x, y) for x, y in sliding_window(2, L)] total = delayed(sum)(L) future = c.compute(total) yield wait(future) yield gen.sleep(1) assert len(a.nbytes) + len(b.nbytes) == 1 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 20) def test_gather_many_small(c, s, a, *workers): a.total_connections = 2 futures = yield c._scatter(list(range(100))) assert all(w.data for w in workers) def f(*args): return 10 future = c.submit(f, *futures, workers=a.address) yield wait(future) types = list(pluck(0, a.log)) req = [i for i, t in enumerate(types) if t == 'request-dep'] recv = [i for i, t in enumerate(types) if t == 'receive-dep'] assert min(recv) > max(req) assert a.comm_nbytes == 0 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_multiple_transfers(c, s, w1, w2, w3): x = c.submit(inc, 1, workers=w1.address) y = c.submit(inc, 2, workers=w2.address) z = c.submit(add, x, y, workers=w3.address) yield wait(z) r = w3.startstops[z.key] transfers = [t for t in r if t[0] == 'transfer'] assert len(transfers) == 2 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 3) def test_share_communication(c, s, w1, w2, w3): x = c.submit(mul, b'1', int(w3.target_message_size + 1), workers=w1.address) y = c.submit(mul, b'2', int(w3.target_message_size + 1), workers=w2.address) yield wait([x, y]) yield c._replicate([x, y], workers=[w1.address, w2.address]) z = c.submit(add, x, y, workers=w3.address) yield wait(z) assert len(w3.incoming_transfer_log) == 2 assert w1.outgoing_transfer_log assert w2.outgoing_transfer_log @gen_cluster(client=True) def test_dont_overlap_communications_to_same_worker(c, s, a, b): x = c.submit(mul, b'1', int(b.target_message_size + 1), workers=a.address) y = c.submit(mul, b'2', int(b.target_message_size + 1), workers=a.address) yield wait([x, y]) z = c.submit(add, x, y, workers=b.address) yield wait(z) assert len(b.incoming_transfer_log) == 2 l1, l2 = b.incoming_transfer_log assert l1['stop'] < l2['start'] @pytest.mark.avoid_travis @gen_cluster(client=True) def test_log_exception_on_failed_task(c, s, a, b): with tmpfile() as fn: fh = logging.FileHandler(fn) try: from distributed.worker import logger logger.addHandler(fh) future = c.submit(div, 1, 0) yield wait(future) yield gen.sleep(0.1) fh.flush() with open(fn) as f: text = f.read() assert "ZeroDivisionError" in text assert "Exception" in text finally: logger.removeHandler(fh) @gen_cluster(client=True) def test_clean_up_dependencies(c, s, a, b): x = delayed(inc)(1) y = delayed(inc)(2) xx = delayed(inc)(x) yy = delayed(inc)(y) z = delayed(add)(xx, yy) zz = c.persist(z) yield wait(zz) start = time() while len(a.data) + len(b.data) > 1: yield gen.sleep(0.01) assert time() < start + 2 assert set(a.data) | set(b.data) == {zz.key} @gen_cluster(client=True) def test_hold_onto_dependents(c, s, a, b): x = c.submit(inc, 1, workers=a.address) y = c.submit(inc, x, workers=b.address) yield wait(y) assert x.key in b.data yield c._cancel(y) yield gen.sleep(0.1) assert x.key in b.data @slow @gen_cluster(client=False, ncores=[]) def test_worker_death_timeout(s): yield s.close() w = Worker(s.address, death_timeout=1) yield w._start() yield gen.sleep(3) assert w.status == 'closed' @gen_cluster(client=True) def test_stop_doing_unnecessary_work(c, s, a, b): futures = c.map(slowinc, range(1000), delay=0.01) yield gen.sleep(0.1) del futures start = time() while a.executing: yield gen.sleep(0.01) assert time() - start < 0.5 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)]) def test_priorities(c, s, w): a = delayed(slowinc)(1, dask_key_name='a', delay=0.05) b = delayed(slowinc)(2, dask_key_name='b', delay=0.05) a1 = delayed(slowinc)(a, dask_key_name='a1', delay=0.05) a2 = delayed(slowinc)(a1, dask_key_name='a2', delay=0.05) b1 = delayed(slowinc)(b, dask_key_name='b1', delay=0.05) z = delayed(add)(a2, b1) future = yield c.compute(z) log = [t for t in w.log if t[1] == 'executing' and t[2] == 'memory'] assert [t[0] for t in log[:5]] == ['a', 'b', 'a1', 'b1', 'a2'] @gen_cluster(client=True, ncores=[('127.0.0.1', 1)]) def test_priorities_2(c, s, w): values = [] for i in range(10): a = delayed(slowinc)(i, dask_key_name='a-%d' % i, delay=0.01) a1 = delayed(inc)(a, dask_key_name='a1-%d' % i) a2 = delayed(inc)(a1, dask_key_name='a2-%d' % i) b1 = delayed(dec)(a, dask_key_name='b1-%d' % i) # <<-- least favored values.append(a2) values.append(b1) futures = c.compute(values) yield wait(futures) log = [t[0] for t in w.log if t[1] == 'executing' and t[2] == 'memory' and not t[0].startswith('finalize')] assert any(key.startswith('b1') for key in log[:len(log) // 2]) @gen_cluster(client=True) def test_heartbeats(c, s, a, b): x = s.worker_info[a.address]['last-seen'] yield gen.sleep(a.periodic_callbacks['heartbeat'].callback_time / 1000 + 0.1) y = s.worker_info[a.address]['last-seen'] assert x != y assert a.periodic_callbacks['heartbeat'].callback_time < 1000 @pytest.mark.parametrize('worker', [Worker, Nanny]) def test_worker_dir(worker): with tmpfile() as fn: @gen_cluster(client=True, worker_kwargs={'local_dir': fn}) def test_worker_dir(c, s, a, b): directories = [info['local_directory'] for info in s.worker_info.values()] assert all(d.startswith(fn) for d in directories) assert len(set(directories)) == 2 # distinct test_worker_dir() @gen_cluster(client=True) def test_dataframe_attribute_error(c, s, a, b): class BadSize(object): def __init__(self, data): self.data = data def __sizeof__(self): raise TypeError('Hello') future = c.submit(BadSize, 123) result = yield future assert result.data == 123 @gen_cluster(client=True) def test_fail_write_to_disk(c, s, a, b): class Bad(object): def __getstate__(self): raise TypeError() def __sizeof__(self): return int(100e9) future = c.submit(Bad) yield wait(future) assert future.status == 'error' with pytest.raises(TypeError): yield future futures = c.map(inc, range(10)) results = yield c._gather(futures) assert results == list(map(inc, range(10))) @pytest.mark.skip(reason="Our logic here is faulty") @gen_cluster(ncores=[('127.0.0.1', 2)], client=True, worker_kwargs={'memory_limit': 10e9}) def test_fail_write_many_to_disk(c, s, a): a.validate = False yield gen.sleep(0.1) assert not a.paused class Bad(object): def __init__(self, x): pass def __getstate__(self): raise TypeError() def __sizeof__(self): return int(2e9) futures = c.map(Bad, range(11)) future = c.submit(lambda *args: 123, *futures) yield wait(future) with pytest.raises(Exception) as info: yield future # workers still operational result = yield c.submit(inc, 1, workers=a.address) assert result == 2 @gen_cluster() def test_pid(s, a, b): assert s.worker_info[a.address]['pid'] == os.getpid() @gen_cluster(client=True) def test_get_client(c, s, a, b): def f(x): cc = get_client() future = cc.submit(inc, x) return future.result() assert default_client() is c future = c.submit(f, 10, workers=a.address) result = yield future assert result == 11 assert a._client assert not b._client assert a._client is c assert default_client() is c a_client = a._client for i in range(10): yield wait(c.submit(f, i)) assert a._client is a_client def test_get_client_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: def f(x): cc = get_client() future = cc.submit(inc, x) return future.result() future = c.submit(f, 10) assert future.result() == 11 @gen_cluster(client=True) def test_get_client_coroutine(c, s, a, b): @gen.coroutine def f(): client = yield get_client() future = client.submit(inc, 10) result = yield future raise gen.Return(result) results = yield c.run_coroutine(f) assert results == {a.address: 11, b.address: 11} def test_get_client_coroutine_sync(loop): with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: @gen.coroutine def f(): client = yield get_client() future = client.submit(inc, 10) result = yield future raise gen.Return(result) results = c.run_coroutine(f) assert results == {a['address']: 11, b['address']: 11} @gen_cluster() def test_global_workers(s, a, b): from distributed.worker import _global_workers n = len(_global_workers) w = _global_workers[-1]() assert w is a or w is b yield a._close() yield b._close() assert len(_global_workers) == n - 2 @pytest.mark.skipif(WINDOWS, reason="file descriptors") @gen_cluster(ncores=[]) def test_worker_fds(s): psutil = pytest.importorskip('psutil') start = psutil.Process().num_fds() worker = Worker(s.address, loop=s.loop) yield worker._start() middle = psutil.Process().num_fds() assert middle > start yield worker._close() start = time() while psutil.Process().num_fds() > start: yield gen.sleep(0.01) assert time() < start + 0.5 @pytest.mark.skipif(not sys.platform.startswith('linux'), reason="Need 127.0.0.2 to mean localhost") @gen_cluster(ncores=[]) def test_service_hosts_match_worker(s): pytest.importorskip('bokeh') from distributed.bokeh.worker import BokehWorker services = {('bokeh', 0): BokehWorker} for host in ['tcp://0.0.0.0', 'tcp://127.0.0.2']: w = Worker(s.address, services=services) yield w._start(host) sock = first(w.services['bokeh'].server._http._sockets.values()) assert sock.getsockname()[0] == host.split('://')[1] yield w._close() @gen_test() def test_scheduler_file(): with tmpfile() as fn: s = Scheduler(scheduler_file=fn) s.start(8009) w = Worker(scheduler_file=fn) yield w._start() assert s.workers == {w.address} yield w._close() s.stop() @gen_cluster(client=True) def test_scheduler_delay(c, s, a, b): old = a.scheduler_delay assert abs(a.scheduler_delay) < 0.1 assert abs(b.scheduler_delay) < 0.1 yield gen.sleep(a.periodic_callbacks['heartbeat'].callback_time / 1000 + .3) assert a.scheduler_delay != old @gen_cluster(client=True) def test_statistical_profiling(c, s, a, b): futures = c.map(slowinc, range(10), delay=0.1) yield wait(futures) profile = a.profile_keys['slowinc'] assert profile['count'] @nodebug @gen_cluster(client=True) def test_statistical_profiling_2(c, s, a, b): da = pytest.importorskip('dask.array') for i in range(5): x = da.random.random(1000000, chunks=(10000,)) y = (x + x * 2) - x.sum().persist() yield wait(y) profile = a.get_profile() assert profile['count'] assert 'sum' in str(profile) or 'random' in str(profile) @gen_cluster(ncores=[('127.0.0.1', 1)], client=True, worker_kwargs={'memory_monitor_interval': 10}) def test_robust_to_bad_sizeof_estimates(c, s, a): np = pytest.importorskip('numpy') memory = psutil.Process().memory_info().rss a.memory_limit = memory / 0.7 + 400e6 class BadAccounting(object): def __init__(self, data): self.data = data def __sizeof__(self): return 10 def f(n): x = np.ones(int(n), dtype='u1') result = BadAccounting(x) return result futures = c.map(f, [100e6] * 8, pure=False) start = time() while not a.data.slow: yield gen.sleep(0.1) assert time() < start + 5 @pytest.mark.slow @gen_cluster(ncores=[('127.0.0.1', 2)], client=True, worker_kwargs={'memory_monitor_interval': 10}, timeout=20) def test_pause_executor(c, s, a): memory = psutil.Process().memory_info().rss a.memory_limit = memory / 0.8 + 200e6 np = pytest.importorskip('numpy') def f(): x = np.ones(int(300e6), dtype='u1') sleep(1) with captured_logger(logging.getLogger('distributed.worker')) as logger: future = c.submit(f) futures = c.map(slowinc, range(10), delay=0.1) yield gen.sleep(0.3) assert a.paused out = logger.getvalue() assert 'memory' in out.lower() assert 'pausing' in out.lower() assert sum(f.status == 'finished' for f in futures) < 4 yield wait(futures) @gen_cluster(client=True, worker_kwargs={'profile_cycle_interval': 100}) def test_statistical_profiling_cycle(c, s, a, b): futures = c.map(slowinc, range(20), delay=0.05) yield wait(futures) yield gen.sleep(0.01) assert len(a.profile_history) > 3 x = a.get_profile(start=time() + 10, stop=time() + 20) assert not x['count'] x = a.get_profile(start=0, stop=time()) assert x['count'] == sum(p['count'] for _, p in a.profile_history) + a.profile_recent['count'] y = a.get_profile(start=time() - 0.300, stop=time()) assert 0 < y['count'] < x['count'] @gen_cluster(client=True) def test_get_current_task(c, s, a, b): def some_name(): return get_worker().get_current_task() result = yield c.submit(some_name) assert result.startswith('some_name') @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_reschedule(c, s, a, b): s.extensions['stealing']._pc.stop() a_address = a.address def f(x): sleep(0.1) if get_worker().address == a_address: raise Reschedule() futures = c.map(f, range(4)) futures2 = c.map(slowinc, range(10), delay=0.1, workers=a.address) yield wait(futures) assert all(f.key in b.data for f in futures) def test_deque_handler(): from distributed.worker import logger w = Worker('127.0.0.1', 8019) deque_handler = w._deque_handler logger.info('foo456') assert deque_handler.deque msg = deque_handler.deque[-1] assert 'distributed.worker' in deque_handler.format(msg) assert any(msg.msg == 'foo456' for msg in deque_handler.deque) @gen_cluster(ncores=[], client=True) def test_avoid_memory_monitor_if_zero_limit(c, s): worker = Worker(s.address, loop=s.loop, memory_limit=0, memory_monitor_interval=10) yield worker._start() assert type(worker.data) is dict assert 'memory' not in worker.periodic_callbacks future = c.submit(inc, 1) assert (yield future) == 2 yield gen.sleep(worker.memory_monitor_interval / 1000) yield c.submit(inc, 2) # worker doesn't pause yield worker._close() @gen_cluster(ncores=[('127.0.0.1', 1)], worker_kwargs={'memory_limit': '2e3 MB'}) def test_parse_memory_limit(s, w): assert w.memory_limit == 2e9 distributed-1.20.2/distributed/tests/test_worker_client.py000066400000000000000000000162761321233345200240740ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import random from time import sleep import warnings import dask from dask import delayed import pytest from tornado import gen from distributed import (worker_client, Client, as_completed, get_worker, wait, get_client) from distributed.metrics import time from distributed.utils_test import cluster, double, gen_cluster, inc from distributed.utils_test import loop # flake8: noqa @gen_cluster(client=True) def test_submit_from_worker(c, s, a, b): def func(x): with worker_client() as c: x = c.submit(inc, x) y = c.submit(double, x) result = x.result() + y.result() return result x, y = c.map(func, [10, 20]) xx, yy = yield c._gather([x, y]) assert xx == 10 + 1 + (10 + 1) * 2 assert yy == 20 + 1 + (20 + 1) * 2 assert len(s.transition_log) > 10 assert len([id for id in s.wants_what if id.lower().startswith('client')]) == 1 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_scatter_from_worker(c, s, a, b): def func(): with worker_client() as c: futures = c.scatter([1, 2, 3, 4, 5]) assert isinstance(futures, (list, tuple)) assert len(futures) == 5 x = dict(get_worker().data) y = {f.key: i for f, i in zip(futures, [1, 2, 3, 4, 5])} assert x == y total = c.submit(sum, futures) return total.result() future = c.submit(func) result = yield future assert result == sum([1, 2, 3, 4, 5]) def func(): with worker_client() as c: correct = True for data in [[1, 2], (1, 2), {1, 2}]: futures = c.scatter(data) correct &= type(futures) == type(data) o = object() futures = c.scatter({'x': o}) correct &= get_worker().data['x'] is o return correct future = c.submit(func) result = yield future assert result is True start = time() while not all(v == 1 for v in s.ncores.values()): yield gen.sleep(0.1) assert time() < start + 5 @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_scatter_singleton(c, s, a, b): np = pytest.importorskip('numpy') def func(): with worker_client() as c: x = np.ones(5) future = c.scatter(x) assert future.type == np.ndarray yield c.submit(func) @gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 2) def test_gather_multi_machine(c, s, a, b): a_address = a.address b_address = b.address assert a_address != b_address def func(): with worker_client() as ee: x = ee.submit(inc, 1, workers=a_address) y = ee.submit(inc, 2, workers=b_address) xx, yy = ee.gather([x, y]) return xx, yy future = c.submit(func) result = yield future assert result == (2, 3) @gen_cluster(client=True) def test_same_loop(c, s, a, b): def f(): with worker_client() as lc: return lc.loop is get_worker().loop future = c.submit(f) result = yield future assert result def test_sync(loop): def mysum(): result = 0 sub_tasks = [delayed(double)(i) for i in range(100)] with worker_client() as lc: futures = lc.compute(sub_tasks) for f in as_completed(futures): result += f.result() return result with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: assert delayed(mysum)().compute(get=c.get) == 9900 @gen_cluster(client=True) def test_async(c, s, a, b): def mysum(): result = 0 sub_tasks = [delayed(double)(i) for i in range(100)] with worker_client() as lc: futures = lc.compute(sub_tasks) for f in as_completed(futures): result += f.result() return result future = c.compute(delayed(mysum)()) yield future start = time() while len(a.data) + len(b.data) > 1: yield gen.sleep(0.1) assert time() < start + 3 @gen_cluster(client=True, ncores=[('127.0.0.1', 3)]) def test_separate_thread_false(c, s, a): a.count = 0 def f(i): with worker_client(separate_thread=False) as client: get_worker().count += 1 assert get_worker().count <= 3 sleep(random.random() / 40) assert get_worker().count <= 3 get_worker().count -= 1 return i futures = c.map(f, range(20)) results = yield c._gather(futures) assert list(results) == list(range(20)) @gen_cluster(client=True) def test_client_executor(c, s, a, b): def mysum(): with worker_client() as c: with c.get_executor() as e: return sum(e.map(double, range(30))) future = c.submit(mysum) result = yield future assert result == 30 * 29 def test_dont_override_default_get(loop): import dask.bag as db def f(x): with worker_client() as c: return True b = db.from_sequence([1, 2]) b2 = b.map(f) with Client(loop=loop, processes=False, set_as_default=True) as c: assert dask.context._globals['get'] == c.get for i in range(2): b2.compute() assert dask.context._globals['get'] == c.get @gen_cluster(client=True) def test_local_client_warning(c, s, a, b): from distributed import local_client def func(x): with warnings.catch_warnings(record=True) as record: with local_client() as c: x = c.submit(inc, x) result = x.result() assert any("worker_client" in str(r.message) for r in record) return result future = c.submit(func, 10) result = yield future assert result == 11 @gen_cluster(client=True) def test_closing_worker_doesnt_close_client(c, s, a, b): def func(x): get_client() return yield wait(c.map(func, range(10))) yield a._close() assert c.status == 'running' def test_timeout(loop): def func(): with worker_client(timeout=0) as wc: print('hello') with cluster() as (s, [a, b]): with Client(s['address'], loop=loop) as c: future = c.submit(func) with pytest.raises(EnvironmentError): result = future.result() def test_secede_without_stealing_issue_1262(): """ Tests that seceding works with the Stealing extension disabled https://github.com/dask/distributed/issues/1262 """ # turn off all extensions extensions = [] # run the loop as an inner function so all workers are closed # and exceptions can be examined @gen_cluster(client=True, scheduler_kwargs={'extensions': extensions}) def secede_test(c, s, a, b): def func(x): with worker_client() as wc: y = wc.submit(lambda: 1 + x) return wc.gather(y) f = yield c.gather(c.submit(func, 1)) raise gen.Return((c, s, a, b, f)) c, s, a, b, f = secede_test() assert f == 2 # ensure no workers had errors assert all([f.exception() is None for f in s._worker_coroutines]) distributed-1.20.2/distributed/tests/test_worker_failure.py000066400000000000000000000243561321233345200242430ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from concurrent.futures import CancelledError from operator import add import os from time import sleep import pytest from toolz import partition_all from tornado import gen from dask import delayed from distributed import Client, Nanny, wait from distributed.comm import CommClosedError from distributed.client import wait from distributed.metrics import time from distributed.utils import sync, ignoring from distributed.utils_test import (gen_cluster, cluster, inc, slow, div, slowinc, slowadd, captured_logger) from distributed.utils_test import loop # flake8: noqa def test_submit_after_failed_worker_sync(loop): with cluster(active_rpc_timeout=10) as (s, [a, b]): with Client(s['address'], loop=loop) as c: L = c.map(inc, range(10)) wait(L) a['proc']().terminate() total = c.submit(sum, L) assert total.result() == sum(map(inc, range(10))) @gen_cluster(client=True, timeout=60, active_rpc_timeout=10) def test_submit_after_failed_worker_async(c, s, a, b): n = Nanny(s.ip, s.port, ncores=2, loop=s.loop) n.start(0) while len(s.workers) < 3: yield gen.sleep(0.1) L = c.map(inc, range(10)) yield wait(L) s.loop.add_callback(n.kill) total = c.submit(sum, L) result = yield total assert result == sum(map(inc, range(10))) yield n._close() @gen_cluster(client=True) def test_submit_after_failed_worker(c, s, a, b): L = c.map(inc, range(10)) yield wait(L) yield a._close() total = c.submit(sum, L) result = yield total assert result == sum(map(inc, range(10))) def test_gather_after_failed_worker(loop): with cluster(active_rpc_timeout=10) as (s, [a, b]): with Client(s['address'], loop=loop) as c: L = c.map(inc, range(10)) wait(L) a['proc']().terminate() result = c.gather(L) assert result == list(map(inc, range(10))) @slow def test_gather_then_submit_after_failed_workers(loop): with cluster(nworkers=4, active_rpc_timeout=10) as (s, [w, x, y, z]): with Client(s['address'], loop=loop) as c: L = c.map(inc, range(20)) wait(L) w['proc']().terminate() total = c.submit(sum, L) wait([total]) addr = c.who_has()[total.key][0] for d in [x, y, z]: if d['address'] == addr: d['proc']().terminate() break else: assert 0, "Could not find worker %r" % (addr,) result = c.gather([total]) assert result == [sum(map(inc, range(20)))] @gen_cluster(Worker=Nanny, timeout=60, client=True) def test_failed_worker_without_warning(c, s, a, b): L = c.map(inc, range(10)) yield wait(L) original_pid = a.pid with ignoring(CommClosedError): yield c._run(os._exit, 1, workers=[a.worker_address]) start = time() while a.pid == original_pid: yield gen.sleep(0.01) assert time() - start < 10 yield gen.sleep(0.5) start = time() while len(s.ncores) < 2: yield gen.sleep(0.01) assert time() - start < 10 yield wait(L) L2 = c.map(inc, range(10, 20)) yield wait(L2) assert all(len(keys) > 0 for keys in s.has_what.values()) ncores2 = s.ncores.copy() yield c._restart() L = c.map(inc, range(10)) yield wait(L) assert all(len(keys) > 0 for keys in s.has_what.values()) assert not (set(ncores2) & set(s.ncores)) # no overlap @gen_cluster(Worker=Nanny, client=True) def test_restart(c, s, a, b): assert s.ncores == {a.worker_address: 1, b.worker_address: 2} x = c.submit(inc, 1) y = c.submit(inc, x) z = c.submit(div, 1, 0) yield y assert set(s.who_has) == {x.key, y.key} f = yield c._restart() assert f is c assert len(s.processing) == 2 assert len(s.occupancy) == 2 assert not any(s.occupancy.values()) assert not s.who_has assert x.cancelled() assert y.cancelled() assert z.cancelled() assert z.key not in s.exceptions assert not s.who_wants assert not s.wants_what @gen_cluster(Worker=Nanny, client=True) def test_restart_cleared(c, s, a, b): x = 2 * delayed(1) + 1 f = c.compute(x) yield wait([f]) assert s.released yield c._restart() for coll in [s.tasks, s.dependencies, s.dependents, s.waiting, s.waiting_data, s.who_has, s.host_restrictions, s.worker_restrictions, s.loose_restrictions, s.released, s.priority, s.exceptions, s.who_wants, s.exceptions_blame]: assert not coll def test_restart_sync_no_center(loop): with cluster(nanny=True) as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(inc, 1) c.restart() assert x.cancelled() y = c.submit(inc, 2) assert y.result() == 3 assert len(c.ncores()) == 2 def test_restart_sync(loop): with cluster(nanny=True) as (s, [a, b]): with Client(s['address'], loop=loop) as c: x = c.submit(div, 1, 2) x.result() assert sync(loop, c.scheduler.who_has) c.restart() assert not sync(loop, c.scheduler.who_has) assert x.cancelled() assert len(c.ncores()) == 2 with pytest.raises(CancelledError): x.result() y = c.submit(div, 1, 3) assert y.result() == 1 / 3 @gen_cluster(Worker=Nanny, client=True, timeout=20) def test_restart_fast(c, s, a, b): L = c.map(sleep, range(10)) start = time() yield c._restart() assert time() - start < 10 assert len(s.ncores) == 2 assert all(x.status == 'cancelled' for x in L) x = c.submit(inc, 1) result = yield x assert result == 2 def test_restart_fast_sync(loop): with cluster(nanny=True) as (s, [a, b]): with Client(s['address'], loop=loop) as c: L = c.map(sleep, range(10)) start = time() c.restart() assert time() - start < 10 assert len(c.ncores()) == 2 assert all(x.status == 'cancelled' for x in L) x = c.submit(inc, 1) assert x.result() == 2 @gen_cluster(Worker=Nanny, client=True, timeout=20) def test_fast_kill(c, s, a, b): L = c.map(sleep, range(10)) start = time() yield c._restart() assert time() - start < 10 assert all(x.status == 'cancelled' for x in L) x = c.submit(inc, 1) result = yield x assert result == 2 @gen_cluster(Worker=Nanny) def test_multiple_clients_restart(s, a, b): e1 = yield Client((s.ip, s.port), asynchronous=True) e2 = yield Client((s.ip, s.port), asynchronous=True) x = e1.submit(inc, 1) y = e2.submit(inc, 2) xx = yield x yy = yield y assert xx == 2 assert yy == 3 yield e1._restart() assert x.cancelled() assert y.cancelled() yield e1._close(fast=True) yield e2._close(fast=True) @gen_cluster(Worker=Nanny) def test_restart_scheduler(s, a, b): import gc gc.collect() addrs = (a.worker_address, b.worker_address) yield s.restart() assert len(s.ncores) == 2 addrs2 = (a.worker_address, b.worker_address) assert addrs != addrs2 @gen_cluster(Worker=Nanny, client=True) def test_forgotten_futures_dont_clean_up_new_futures(c, s, a, b): x = c.submit(inc, 1) yield c._restart() y = c.submit(inc, 1) del x import gc gc.collect() yield gen.sleep(0.1) yield y @gen_cluster(client=True, timeout=60, active_rpc_timeout=10) def test_broken_worker_during_computation(c, s, a, b): s.allowed_failures = 100 n = Nanny(s.ip, s.port, ncores=2, loop=s.loop) n.start(0) start = time() while len(s.ncores) < 3: yield gen.sleep(0.01) assert time() < start + 5 L = c.map(inc, range(256)) for i in range(8): L = c.map(add, *zip(*partition_all(2, L))) from random import random yield gen.sleep(random() / 2) with ignoring(CommClosedError): # comm will be closed abrupty yield c._run(os._exit, 1, workers=[n.worker_address]) yield gen.sleep(random() / 2) with ignoring(CommClosedError, EnvironmentError): # perhaps new worker can't be contacted yet yield c._run(os._exit, 1, workers=[n.worker_address]) result = yield c.gather(L) assert isinstance(result[0], int) yield n._close() @gen_cluster(client=True, Worker=Nanny) def test_restart_during_computation(c, s, a, b): xs = [delayed(slowinc)(i, delay=0.01) for i in range(50)] ys = [delayed(slowinc)(i, delay=0.01) for i in xs] zs = [delayed(slowadd)(x, y, delay=0.01) for x, y in zip(xs, ys)] total = delayed(sum)(zs) result = c.compute(total) yield gen.sleep(0.5) assert s.rprocessing yield c._restart() assert not s.rprocessing assert len(s.ncores) == 2 assert not s.task_state @gen_cluster(client=True, timeout=None) def test_worker_who_has_clears_after_failed_connection(c, s, a, b): n = Nanny(s.ip, s.port, ncores=2, loop=s.loop) n.start(0) start = time() while len(s.ncores) < 3: yield gen.sleep(0.01) assert time() < start + 5 futures = c.map(slowinc, range(20), delay=0.01) yield wait(futures) result = yield c.submit(sum, futures, workers=a.address) for dep in set(a.dep_state) - set(a.task_state): a.release_dep(dep, report=True) n_worker_address = n.worker_address with ignoring(CommClosedError): yield c._run(os._exit, 1, workers=[n_worker_address]) while len(s.workers) > 2: yield gen.sleep(0.01) total = c.submit(sum, futures, workers=a.address) yield total assert not a.has_what.get(n_worker_address) assert not any(n_worker_address in s for s in a.who_has.values()) yield n._close() @slow @gen_cluster(client=True, timeout=60, Worker=Nanny, ncores=[('127.0.0.1', 1)]) def test_restart_timeout_on_long_running_task(c, s, a): with captured_logger('distributed.scheduler') as sio: future = c.submit(sleep, 3600) yield gen.sleep(0.1) yield c.restart(timeout=20) text = sio.getvalue() assert 'timeout' not in text.lower() distributed-1.20.2/distributed/tests/tls-ca-cert.pem000066400000000000000000000101441321233345200224210ustar00rootroot00000000000000Certificate: Data: Version: 3 (0x2) Serial Number: a4:da:fb:14:c7:46:fe:d2 Signature Algorithm: sha1WithRSAEncryption Issuer: C=XY, O=Dask CA, CN=our-ca-server Validity Not Before: Apr 5 14:36:25 2017 GMT Not After : Apr 3 14:36:25 2027 GMT Subject: C=XY, O=Dask CA, CN=our-ca-server Subject Public Key Info: Public Key Algorithm: rsaEncryption Public-Key: (2048 bit) Modulus: 00:d2:ab:69:b2:9e:76:de:6e:85:7d:18:6c:8b:d4: 68:fd:4a:96:e2:03:2f:8e:dd:ab:46:53:c4:46:af: 8a:2d:5a:c8:39:7f:7b:ce:39:69:ae:f0:b1:b3:5d: 84:b1:81:b0:94:bd:b6:de:22:38:bd:7a:cc:c9:d6: ed:e0:ba:9c:a6:59:67:bb:9f:ca:01:c3:11:2c:15: 05:24:9a:a3:31:1e:4f:f5:40:ad:97:54:24:f4:ca: e6:68:ad:ef:33:fd:d1:91:3f:bc:36:7b:45:99:53: fd:3c:50:2a:2c:84:38:b7:83:81:bf:72:b4:d0:eb: e3:43:be:8c:cb:91:92:bc:c6:76:98:7a:28:14:cc: a1:29:1d:ff:3f:22:5f:8d:21:e4:88:4c:b1:81:93: 4c:d8:ca:1c:55:cf:c9:a1:4d:0f:de:7d:b9:87:38: 0f:d4:a8:2f:ec:35:d4:76:0f:1b:7f:20:9b:c9:2e: 50:4d:89:d0:a5:61:a9:ae:8e:73:48:50:1a:33:1e: 28:55:c4:31:80:95:11:03:27:97:83:8a:49:c0:99: 6d:93:32:80:c1:5d:60:2b:52:0a:ca:ca:f7:54:32: 10:3c:62:8c:73:14:a6:25:21:42:1a:45:a2:42:6a: c7:fe:c9:04:c3:fe:ef:ee:4a:34:61:3c:c5:80:a9: 48:91 Exponent: 65537 (0x10001) X509v3 extensions: X509v3 Subject Key Identifier: 92:E3:52:19:13:1F:38:4E:31:EC:D4:66:CA:3C:E6:74:ED:9D:2E:3B X509v3 Authority Key Identifier: keyid:92:E3:52:19:13:1F:38:4E:31:EC:D4:66:CA:3C:E6:74:ED:9D:2E:3B X509v3 Basic Constraints: CA:TRUE Signature Algorithm: sha1WithRSAEncryption 41:dc:da:b5:ba:a6:1b:81:9d:d8:72:9e:c9:14:ab:b6:b1:e2: 36:7b:6f:d4:b7:7e:8a:c6:ad:f5:15:45:ac:1f:f9:38:4d:cd: de:05:00:4f:c2:74:ba:fb:4d:19:25:0f:c5:8e:d7:02:e2:e6: a9:69:a2:b4:36:4a:6c:10:49:00:09:0d:9b:3a:5e:4a:14:57: 7b:5a:63:bf:83:33:6c:af:88:64:e0:a5:e1:d5:56:97:d1:4d: 79:db:8e:b3:a9:b7:a9:aa:41:e8:26:2d:36:05:9c:fc:9d:60: c4:23:a2:0c:92:48:b1:cf:3a:c6:e9:67:9e:71:b2:d6:97:1f: be:fa:e0:ff:72:52:a7:78:50:87:26:0a:28:e6:cc:46:e8:88: 3f:a0:aa:86:0e:b8:20:b1:b8:fd:45:0c:94:4c:67:e2:05:99: 25:65:92:21:16:85:df:91:1a:0c:02:36:3b:7a:f1:3d:08:29: 5a:47:83:45:0b:2e:ff:ce:cb:97:64:8f:fa:d8:99:1d:eb:c1: 20:2e:ca:61:11:e9:08:f9:c7:ce:9f:14:a5:c6:55:e5:ed:07: 4a:eb:2f:0a:18:90:fb:5d:23:b4:19:f7:29:1f:66:6e:32:e4: 61:1c:7f:78:73:1d:56:24:ef:1a:ab:4c:85:72:a8:0a:63:ef: 67:f7:03:6f -----BEGIN CERTIFICATE----- MIIDQTCCAimgAwIBAgIJAKTa+xTHRv7SMA0GCSqGSIb3DQEBBQUAMDcxCzAJBgNV BAYTAlhZMRAwDgYDVQQKDAdEYXNrIENBMRYwFAYDVQQDDA1vdXItY2Etc2VydmVy MB4XDTE3MDQwNTE0MzYyNVoXDTI3MDQwMzE0MzYyNVowNzELMAkGA1UEBhMCWFkx EDAOBgNVBAoMB0Rhc2sgQ0ExFjAUBgNVBAMMDW91ci1jYS1zZXJ2ZXIwggEiMA0G CSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQDSq2mynnbeboV9GGyL1Gj9SpbiAy+O 3atGU8RGr4otWsg5f3vOOWmu8LGzXYSxgbCUvbbeIji9eszJ1u3gupymWWe7n8oB wxEsFQUkmqMxHk/1QK2XVCT0yuZore8z/dGRP7w2e0WZU/08UCoshDi3g4G/crTQ 6+NDvozLkZK8xnaYeigUzKEpHf8/Il+NIeSITLGBk0zYyhxVz8mhTQ/efbmHOA/U qC/sNdR2Dxt/IJvJLlBNidClYamujnNIUBozHihVxDGAlREDJ5eDiknAmW2TMoDB XWArUgrKyvdUMhA8YoxzFKYlIUIaRaJCasf+yQTD/u/uSjRhPMWAqUiRAgMBAAGj UDBOMB0GA1UdDgQWBBSS41IZEx84TjHs1GbKPOZ07Z0uOzAfBgNVHSMEGDAWgBSS 41IZEx84TjHs1GbKPOZ07Z0uOzAMBgNVHRMEBTADAQH/MA0GCSqGSIb3DQEBBQUA A4IBAQBB3Nq1uqYbgZ3Ycp7JFKu2seI2e2/Ut36Kxq31FUWsH/k4Tc3eBQBPwnS6 +00ZJQ/FjtcC4uapaaK0NkpsEEkACQ2bOl5KFFd7WmO/gzNsr4hk4KXh1VaX0U15 246zqbepqkHoJi02BZz8nWDEI6IMkkixzzrG6WeecbLWlx+++uD/clKneFCHJgoo 5sxG6Ig/oKqGDrggsbj9RQyUTGfiBZklZZIhFoXfkRoMAjY7evE9CClaR4NFCy7/ zsuXZI/62Jkd68EgLsphEekI+cfOnxSlxlXl7QdK6y8KGJD7XSO0GfcpH2ZuMuRh HH94cx1WJO8aq0yFcqgKY+9n9wNv -----END CERTIFICATE----- distributed-1.20.2/distributed/tests/tls-ca-key.pem000066400000000000000000000032501321233345200222540ustar00rootroot00000000000000-----BEGIN PRIVATE KEY----- MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQDSq2mynnbeboV9 GGyL1Gj9SpbiAy+O3atGU8RGr4otWsg5f3vOOWmu8LGzXYSxgbCUvbbeIji9eszJ 1u3gupymWWe7n8oBwxEsFQUkmqMxHk/1QK2XVCT0yuZore8z/dGRP7w2e0WZU/08 UCoshDi3g4G/crTQ6+NDvozLkZK8xnaYeigUzKEpHf8/Il+NIeSITLGBk0zYyhxV z8mhTQ/efbmHOA/UqC/sNdR2Dxt/IJvJLlBNidClYamujnNIUBozHihVxDGAlRED J5eDiknAmW2TMoDBXWArUgrKyvdUMhA8YoxzFKYlIUIaRaJCasf+yQTD/u/uSjRh PMWAqUiRAgMBAAECggEBAKWJred+HzePLvNqQrjpcoo9lywHhtJDGreLUPqeKvyS jiyoIPeeI44kce3YOVFl8AQgwtXvnBg/qzWa3pas7DTGof1LhOc3iYHTDpSBsmQB RqEXbpAvvU5hNQXiTsjS1F7G7zT8Tmc6GQ3QjCMozoxdGJIAJ1bXVGBCFc5MGDqe ZlVRImZKLopvc8N7brvqjLZ2WWg4aeXUCtV2AsBQzlU7SzjLUKfKi+QtwKIUX//K Iv5Qtx0yR+ZgxQ8ONlz89PCUsDKfMG+2opINyII91t5Mk9FUJBU+xgtarwIzAy5q uG/v16y3nz0slyKMe92RYzQJQGc67Prz65EU5P+m0VUCgYEA+ek2oZDYmkbXoR9L hv205lp6pWu2b+QWgDwRMzJV9bUCuapDfbsqpXWgSojzPbYPWL8jI4gIPtADixTk Ql/UI+r6T0ypkcAbqpwL5ivsvwTL84vy8K3I55B06y6fsgKQX9+PAophN8PFNDwQ r/nG361DEbOF6ZoEmm5S/lZxM9MCgYEA181vuHLcBuaUdrJ5ARQW+9FP0sFPIniu aqUaFLHFGp6DcXa0rx2gGhs50l3o07fOj1RVi9xAkPXSMGXb5Ufk6JvBwL/4ppuJ wjoHJOWVypUhNyB6JqBRvJ1ir9gNMsTyusOUMW81i+9U9ri4ntuAiKlFntSfj2UL imJYMS1fJ4sCgYAY5pXc71PFu31MvpOF9c2KjkVeKidZar6bNGD6Wu+Ztdl5qm9j Hb0YE2pPQuoJohRRYWoc5jb8GznfJ8u+mVHH6mfacucKcLvQ+g9tdb73IYE42R7x wD8eY5RiwpiC4iKEiv/y3trk9Yq2s61t+1NHgjoGj36uUfxXbtqyIObSpQKBgBwZ BoPYiKdpXEk8IZ8vU3LEJQWG39vYJLWD+6pYHxbBLtIWJ55pWBAuHVAzo/30/7MP wver3MhDCXP9vyzSOI7EwOxs0l43AC/GyGlYbzFqS2HXG7BtzVamk1Pb6JQEqqoA jwoEtc8bgU+Ca59baxfgGk18162e9Wq2NU9p3erTAoGBAMXtJsVWvrT0pY+nmMFo oV2cdDaJ1U/BBNtc2+cmxyHURIgoGgJygs/ioV6Lwcj+1zQHzkNeIVcEhX3hH4MX f1ibf0NNTMLuCRydEHBS5dTRdqePe1WJoTQFBKiuW2pFfZh13MAcYu6CbqG+VwFB sjuaJuu2e8mRKoaHWsjLopUs -----END PRIVATE KEY----- distributed-1.20.2/distributed/tests/tls-cert-chain.pem000066400000000000000000000160711321233345200231250ustar00rootroot00000000000000Certificate: Data: Version: 1 (0x0) Serial Number: a4:da:fb:14:c7:46:fe:d3 Signature Algorithm: sha1WithRSAEncryption Issuer: C=XY, O=Dask CA, CN=our-ca-server Validity Not Before: Apr 5 14:36:25 2017 GMT Not After : Feb 12 14:36:25 2027 GMT Subject: C=XY, L=Dask-distributed, O=Dask, CN=localhost Subject Public Key Info: Public Key Algorithm: rsaEncryption Public-Key: (1024 bit) Modulus: 00:d0:de:a3:22:83:df:a6:f8:73:f1:7c:f7:2b:33: 5a:d4:0d:c8:c4:e6:c3:b3:12:13:fd:b0:9c:b2:eb: 77:f7:d4:c9:f2:01:52:85:e5:3c:f0:5b:fd:8d:21: f9:92:6c:35:9c:c4:c9:75:32:85:bd:5b:46:10:e1: 8b:ad:1b:00:93:f1:d8:df:11:af:08:fc:c7:ba:61: fa:e2:ce:85:22:ed:d1:5c:f1:20:82:5f:8d:fe:9e: 9d:8b:fc:8f:3c:3e:e2:e3:ce:d6:c0:05:ce:14:58: 32:f7:c7:a0:53:3b:5b:63:7e:5f:59:da:14:fd:b4: 86:17:cd:3a:11:1b:bf:36:95 Exponent: 65537 (0x10001) Signature Algorithm: sha1WithRSAEncryption 15:c0:ca:1b:ec:f1:3b:ba:08:65:fb:6a:f6:60:6f:29:15:48: f6:4c:ba:f0:56:56:5e:2a:6c:3c:68:94:d6:6b:eb:23:d7:ec: 09:2f:72:2b:21:91:fa:4a:e8:b6:f3:79:28:db:00:8e:66:8d: 2d:63:69:fa:ac:e5:3f:e0:71:d5:ae:ea:d6:f2:26:7e:a0:e6: 04:db:f0:98:03:d1:0a:67:fd:c5:53:8a:6d:46:7a:7a:8e:48: 57:42:76:0d:11:f2:d0:34:ee:e4:a5:ca:25:22:09:d5:74:f9: 30:6a:03:30:5b:2a:ec:d0:2e:09:ef:8e:f1:f8:af:1b:3f:bf: 29:08:89:cf:a2:58:f1:4f:01:60:82:59:9e:6c:33:e1:9b:4a: cd:ec:c1:e7:73:c9:eb:68:34:e0:d3:08:f8:7c:41:08:d7:1f: b1:00:01:d4:e0:0b:5b:52:39:be:1c:d0:75:d2:05:55:7d:14: 82:4c:8c:16:9f:30:ff:03:c1:ac:09:53:19:76:71:e5:f1:d9: 07:4b:db:d3:99:71:a8:31:7c:1f:99:27:3f:1c:6f:c9:02:e9: 0c:a8:d3:62:fd:a8:b9:57:28:c4:8f:f8:33:9c:c1:f1:a9:47: fb:bb:c6:60:31:b3:06:1e:ab:89:eb:e4:23:32:09:17:a3:5e: 04:8e:88:c4 -----BEGIN CERTIFICATE----- MIICejCCAWICCQCk2vsUx0b+0zANBgkqhkiG9w0BAQUFADA3MQswCQYDVQQGEwJY WTEQMA4GA1UECgwHRGFzayBDQTEWMBQGA1UEAwwNb3VyLWNhLXNlcnZlcjAeFw0x NzA0MDUxNDM2MjVaFw0yNzAyMTIxNDM2MjVaMEsxCzAJBgNVBAYTAlhZMRkwFwYD VQQHDBBEYXNrLWRpc3RyaWJ1dGVkMQ0wCwYDVQQKDAREYXNrMRIwEAYDVQQDDAls b2NhbGhvc3QwgZ8wDQYJKoZIhvcNAQEBBQADgY0AMIGJAoGBANDeoyKD36b4c/F8 9yszWtQNyMTmw7MSE/2wnLLrd/fUyfIBUoXlPPBb/Y0h+ZJsNZzEyXUyhb1bRhDh i60bAJPx2N8Rrwj8x7ph+uLOhSLt0VzxIIJfjf6enYv8jzw+4uPO1sAFzhRYMvfH oFM7W2N+X1naFP20hhfNOhEbvzaVAgMBAAEwDQYJKoZIhvcNAQEFBQADggEBABXA yhvs8Tu6CGX7avZgbykVSPZMuvBWVl4qbDxolNZr6yPX7AkvcishkfpK6LbzeSjb AI5mjS1jafqs5T/gcdWu6tbyJn6g5gTb8JgD0Qpn/cVTim1GenqOSFdCdg0R8tA0 7uSlyiUiCdV0+TBqAzBbKuzQLgnvjvH4rxs/vykIic+iWPFPAWCCWZ5sM+GbSs3s wedzyetoNODTCPh8QQjXH7EAAdTgC1tSOb4c0HXSBVV9FIJMjBafMP8DwawJUxl2 ceXx2QdL29OZcagxfB+ZJz8cb8kC6Qyo02L9qLlXKMSP+DOcwfGpR/u7xmAxswYe q4nr5CMyCRejXgSOiMQ= -----END CERTIFICATE----- Certificate: Data: Version: 3 (0x2) Serial Number: a4:da:fb:14:c7:46:fe:d2 Signature Algorithm: sha1WithRSAEncryption Issuer: C=XY, O=Dask CA, CN=our-ca-server Validity Not Before: Apr 5 14:36:25 2017 GMT Not After : Apr 3 14:36:25 2027 GMT Subject: C=XY, O=Dask CA, CN=our-ca-server Subject Public Key Info: Public Key Algorithm: rsaEncryption Public-Key: (2048 bit) Modulus: 00:d2:ab:69:b2:9e:76:de:6e:85:7d:18:6c:8b:d4: 68:fd:4a:96:e2:03:2f:8e:dd:ab:46:53:c4:46:af: 8a:2d:5a:c8:39:7f:7b:ce:39:69:ae:f0:b1:b3:5d: 84:b1:81:b0:94:bd:b6:de:22:38:bd:7a:cc:c9:d6: ed:e0:ba:9c:a6:59:67:bb:9f:ca:01:c3:11:2c:15: 05:24:9a:a3:31:1e:4f:f5:40:ad:97:54:24:f4:ca: e6:68:ad:ef:33:fd:d1:91:3f:bc:36:7b:45:99:53: fd:3c:50:2a:2c:84:38:b7:83:81:bf:72:b4:d0:eb: e3:43:be:8c:cb:91:92:bc:c6:76:98:7a:28:14:cc: a1:29:1d:ff:3f:22:5f:8d:21:e4:88:4c:b1:81:93: 4c:d8:ca:1c:55:cf:c9:a1:4d:0f:de:7d:b9:87:38: 0f:d4:a8:2f:ec:35:d4:76:0f:1b:7f:20:9b:c9:2e: 50:4d:89:d0:a5:61:a9:ae:8e:73:48:50:1a:33:1e: 28:55:c4:31:80:95:11:03:27:97:83:8a:49:c0:99: 6d:93:32:80:c1:5d:60:2b:52:0a:ca:ca:f7:54:32: 10:3c:62:8c:73:14:a6:25:21:42:1a:45:a2:42:6a: c7:fe:c9:04:c3:fe:ef:ee:4a:34:61:3c:c5:80:a9: 48:91 Exponent: 65537 (0x10001) X509v3 extensions: X509v3 Subject Key Identifier: 92:E3:52:19:13:1F:38:4E:31:EC:D4:66:CA:3C:E6:74:ED:9D:2E:3B X509v3 Authority Key Identifier: keyid:92:E3:52:19:13:1F:38:4E:31:EC:D4:66:CA:3C:E6:74:ED:9D:2E:3B X509v3 Basic Constraints: CA:TRUE Signature Algorithm: sha1WithRSAEncryption 41:dc:da:b5:ba:a6:1b:81:9d:d8:72:9e:c9:14:ab:b6:b1:e2: 36:7b:6f:d4:b7:7e:8a:c6:ad:f5:15:45:ac:1f:f9:38:4d:cd: de:05:00:4f:c2:74:ba:fb:4d:19:25:0f:c5:8e:d7:02:e2:e6: a9:69:a2:b4:36:4a:6c:10:49:00:09:0d:9b:3a:5e:4a:14:57: 7b:5a:63:bf:83:33:6c:af:88:64:e0:a5:e1:d5:56:97:d1:4d: 79:db:8e:b3:a9:b7:a9:aa:41:e8:26:2d:36:05:9c:fc:9d:60: c4:23:a2:0c:92:48:b1:cf:3a:c6:e9:67:9e:71:b2:d6:97:1f: be:fa:e0:ff:72:52:a7:78:50:87:26:0a:28:e6:cc:46:e8:88: 3f:a0:aa:86:0e:b8:20:b1:b8:fd:45:0c:94:4c:67:e2:05:99: 25:65:92:21:16:85:df:91:1a:0c:02:36:3b:7a:f1:3d:08:29: 5a:47:83:45:0b:2e:ff:ce:cb:97:64:8f:fa:d8:99:1d:eb:c1: 20:2e:ca:61:11:e9:08:f9:c7:ce:9f:14:a5:c6:55:e5:ed:07: 4a:eb:2f:0a:18:90:fb:5d:23:b4:19:f7:29:1f:66:6e:32:e4: 61:1c:7f:78:73:1d:56:24:ef:1a:ab:4c:85:72:a8:0a:63:ef: 67:f7:03:6f -----BEGIN CERTIFICATE----- MIIDQTCCAimgAwIBAgIJAKTa+xTHRv7SMA0GCSqGSIb3DQEBBQUAMDcxCzAJBgNV BAYTAlhZMRAwDgYDVQQKDAdEYXNrIENBMRYwFAYDVQQDDA1vdXItY2Etc2VydmVy MB4XDTE3MDQwNTE0MzYyNVoXDTI3MDQwMzE0MzYyNVowNzELMAkGA1UEBhMCWFkx EDAOBgNVBAoMB0Rhc2sgQ0ExFjAUBgNVBAMMDW91ci1jYS1zZXJ2ZXIwggEiMA0G CSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQDSq2mynnbeboV9GGyL1Gj9SpbiAy+O 3atGU8RGr4otWsg5f3vOOWmu8LGzXYSxgbCUvbbeIji9eszJ1u3gupymWWe7n8oB wxEsFQUkmqMxHk/1QK2XVCT0yuZore8z/dGRP7w2e0WZU/08UCoshDi3g4G/crTQ 6+NDvozLkZK8xnaYeigUzKEpHf8/Il+NIeSITLGBk0zYyhxVz8mhTQ/efbmHOA/U qC/sNdR2Dxt/IJvJLlBNidClYamujnNIUBozHihVxDGAlREDJ5eDiknAmW2TMoDB XWArUgrKyvdUMhA8YoxzFKYlIUIaRaJCasf+yQTD/u/uSjRhPMWAqUiRAgMBAAGj UDBOMB0GA1UdDgQWBBSS41IZEx84TjHs1GbKPOZ07Z0uOzAfBgNVHSMEGDAWgBSS 41IZEx84TjHs1GbKPOZ07Z0uOzAMBgNVHRMEBTADAQH/MA0GCSqGSIb3DQEBBQUA A4IBAQBB3Nq1uqYbgZ3Ycp7JFKu2seI2e2/Ut36Kxq31FUWsH/k4Tc3eBQBPwnS6 +00ZJQ/FjtcC4uapaaK0NkpsEEkACQ2bOl5KFFd7WmO/gzNsr4hk4KXh1VaX0U15 246zqbepqkHoJi02BZz8nWDEI6IMkkixzzrG6WeecbLWlx+++uD/clKneFCHJgoo 5sxG6Ig/oKqGDrggsbj9RQyUTGfiBZklZZIhFoXfkRoMAjY7evE9CClaR4NFCy7/ zsuXZI/62Jkd68EgLsphEekI+cfOnxSlxlXl7QdK6y8KGJD7XSO0GfcpH2ZuMuRh HH94cx1WJO8aq0yFcqgKY+9n9wNv -----END CERTIFICATE----- distributed-1.20.2/distributed/tests/tls-cert.pem000066400000000000000000000057251321233345200220510ustar00rootroot00000000000000Certificate: Data: Version: 1 (0x0) Serial Number: a4:da:fb:14:c7:46:fe:d3 Signature Algorithm: sha1WithRSAEncryption Issuer: C=XY, O=Dask CA, CN=our-ca-server Validity Not Before: Apr 5 14:36:25 2017 GMT Not After : Feb 12 14:36:25 2027 GMT Subject: C=XY, L=Dask-distributed, O=Dask, CN=localhost Subject Public Key Info: Public Key Algorithm: rsaEncryption Public-Key: (1024 bit) Modulus: 00:d0:de:a3:22:83:df:a6:f8:73:f1:7c:f7:2b:33: 5a:d4:0d:c8:c4:e6:c3:b3:12:13:fd:b0:9c:b2:eb: 77:f7:d4:c9:f2:01:52:85:e5:3c:f0:5b:fd:8d:21: f9:92:6c:35:9c:c4:c9:75:32:85:bd:5b:46:10:e1: 8b:ad:1b:00:93:f1:d8:df:11:af:08:fc:c7:ba:61: fa:e2:ce:85:22:ed:d1:5c:f1:20:82:5f:8d:fe:9e: 9d:8b:fc:8f:3c:3e:e2:e3:ce:d6:c0:05:ce:14:58: 32:f7:c7:a0:53:3b:5b:63:7e:5f:59:da:14:fd:b4: 86:17:cd:3a:11:1b:bf:36:95 Exponent: 65537 (0x10001) Signature Algorithm: sha1WithRSAEncryption 15:c0:ca:1b:ec:f1:3b:ba:08:65:fb:6a:f6:60:6f:29:15:48: f6:4c:ba:f0:56:56:5e:2a:6c:3c:68:94:d6:6b:eb:23:d7:ec: 09:2f:72:2b:21:91:fa:4a:e8:b6:f3:79:28:db:00:8e:66:8d: 2d:63:69:fa:ac:e5:3f:e0:71:d5:ae:ea:d6:f2:26:7e:a0:e6: 04:db:f0:98:03:d1:0a:67:fd:c5:53:8a:6d:46:7a:7a:8e:48: 57:42:76:0d:11:f2:d0:34:ee:e4:a5:ca:25:22:09:d5:74:f9: 30:6a:03:30:5b:2a:ec:d0:2e:09:ef:8e:f1:f8:af:1b:3f:bf: 29:08:89:cf:a2:58:f1:4f:01:60:82:59:9e:6c:33:e1:9b:4a: cd:ec:c1:e7:73:c9:eb:68:34:e0:d3:08:f8:7c:41:08:d7:1f: b1:00:01:d4:e0:0b:5b:52:39:be:1c:d0:75:d2:05:55:7d:14: 82:4c:8c:16:9f:30:ff:03:c1:ac:09:53:19:76:71:e5:f1:d9: 07:4b:db:d3:99:71:a8:31:7c:1f:99:27:3f:1c:6f:c9:02:e9: 0c:a8:d3:62:fd:a8:b9:57:28:c4:8f:f8:33:9c:c1:f1:a9:47: fb:bb:c6:60:31:b3:06:1e:ab:89:eb:e4:23:32:09:17:a3:5e: 04:8e:88:c4 -----BEGIN CERTIFICATE----- MIICejCCAWICCQCk2vsUx0b+0zANBgkqhkiG9w0BAQUFADA3MQswCQYDVQQGEwJY WTEQMA4GA1UECgwHRGFzayBDQTEWMBQGA1UEAwwNb3VyLWNhLXNlcnZlcjAeFw0x NzA0MDUxNDM2MjVaFw0yNzAyMTIxNDM2MjVaMEsxCzAJBgNVBAYTAlhZMRkwFwYD VQQHDBBEYXNrLWRpc3RyaWJ1dGVkMQ0wCwYDVQQKDAREYXNrMRIwEAYDVQQDDAls b2NhbGhvc3QwgZ8wDQYJKoZIhvcNAQEBBQADgY0AMIGJAoGBANDeoyKD36b4c/F8 9yszWtQNyMTmw7MSE/2wnLLrd/fUyfIBUoXlPPBb/Y0h+ZJsNZzEyXUyhb1bRhDh i60bAJPx2N8Rrwj8x7ph+uLOhSLt0VzxIIJfjf6enYv8jzw+4uPO1sAFzhRYMvfH oFM7W2N+X1naFP20hhfNOhEbvzaVAgMBAAEwDQYJKoZIhvcNAQEFBQADggEBABXA yhvs8Tu6CGX7avZgbykVSPZMuvBWVl4qbDxolNZr6yPX7AkvcishkfpK6LbzeSjb AI5mjS1jafqs5T/gcdWu6tbyJn6g5gTb8JgD0Qpn/cVTim1GenqOSFdCdg0R8tA0 7uSlyiUiCdV0+TBqAzBbKuzQLgnvjvH4rxs/vykIic+iWPFPAWCCWZ5sM+GbSs3s wedzyetoNODTCPh8QQjXH7EAAdTgC1tSOb4c0HXSBVV9FIJMjBafMP8DwawJUxl2 ceXx2QdL29OZcagxfB+ZJz8cb8kC6Qyo02L9qLlXKMSP+DOcwfGpR/u7xmAxswYe q4nr5CMyCRejXgSOiMQ= -----END CERTIFICATE----- distributed-1.20.2/distributed/tests/tls-key-cert.pem000066400000000000000000000075511321233345200226360ustar00rootroot00000000000000-----BEGIN PRIVATE KEY----- MIICdwIBADANBgkqhkiG9w0BAQEFAASCAmEwggJdAgEAAoGBANDeoyKD36b4c/F8 9yszWtQNyMTmw7MSE/2wnLLrd/fUyfIBUoXlPPBb/Y0h+ZJsNZzEyXUyhb1bRhDh i60bAJPx2N8Rrwj8x7ph+uLOhSLt0VzxIIJfjf6enYv8jzw+4uPO1sAFzhRYMvfH oFM7W2N+X1naFP20hhfNOhEbvzaVAgMBAAECgYAxFJ/w9FwRuEBGdr79/8tQGQ0c d710zsKz0lSAFqTSL8Zkf7sMi5mOQY+LIniq15Khi/xq9bxHirMkNFvGCZpQvM6m DuKAHf/0zFQOfM3QtXxEb64uupS8UTsiTkiq84Zwb3g6uWYXB+3zvY+FDbi5SkXp P1c6c58wu1WSSqBh4QJBAOijVjTxu1FtvrVN/e4J+lfuXKYs7cO9tbC7L2cAhB0A 6lAD2w1NRbIoY9IXwFc0E+vbyNqp18wYWuUNi+dYTEcCQQDl2ENZ3vc4LvtWWkMm fmQ95ks7WTWCnxkWdFKWLEnRAOUM18dJnJgoJxvySaCifrMhOObz+UgJx/a23Sqj IsBDAkBRXshJbn/xhGxfKC94cj8BQJtheHeVrqZ9cYusAG81YduTLzYQWgXVwfDg THR3S/zn05FWgm+7sI0ibvGPcsADAkEAjr/dPLDk/Aj54qzWrc6r9WHVcbu7JzAm xbJjJI/HHL6gg3R0enx+Z9m6WqVt1d445LZRgk9ddW2zBemgtIc9vwJBAJkuovHW wq/hmDkBuPqbe2KX5id4sQZqCr7V+fAs12lxjEIwk/fe2Oop0E1dsLGXQaCzqKmp GXaAzcrQXi7JtZc= -----END PRIVATE KEY----- Certificate: Data: Version: 1 (0x0) Serial Number: a4:da:fb:14:c7:46:fe:d3 Signature Algorithm: sha1WithRSAEncryption Issuer: C=XY, O=Dask CA, CN=our-ca-server Validity Not Before: Apr 5 14:36:25 2017 GMT Not After : Feb 12 14:36:25 2027 GMT Subject: C=XY, L=Dask-distributed, O=Dask, CN=localhost Subject Public Key Info: Public Key Algorithm: rsaEncryption Public-Key: (1024 bit) Modulus: 00:d0:de:a3:22:83:df:a6:f8:73:f1:7c:f7:2b:33: 5a:d4:0d:c8:c4:e6:c3:b3:12:13:fd:b0:9c:b2:eb: 77:f7:d4:c9:f2:01:52:85:e5:3c:f0:5b:fd:8d:21: f9:92:6c:35:9c:c4:c9:75:32:85:bd:5b:46:10:e1: 8b:ad:1b:00:93:f1:d8:df:11:af:08:fc:c7:ba:61: fa:e2:ce:85:22:ed:d1:5c:f1:20:82:5f:8d:fe:9e: 9d:8b:fc:8f:3c:3e:e2:e3:ce:d6:c0:05:ce:14:58: 32:f7:c7:a0:53:3b:5b:63:7e:5f:59:da:14:fd:b4: 86:17:cd:3a:11:1b:bf:36:95 Exponent: 65537 (0x10001) Signature Algorithm: sha1WithRSAEncryption 15:c0:ca:1b:ec:f1:3b:ba:08:65:fb:6a:f6:60:6f:29:15:48: f6:4c:ba:f0:56:56:5e:2a:6c:3c:68:94:d6:6b:eb:23:d7:ec: 09:2f:72:2b:21:91:fa:4a:e8:b6:f3:79:28:db:00:8e:66:8d: 2d:63:69:fa:ac:e5:3f:e0:71:d5:ae:ea:d6:f2:26:7e:a0:e6: 04:db:f0:98:03:d1:0a:67:fd:c5:53:8a:6d:46:7a:7a:8e:48: 57:42:76:0d:11:f2:d0:34:ee:e4:a5:ca:25:22:09:d5:74:f9: 30:6a:03:30:5b:2a:ec:d0:2e:09:ef:8e:f1:f8:af:1b:3f:bf: 29:08:89:cf:a2:58:f1:4f:01:60:82:59:9e:6c:33:e1:9b:4a: cd:ec:c1:e7:73:c9:eb:68:34:e0:d3:08:f8:7c:41:08:d7:1f: b1:00:01:d4:e0:0b:5b:52:39:be:1c:d0:75:d2:05:55:7d:14: 82:4c:8c:16:9f:30:ff:03:c1:ac:09:53:19:76:71:e5:f1:d9: 07:4b:db:d3:99:71:a8:31:7c:1f:99:27:3f:1c:6f:c9:02:e9: 0c:a8:d3:62:fd:a8:b9:57:28:c4:8f:f8:33:9c:c1:f1:a9:47: fb:bb:c6:60:31:b3:06:1e:ab:89:eb:e4:23:32:09:17:a3:5e: 04:8e:88:c4 -----BEGIN CERTIFICATE----- MIICejCCAWICCQCk2vsUx0b+0zANBgkqhkiG9w0BAQUFADA3MQswCQYDVQQGEwJY WTEQMA4GA1UECgwHRGFzayBDQTEWMBQGA1UEAwwNb3VyLWNhLXNlcnZlcjAeFw0x NzA0MDUxNDM2MjVaFw0yNzAyMTIxNDM2MjVaMEsxCzAJBgNVBAYTAlhZMRkwFwYD VQQHDBBEYXNrLWRpc3RyaWJ1dGVkMQ0wCwYDVQQKDAREYXNrMRIwEAYDVQQDDAls b2NhbGhvc3QwgZ8wDQYJKoZIhvcNAQEBBQADgY0AMIGJAoGBANDeoyKD36b4c/F8 9yszWtQNyMTmw7MSE/2wnLLrd/fUyfIBUoXlPPBb/Y0h+ZJsNZzEyXUyhb1bRhDh i60bAJPx2N8Rrwj8x7ph+uLOhSLt0VzxIIJfjf6enYv8jzw+4uPO1sAFzhRYMvfH oFM7W2N+X1naFP20hhfNOhEbvzaVAgMBAAEwDQYJKoZIhvcNAQEFBQADggEBABXA yhvs8Tu6CGX7avZgbykVSPZMuvBWVl4qbDxolNZr6yPX7AkvcishkfpK6LbzeSjb AI5mjS1jafqs5T/gcdWu6tbyJn6g5gTb8JgD0Qpn/cVTim1GenqOSFdCdg0R8tA0 7uSlyiUiCdV0+TBqAzBbKuzQLgnvjvH4rxs/vykIic+iWPFPAWCCWZ5sM+GbSs3s wedzyetoNODTCPh8QQjXH7EAAdTgC1tSOb4c0HXSBVV9FIJMjBafMP8DwawJUxl2 ceXx2QdL29OZcagxfB+ZJz8cb8kC6Qyo02L9qLlXKMSP+DOcwfGpR/u7xmAxswYe q4nr5CMyCRejXgSOiMQ= -----END CERTIFICATE----- distributed-1.20.2/distributed/tests/tls-key.pem000066400000000000000000000016241321233345200216760ustar00rootroot00000000000000-----BEGIN PRIVATE KEY----- MIICdwIBADANBgkqhkiG9w0BAQEFAASCAmEwggJdAgEAAoGBANDeoyKD36b4c/F8 9yszWtQNyMTmw7MSE/2wnLLrd/fUyfIBUoXlPPBb/Y0h+ZJsNZzEyXUyhb1bRhDh i60bAJPx2N8Rrwj8x7ph+uLOhSLt0VzxIIJfjf6enYv8jzw+4uPO1sAFzhRYMvfH oFM7W2N+X1naFP20hhfNOhEbvzaVAgMBAAECgYAxFJ/w9FwRuEBGdr79/8tQGQ0c d710zsKz0lSAFqTSL8Zkf7sMi5mOQY+LIniq15Khi/xq9bxHirMkNFvGCZpQvM6m DuKAHf/0zFQOfM3QtXxEb64uupS8UTsiTkiq84Zwb3g6uWYXB+3zvY+FDbi5SkXp P1c6c58wu1WSSqBh4QJBAOijVjTxu1FtvrVN/e4J+lfuXKYs7cO9tbC7L2cAhB0A 6lAD2w1NRbIoY9IXwFc0E+vbyNqp18wYWuUNi+dYTEcCQQDl2ENZ3vc4LvtWWkMm fmQ95ks7WTWCnxkWdFKWLEnRAOUM18dJnJgoJxvySaCifrMhOObz+UgJx/a23Sqj IsBDAkBRXshJbn/xhGxfKC94cj8BQJtheHeVrqZ9cYusAG81YduTLzYQWgXVwfDg THR3S/zn05FWgm+7sI0ibvGPcsADAkEAjr/dPLDk/Aj54qzWrc6r9WHVcbu7JzAm xbJjJI/HHL6gg3R0enx+Z9m6WqVt1d445LZRgk9ddW2zBemgtIc9vwJBAJkuovHW wq/hmDkBuPqbe2KX5id4sQZqCr7V+fAs12lxjEIwk/fe2Oop0E1dsLGXQaCzqKmp GXaAzcrQXi7JtZc= -----END PRIVATE KEY----- distributed-1.20.2/distributed/tests/tls-self-signed-cert.pem000066400000000000000000000014561321233345200242440ustar00rootroot00000000000000-----BEGIN CERTIFICATE----- MIICLDCCAZWgAwIBAgIJAMA9Gen+zrxJMA0GCSqGSIb3DQEBCwUAMEsxCzAJBgNV BAYTAlhZMRkwFwYDVQQHDBBEYXNrLWRpc3RyaWJ1dGVkMQ0wCwYDVQQKDAREYXNr MRIwEAYDVQQDDAlsb2NhbGhvc3QwHhcNMTcwNDA1MTQzNjI1WhcNMjcwNDAzMTQz NjI1WjBLMQswCQYDVQQGEwJYWTEZMBcGA1UEBwwQRGFzay1kaXN0cmlidXRlZDEN MAsGA1UECgwERGFzazESMBAGA1UEAwwJbG9jYWxob3N0MIGfMA0GCSqGSIb3DQEB AQUAA4GNADCBiQKBgQDE7N1D4bg1tmhg5xAa+H85TPZCG+JOGqzZ8dpxjl5OdKw4 0x3c4KTySAXUY7TtOCG7Woc0/Vam7PJTdU3ZAcU3xzBdYBCze2umY2zsV2lNFQmC XuJ1Dnv90DEgljcFNOD4vzfb6kyGUcZeacId6+7D3bdq7r4zEeqCwSBdiIbo3wID AQABoxgwFjAUBgNVHREEDTALgglsb2NhbGhvc3QwDQYJKoZIhvcNAQELBQADgYEA Mzek0ecKBNJe0d7bD+cilK4S+Gu9NxnXiFw3AUGBhJGmQWrrRaRWuwVZWwrW4qoj NnxyB/U0VRiTH2jOMzV/ejfhRp9YPXJj2J0EMEvbnxsWqcSnCg5Y0Px/VdXDHx5P V5XJHrcfYAiR/KeDclLoKytf5lTMAZ/2fXUoWa2oPm0= -----END CERTIFICATE----- distributed-1.20.2/distributed/tests/tls-self-signed-key.pem000066400000000000000000000016241321233345200240740ustar00rootroot00000000000000-----BEGIN PRIVATE KEY----- MIICdgIBADANBgkqhkiG9w0BAQEFAASCAmAwggJcAgEAAoGBAMTs3UPhuDW2aGDn EBr4fzlM9kIb4k4arNnx2nGOXk50rDjTHdzgpPJIBdRjtO04IbtahzT9Vqbs8lN1 TdkBxTfHMF1gELN7a6ZjbOxXaU0VCYJe4nUOe/3QMSCWNwU04Pi/N9vqTIZRxl5p wh3r7sPdt2ruvjMR6oLBIF2IhujfAgMBAAECgYAHh68mS1lJ5IG1QuCgx2PGAyVf fcV3ixmU8+SQdwswINH9EkWQDp8ADLT8XBus/np7uYh2BEHt/B2AB3pElAbAfBQU nTLcdwTvS5lVgvBtTzw9mPCaISlZ/YUmnRvFemXu5VP+EwLvCHim7XkLz3t9ldgY rjk8FlypH6Xx1cdlYQJBAOmgYqOu6GtwE6toCTGbP+xeNyb0kMqcjoV+B6sBAeCy cNyyqR6M2Mgm38+GvcEZ2fT2JbbFb6KRFdq7e1+MhSkCQQDXyLTvg1WYuilx4Ggg Fys6FNY7liIx8lFfoH3SPvFVDyOk7q7p4jOgIRaYmaK34HNF8+K1bTArPN/lDyBM ZvbHAkEArKAm4n12HRTWY9velzn41MWVCQLDDgL6SGckcVWwNoCDZn/Y+Mrnz5Tx 4RzVjx5z704F9fQTFdMQi+vmMGbbaQJAXTBpq5KNkeK76VkfvtQir/e1uZAR/n4G 1ISMMhNKyBA8tYr68TjmUL5ZctfO5RhC0DAOdYnNJq8ohlTs/vHsmwJAJL5R/NBo CeffVkMArufyG9yXhPHPhWQCH4Q1h40A+5H9VCeiW7uGs2hWeVRDAqWEAbo7tj2+ 0BZ5fzdy3VCleA== -----END PRIVATE KEY----- distributed-1.20.2/distributed/threadpoolexecutor.py000066400000000000000000000151151321233345200227330ustar00rootroot00000000000000""" Modified ThreadPoolExecutor to support threads leaving the thread pool This includes a global `secede` method that a submitted function can call to have its thread leave the ThreadPoolExecutor's thread pool. This allows the thread pool to allocate another thread if necessary and so is useful when a function realises that it is going to be a long-running job that doesn't want to take up space. When the function finishes its thread will terminate gracefully. This code copies and modifies two functions from the `concurrent.futures.thread` module, notably `_worker` and ThreadPoolExecutor._adjust_thread_count` to allow for checking against a global `threading.local` state. These functions are subject to the following license, which is included as a comment at the end of this file: https://docs.python.org/3/license.html ... and are under copyright by the Python Software Foundation Copyright 2001-2016 Python Software Foundation; All Rights Reserved """ from __future__ import print_function, division, absolute_import from . import _concurrent_futures_thread as thread import logging import threading from .metrics import time logger = logging.getLogger(__name__) thread_state = threading.local() def _worker(executor, work_queue): thread_state.proceed = True thread_state.executor = executor try: while thread_state.proceed: with executor._rejoin_lock: if executor._rejoin_list: rejoin_thread, rejoin_event = executor._rejoin_list.pop() executor._threads.add(rejoin_thread) executor._threads.remove(threading.current_thread()) rejoin_event.set() break task = work_queue.get() if task is not None: # sentinel task.run() del task elif thread._shutdown or executor is None or executor._shutdown: work_queue.put(None) return del executor except BaseException: logger.critical('Exception in worker', exc_info=True) finally: del thread_state.proceed del thread_state.executor class ThreadPoolExecutor(thread.ThreadPoolExecutor): def __init__(self, *args, **kwargs): super(ThreadPoolExecutor, self).__init__(*args, **kwargs) self._rejoin_list = [] self._rejoin_lock = threading.Lock() def _adjust_thread_count(self): if len(self._threads) < self._max_workers: t = threading.Thread(target=_worker, name="ThreadPool worker %d" % len(self._threads,), args=(self, self._work_queue)) t.daemon = True self._threads.add(t) t.start() def shutdown(self, wait=True, timeout=None): with threads_lock: with self._shutdown_lock: self._shutdown = True self._work_queue.put(None) if timeout is not None: deadline = time() + timeout for t in self._threads: if timeout is not None: timeout2 = max(deadline - time(), 0) else: timeout2 = None t.join(timeout=timeout2) def secede(adjust=True): """ Have this thread secede from the ThreadPoolExecutor See Also -------- rejoin: rejoin the thread pool """ thread_state.proceed = False with threads_lock: thread_state.executor._threads.remove(threading.current_thread()) if adjust: thread_state.executor._adjust_thread_count() def rejoin(): """ Have this thread rejoin the ThreadPoolExecutor This will block until a new slot opens up in the executor. The next thread to finish a task will leave the pool to allow this one to join. See Also -------- secede: leave the thread pool """ thread = threading.current_thread() event = threading.Event() e = thread_state.executor with e._rejoin_lock: e._rejoin_list.append((thread, event)) e.submit(lambda: None) event.wait() thread_state.proceed = True threads_lock = threading.Lock() """ PSF LICENSE AGREEMENT FOR PYTHON 3.5.2 ====================================== 1. This LICENSE AGREEMENT is between the Python Software Foundation ("PSF"), and the Individual or Organization ("Licensee") accessing and otherwise using Python 3.5.2 software in source or binary form and its associated documentation. 2. Subject to the terms and conditions of this License Agreement, PSF hereby grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, analyze, test, perform and/or display publicly, prepare derivative works, distribute, and otherwise use Python 3.5.2 alone or in any derivative version, provided, however, that PSF's License Agreement and PSF's notice of copyright, i.e., "Copyright c 2001-2016 Python Software Foundation; All Rights Reserved" are retained in Python 3.5.2 alone or in any derivative version prepared by Licensee. 3. In the event Licensee prepares a derivative work that is based on or incorporates Python 3.5.2 or any part thereof, and wants to make the derivative work available to others as provided herein, then Licensee hereby agrees to include in any such work a brief summary of the changes made to Python 3.5.2. 4. PSF is making Python 3.5.2 available to Licensee on an "AS IS" basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON 3.5.2 WILL NOT INFRINGE ANY THIRD PARTY RIGHTS. 5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON 3.5.2 FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON 3.5.2, OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. 6. This License Agreement will automatically terminate upon a material breach of its terms and conditions. 7. Nothing in this License Agreement shall be deemed to create any relationship of agency, partnership, or joint venture between PSF and Licensee. This License Agreement does not grant permission to use PSF trademarks or trade name in a trademark sense to endorse or promote products or services of Licensee, or any third party. 8. By copying, installing or otherwise using Python 3.5.2, Licensee agrees to be bound by the terms and conditions of this License Agreement. """ distributed-1.20.2/distributed/utils.py000066400000000000000000001103031321233345200201460ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import atexit from collections import Iterable, deque from contextlib import contextmanager from datetime import timedelta import functools import gc import json import logging import math import multiprocessing import operator import os import re import shutil import socket from time import sleep from importlib import import_module import sys import tempfile import threading import warnings import weakref import six import tblib.pickling_support from .compatibility import cache_from_source, getargspec, invalidate_caches, reload try: import resource except ImportError: resource = None from dask import istask from toolz import memoize, valmap import tornado from tornado import gen from tornado.ioloop import IOLoop, PollIOLoop from .compatibility import Queue, PY3, PY2, get_thread_identity, unicode from .config import config from .metrics import time try: from dask.context import thread_state except ImportError: thread_state = threading.local() logger = _logger = logging.getLogger(__name__) no_default = '__no_default__' def _initialize_mp_context(): if PY3 and not sys.platform.startswith('win') and 'PyPy' not in sys.version: method = config.get('multiprocessing-method', 'forkserver') ctx = multiprocessing.get_context(method) # Makes the test suite much faster preload = ['distributed'] if 'pkg_resources' in sys.modules: preload.append('pkg_resources') ctx.set_forkserver_preload(preload) else: ctx = multiprocessing return ctx mp_context = _initialize_mp_context() def funcname(func): """Get the name of a function.""" while hasattr(func, 'func'): func = func.func try: return func.__name__ except AttributeError: return str(func) def has_arg(func, argname): """ Whether the function takes an argument with the given name. """ while True: try: if argname in getargspec(func).args: return True except TypeError: break try: # For Tornado coroutines and other decorated functions func = func.__wrapped__ except AttributeError: break return False def get_fileno_limit(): """ Get the maximum number of open files per process. """ if resource is not None: return resource.getrlimit(resource.RLIMIT_NOFILE)[0] else: # Default ceiling for Windows when using the CRT, though it # is settable using _setmaxstdio(). return 512 @memoize def _get_ip(host, port, family, default): # By using a UDP socket, we don't actually try to connect but # simply select the local address through which *host* is reachable. sock = socket.socket(family, socket.SOCK_DGRAM) try: sock.connect((host, port)) ip = sock.getsockname()[0] return ip except EnvironmentError as e: # XXX Should first try getaddrinfo() on socket.gethostname() and getfqdn() warnings.warn("Couldn't detect a suitable IP address for " "reaching %r, defaulting to %r: %s" % (host, default, e), RuntimeWarning) return default finally: sock.close() def get_ip(host='8.8.8.8', port=80): """ Get the local IP address through which the *host* is reachable. *host* defaults to a well-known Internet host (one of Google's public DNS servers). """ return _get_ip(host, port, family=socket.AF_INET, default='127.0.0.1') def get_ipv6(host='2001:4860:4860::8888', port=80): """ The same as get_ip(), but for IPv6. """ return _get_ip(host, port, family=socket.AF_INET6, default='::1') def get_ip_interface(ifname): """ Get the local IPv4 address of a network interface. KeyError is raised if the interface doesn't exist. ValueError is raised if the interface does no have an IPv4 address associated with it. """ import psutil for info in psutil.net_if_addrs()[ifname]: if info.family == socket.AF_INET: return info.address raise ValueError("interface %r doesn't have an IPv4 address" % (ifname,)) @contextmanager def ignoring(*exceptions): try: yield except exceptions as e: pass @gen.coroutine def ignore_exceptions(coroutines, *exceptions): """ Process list of coroutines, ignoring certain exceptions >>> coroutines = [cor(...) for ...] # doctest: +SKIP >>> x = yield ignore_exceptions(coroutines, TypeError) # doctest: +SKIP """ wait_iterator = gen.WaitIterator(*coroutines) results = [] while not wait_iterator.done(): with ignoring(*exceptions): result = yield wait_iterator.next() results.append(result) raise gen.Return(results) @gen.coroutine def All(*args): """ Wait on many tasks at the same time Err once any of the tasks err. See https://github.com/tornadoweb/tornado/issues/1546 """ if len(args) == 1 and isinstance(args[0], Iterable): args = args[0] tasks = gen.WaitIterator(*args) results = [None for _ in args] while not tasks.done(): result = yield tasks.next() results[tasks.current_index] = result raise gen.Return(results) def sync(loop, func, *args, **kwargs): """ Run coroutine in loop running in separate thread. """ # Tornado's PollIOLoop doesn't raise when using closed, do it ourselves if isinstance(loop, PollIOLoop) and getattr(loop, '_closing', False): raise RuntimeError("IOLoop is closed") timeout = kwargs.pop('callback_timeout', None) def make_coro(): coro = gen.maybe_future(func(*args, **kwargs)) if timeout is None: return coro else: return gen.with_timeout(timedelta(seconds=timeout), coro) e = threading.Event() main_tid = get_thread_identity() result = [None] error = [False] @gen.coroutine def f(): try: if main_tid == get_thread_identity(): raise RuntimeError("sync() called from thread of running loop") yield gen.moment thread_state.asynchronous = True result[0] = yield make_coro() except Exception as exc: logger.exception(exc) error[0] = sys.exc_info() finally: thread_state.asynchronous = False e.set() loop.add_callback(f) if timeout is not None: if not e.wait(timeout): raise gen.TimeoutError("timed out after %s s." % (timeout,)) else: while not e.is_set(): e.wait(1000000) if error[0]: six.reraise(*error[0]) else: return result[0] class LoopRunner(object): """ A helper to start and stop an IO loop in a controlled way. Several loop runners can associate safely to the same IO loop. Parameters ---------- loop: IOLoop (optional) If given, this loop will be re-used, otherwise an appropriate one will be looked up or created. asynchronous: boolean (optional, default False) If false (the default), the loop is meant to run in a separate thread and will be started if necessary. If true, the loop is meant to run in the thread this object is instantiated from, and will not be started automatically. """ # All loops currently associated to loop runners _all_loops = weakref.WeakKeyDictionary() _lock = threading.Lock() def __init__(self, loop=None, asynchronous=False): if loop is None: if asynchronous: self._loop = IOLoop.current() else: # We're expecting the loop to run in another thread, # avoid re-using this thread's assigned loop self._loop = IOLoop() self._should_close_loop = True else: self._loop = loop self._should_close_loop = False self._asynchronous = asynchronous self._loop_thread = None self._started = False with self._lock: self._all_loops.setdefault(self._loop, (0, None)) def start(self): """ Start the IO loop if required. The loop is run in a dedicated thread. If the loop is already running, this method does nothing. """ with self._lock: self._start_unlocked() def _start_unlocked(self): assert not self._started count, real_runner = self._all_loops[self._loop] if (self._asynchronous or real_runner is not None or count > 0): self._all_loops[self._loop] = count + 1, real_runner self._started = True return assert self._loop_thread is None assert count == 0 loop_evt = threading.Event() done_evt = threading.Event() in_thread = [None] start_exc = [None] def loop_cb(): in_thread[0] = threading.current_thread() loop_evt.set() def run_loop(loop=self._loop): loop.add_callback(loop_cb) try: loop.start() except Exception as e: start_exc[0] = e finally: done_evt.set() thread = threading.Thread(target=run_loop, name="IO loop") thread.daemon = True thread.start() loop_evt.wait(timeout=1000) self._started = True actual_thread = in_thread[0] if actual_thread is not thread: # Loop already running in other thread (user-launched) done_evt.wait(5) if not isinstance(start_exc[0], RuntimeError): raise start_exc[0] self._all_loops[self._loop] = count + 1, None else: assert start_exc[0] is None, start_exc self._loop_thread = thread self._all_loops[self._loop] = count + 1, self def stop(self, timeout=10): """ Stop and close the loop if it was created by us. Otherwise, just mark this object "stopped". """ with self._lock: self._stop_unlocked(timeout) def _stop_unlocked(self, timeout): if not self._started: return self._started = False count, real_runner = self._all_loops[self._loop] if count > 1: self._all_loops[self._loop] = count - 1, real_runner else: assert count == 1 del self._all_loops[self._loop] if real_runner is not None: real_runner._real_stop(timeout) def _real_stop(self, timeout): assert self._loop_thread is not None if self._loop_thread is not None: try: self._loop.add_callback(self._loop.stop) self._loop_thread.join(timeout=timeout) self._loop.close() finally: self._loop_thread = None def is_started(self): """ Return True between start() and stop() calls, False otherwise. """ return self._started def run_sync(self, func, *args, **kwargs): """ Convenience helper: start the loop if needed, run sync(func, *args, **kwargs), then stop the loop again. """ if self._started: return sync(self.loop, func, *args, **kwargs) else: self.start() try: return sync(self.loop, func, *args, **kwargs) finally: self.stop() @property def loop(self): return self._loop @contextmanager def set_thread_state(**kwargs): old = {} for k in kwargs: try: old[k] = getattr(thread_state, k) except AttributeError: pass for k, v in kwargs.items(): setattr(thread_state, k, v) try: yield finally: for k in kwargs: try: v = old[k] except KeyError: delattr(thread_state, k) else: setattr(thread_state, k, v) @contextmanager def tmp_text(filename, text): fn = os.path.join(tempfile.gettempdir(), filename) with open(fn, 'w') as f: f.write(text) try: yield fn finally: if os.path.exists(fn): os.remove(fn) def clear_queue(q): while not q.empty(): q.get_nowait() def is_kernel(): """ Determine if we're running within an IPython kernel >>> is_kernel() False """ # http://stackoverflow.com/questions/34091701/determine-if-were-in-an-ipython-notebook-session if 'IPython' not in sys.modules: # IPython hasn't been imported return False from IPython import get_ipython # check for `kernel` attribute on the IPython instance return getattr(get_ipython(), 'kernel', None) is not None hex_pattern = re.compile('[a-f]+') def key_split(s): """ >>> key_split('x') 'x' >>> key_split('x-1') 'x' >>> key_split('x-1-2-3') 'x' >>> key_split(('x-2', 1)) 'x' >>> key_split("('x-2', 1)") 'x' >>> key_split("('x', 1)") 'x' >>> key_split('hello-world-1') 'hello-world' >>> key_split(b'hello-world-1') 'hello-world' >>> key_split('ae05086432ca935f6eba409a8ecd4896') 'data' >>> key_split('>> key_split(None) 'Other' >>> key_split('x-abcdefab') # ignores hex 'x' """ if type(s) is bytes: s = s.decode() if type(s) is tuple: s = s[0] try: words = s.split('-') if not words[0][0].isalpha(): result = words[0].split(",")[0].strip("'(\"") else: result = words[0] for word in words[1:]: if word.isalpha() and not (len(word) == 8 and hex_pattern.match(word) is not None): result += '-' + word else: break if len(result) == 32 and re.match(r'[a-f0-9]{32}', result): return 'data' else: if result[0] == '<': result = result.strip('<>').split()[0].split('.')[-1] return result except Exception: return 'Other' try: from functools import lru_cache except ImportError: pass else: key_split = lru_cache(100000)(key_split) if PY3: def key_split_group(x): """A more fine-grained version of key_split >>> key_split_group('x') 'x' >>> key_split_group('x-1') 'x-1' >>> key_split_group('x-1-2-3') 'x-1-2-3' >>> key_split_group(('x-2', 1)) 'x-2' >>> key_split_group("('x-2', 1)") 'x-2' >>> key_split_group('hello-world-1') 'hello-world-1' >>> key_split_group(b'hello-world-1') 'hello-world-1' >>> key_split_group('ae05086432ca935f6eba409a8ecd4896') 'data' >>> key_split_group('>> key_split_group(None) 'Other' >>> key_split_group('x-abcdefab') # ignores hex 'x-abcdefab' """ typ = type(x) if typ is tuple: return x[0] elif typ is str: if x[0] == '(': return x.split(',', 1)[0].strip('()"\'') elif len(x) == 32 and re.match(r'[a-f0-9]{32}', x): return 'data' elif x[0] == '<': return x.strip('<>').split()[0].split('.')[-1] else: return x elif typ is bytes: return key_split_group(x.decode()) else: return 'Other' else: def key_split_group(x): """A more fine-grained version of key_split >>> key_split_group('x') 'x' >>> key_split_group('x-1') 'x-1' >>> key_split_group('x-1-2-3') 'x-1-2-3' >>> key_split_group(('x-2', 1)) 'x-2' >>> key_split_group("('x-2', 1)") 'x-2' >>> key_split_group('hello-world-1') 'hello-world-1' >>> key_split_group(b'hello-world-1') 'hello-world-1' >>> key_split_group('ae05086432ca935f6eba409a8ecd4896') 'data' >>> key_split_group('>> key_split_group(None) 'Other' >>> key_split_group('x-abcdefab') # ignores hex 'x-abcdefab' """ typ = type(x) if typ is tuple: return x[0] elif typ is str or typ is unicode: if x[0] == '(': return x.split(',', 1)[0].strip('()"\'') elif len(x) == 32 and re.match(r'[a-f0-9]{32}', x): return 'data' elif x[0] == '<': return x.strip('<>').split()[0].split('.')[-1] else: return x else: return 'Other' @contextmanager def log_errors(pdb=False): from .comm import CommClosedError try: yield except (CommClosedError, gen.Return): raise except Exception as e: try: logger.exception(e) except TypeError: # logger becomes None during process cleanup pass if pdb: import pdb pdb.set_trace() raise def silence_logging(level, root='distributed'): """ Force all existing loggers below *root* to the given level at least (or keep the existing level if less verbose). """ if isinstance(level, str): level = logging_names[level.upper()] for name, logger in logging.root.manager.loggerDict.items(): if (isinstance(logger, logging.Logger) and logger.name.startswith(root + '.') and logger.level < level): logger.setLevel(level) @memoize def ensure_ip(hostname): """ Ensure that address is an IP address Examples -------- >>> ensure_ip('localhost') '127.0.0.1' >>> ensure_ip('123.123.123.123') # pass through IP addresses '123.123.123.123' """ # Prefer IPv4 over IPv6, for compatibility families = [socket.AF_INET, socket.AF_INET6] for fam in families: try: results = socket.getaddrinfo(hostname, 1234, # dummy port number fam, socket.SOCK_STREAM) except socket.gaierror as e: exc = e else: return results[0][4][0] raise exc tblib.pickling_support.install() def get_traceback(): exc_type, exc_value, exc_traceback = sys.exc_info() bad = [os.path.join('distributed', 'worker'), os.path.join('distributed', 'scheduler'), os.path.join('tornado', 'gen.py'), os.path.join('concurrent', 'futures')] while exc_traceback and any(b in exc_traceback.tb_frame.f_code.co_filename for b in bad): exc_traceback = exc_traceback.tb_next return exc_traceback def truncate_exception(e, n=10000): """ Truncate exception to be about a certain length """ if len(str(e)) > n: try: return type(e)("Long error message", str(e)[:n]) except Exception: return Exception("Long error message", type(e), str(e)[:n]) else: 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. Examples -------- >>> tokey(b'x') 'x' >>> tokey('x') 'x' >>> tokey(1) '1' """ typ = type(o) if typ is unicode or typ is bytes: return o else: return str(o) def validate_key(k): """Validate a key as received on a stream. """ typ = type(k) if typ is not unicode and typ is not bytes: raise TypeError("Unexpected key type %s (value: %r)" % (typ, k)) def _maybe_complex(task): """ Possibly contains a nested task """ return (istask(task) or type(task) is list and any(map(_maybe_complex, task)) or type(task) is dict and any(map(_maybe_complex, task.values()))) def str_graph(dsk, extra_values=()): def convert(task): if type(task) is list: return [convert(v) for v in task] if type(task) is dict: return valmap(convert, task) if istask(task): return (task[0],) + tuple(map(convert, task[1:])) try: if task in dsk or task in extra_values: return tokey(task) except TypeError: pass return task return {tokey(k): convert(v) for k, v in dsk.items()} def seek_delimiter(file, delimiter, blocksize): """ Seek current file to next byte after a delimiter bytestring This seeks the file to the next byte following the delimiter. It does not return anything. Use ``file.tell()`` to see location afterwards. Parameters ---------- file: a file delimiter: bytes a delimiter like ``b'\n'`` or message sentinel blocksize: int Number of bytes to read from the file at once. """ if file.tell() == 0: return last = b'' while True: current = file.read(blocksize) if not current: return full = last + current try: i = full.index(delimiter) file.seek(file.tell() - (len(full) - i) + len(delimiter)) return except ValueError: pass last = full[-len(delimiter):] def read_block(f, offset, length, delimiter=None): """ Read a block of bytes from a file Parameters ---------- f: file File-like object supporting seek, read, tell, etc.. offset: int Byte offset to start read length: int Number of bytes to read delimiter: bytes (optional) Ensure reading starts and stops at delimiter bytestring If using the ``delimiter=`` keyword argument we ensure that the read starts and stops at delimiter boundaries that follow the locations ``offset`` and ``offset + length``. If ``offset`` is zero then we start at zero. The bytestring returned WILL include the terminating delimiter string. Examples -------- >>> from io import BytesIO # doctest: +SKIP >>> f = BytesIO(b'Alice, 100\\nBob, 200\\nCharlie, 300') # doctest: +SKIP >>> read_block(f, 0, 13) # doctest: +SKIP b'Alice, 100\\nBo' >>> read_block(f, 0, 13, delimiter=b'\\n') # doctest: +SKIP b'Alice, 100\\nBob, 200\\n' >>> read_block(f, 10, 10, delimiter=b'\\n') # doctest: +SKIP b'Bob, 200\\nCharlie, 300' """ if delimiter: f.seek(offset) seek_delimiter(f, delimiter, 2**16) start = f.tell() length -= start - offset f.seek(start + length) seek_delimiter(f, delimiter, 2**16) end = f.tell() offset = start length = end - start f.seek(offset) bytes = f.read(length) return bytes @contextmanager def tmpfile(extension=''): extension = '.' + extension.lstrip('.') handle, filename = tempfile.mkstemp(extension) os.close(handle) os.remove(filename) yield filename if os.path.exists(filename): if os.path.isdir(filename): shutil.rmtree(filename) else: try: os.remove(filename) except OSError: # sometimes we can't remove a generated temp file pass def ensure_bytes(s): """ Turn string or bytes to bytes >>> ensure_bytes('123') b'123' >>> ensure_bytes(b'123') b'123' """ if isinstance(s, bytes): return s if isinstance(s, memoryview): return s.tobytes() if isinstance(s, bytearray) or PY2 and isinstance(s, buffer): # flake8: noqa return bytes(s) if hasattr(s, 'encode'): return s.encode() raise TypeError( "Object %s is neither a bytes object nor has an encode method" % s) def divide_n_among_bins(n, bins): """ >>> divide_n_among_bins(12, [1, 1]) [6, 6] >>> divide_n_among_bins(12, [1, 2]) [4, 8] >>> divide_n_among_bins(12, [1, 2, 1]) [3, 6, 3] >>> divide_n_among_bins(11, [1, 2, 1]) [2, 6, 3] >>> divide_n_among_bins(11, [.1, .2, .1]) [2, 6, 3] """ total = sum(bins) acc = 0.0 out = [] for b in bins: now = n / total * b + acc now, acc = divmod(now, 1) out.append(int(now)) return out def mean(seq): seq = list(seq) return sum(seq) / len(seq) if hasattr(sys, "is_finalizing"): def shutting_down(is_finalizing=sys.is_finalizing): return is_finalizing() else: _shutting_down = [False] def _at_shutdown(l=_shutting_down): l[0] = True def shutting_down(l=_shutting_down): return l[0] atexit.register(_at_shutdown) shutting_down.__doc__ = """ Whether the interpreter is currently shutting down. For use in finalizers, __del__ methods, and similar; it is advised to early bind this function rather than look it up when calling it, since at shutdown module globals may be cleared. """ def open_port(host=''): """ Return a probably-open port There is a chance that this port will be taken by the operating system soon after returning from this function. """ # http://stackoverflow.com/questions/2838244/get-open-tcp-port-in-python s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) s.bind((host, 0)) s.listen(1) port = s.getsockname()[1] s.close() return port def import_file(path): """ Loads modules for a file (.py, .pyc, .zip, .egg) """ directory, filename = os.path.split(path) name, ext = os.path.splitext(filename) names_to_import = [] tmp_python_path = None if ext in ('.py', '.pyc'): if directory not in sys.path: tmp_python_path = directory names_to_import.append(name) # Ensures that no pyc file will be reused cache_file = cache_from_source(path) if os.path.exists(cache_file): os.remove(cache_file) if ext in ('.egg', '.zip'): 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 == '.zip': names_to_import.append(name) loaded = [] if not names_to_import: logger.warning("Found nothing to import from %s", filename) else: invalidate_caches() if tmp_python_path is not None: sys.path.insert(0, tmp_python_path) try: for name in names_to_import: logger.info("Reload module %s from %s file", name, ext) loaded.append(reload(import_module(name))) finally: if tmp_python_path is not None: sys.path.remove(tmp_python_path) return loaded class itemgetter(object): """A picklable itemgetter. Examples -------- >>> data = [0, 1, 2] >>> get_1 = itemgetter(1) >>> get_1(data) 1 """ __slots__ = ('index',) def __init__(self, index): self.index = index def __call__(self, x): return x[self.index] def __reduce__(self): return (itemgetter, (self.index,)) def format_bytes(n): """ Format bytes as text >>> format_bytes(1) '1 B' >>> format_bytes(1234) '1.23 kB' >>> format_bytes(12345678) '12.35 MB' >>> format_bytes(1234567890) '1.23 GB' """ if n > 1e9: return '%0.2f GB' % (n / 1e9) if n > 1e6: return '%0.2f MB' % (n / 1e6) if n > 1e3: return '%0.2f kB' % (n / 1000) return '%d B' % n byte_sizes = { 'kB': 10**3, 'MB': 10**6, 'GB': 10**9, 'TB': 10**12, 'PB': 10**15, 'KiB': 2**10, 'MiB': 2**20, 'GiB': 2**30, 'TiB': 2**40, 'PiB': 2**50, 'B': 1, '': 1, } byte_sizes = {k.lower(): v for k, v in byte_sizes.items()} byte_sizes.update({k[0]: v for k, v in byte_sizes.items() if k and 'i' not in k}) byte_sizes.update({k[:-1]: v for k, v in byte_sizes.items() if k and 'i' in k}) def parse_bytes(s): """ Parse byte string to numbers >>> parse_bytes('100') 100 >>> parse_bytes('100 MB') 100000000 >>> parse_bytes('100M') 100000000 >>> parse_bytes('5kB') 5000 >>> parse_bytes('5.4 kB') 5400 >>> parse_bytes('1kiB') 1024 >>> parse_bytes('1e6') 1000000 >>> parse_bytes('1e6 kB') 1000000000 >>> parse_bytes('MB') 1000000 """ s = s.replace(' ', '') if not s[0].isdigit(): s = '1' + s for i in range(len(s) - 1, -1, -1): if not s[i].isalpha(): break index = i + 1 prefix = s[:index] suffix = s[index:] n = float(prefix) multiplier = byte_sizes[suffix.lower()] result = n * multiplier return int(result) def asciitable(columns, rows): """Formats an ascii table for given columns and rows. Parameters ---------- columns : list The column names rows : list of tuples The rows in the table. Each tuple must be the same length as ``columns``. """ rows = [tuple(str(i) for i in r) for r in rows] columns = tuple(str(i) for i in columns) widths = tuple(max(max(map(len, x)), len(c)) for x, c in zip(zip(*rows), columns)) row_template = ('|' + (' %%-%ds |' * len(columns))) % widths header = row_template % tuple(columns) bar = '+%s+' % '+'.join('-' * (w + 2) for w in widths) data = '\n'.join(row_template % r for r in rows) return '\n'.join([bar, header, bar, data, bar]) if PY2: def nbytes(frame, _bytes_like=(bytes, bytearray, buffer)): """ Number of bytes of a frame or memoryview """ if isinstance(frame, _bytes_like): return len(frame) elif isinstance(frame, memoryview): if frame.shape is None: return frame.itemsize else: return functools.reduce(operator.mul, frame.shape, frame.itemsize) else: return frame.nbytes else: def nbytes(frame, _bytes_like=(bytes, bytearray)): """ Number of bytes of a frame or memoryview """ if isinstance(frame, _bytes_like): return len(frame) else: return frame.nbytes def PeriodicCallback(callback, callback_time, io_loop=None): """ Wrapper around tornado.IOLoop.PeriodicCallback, for compatibility with removal of the `io_loop` parameter in Tornado 5.0. """ if tornado.version_info >= (5,): return tornado.ioloop.PeriodicCallback(callback, callback_time) else: return tornado.ioloop.PeriodicCallback(callback, callback_time, io_loop) @contextmanager def time_warn(duration, text): start = time() yield end = time() if end - start > duration: print('TIME WARNING', text, end - start) def json_load_robust(fn, load=json.load): """ Reads a JSON file from disk that may be being written as we read """ while not os.path.exists(fn): sleep(0.01) for i in range(10): try: with open(fn) as f: cfg = load(f) if cfg: return cfg except (ValueError, KeyError): # race with writing process pass sleep(0.1) def format_time(n): """ format integers as time >>> format_time(1) '1.00 s' >>> format_time(0.001234) '1.23 ms' >>> format_time(0.00012345) '123.45 us' >>> format_time(123.456) '123.46 s' """ if n >= 1: return '%.2f s' % n if n >= 1e-3: return '%.2f ms' % (n * 1e3) return '%.2f us' % (n * 1e6) class DequeHandler(logging.Handler): """ A logging.Handler that records records into a deque """ _instances = weakref.WeakSet() def __init__(self, *args, **kwargs): n = kwargs.pop('n', 10000) self.deque = deque(maxlen=n) super(DequeHandler, self).__init__(*args, **kwargs) self._instances.add(self) def emit(self, record): self.deque.append(record) def clear(self): """ Clear internal storage. """ self.deque.clear() @classmethod def clear_all_instances(cls): """ Clear the internal storage of all live DequeHandlers. """ for inst in list(cls._instances): inst.clear() class ThrottledGC(object): """Wrap gc.collect to protect against excessively repeated calls. Allows to run throttled garbage collection in the workers as a countermeasure to e.g.: https://github.com/dask/zict/issues/19 collect() does nothing when repeated calls are so costly and so frequent that the thread would spend more than max_in_gc_frac doing GC. warn_if_longer is a duration in seconds (10s by default) that can be used to log a warning level message whenever an actual call to gc.collect() lasts too long. """ def __init__(self, max_in_gc_frac=0.05, warn_if_longer=1, logger=None): self.max_in_gc_frac = max_in_gc_frac self.warn_if_longer = warn_if_longer self.last_collect = time() self.last_gc_duration = 0 self.logger = logger if logger is not None else _logger def collect(self): # In case of non-monotonicity in the clock, assume that any Python # operation lasts at least 1e-6 second. collect_start = time() elapsed = max(collect_start - self.last_collect, 1e-6) if self.last_gc_duration / elapsed < self.max_in_gc_frac: self.logger.debug("Calling gc.collect(). %0.3fs elapsed since " "previous call.", elapsed) gc.collect() self.last_collect = collect_start self.last_gc_duration = max(time() - collect_start, 1e-6) if self.last_gc_duration > self.warn_if_longer: self.logger.warning("gc.collect() took %0.3fs. This is usually" " a sign that the some tasks handle too" " many Python objects at the same time." " Rechunking the work into smaller tasks" " might help.", self.last_gc_duration) else: self.logger.debug("gc.collect() took %0.3fs", self.last_gc_duration) else: self.logger.debug("gc.collect() lasts %0.3fs but only %0.3fs " "elapsed since last call: throttling.", self.last_gc_duration, elapsed) def fix_asyncio_event_loop_policy(asyncio): """ Work around https://github.com/tornadoweb/tornado/issues/2183 """ class PatchedDefaultEventLoopPolicy(asyncio.DefaultEventLoopPolicy): def get_event_loop(self): """Get the event loop. This may be None or an instance of EventLoop. """ try: return super().get_event_loop() except RuntimeError: # "There is no current event loop in thread" loop = self.new_event_loop() self.set_event_loop(loop) return loop asyncio.set_event_loop_policy(PatchedDefaultEventLoopPolicy()) # Only bother if asyncio has been loaded by Tornado if 'asyncio' in sys.modules: fix_asyncio_event_loop_policy(sys.modules['asyncio']) distributed-1.20.2/distributed/utils_comm.py000066400000000000000000000142241321233345200211660ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import defaultdict from itertools import cycle import random from tornado import gen from tornado.gen import Return from toolz import merge, concat, groupby, drop from .core import rpc from .utils import All, tokey @gen.coroutine def gather_from_workers(who_has, rpc, close=True): """ Gather data directly from peers Parameters ---------- who_has: dict Dict mapping keys to sets of workers that may have that key rpc: callable Returns dict mapping key to value See Also -------- gather _gather """ bad_addresses = set() missing_workers = set() original_who_has = who_has who_has = {k: set(v) for k, v in who_has.items()} results = dict() all_bad_keys = set() while len(results) + len(all_bad_keys) < len(who_has): d = defaultdict(list) rev = dict() bad_keys = set() for key, addresses in who_has.items(): if key in results: continue try: addr = random.choice(list(addresses - bad_addresses)) d[addr].append(key) rev[key] = addr except IndexError: bad_keys.add(key) if bad_keys: all_bad_keys |= bad_keys rpcs = {addr: rpc(addr) for addr in d} try: coroutines = {address: rpcs[address].get_data(keys=keys, close=close) for address, keys in d.items()} response = {} for worker, c in coroutines.items(): try: r = yield c except EnvironmentError: missing_workers.add(worker) else: response.update(r) finally: for r in rpcs.values(): r.close_rpc() bad_addresses |= {v for k, v in rev.items() if k not in response} results.update(response) bad_keys = {k: list(original_who_has[k]) for k in all_bad_keys} raise Return((results, bad_keys, list(missing_workers))) class WrappedKey(object): """ Interface for a key in a dask graph. Subclasses must have .key attribute that refers to a key in a dask graph. Sometimes we want to associate metadata to keys in a dask graph. For example we might know that that key lives on a particular machine or can only be accessed in a certain way. Schedulers may have particular needs that can only be addressed by additional metadata. """ def __init__(self, key): self.key = key _round_robin_counter = [0] @gen.coroutine def scatter_to_workers(ncores, data, rpc=rpc, report=True): """ Scatter data directly to workers This distributes data in a round-robin fashion to a set of workers based on how many cores they have. ncores should be a dictionary mapping worker identities to numbers of cores. See scatter for parameter docstring """ assert isinstance(ncores, dict) assert isinstance(data, dict) workers = list(concat([w] * nc for w, nc in ncores.items())) names, data = list(zip(*data.items())) worker_iter = drop(_round_robin_counter[0] % len(workers), cycle(workers)) _round_robin_counter[0] += len(data) L = list(zip(worker_iter, names, data)) d = groupby(0, L) d = {worker: {key: value for _, key, value in v} for worker, v in d.items()} rpcs = {addr: rpc(addr) for addr in d} try: out = yield All([rpcs[address].update_data(data=v, report=report) for address, v in d.items()]) finally: for r in rpcs.values(): r.close_rpc() nbytes = merge(o['nbytes'] for o in out) who_has = {k: [w for w, _, _ in v] for k, v in groupby(1, L).items()} raise Return((names, who_has, nbytes)) collection_types = (tuple, list, set, frozenset) def unpack_remotedata(o, byte_keys=False, myset=None): """ Unpack WrappedKey objects from collection Returns original collection and set of all found keys Examples -------- >>> rd = WrappedKey('mykey') >>> unpack_remotedata(1) (1, set()) >>> unpack_remotedata(()) ((), set()) >>> unpack_remotedata(rd) ('mykey', {'mykey'}) >>> unpack_remotedata([1, rd]) ([1, 'mykey'], {'mykey'}) >>> unpack_remotedata({1: rd}) ({1: 'mykey'}, {'mykey'}) >>> unpack_remotedata({1: [rd]}) ({1: ['mykey']}, {'mykey'}) Use the ``byte_keys=True`` keyword to force string keys >>> rd = WrappedKey(('x', 1)) >>> unpack_remotedata(rd, byte_keys=True) ("('x', 1)", {"('x', 1)"}) """ if myset is None: myset = set() out = unpack_remotedata(o, byte_keys, myset) return out, myset typ = type(o) if typ in collection_types: if not o: return o outs = [unpack_remotedata(item, byte_keys, myset) for item in o] return type(o)(outs) elif typ is dict: if o: values = [unpack_remotedata(v, byte_keys, myset) for v in o.values()] return dict(zip(o.keys(), values)) else: return o elif issubclass(typ, WrappedKey): # TODO use type is Future k = o.key if byte_keys: k = tokey(k) myset.add(k) return k else: return o def pack_data(o, d, key_types=object): """ Merge known data into tuple or dict Parameters ---------- o: core data structures containing literals and keys d: dict mapping of keys to data Examples -------- >>> data = {'x': 1} >>> pack_data(('x', 'y'), data) (1, 'y') >>> pack_data({'a': 'x', 'b': 'y'}, data) # doctest: +SKIP {'a': 1, 'b': 'y'} >>> pack_data({'a': ['x'], 'b': 'y'}, data) # doctest: +SKIP {'a': [1], 'b': 'y'} """ typ = type(o) try: if isinstance(o, key_types) and o in d: return d[o] except TypeError: pass if typ in collection_types: return typ([pack_data(x, d, key_types=key_types) for x in o]) elif typ is dict: return {k: pack_data(v, d, key_types=key_types) for k, v in o.items()} else: return o distributed-1.20.2/distributed/utils_test.py000066400000000000000000000773671321233345200212330ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from contextlib import contextmanager from datetime import timedelta import functools import gc from glob import glob import itertools import inspect import logging import logging.config import os import re import shutil import signal import socket import subprocess import sys import tempfile import textwrap import threading from time import sleep import uuid import warnings import weakref try: import ssl except ImportError: ssl = None import psutil import pytest import six from dask.context import _globals from toolz import merge, memoize from tornado import gen, queues from tornado.gen import TimeoutError from tornado.ioloop import IOLoop from .compatibility import WINDOWS, PY3 from .config import config, initialize_logging from .core import connect, rpc, CommClosedError from .metrics import time from .nanny import Nanny from .proctitle import enable_proctitle_on_children from .security import Security from .utils import (ignoring, log_errors, sync, mp_context, get_ip, get_ipv6, DequeHandler) from .worker import Worker, TOTAL_MEMORY logger = logging.getLogger(__name__) logging_levels = {name: logger.level for name, logger in logging.root.manager.loggerDict.items() if isinstance(logger, logging.Logger)} @pytest.fixture(scope='session') def valid_python_script(tmpdir_factory): local_file = tmpdir_factory.mktemp('data').join('file.py') local_file.write("print('hello world!')") return local_file @pytest.fixture(scope='session') def client_contract_script(tmpdir_factory): local_file = tmpdir_factory.mktemp('data').join('distributed_script.py') lines = ("from distributed import Client", "e = Client('127.0.0.1:8989')", 'print(e)') local_file.write('\n'.join(lines)) return local_file @pytest.fixture(scope='session') def invalid_python_script(tmpdir_factory): local_file = tmpdir_factory.mktemp('data').join('file.py') local_file.write("a+1") return local_file @pytest.fixture def loop(): 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() loop.start = start yield loop # Stop the loop in case it's still running try: loop.add_callback(loop.stop) except RuntimeError as e: if not re.match("IOLoop is clos(ed|ing)", str(e)): raise else: is_stopped.wait() @pytest.fixture def loop_in_thread(): with pristine_loop() as loop: thread = threading.Thread(target=loop.start, name="test IOLoop") thread.daemon = True thread.start() loop_started = threading.Event() loop.add_callback(loop_started.set) loop_started.wait() yield loop loop.add_callback(loop.stop) thread.join(timeout=5) @pytest.fixture def zmq_ctx(): import zmq ctx = zmq.Context.instance() yield ctx ctx.destroy(linger=0) @contextmanager def pristine_loop(): IOLoop.clear_instance() IOLoop.clear_current() loop = IOLoop() loop.make_current() assert IOLoop.current() is loop try: yield loop finally: try: loop.close(all_fds=True) except ValueError: pass IOLoop.clear_instance() IOLoop.clear_current() @contextmanager def mock_ipython(): import mock ip = mock.Mock() ip.user_ns = {} ip.kernel = None def get_ip(): return ip with mock.patch('IPython.get_ipython', get_ip), \ mock.patch('distributed._ipython_utils.get_ipython', get_ip): yield ip def nodebug(func): """ A decorator to disable debug facilities during timing-sensitive tests. Warning: this doesn't affect already created IOLoops. """ if not PY3: # py.test's runner magic breaks horridly on Python 2 # when a test function is wrapped, so avoid it # (incidently, asyncio is irrelevant anyway) return func @functools.wraps(func) def wrapped(*args, **kwargs): old_asyncio_debug = os.environ.get("PYTHONASYNCIODEBUG") if old_asyncio_debug is not None: del os.environ["PYTHONASYNCIODEBUG"] try: return func(*args, **kwargs) finally: if old_asyncio_debug is not None: os.environ["PYTHONASYNCIODEBUG"] = old_asyncio_debug return wrapped def nodebug_setup_module(module): """ A setup_module() that you can install in a test module to disable debug facilities. """ module._old_asyncio_debug = os.environ.get("PYTHONASYNCIODEBUG") if module._old_asyncio_debug is not None: del os.environ["PYTHONASYNCIODEBUG"] def nodebug_teardown_module(module): """ A teardown_module() that you can install in a test module to reenable debug facilities. """ if module._old_asyncio_debug is not None: os.environ["PYTHONASYNCIODEBUG"] = module._old_asyncio_debug def inc(x): return x + 1 def dec(x): return x - 1 def mul(x, y): return x * y def div(x, y): return x / y def deep(n): if n > 0: return deep(n - 1) else: return True def throws(x): raise RuntimeError('hello!') def double(x): return x * 2 def slowinc(x, delay=0.02): sleep(delay) return x + 1 def slowdec(x, delay=0.02): sleep(delay) return x - 1 def slowdouble(x, delay=0.02): sleep(delay) return 2 * x def randominc(x, scale=1): from random import random sleep(random() * scale) return x + 1 def slowadd(x, y, delay=0.02): sleep(delay) return x + y def slowsum(seq, delay=0.02): sleep(delay) return sum(seq) def slowidentity(*args, **kwargs): delay = kwargs.get('delay', 0.02) sleep(delay) if len(args) == 1: return args[0] else: return args # This dict grows at every varying() invocation _varying_dict = {} _varying_key_gen = itertools.count() class _ModuleSlot(object): def __init__(self, modname, slotname): self.modname = modname self.slotname = slotname def get(self): return getattr(sys.modules[self.modname], self.slotname) def varying(items): """ Return a function that returns a result (or raises an exception) from *items* at each call. """ # cloudpickle would serialize the *values* of all globals # used by *func* below, so we can't use `global `. # Instead look up the module by name to get the original namespace # and not a copy. slot = _ModuleSlot(__name__, '_varying_dict') key = next(_varying_key_gen) _varying_dict[key] = 0 def func(): dct = slot.get() i = dct[key] if i == len(items): raise IndexError else: x = items[i] dct[key] = i + 1 if isinstance(x, Exception): raise x else: return x return func def map_varying(itemslists): """ Like *varying*, but return the full specification for a map() call on multiple items lists. """ def apply(func, *args, **kwargs): return func(*args, **kwargs) return apply, map(varying, itemslists) @gen.coroutine def geninc(x, delay=0.02): yield gen.sleep(delay) raise gen.Return(x + 1) def compile_snippet(code, dedent=True): if dedent: code = textwrap.dedent(code) code = compile(code, '', 'exec') ns = globals() exec(code, ns, ns) if sys.version_info >= (3, 5): compile_snippet(""" async def asyncinc(x, delay=0.02): await gen.sleep(delay) return x + 1 """) assert asyncinc # flake8: noqa else: asyncinc = None _readone_queues = {} @gen.coroutine def readone(comm): """ Read one message at a time from a comm that reads lists of messages. """ try: q = _readone_queues[comm] except KeyError: q = _readone_queues[comm] = queues.Queue() @gen.coroutine def background_read(): while True: try: messages = yield comm.read() except CommClosedError: break for msg in messages: q.put_nowait(msg) q.put_nowait(None) del _readone_queues[comm] background_read() msg = yield q.get() if msg is None: raise CommClosedError else: raise gen.Return(msg) def run_scheduler(q, nputs, **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, **kwargs) done = scheduler.start('127.0.0.1') for i in range(nputs): q.put(scheduler.address) try: loop.start() finally: loop.close(all_fds=True) def run_worker(q, scheduler_q, **kwargs): from distributed import Worker with log_errors(): with pristine_loop() as loop: scheduler_addr = scheduler_q.get() worker = Worker(scheduler_addr, validate=True, **kwargs) loop.run_sync(lambda: worker._start(0)) q.put(worker.address) try: @gen.coroutine def wait_until_closed(): yield worker._closed.wait() loop.run_sync(wait_until_closed) finally: loop.close(all_fds=True) def run_nanny(q, scheduler_q, **kwargs): from distributed import Nanny with log_errors(): with pristine_loop() as loop: scheduler_addr = scheduler_q.get() worker = Nanny(scheduler_addr, validate=True, **kwargs) loop.run_sync(lambda: worker._start(0)) q.put(worker.address) try: loop.start() finally: loop.run_sync(worker._close) loop.close(all_fds=True) @contextmanager def check_active_rpc(loop, active_rpc_timeout=1): active_before = set(rpc.active) if active_before and not PY3: # On Python 2, try to avoid dangling comms before forking workers gc.collect() active_before = set(rpc.active) yield # Some streams can take a bit of time to notice their peer # has closed, and keep a coroutine (*) waiting for a CommClosedError # before calling close_rpc() after a CommClosedError. # This would happen especially if a non-localhost address is used, # as Nanny does. # (*) (example: gather_from_workers()) def fail(): pytest.fail("some RPCs left active by test: %s" % (sorted(set(rpc.active) - active_before))) @gen.coroutine def wait(): yield async_wait_for(lambda: len(set(rpc.active) - active_before) == 0, timeout=active_rpc_timeout, fail_func=fail) loop.run_sync(wait) @contextmanager def cluster(nworkers=2, nanny=False, worker_kwargs={}, active_rpc_timeout=1, scheduler_kwargs={}): ws = weakref.WeakSet() old_globals = _globals.copy() 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 # The scheduler queue will receive the scheduler's address scheduler_q = mp_context.Queue() # Launch scheduler scheduler = mp_context.Process(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) proc = mp_context.Process(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() for worker in workers: worker['address'] = worker['queue'].get() saddr = scheduler_q.get() start = time() try: with rpc(saddr) 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)) loop.run_sync(lambda: disconnect(saddr, timeout=0.5)) 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() 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[:] for fn in glob('_test_worker-*'): shutil.rmtree(fn) _globals.clear() _globals.update(old_globals) assert not ws @gen.coroutine def disconnect(addr, timeout=3): @gen.coroutine def do_disconnect(): with ignoring(EnvironmentError, CommClosedError): with rpc(addr) as w: yield w.terminate(close=True) with ignoring(TimeoutError): yield gen.with_timeout(timedelta(seconds=timeout), do_disconnect()) @gen.coroutine def disconnect_all(addresses, timeout=3): yield [disconnect(addr, timeout) for addr in addresses] def slow(func): try: if not pytest.config.getoption("--runslow"): func = pytest.mark.skip("need --runslow option to run")(func) except AttributeError: # AttributeError: module 'pytest' has no attribute 'config' pass return nodebug(func) def gen_test(timeout=10): """ Coroutine test @gen_test(timeout=5) def test_foo(): yield ... # use tornado coroutines """ def _(func): def test_func(): with pristine_loop() as loop: cor = gen.coroutine(func) try: loop.run_sync(cor, timeout=timeout) finally: loop.stop() return test_func return _ from .scheduler import Scheduler from .worker import Worker @gen.coroutine def start_cluster(ncores, scheduler_addr, loop, security=None, Worker=Worker, scheduler_kwargs={}, worker_kwargs={}): s = Scheduler(loop=loop, validate=True, security=security, **scheduler_kwargs) done = s.start(scheduler_addr) workers = [Worker(s.address, ncores=ncore[1], name=i, security=security, loop=loop, validate=True, **(merge(worker_kwargs, ncore[2]) if len(ncore) > 2 else worker_kwargs)) for i, ncore in enumerate(ncores)] for w in workers: w.rpc = workers[0].rpc yield [w._start(ncore[0]) for ncore, w in zip(ncores, workers)] start = time() while len(s.ncores) < len(ncores): yield gen.sleep(0.01) if time() - start > 5: raise Exception("Cluster creation timeout") raise gen.Return((s, workers)) @gen.coroutine def end_cluster(s, workers): logger.debug("Closing out test cluster") @gen.coroutine def end_worker(w): with ignoring(TimeoutError, CommClosedError, EnvironmentError): yield w._close(report=False) yield [end_worker(w) for w in workers] yield s.close() # wait until scheduler stops completely s.stop() def iscoroutinefunction(f): if sys.version_info >= (3, 5) and inspect.iscoroutinefunction(f): return True return False def gen_cluster(ncores=[('127.0.0.1', 1), ('127.0.0.1', 2)], scheduler='127.0.0.1', timeout=10, security=None, Worker=Worker, client=False, scheduler_kwargs={}, worker_kwargs={}, active_rpc_timeout=1): from distributed import Client """ Coroutine test with small cluster @gen_cluster() def test_foo(scheduler, worker1, worker2): yield ... # use tornado coroutines See also: start end """ worker_kwargs = merge({'memory_limit': TOTAL_MEMORY}, worker_kwargs) def _(func): if not iscoroutinefunction(func): func = gen.coroutine(func) def test_func(): # Restore default logging levels # XXX use pytest hooks/fixtures instead? for name, level in logging_levels.items(): logging.getLogger(name).setLevel(level) old_globals = _globals.copy() result = None workers = [] with pristine_loop() as loop: with check_active_rpc(loop, active_rpc_timeout): @gen.coroutine def coro(): s, ws = yield start_cluster( ncores, scheduler, loop, security=security, Worker=Worker, scheduler_kwargs=scheduler_kwargs, worker_kwargs=worker_kwargs) workers[:] = ws args = [s] + workers if client: c = yield Client(s.address, loop=loop, security=security, asynchronous=True) args = [c] + args try: result = yield func(*args) # for w in workers: # assert not w._comms finally: if client: yield c._close() yield end_cluster(s, workers) _globals.clear() _globals.update(old_globals) raise gen.Return(result) result = loop.run_sync(coro, timeout=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 DequeHandler.clear_all_instances() return result return test_func return _ def raises(func, exc=Exception): try: func() return False except exc: return True def terminate_process(proc): if proc.poll() is None: if sys.platform.startswith('win'): proc.send_signal(signal.CTRL_BREAK_EVENT) else: proc.send_signal(signal.SIGINT) try: if sys.version_info[0] == 3: proc.wait(10) else: start = time() while proc.poll() is None and time() < start + 10: sleep(0.02) finally: # Make sure we don't leave the process lingering around with ignoring(OSError): proc.kill() @contextmanager def popen(*args, **kwargs): kwargs['stdout'] = subprocess.PIPE kwargs['stderr'] = subprocess.PIPE if sys.platform.startswith('win'): # Allow using CTRL_C_EVENT / CTRL_BREAK_EVENT kwargs['creationflags'] = subprocess.CREATE_NEW_PROCESS_GROUP dump_stdout = False proc = subprocess.Popen(*args, **kwargs) try: yield proc except Exception: dump_stdout = True raise finally: try: terminate_process(proc) finally: # XXX Also dump stdout if return code != 0 ? out, err = proc.communicate() if dump_stdout: print('\n\nPrint from stderr\n %s\n=================\n' % args[0][0]) print(err.decode()) print('\n\nPrint from stdout\n=================\n') print(out.decode()) def wait_for_port(address, timeout=5): assert isinstance(address, tuple) deadline = time() + timeout while True: timeout = deadline - time() if timeout < 0: raise RuntimeError("Failed to connect to %s" % (address,)) try: sock = socket.create_connection(address, timeout=timeout) except EnvironmentError: pass else: sock.close() break def wait_for(predicate, timeout, fail_func=None, period=0.001): deadline = time() + timeout while not predicate(): sleep(period) if time() > deadline: if fail_func is not None: fail_func() pytest.fail("condition not reached until %s seconds" % (timeout,)) @gen.coroutine def async_wait_for(predicate, timeout, fail_func=None, period=0.001): deadline = time() + timeout while not predicate(): yield gen.sleep(period) if time() > deadline: if fail_func is not None: fail_func() pytest.fail("condition not reached until %s seconds" % (timeout,)) @memoize def has_ipv6(): """ Return whether IPv6 is locally functional. This doesn't guarantee IPv6 is properly configured outside of localhost. """ serv = cli = None try: serv = socket.socket(socket.AF_INET6, socket.SOCK_STREAM) serv.bind(('::', 0)) serv.listen(5) cli = socket.create_connection(serv.getsockname()[:2]) except EnvironmentError: return False else: return True finally: if cli is not None: cli.close() if serv is not None: serv.close() if has_ipv6(): def requires_ipv6(test_func): return test_func else: requires_ipv6 = pytest.mark.skip("ipv6 required") @gen.coroutine def assert_can_connect(addr, timeout=None, connection_args=None): """ Check that it is possible to connect to the distributed *addr* within the given *timeout*. """ if timeout is None: timeout = 0.2 comm = yield connect(addr, timeout=timeout, connection_args=connection_args) comm.abort() @gen.coroutine def assert_cannot_connect(addr, timeout=None, connection_args=None): """ Check that it is impossible to connect to the distributed *addr* within the given *timeout*. """ if timeout is None: timeout = 0.2 with pytest.raises(EnvironmentError): comm = yield connect(addr, timeout=timeout, connection_args=connection_args) comm.abort() @gen.coroutine def assert_can_connect_from_everywhere_4_6(port, timeout=None, connection_args=None): """ Check that the local *port* is reachable from all IPv4 and IPv6 addresses. """ args = (timeout, connection_args) futures = [ assert_can_connect('tcp://127.0.0.1:%d' % port, *args), assert_can_connect('tcp://%s:%d' % (get_ip(), port), *args), ] if has_ipv6(): futures += [ assert_can_connect('tcp://[::1]:%d' % port, *args), assert_can_connect('tcp://[%s]:%d' % (get_ipv6(), port), *args), ] yield futures @gen.coroutine def assert_can_connect_from_everywhere_4(port, timeout=None, connection_args=None): """ Check that the local *port* is reachable from all IPv4 addresses. """ args = (timeout, connection_args) futures = [ assert_can_connect('tcp://127.0.0.1:%d' % port, *args), assert_can_connect('tcp://%s:%d' % (get_ip(), port), *args), ] if has_ipv6(): futures += [ assert_cannot_connect('tcp://[::1]:%d' % port, *args), assert_cannot_connect('tcp://[%s]:%d' % (get_ipv6(), port), *args), ] yield futures @gen.coroutine def assert_can_connect_locally_4(port, timeout=None, connection_args=None): """ Check that the local *port* is only reachable from local IPv4 addresses. """ args = (timeout, connection_args) futures = [ assert_can_connect('tcp://127.0.0.1:%d' % port, *args), ] if get_ip() != '127.0.0.1': # No outside IPv4 connectivity? futures += [ assert_cannot_connect('tcp://%s:%d' % (get_ip(), port), *args), ] if has_ipv6(): futures += [ assert_cannot_connect('tcp://[::1]:%d' % port, *args), assert_cannot_connect('tcp://[%s]:%d' % (get_ipv6(), port), *args), ] yield futures @gen.coroutine def assert_can_connect_from_everywhere_6(port, timeout=None, connection_args=None): """ Check that the local *port* is reachable from all IPv6 addresses. """ assert has_ipv6() args = (timeout, connection_args) futures = [ assert_cannot_connect('tcp://127.0.0.1:%d' % port, *args), assert_cannot_connect('tcp://%s:%d' % (get_ip(), port), *args), assert_can_connect('tcp://[::1]:%d' % port, *args), assert_can_connect('tcp://[%s]:%d' % (get_ipv6(), port), *args), ] yield futures @gen.coroutine def assert_can_connect_locally_6(port, timeout=None, connection_args=None): """ Check that the local *port* is only reachable from local IPv6 addresses. """ assert has_ipv6() args = (timeout, connection_args) futures = [ assert_cannot_connect('tcp://127.0.0.1:%d' % port, *args), assert_cannot_connect('tcp://%s:%d' % (get_ip(), port), *args), assert_can_connect('tcp://[::1]:%d' % port, *args), ] if get_ipv6() != '::1': # No outside IPv6 connectivity? futures += [ assert_cannot_connect('tcp://[%s]:%d' % (get_ipv6(), port), *args), ] yield futures @contextmanager def captured_logger(logger, level=logging.INFO, propagate=None): """Capture output from the given Logger. """ if isinstance(logger, str): logger = logging.getLogger(logger) orig_level = logger.level orig_handlers = logger.handlers[:] if propagate is not None: orig_propagate = logger.propagate logger.propagate = propagate sio = six.StringIO() logger.handlers[:] = [logging.StreamHandler(sio)] logger.setLevel(level) try: yield sio finally: logger.handlers[:] = orig_handlers logger.setLevel(orig_level) if propagate is not None: logger.propagate = orig_propagate @contextmanager def captured_handler(handler): """Capture output from the given logging.StreamHandler. """ assert isinstance(handler, logging.StreamHandler) orig_stream = handler.stream handler.stream = six.StringIO() try: yield handler.stream finally: handler.stream = orig_stream @contextmanager def new_config(new_config): """ Temporarily change configuration dictionary. """ orig_config = config.copy() try: config.clear() config.update(new_config) initialize_logging(config) yield finally: config.clear() config.update(orig_config) initialize_logging(config) @contextmanager def new_config_file(c): """ Temporarily change configuration file to match dictionary *c*. """ import yaml old_file = os.environ.get('DASK_CONFIG') fd, path = tempfile.mkstemp(prefix='dask-config') try: with os.fdopen(fd, 'w') as f: f.write(yaml.dump(c)) os.environ['DASK_CONFIG'] = path try: yield finally: if old_file: os.environ['DASK_CONFIG'] = old_file else: del os.environ['DASK_CONFIG'] finally: os.remove(path) certs_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), 'tests')) def get_cert(filename): """ Get the path to one of the test TLS certificates. """ path = os.path.join(certs_dir, filename) assert os.path.exists(path), path return path def tls_config(): """ A functional TLS configuration with our test certs. """ ca_file = get_cert('tls-ca-cert.pem') keycert = get_cert('tls-key-cert.pem') c = { 'tls': { 'ca-file': ca_file, 'client': { 'cert': keycert, }, 'scheduler': { 'cert': keycert, }, 'worker': { 'cert': keycert, }, }, } return c def tls_only_config(): """ A functional TLS configuration with our test certs, disallowing plain TCP communications. """ c = tls_config() c['require-encryption'] = True return c def tls_security(): """ A Security object with proper TLS configuration. """ with new_config(tls_config()): sec = Security() return sec def tls_only_security(): """ A Security object with proper TLS configuration and disallowing plain TCP communications. """ with new_config(tls_only_config()): sec = Security() assert sec.require_encryption return sec def get_server_ssl_context(certfile='tls-cert.pem', keyfile='tls-key.pem', ca_file='tls-ca-cert.pem'): ctx = ssl.create_default_context(ssl.Purpose.CLIENT_AUTH, cafile=get_cert(ca_file)) ctx.check_hostname = False ctx.verify_mode = ssl.CERT_REQUIRED ctx.load_cert_chain(get_cert(certfile), get_cert(keyfile)) return ctx def get_client_ssl_context(certfile='tls-cert.pem', keyfile='tls-key.pem', ca_file='tls-ca-cert.pem'): ctx = ssl.create_default_context(ssl.Purpose.SERVER_AUTH, cafile=get_cert(ca_file)) ctx.check_hostname = False ctx.verify_mode = ssl.CERT_REQUIRED ctx.load_cert_chain(get_cert(certfile), get_cert(keyfile)) return ctx def bump_rlimit(limit, desired): resource = pytest.importorskip('resource') try: soft, hard = resource.getrlimit(limit) if soft < desired: resource.setrlimit(limit, (desired, max(hard, desired))) except Exception as e: pytest.skip("rlimit too low (%s) and can't be increased: %s" % (soft, e)) distributed-1.20.2/distributed/variable.py000066400000000000000000000170111321233345200205750ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from collections import defaultdict import logging import uuid from tornado import gen import tornado.locks try: from cytoolz import merge except ImportError: from toolz import merge from .client import Future, _get_global_client, Client from .metrics import time from .utils import tokey, log_errors from .worker import get_client logger = logging.getLogger(__name__) class VariableExtension(object): """ An extension for the scheduler to manage queues This adds the following routes to the scheduler * variable-set * variable-get * variable-delete """ def __init__(self, scheduler): self.scheduler = scheduler self.variables = dict() self.waiting = defaultdict(set) self.waiting_conditions = defaultdict(tornado.locks.Condition) self.started = tornado.locks.Condition() self.scheduler.handlers.update({'variable_set': self.set, 'variable_get': self.get}) self.scheduler.client_handlers['variable-future-release'] = self.future_release self.scheduler.client_handlers['variable_delete'] = self.delete self.scheduler.extensions['variables'] = self def set(self, stream=None, name=None, key=None, data=None, client=None): if key is not None: record = {'type': 'Future', 'value': key} self.scheduler.client_desires_keys(keys=[key], client='variable-%s' % name) else: record = {'type': 'msgpack', 'value': data} try: old = self.variables[name] except KeyError: pass else: if old['type'] == 'Future' and old['value'] != key: self.release(old['value'], name) if name not in self.variables: self.started.notify_all() self.variables[name] = record @gen.coroutine def release(self, key, name): while self.waiting[key, name]: yield self.waiting_conditions[name].wait() self.scheduler.client_releases_keys(keys=[key], client='variable-%s' % name) del self.waiting[key, name] def future_release(self, name=None, key=None, token=None, client=None): self.waiting[key, name].remove(token) if not self.waiting[key, name]: self.waiting_conditions[name].notify_all() @gen.coroutine def get(self, stream=None, name=None, client=None, timeout=None): start = time() while name not in self.variables: if timeout is not None: left = timeout - (time() - start) else: left = None if left and left < 0: raise gen.TimeoutError() yield self.started.wait(timeout=left) record = self.variables[name] if record['type'] == 'Future': key = record['value'] token = uuid.uuid4().hex try: state = self.scheduler.task_state[key] except KeyError: state = 'lost' msg = {'token': token, 'state': state} if state == 'erred': msg['exception'] = self.scheduler.exceptions[self.scheduler.exceptions_blame[key]] msg['traceback'] = self.scheduler.tracebacks[self.scheduler.exceptions_blame[key]] record = merge(record, msg) self.waiting[key, name].add(token) raise gen.Return(record) @gen.coroutine def delete(self, stream=None, name=None, client=None): with log_errors(): try: old = self.variables[name] except KeyError: pass else: if old['type'] == 'Future': yield self.release(old['value'], name) del self.waiting_conditions[name] del self.variables[name] class Variable(object): """ Distributed Global Variable This allows multiple clients to share futures and data between each other with a single mutable variable. All metadata is sequentialized through the scheduler. Race conditions can occur. Values must be either Futures or msgpack-encodable data (ints, lists, strings, etc..) All data will be kept and sent through the scheduler, so it is wise not to send too much. If you want to share a large amount of data then ``scatter`` it and share the future instead. .. warning:: This object is experimental and has known issues in Python 2 Examples -------- >>> from dask.distributed import Client, Variable # doctest: +SKIP >>> client = Client() # doctest: +SKIP >>> x = Variable('x') # doctest: +SKIP >>> x.set(123) # docttest: +SKIP >>> x.get() # docttest: +SKIP 123 >>> future = client.submit(f, x) # doctest: +SKIP >>> x.set(future) # doctest: +SKIP See Also -------- Queue: shared multi-producer/multi-consumer queue between clients """ def __init__(self, name=None, client=None, maxsize=0): self.client = client or _get_global_client() self.name = name or 'variable-' + uuid.uuid4().hex @gen.coroutine def _set(self, value): if isinstance(value, Future): yield self.client.scheduler.variable_set(key=tokey(value.key), name=self.name) else: yield self.client.scheduler.variable_set(data=value, name=self.name) def set(self, value, **kwargs): """ Set the value of this variable Parameters ---------- value: Future or object Must be either a Future or a msgpack-encodable value """ return self.client.sync(self._set, value, **kwargs) @gen.coroutine def _get(self, timeout=None): d = yield self.client.scheduler.variable_get(timeout=timeout, name=self.name, client=self.client.id) if d['type'] == 'Future': value = Future(d['value'], self.client, inform=True, state=d['state']) if d['state'] == 'erred': value._state.set_error(d['exception'], d['traceback']) self.client._send_to_scheduler({'op': 'variable-future-release', 'name': self.name, 'key': d['value'], 'token': d['token']}) else: value = d['value'] raise gen.Return(value) def get(self, timeout=None, **kwargs): """ Get the value of this variable """ return self.client.sync(self._get, timeout=timeout, **kwargs) def delete(self): """ Delete this variable Caution, this affects all clients currently pointing to this variable. """ if self.client.status == 'running': # TODO: can leave zombie futures self.client._send_to_scheduler({'op': 'variable_delete', 'name': self.name}) def __getstate__(self): return (self.name, self.client.scheduler.address) def __setstate__(self, state): name, address = state try: client = get_client(address) assert client.address == address except (AttributeError, AssertionError): client = Client(address, set_as_default=False) self.__init__(name=name, client=client) distributed-1.20.2/distributed/versions.py000066400000000000000000000043221321233345200206610ustar00rootroot00000000000000""" utilities for package version introspection """ from __future__ import print_function, division, absolute_import import platform import struct import os import sys import locale import importlib required_packages = [('dask', lambda p: p.__version__), ('distributed', lambda p: p.__version__), ('msgpack', lambda p: '.'.join([str(v) for v in p.version])), ('cloudpickle', lambda p: p.__version__), ('tornado', lambda p: p.version), ('toolz', lambda p: p.__version__)] optional_packages = [('numpy', lambda p: p.__version__), ('pandas', lambda p: p.__version__), ('bokeh', lambda p: p.__version__), ('lz4', lambda p: p.__version__), ('blosc', lambda p: p.__version__)] def get_versions(): """ Return basic information on our software installation, and out installed versions of packages. """ d = {'host': get_system_info(), 'packages': {'required': get_package_info(required_packages), 'optional': get_package_info(optional_packages)} } return d def get_system_info(): (sysname, nodename, release, version, machine, processor) = platform.uname() host = [("python", "%d.%d.%d.%s.%s" % sys.version_info[:]), ("python-bits", struct.calcsize("P") * 8), ("OS", "%s" % (sysname)), ("OS-release", "%s" % (release)), ("machine", "%s" % (machine)), ("processor", "%s" % (processor)), ("byteorder", "%s" % sys.byteorder), ("LC_ALL", "%s" % os.environ.get('LC_ALL', "None")), ("LANG", "%s" % os.environ.get('LANG', "None")), ("LOCALE", "%s.%s" % locale.getlocale()), ] return host def get_package_info(pkgs): """ get package versions for the passed required & optional packages """ pversions = [] for (modname, ver_f) in pkgs: try: mod = importlib.import_module(modname) ver = ver_f(mod) pversions.append((modname, ver)) except Exception: pversions.append((modname, None)) return pversions distributed-1.20.2/distributed/worker.py000066400000000000000000002753601321233345200203360ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import import bisect from collections import defaultdict, deque from datetime import timedelta import heapq import logging import os from pickle import PicklingError import random import threading import sys import weakref from dask.core import istask from dask.compatibility import apply try: from cytoolz import pluck except ImportError: from toolz import pluck from tornado.gen import Return from tornado import gen from tornado.ioloop import IOLoop from tornado.locks import Event from . import profile from .batched import BatchedSend from .comm import get_address_host, get_local_address_for from .comm.utils import offload from .config import config, log_format from .compatibility import unicode, get_thread_identity, finalize from .core import (error_message, CommClosedError, rpc, pingpong, coerce_to_address) from .diskutils import WorkSpace from .metrics import time from .node import ServerNode from .preloading import preload_modules from .proctitle import setproctitle from .protocol import (pickle, to_serialize, deserialize_bytes, serialize_bytelist) from .security import Security from .sizeof import safe_sizeof as sizeof from .threadpoolexecutor import ThreadPoolExecutor, secede as tpe_secede from .utils import (funcname, get_ip, has_arg, _maybe_complex, log_errors, ignoring, validate_key, mp_context, import_file, silence_logging, thread_state, json_load_robust, key_split, format_bytes, DequeHandler, ThrottledGC, PeriodicCallback, parse_bytes) from .utils_comm import pack_data, gather_from_workers _ncores = mp_context.cpu_count() logger = logging.getLogger(__name__) LOG_PDB = config.get('pdb-on-err') no_value = '--no-value-sentinel--' try: import psutil TOTAL_MEMORY = psutil.virtual_memory().total except ImportError: logger.warning("Please install psutil to estimate worker memory use") TOTAL_MEMORY = 8e9 psutil = None IN_PLAY = ('waiting', 'ready', 'executing', 'long-running') PENDING = ('waiting', 'ready', 'constrained') PROCESSING = ('waiting', 'ready', 'constrained', 'executing', 'long-running') READY = ('ready', 'constrained') _global_workers = [] class WorkerBase(ServerNode): def __init__(self, scheduler_ip=None, scheduler_port=None, scheduler_file=None, ncores=None, loop=None, local_dir=None, services=None, service_ports=None, name=None, reconnect=True, memory_limit='auto', executor=None, resources=None, silence_logs=None, death_timeout=None, preload=(), security=None, contact_address=None, memory_monitor_interval=200, **kwargs): self._setup_logging() if scheduler_file: cfg = json_load_robust(scheduler_file) scheduler_addr = cfg['address'] elif scheduler_port is None: scheduler_addr = coerce_to_address(scheduler_ip) else: scheduler_addr = coerce_to_address((scheduler_ip, scheduler_port)) self._port = 0 self.ncores = ncores or _ncores self.total_resources = resources or {} self.available_resources = (resources or {}).copy() self.death_timeout = death_timeout self.preload = preload self.contact_address = contact_address self.memory_monitor_interval = memory_monitor_interval if silence_logs: silence_logging(level=silence_logs) if local_dir: local_dir = os.path.abspath(local_dir) else: local_dir = 'dask-worker-space' self._workspace = WorkSpace(local_dir) self._workdir = self._workspace.new_work_dir(prefix='worker-') self.local_dir = self._workdir.dir_path 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.memory_limit = parse_memory_limit(memory_limit, self.ncores) self.paused = False if 'memory_target_fraction' in kwargs: self.memory_target_fraction = kwargs.pop('memory_target_fraction') else: self.memory_target_fraction = config.get('worker-memory-target', 0.6) if 'memory_spill_fraction' in kwargs: self.memory_spill_fraction = kwargs.pop('memory_spill_fraction') else: self.memory_spill_fraction = config.get('worker-memory-spill', 0.7) if 'memory_pause_fraction' in kwargs: self.memory_pause_fraction = kwargs.pop('memory_pause_fraction') else: self.memory_pause_fraction = config.get('worker-memory-pause', 0.8) if self.memory_limit: try: from zict import Buffer, File, Func except ImportError: raise ImportError("Please `pip install zict` for spill-to-disk workers") path = os.path.join(self.local_dir, 'storage') storage = Func(serialize_bytelist, deserialize_bytes, File(path)) target = int(float(self.memory_limit) * self.memory_target_fraction) self.data = Buffer({}, storage, target, weight) else: self.data = dict() self.loop = loop or IOLoop.current() self.status = None self._closed = Event() self.reconnect = reconnect self.executor = executor or ThreadPoolExecutor(self.ncores) self.scheduler = rpc(scheduler_addr, connection_args=self.connection_args) self.name = name self.scheduler_delay = 0 self.heartbeat_active = False self.execution_state = {'scheduler': self.scheduler.address, 'ioloop': self.loop, 'worker': self} self._ipython_kernel = None if self.local_dir not in sys.path: sys.path.insert(0, self.local_dir) self.services = {} self.service_ports = service_ports or {} self.service_specs = services or {} handlers = { 'gather': self.gather, 'compute-stream': self.compute_stream, 'run': self.run, 'run_coroutine': self.run_coroutine, 'get_data': self.get_data, 'update_data': self.update_data, 'delete_data': self.delete_data, 'terminate': self.terminate, 'ping': pingpong, 'upload_file': self.upload_file, 'start_ipython': self.start_ipython, 'call_stack': self.get_call_stack, 'profile': self.get_profile, 'profile_metadata': self.get_profile_metadata, 'get_logs': self.get_logs, 'keys': self.keys, } super(WorkerBase, self).__init__(handlers, io_loop=self.loop, connection_args=self.connection_args, **kwargs) pc = PeriodicCallback(self.heartbeat, 1000) self.periodic_callbacks['heartbeat'] = pc self._address = contact_address if self.memory_limit: self._memory_monitoring = False pc = PeriodicCallback(self.memory_monitor, self.memory_monitor_interval) self.periodic_callbacks['memory'] = pc self._throttled_gc = ThrottledGC(logger=logger) setproctitle("dask-worker [not started]") def _setup_logging(self): self._deque_handler = DequeHandler(n=config.get('log-length', 10000)) self._deque_handler.setFormatter(logging.Formatter(log_format)) logger.addHandler(self._deque_handler) finalize(self, logger.removeHandler, self._deque_handler) @property def worker_address(self): """ For API compatibility with Nanny """ return self.address @gen.coroutine def heartbeat(self): if not self.heartbeat_active: self.heartbeat_active = True logger.debug("Heartbeat: %s" % self.address) try: start = time() response = yield self.scheduler.register( address=self.contact_address, name=self.name, ncores=self.ncores, now=time(), services=self.service_ports, memory_limit=self.memory_limit, executing=len(self.executing), in_memory=len(self.data), ready=len(self.ready), in_flight=len(self.in_flight_tasks), **self.monitor.recent()) end = time() middle = (start + end) / 2 self.scheduler_delay = response['time'] - middle self.periodic_callbacks['heartbeat'].callback_time = response['heartbeat-interval'] * 1000 finally: self.heartbeat_active = False else: logger.debug("Heartbeat skipped: channel busy") @gen.coroutine def _register_with_scheduler(self): self.periodic_callbacks['heartbeat'].stop() start = time() if self.contact_address is None: self.contact_address = self.address while True: if self.death_timeout and time() > start + self.death_timeout: yield self._close(timeout=1) return if self.status in ('closed', 'closing'): raise gen.Return try: _start = time() future = self.scheduler.register( ncores=self.ncores, address=self.contact_address, keys=list(self.data), name=self.name, nbytes=self.nbytes, now=time(), services=self.service_ports, memory_limit=self.memory_limit, local_directory=self.local_dir, resources=self.total_resources, pid=os.getpid(), **self.monitor.recent()) if self.death_timeout: diff = self.death_timeout - (time() - start) future = gen.with_timeout(timedelta(seconds=diff), future) response = yield future _end = time() middle = (_start + _end) / 2 self.scheduler_delay = response['time'] - middle self.status = 'running' break except EnvironmentError: logger.info("Trying to connect to scheduler: %s" % str(self.scheduler.address)) yield gen.sleep(0.1) except gen.TimeoutError: pass if response['status'] != 'OK': raise ValueError("Unexpected response from register: %r" % (response,)) self.periodic_callbacks['heartbeat'].start() def start_services(self, listen_ip=''): for k, v in self.service_specs.items(): if isinstance(k, tuple): k, port = k else: port = 0 if isinstance(v, tuple): v, kwargs = v else: v, kwargs = v, {} self.services[k] = v(self, io_loop=self.loop, **kwargs) self.services[k].listen((listen_ip, port)) self.service_ports[k] = self.services[k].port @gen.coroutine def _start(self, addr_or_port=0): assert self.status is None # XXX Factor this out if not addr_or_port: # Default address is the required one to reach the scheduler listen_host = get_address_host(self.scheduler.address) self.listen(get_local_address_for(self.scheduler.address), listen_args=self.listen_args) self.ip = get_address_host(self.address) elif isinstance(addr_or_port, int): # addr_or_port is an integer => assume TCP listen_host = self.ip = get_ip( get_address_host(self.scheduler.address) ) self.listen((listen_host, addr_or_port), listen_args=self.listen_args) else: self.listen(addr_or_port, listen_args=self.listen_args) self.ip = get_address_host(self.address) try: listen_host = get_address_host(addr_or_port) except ValueError: listen_host = addr_or_port if '://' in listen_host: protocol, listen_host = listen_host.split('://') self.name = self.name or self.address preload_modules(self.preload, parameter=self, file_dir=self.local_dir) # Services listen on all addresses # Note Nanny is not a "real" service, just some metadata # passed in service_ports... self.start_services(listen_host) try: listening_address = '%s%s:%d' % (self.listener.prefix, listen_host, self.port) except Exception: listening_address = '%s%s' % (self.listener.prefix, listen_host) logger.info(' Start worker at: %26s', self.address) logger.info(' Listening to: %26s', listening_address) for k, v in self.service_ports.items(): logger.info(' %16s at: %26s' % (k, listen_host + ':' + str(v))) logger.info('Waiting to connect to: %26s', self.scheduler.address) logger.info('-' * 49) logger.info(' Threads: %26d', self.ncores) if self.memory_limit: logger.info(' Memory: %26s', format_bytes(self.memory_limit)) logger.info(' Local Directory: %26s', self.local_dir) logger.info('-' * 49) setproctitle("dask-worker [%s]" % self.address) yield self._register_with_scheduler() if self.status == 'running': logger.info(' Registered to: %26s', self.scheduler.address) logger.info('-' * 49) self.start_periodic_callbacks() def start(self, port=0): self.loop.add_callback(self._start, port) def identity(self, comm): return {'type': type(self).__name__, 'id': self.id, 'scheduler': self.scheduler.address, 'ncores': self.ncores, 'memory_limit': self.memory_limit} @gen.coroutine def _close(self, report=True, timeout=10, nanny=True): if self.status in ('closed', 'closing'): return logger.info("Stopping worker at %s", self.address) self.status = 'closing' setproctitle("dask-worker [closing]") self.stop() for pc in self.periodic_callbacks.values(): pc.stop() with ignoring(EnvironmentError, gen.TimeoutError): if report: yield gen.with_timeout(timedelta(seconds=timeout), self.scheduler.unregister(address=self.contact_address)) self.scheduler.close_rpc() if isinstance(self.executor, ThreadPoolExecutor): self.executor.shutdown(timeout=timeout) else: self.executor.shutdown(wait=False) self._workdir.release() for k, v in self.services.items(): v.stop() self.status = 'closed' if nanny and 'nanny' in self.service_ports: with self.rpc((self.ip, self.service_ports['nanny'])) as r: yield r.terminate() self.rpc.close() self._closed.set() self._remove_from_global_workers() yield super(WorkerBase, self).close() setproctitle("dask-worker [closed]") def __del__(self): self._remove_from_global_workers() def _remove_from_global_workers(self): for ref in list(_global_workers): if ref() is self: _global_workers.remove(ref) if ref() is None: _global_workers.remove(ref) @gen.coroutine def terminate(self, comm, report=True): yield self._close(report=report) raise Return('OK') @gen.coroutine def wait_until_closed(self): yield self._closed.wait() assert self.status == 'closed' @gen.coroutine def executor_submit(self, key, function, *args, **kwargs): """ Safely run function in thread pool executor We've run into issues running concurrent.future futures within tornado. Apparently it's advantageous to use timeouts and periodic callbacks to ensure things run smoothly. This can get tricky, so we pull it off into an separate method. """ job_counter[0] += 1 # logger.info("%s:%d Starts job %d, %s", self.ip, self.port, i, key) future = self.executor.submit(function, *args, **kwargs) pc = PeriodicCallback(lambda: logger.debug("future state: %s - %s", key, future._state), 1000) pc.start() try: yield future finally: pc.stop() result = future.result() # logger.info("Finish job %d, %s", i, key) raise gen.Return(result) def run(self, comm, function, args=(), kwargs={}): return run(self, comm, function=function, args=args, kwargs=kwargs) def run_coroutine(self, comm, function, args=(), kwargs={}, wait=True): return run(self, comm, function=function, args=args, kwargs=kwargs, is_coro=True, wait=wait) def update_data(self, comm=None, data=None, report=True): for key, value in data.items(): if key in self.task_state: self.transition(key, 'memory', value=value) else: self.put_key_in_memory(key, value) self.task_state[key] = 'memory' self.tasks[key] = None self.priorities[key] = None self.durations[key] = None self.dependencies[key] = set() if key in self.dep_state: self.transition_dep(key, 'memory', value=value) self.log.append((key, 'receive-from-scatter')) if report: self.batched_stream.send({'op': 'add-keys', 'keys': list(data)}) info = {'nbytes': {k: sizeof(v) for k, v in data.items()}, 'status': 'OK'} return info @gen.coroutine def delete_data(self, comm=None, keys=None, report=True): if keys: for key in list(keys): self.log.append((key, 'delete')) if key in self.task_state: self.release_key(key) if key in self.dep_state: self.release_dep(key) logger.debug("Deleted %d keys", len(keys)) if report: logger.debug("Reporting loss of keys to scheduler") # TODO: this route seems to not exist? yield self.scheduler.remove_keys(address=self.contact_address, keys=list(keys)) raise Return('OK') @gen.coroutine def get_data(self, comm, keys=None, who=None): start = time() msg = {k: to_serialize(self.data[k]) for k in keys if k in self.data} nbytes = {k: self.nbytes.get(k) for k in keys if k in self.data} stop = time() if self.digests is not None: self.digests['get-data-load-duration'].add(stop - start) start = time() try: compressed = yield comm.write(msg) except EnvironmentError: logger.exception('failed during get data', exc_info=True) comm.abort() raise stop = time() if self.digests is not None: self.digests['get-data-send-duration'].add(stop - start) total_bytes = sum(filter(None, nbytes.values())) self.outgoing_count += 1 duration = (stop - start) or 0.5 # windows self.outgoing_transfer_log.append({ 'start': start + self.scheduler_delay, 'stop': stop + self.scheduler_delay, 'middle': (start + stop) / 2, 'duration': duration, 'who': who, 'keys': nbytes, 'total': total_bytes, 'compressed': compressed, 'bandwidth': total_bytes / duration }) raise gen.Return('dont-reply') @gen.coroutine def set_resources(self, **resources): for r, quantity in resources.items(): if r in self.total_resources: self.available_resources[r] += quantity - self.total_resources[r] else: self.available_resources[r] = quantity self.total_resources[r] = quantity yield self.scheduler.set_resources(resources=self.total_resources, worker=self.contact_address) def start_ipython(self, comm): """Start an IPython kernel Returns Jupyter connection info dictionary. """ from ._ipython_utils import start_ipython if self._ipython_kernel is None: self._ipython_kernel = start_ipython( ip=self.ip, ns={'worker': self}, log=logger, ) return self._ipython_kernel.get_connection_info() @gen.coroutine def upload_file(self, comm, filename=None, data=None, load=True): out_filename = os.path.join(self.local_dir, filename) def func(data): if isinstance(data, unicode): data = data.encode() with open(out_filename, 'wb') as f: f.write(data) f.flush() return data if len(data) < 10000: data = func(data) else: data = yield offload(func, data) if load: try: import_file(out_filename) except Exception as e: logger.exception(e) raise gen.Return({'status': 'error', 'exception': pickle.dumps(e)}) raise gen.Return({'status': 'OK', 'nbytes': len(data)}) def keys(self, comm=None): return list(self.data) @gen.coroutine def gather(self, comm=None, who_has=None): who_has = {k: [coerce_to_address(addr) for addr in v] for k, v in who_has.items() if k not in self.data} result, missing_keys, missing_workers = yield gather_from_workers( who_has, rpc=self.rpc) if missing_keys: logger.warning("Could not find data: %s on workers: %s (who_has: %s)", missing_keys, missing_workers, who_has) raise Return({'status': 'missing-data', 'keys': missing_keys}) else: self.update_data(data=result, report=False) raise Return({'status': 'OK'}) job_counter = [0] def _deserialize(function=None, args=None, kwargs=None, task=None): """ Deserialize task inputs and regularize to func, args, kwargs """ if function is not None: function = pickle.loads(function) if args: args = pickle.loads(args) if kwargs: kwargs = pickle.loads(kwargs) if task is not None: assert not function and not args and not kwargs function = execute_task args = (task,) return function, args or (), kwargs or {} def execute_task(task): """ Evaluate a nested task >>> inc = lambda x: x + 1 >>> execute_task((inc, 1)) 2 >>> execute_task((sum, [1, 2, (inc, 3)])) 7 """ if istask(task): func, args = task[0], task[1:] return func(*map(execute_task, args)) elif isinstance(task, list): return list(map(execute_task, task)) else: return task cache = dict() def dumps_function(func): """ Dump a function to bytes, cache functions """ if func not in cache: b = pickle.dumps(func) if len(b) < 100000: cache[func] = b else: return b return cache[func] def dumps_task(task): """ Serialize a dask task Returns a dict of bytestrings that can each be loaded with ``loads`` Examples -------- Either returns a task as a function, args, kwargs dict >>> from operator import add >>> dumps_task((add, 1)) # doctest: +SKIP {'function': b'\x80\x04\x95\x00\x8c\t_operator\x94\x8c\x03add\x94\x93\x94.' 'args': b'\x80\x04\x95\x07\x00\x00\x00K\x01K\x02\x86\x94.'} Or as a single task blob if it can't easily decompose the result. This happens either if the task is highly nested, or if it isn't a task at all >>> dumps_task(1) # doctest: +SKIP {'task': b'\x80\x04\x95\x03\x00\x00\x00\x00\x00\x00\x00K\x01.'} """ if istask(task): if task[0] is apply and not any(map(_maybe_complex, task[2:])): d = {'function': dumps_function(task[1]), 'args': pickle.dumps(task[2])} if len(task) == 4: d['kwargs'] = pickle.dumps(task[3]) return d elif not any(map(_maybe_complex, task[1:])): return {'function': dumps_function(task[0]), 'args': pickle.dumps(task[1:])} return to_serialize(task) def apply_function(function, args, kwargs, execution_state, key, active_threads, active_threads_lock, time_delay): """ Run a function, collect information Returns ------- msg: dictionary with status, result/error, timings, etc.. """ ident = get_thread_identity() with active_threads_lock: active_threads[ident] = key thread_state.start_time = time() thread_state.execution_state = execution_state thread_state.key = key start = time() try: result = function(*args, **kwargs) except Exception as e: msg = error_message(e) msg['op'] = 'task-erred' msg['actual-exception'] = e else: msg = {'op': 'task-finished', 'status': 'OK', 'result': result, 'nbytes': sizeof(result), 'type': type(result) if result is not None else None} finally: end = time() msg['start'] = start + time_delay msg['stop'] = end + time_delay msg['thread'] = ident with active_threads_lock: del active_threads[ident] return msg def get_msg_safe_str(msg): """ Make a worker msg, which contains args and kwargs, safe to cast to str: allowing for some arguments to raise exceptions during conversion and ignoring them. """ class Repr(object): def __init__(self, f, val): self._f = f self._val = val def __repr__(self): return self._f(self._val) msg = msg.copy() if "args" in msg: msg["args"] = Repr(convert_args_to_str, msg["args"]) if "kwargs" in msg: msg["kwargs"] = Repr(convert_kwargs_to_str, msg["kwargs"]) return msg def convert_args_to_str(args, max_len=None): """ Convert args to a string, allowing for some arguments to raise exceptions during conversion and ignoring them. """ length = 0 strs = ["" for i in range(len(args))] for i, arg in enumerate(args): try: sarg = repr(arg) except Exception: sarg = "< could not convert arg to str >" strs[i] = sarg length += len(sarg) + 2 if max_len is not None and length > max_len: return "({}".format(", ".join(strs[:i + 1]))[:max_len] else: return "({})".format(", ".join(strs)) def convert_kwargs_to_str(kwargs, max_len=None): """ Convert kwargs to a string, allowing for some arguments to raise exceptions during conversion and ignoring them. """ length = 0 strs = ["" for i in range(len(kwargs))] for i, (argname, arg) in enumerate(kwargs.items()): try: sarg = repr(arg) except Exception: sarg = "< could not convert arg to str >" skwarg = repr(argname) + ": " + sarg strs[i] = skwarg length += len(skwarg) + 2 if max_len is not None and length > max_len: return "{{{}".format(", ".join(strs[:i + 1]))[:max_len] else: return "{{{}}}".format(", ".join(strs)) def weight(k, v): return sizeof(v) @gen.coroutine def run(server, comm, function, args=(), kwargs={}, is_coro=False, wait=True): assert wait or is_coro, "Combination not supported" function = pickle.loads(function) if args: args = pickle.loads(args) if kwargs: kwargs = pickle.loads(kwargs) if has_arg(function, 'dask_worker'): kwargs['dask_worker'] = server if has_arg(function, 'dask_scheduler'): kwargs['dask_scheduler'] = server logger.info("Run out-of-band function %r", funcname(function)) try: result = function(*args, **kwargs) if is_coro: result = (yield result) if wait else None except Exception as e: logger.warning(" Run Failed\n" "Function: %s\n" "args: %s\n" "kwargs: %s\n", str(funcname(function))[:1000], convert_args_to_str(args, max_len=1000), convert_kwargs_to_str(kwargs, max_len=1000), exc_info=True) response = error_message(e) else: response = { 'status': 'OK', 'result': to_serialize(result), } raise Return(response) class Worker(WorkerBase): """ Worker node in a Dask distributed cluster Workers perform two functions: 1. **Serve data** from a local dictionary 2. **Perform computation** on that data and on data from peers Workers keep the scheduler informed of their data and use that scheduler to gather data from other workers when necessary to perform a computation. You can start a worker with the ``dask-worker`` command line application:: $ dask-worker scheduler-ip:port Use the ``--help`` flag to see more options $ dask-worker --help The rest of this docstring is about the internal state the the worker uses to manage and track internal computations. **State** **Informational State** These attributes don't change significantly during execution. * **ncores:** ``int``: Number of cores used by this worker process * **executor:** ``concurrent.futures.ThreadPoolExecutor``: Executor used to perform computation * **local_dir:** ``path``: Path on local machine to store temporary files * **scheduler:** ``rpc``: Location of scheduler. See ``.ip/.port`` attributes. * **name:** ``string``: Alias * **services:** ``{str: Server}``: Auxiliary web servers running on this worker * **service_ports:** ``{str: port}``: * **total_connections**: ``int`` The maximum number of concurrent connections we want to see * **total_comm_nbytes**: ``int`` * **batched_stream**: ``BatchedSend`` A batched stream along which we communicate to the scheduler * **log**: ``[(message)]`` A structured and queryable log. See ``Worker.story`` **Volatile State** This attributes track the progress of tasks that this worker is trying to complete. In the descriptions below a ``key`` is the name of a task that we want to compute and ``dep`` is the name of a piece of dependent data that we want to collect from others. * **data:** ``{key: object}``: Dictionary mapping keys to actual values * **task_state**: ``{key: string}``: The state of all tasks that the scheduler has asked us to compute. Valid states include waiting, constrained, exeucuting, memory, erred * **tasks**: ``{key: dict}`` The function, args, kwargs of a task. We run this when appropriate * **dependencies**: ``{key: {deps}}`` The data needed by this key to run * **dependents**: ``{dep: {keys}}`` The keys that use this dependency * **data_needed**: deque(keys) The keys whose data we still lack, arranged in a deque * **waiting_for_data**: ``{kep: {deps}}`` A dynamic verion of dependencies. All dependencies that we still don't have for a particular key. * **ready**: [keys] Keys that are ready to run. Stored in a LIFO stack * **constrained**: [keys] Keys for which we have the data to run, but are waiting on abstract resources like GPUs. Stored in a FIFO deque * **executing**: {keys} Keys that are currently executing * **executed_count**: int A number of tasks that this worker has run in its lifetime * **long_running**: {keys} A set of keys of tasks that are running and have started their own long-running clients. * **dep_state**: ``{dep: string}``: The state of all dependencies required by our tasks Valid states include waiting, flight, and memory * **who_has**: ``{dep: {worker}}`` Workers that we believe have this data * **has_what**: ``{worker: {deps}}`` The data that we care about that we think a worker has * **pending_data_per_worker**: ``{worker: [dep]}`` The data on each worker that we still want, prioritized as a deque * **in_flight_tasks**: ``{task: worker}`` All dependencies that are coming to us in current peer-to-peer connections and the workers from which they are coming. * **in_flight_workers**: ``{worker: {task}}`` The workers from which we are currently gathering data and the dependencies we expect from those connections * **comm_bytes**: ``int`` The total number of bytes in flight * **suspicious_deps**: ``{dep: int}`` The number of times a dependency has not been where we expected it * **nbytes**: ``{key: int}`` The size of a particular piece of data * **types**: ``{key: type}`` The type of a particular piece of data * **threads**: ``{key: int}`` The ID of the thread on which the task ran * **active_threads**: ``{int: key}`` The keys currently running on active threads * **exceptions**: ``{key: exception}`` The exception caused by running a task if it erred * **tracebacks**: ``{key: traceback}`` The exception caused by running a task if it erred * **startstops**: ``{key: [(str, float, float)]}`` Log of transfer, load, and compute times for a task * **priorities**: ``{key: tuple}`` The priority of a key given by the scheduler. Determines run order. * **durations**: ``{key: float}`` Expected duration of a task * **resource_restrictions**: ``{key: {str: number}}`` Abstract resources required to run a task Parameters ---------- scheduler_ip: str scheduler_port: int ip: str, optional ncores: int, optional loop: tornado.ioloop.IOLoop local_dir: str, optional Directory where we place local resources name: str, optional heartbeat_interval: int Milliseconds between heartbeats to scheduler memory_limit: int, float, string Number of bytes of memory that this worker should use. Set to zero for no limit. Set to 'auto' for 60% of memory use. Use strings or numbers like 5GB or 5e9 memory_target_fraction: float Fraction of memory to try to stay beneath memory_spill_fraction: float Fraction of memory at which we start spilling to disk memory_pause_fraction: float Fraction of memory at which we stop running new tasks executor: concurrent.futures.Executor resources: dict Resources that thiw worker has like ``{'GPU': 2}`` Examples -------- Use the command line to start a worker:: $ dask-scheduler Start scheduler at 127.0.0.1:8786 $ dask-worker 127.0.0.1:8786 Start worker at: 127.0.0.1:1234 Registered with scheduler at: 127.0.0.1:8786 See Also -------- distributed.scheduler.Scheduler distributed.nanny.Nanny """ def __init__(self, *args, **kwargs): self.tasks = dict() self.task_state = dict() self.dep_state = dict() self.dependencies = dict() self.dependents = dict() self.waiting_for_data = dict() self.who_has = dict() self.has_what = defaultdict(set) self.pending_data_per_worker = defaultdict(deque) self.extensions = {} self._lock = threading.Lock() self.data_needed = deque() # TODO: replace with heap? self.in_flight_tasks = dict() self.in_flight_workers = dict() self.total_connections = 50 self.total_comm_nbytes = 10e6 self.comm_nbytes = 0 self.suspicious_deps = defaultdict(lambda: 0) self._missing_dep_flight = set() self.nbytes = dict() self.types = dict() self.threads = dict() self.exceptions = dict() self.tracebacks = dict() self.active_threads_lock = threading.Lock() self.active_threads = dict() self.profile_keys = defaultdict(profile.create) self.profile_keys_history = deque(maxlen=3600) self.profile_recent = profile.create() self.profile_history = deque(maxlen=3600) self.priorities = dict() self.priority_counter = 0 self.durations = dict() self.startstops = defaultdict(list) self.resource_restrictions = dict() self.ready = list() self.constrained = deque() self.executing = set() self.executed_count = 0 self.long_running = set() self.batched_stream = None self.recent_messages_log = deque(maxlen=10000) self.target_message_size = 50e6 # 50 MB self.log = deque(maxlen=100000) self.validate = kwargs.pop('validate', False) self._transitions = { ('waiting', 'ready'): self.transition_waiting_ready, ('waiting', 'memory'): self.transition_waiting_done, ('waiting', 'error'): self.transition_waiting_done, ('ready', 'executing'): self.transition_ready_executing, ('ready', 'memory'): self.transition_ready_memory, ('constrained', 'executing'): self.transition_constrained_executing, ('executing', 'memory'): self.transition_executing_done, ('executing', 'error'): self.transition_executing_done, ('executing', 'rescheduled'): self.transition_executing_done, ('executing', 'long-running'): self.transition_executing_long_running, ('long-running', 'error'): self.transition_executing_done, ('long-running', 'memory'): self.transition_executing_done, ('long-running', 'rescheduled'): self.transition_executing_done, } self._dep_transitions = { ('waiting', 'flight'): self.transition_dep_waiting_flight, ('waiting', 'memory'): self.transition_dep_waiting_memory, ('flight', 'waiting'): self.transition_dep_flight_waiting, ('flight', 'memory'): self.transition_dep_flight_memory, } self.incoming_transfer_log = deque(maxlen=(100000)) self.incoming_count = 0 self.outgoing_transfer_log = deque(maxlen=(100000)) self.outgoing_count = 0 self._client = None profile_cycle_interval = kwargs.pop('profile_cycle_interval', config.get('profile-cycle-interval', 1000)) WorkerBase.__init__(self, *args, **kwargs) pc = PeriodicCallback(self.trigger_profile, config.get('profile-interval', 10)) self.periodic_callbacks['profile'] = pc pc = PeriodicCallback(self.cycle_profile, profile_cycle_interval) self.periodic_callbacks['profile-cycle'] = pc _global_workers.append(weakref.ref(self)) def __repr__(self): return "<%s: %s, %s, stored: %d, running: %d/%d, ready: %d, comm: %d, waiting: %d>" % ( self.__class__.__name__, self.address, self.status, len(self.data), len(self.executing), self.ncores, len(self.ready), len(self.in_flight_tasks), len(self.waiting_for_data)) ################ # Update Graph # ################ @gen.coroutine def compute_stream(self, comm): try: self.batched_stream = BatchedSend(interval=2, loop=self.loop) self.batched_stream.start(comm) def on_closed(): if self.reconnect and self.status not in ('closed', 'closing'): logger.info("Connection to scheduler broken. Reregistering") self._register_with_scheduler() else: self._close(report=False) closed = False while not closed: try: msgs = yield comm.read() except CommClosedError: on_closed() break except EnvironmentError as e: break except Exception as e: logger.error("Worker failed to read message. " "This will likely cause the cluster to fail.", exc_info=True) raise start = time() for msg in msgs: self.recent_messages_log.append(msg) op = msg.pop('op', None) if 'key' in msg: validate_key(msg['key']) if op == 'close': closed = True self._close() break elif op == 'compute-task': self.add_task(**msg) elif op == 'release-task': self.log.append((msg['key'], 'release-task', msg.get('reason'))) self.release_key(report=False, **msg) elif op == 'delete-data': self.delete_data(**msg) elif op == 'steal-request': self.steal_request(**msg) else: logger.warning("Unknown operation %s, %s", op, msg) self.priority_counter -= 1 self.ensure_communicating() self.ensure_computing() end = time() if self.digests is not None: self.digests['handle-messages-duration'].add(end - start) yield self.batched_stream.close() logger.info('Close compute stream') except Exception as e: logger.exception(e) raise def add_task(self, key, function=None, args=None, kwargs=None, task=None, who_has=None, nbytes=None, priority=None, duration=None, resource_restrictions=None, **kwargs2): if isinstance(priority, list): priority.insert(1, self.priority_counter) try: if key in self.tasks: state = self.task_state[key] if state in ('memory', 'error'): if state == 'memory': assert key in self.data logger.debug("Asked to compute pre-existing result: %s: %s", key, state) self.send_task_state_to_scheduler(key) return if state in IN_PLAY: return if self.dep_state.get(key) == 'memory': self.task_state[key] = 'memory' self.send_task_state_to_scheduler(key) self.tasks[key] = None self.log.append((key, 'new-task-already-in-memory')) self.priorities[key] = priority self.durations[key] = duration return self.log.append((key, 'new')) try: start = time() self.tasks[key] = _deserialize(function, args, kwargs, task) stop = time() if stop - start > 0.010: self.startstops[key].append(('deserialize', start, stop)) except Exception as e: logger.warning("Could not deserialize task", exc_info=True) emsg = error_message(e) emsg['key'] = key emsg['op'] = 'task-erred' self.batched_stream.send(emsg) self.log.append((key, 'deserialize-error')) return self.priorities[key] = priority self.durations[key] = duration if resource_restrictions: self.resource_restrictions[key] = resource_restrictions self.task_state[key] = 'waiting' if nbytes is not None: self.nbytes.update(nbytes) who_has = who_has or {} self.dependencies[key] = set(who_has) self.waiting_for_data[key] = set() for dep in who_has: if dep not in self.dependents: self.dependents[dep] = set() self.dependents[dep].add(key) if dep not in self.dep_state: if self.task_state.get(dep) == 'memory': state = 'memory' else: state = 'waiting' self.dep_state[dep] = state self.log.append((dep, 'new-dep', state)) if self.dep_state[dep] != 'memory': self.waiting_for_data[key].add(dep) for dep, workers in who_has.items(): assert workers if dep not in self.who_has: self.who_has[dep] = set(workers) self.who_has[dep].update(workers) for worker in workers: self.has_what[worker].add(dep) if self.dep_state[dep] != 'memory': self.pending_data_per_worker[worker].append(dep) if self.waiting_for_data[key]: self.data_needed.append(key) else: self.transition(key, 'ready') if self.validate: if who_has: assert all(dep in self.dep_state for dep in who_has) assert all(dep in self.nbytes for dep in who_has) for dep in who_has: self.validate_dep(dep) self.validate_key(key) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise ############### # Transitions # ############### def transition_dep(self, dep, finish, **kwargs): try: start = self.dep_state[dep] except KeyError: return if start == finish: return func = self._dep_transitions[start, finish] state = func(dep, **kwargs) self.log.append(('dep', dep, start, state or finish)) if dep in self.dep_state: self.dep_state[dep] = state or finish if self.validate: self.validate_dep(dep) def transition_dep_waiting_flight(self, dep, worker=None): try: if self.validate: assert dep not in self.in_flight_tasks assert self.dependents[dep] self.in_flight_tasks[dep] = worker except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_dep_flight_waiting(self, dep, worker=None): try: if self.validate: assert dep in self.in_flight_tasks del self.in_flight_tasks[dep] try: self.who_has[dep].remove(worker) except KeyError: pass try: self.has_what[worker].remove(dep) except KeyError: pass if not self.who_has.get(dep): if dep not in self._missing_dep_flight: self._missing_dep_flight.add(dep) self.loop.add_callback(self.handle_missing_dep, dep) for key in self.dependents.get(dep, ()): if self.task_state[key] == 'waiting': self.data_needed.appendleft(key) if not self.dependents[dep]: self.release_dep(dep) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_dep_flight_memory(self, dep, value=None): try: if self.validate: assert dep in self.in_flight_tasks del self.in_flight_tasks[dep] if self.dependents[dep]: self.dep_state[dep] = 'memory' self.put_key_in_memory(dep, value) self.batched_stream.send({'op': 'add-keys', 'keys': [dep]}) else: self.release_dep(dep) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_dep_waiting_memory(self, dep, value=None): try: if self.validate: try: assert dep in self.data assert dep in self.nbytes assert dep in self.types assert self.task_state[dep] == 'memory' except Exception as e: logger.exception(e) import pdb pdb.set_trace() except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition(self, key, finish, **kwargs): start = self.task_state[key] if start == finish: return func = self._transitions[start, finish] state = func(key, **kwargs) self.log.append((key, start, state or finish)) self.task_state[key] = state or finish if self.validate: self.validate_key(key) def transition_waiting_ready(self, key): try: if self.validate: assert self.task_state[key] == 'waiting' assert key in self.waiting_for_data assert not self.waiting_for_data[key] assert all(dep in self.data for dep in self.dependencies[key]) assert key not in self.executing assert key not in self.ready self.waiting_for_data.pop(key, None) if key in self.resource_restrictions: self.constrained.append(key) return 'constrained' else: heapq.heappush(self.ready, (self.priorities[key], key)) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_waiting_done(self, key, value=None): try: if self.validate: assert self.task_state[key] == 'waiting' assert key in self.waiting_for_data assert key not in self.executing assert key not in self.ready del self.waiting_for_data[key] self.send_task_state_to_scheduler(key) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_ready_executing(self, key): try: if self.validate: assert key not in self.waiting_for_data # assert key not in self.data assert self.task_state[key] in READY assert key not in self.ready assert all(dep in self.data for dep in self.dependencies[key]) self.executing.add(key) self.loop.add_callback(self.execute, key) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_ready_memory(self, key, value=None): self.send_task_state_to_scheduler(key) def transition_constrained_executing(self, key): self.transition_ready_executing(key) for resource, quantity in self.resource_restrictions[key].items(): self.available_resources[resource] -= quantity if self.validate: assert all(v >= 0 for v in self.available_resources.values()) def transition_executing_done(self, key, value=no_value, report=True): try: if self.validate: assert key in self.executing or key in self.long_running assert key not in self.waiting_for_data assert key not in self.ready out = None if key in self.resource_restrictions: for resource, quantity in self.resource_restrictions[key].items(): self.available_resources[resource] += quantity if self.task_state[key] == 'executing': self.executing.remove(key) self.executed_count += 1 elif self.task_state[key] == 'long-running': self.long_running.remove(key) if value is not no_value: try: self.task_state[key] = 'memory' self.put_key_in_memory(key, value, transition=False) except Exception as e: logger.info("Failed to put key in memory", exc_info=True) msg = error_message(e) self.exceptions[key] = msg['exception'] self.tracebacks[key] = msg['traceback'] self.task_state[key] = 'error' out = 'error' if key in self.dep_state: self.transition_dep(key, 'memory') if report and self.batched_stream: self.send_task_state_to_scheduler(key) else: raise CommClosedError return out except EnvironmentError: logger.info("Comm closed") except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def transition_executing_long_running(self, key, compute_duration=None): try: if self.validate: assert key in self.executing self.executing.remove(key) self.long_running.add(key) self.batched_stream.send({'op': 'long-running', 'key': key, 'compute_duration': compute_duration}) self.ensure_computing() except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def maybe_transition_long_running(self, key, compute_duration=None): if self.task_state.get(key) == 'executing': self.transition(key, 'long-running', compute_duration=compute_duration) ########################## # Gather Data from Peers # ########################## def ensure_communicating(self): changed = True try: while changed and self.data_needed and len(self.in_flight_workers) < self.total_connections: changed = False logger.debug("Ensure communicating. Pending: %d. Connections: %d/%d", len(self.data_needed), len(self.in_flight_workers), self.total_connections) key = self.data_needed[0] if key not in self.tasks: self.data_needed.popleft() changed = True continue if self.task_state.get(key) != 'waiting': self.log.append((key, 'communication pass')) self.data_needed.popleft() changed = True continue deps = self.dependencies[key] if self.validate: assert all(dep in self.dep_state for dep in deps) deps = [dep for dep in deps if self.dep_state[dep] == 'waiting'] missing_deps = {dep for dep in deps if not self.who_has.get(dep)} if missing_deps: logger.info("Can't find dependencies for key %s", key) missing_deps2 = {dep for dep in missing_deps if dep not in self._missing_dep_flight} for dep in missing_deps2: self._missing_dep_flight.add(dep) self.loop.add_callback(self.handle_missing_dep, *missing_deps2) deps = [dep for dep in deps if dep not in missing_deps] self.log.append(('gather-dependencies', key, deps)) in_flight = False while deps and (len(self.in_flight_workers) < self.total_connections or self.comm_nbytes < self.total_comm_nbytes): dep = deps.pop() if self.dep_state[dep] != 'waiting': continue if dep not in self.who_has: continue workers = [w for w in self.who_has[dep] if w not in self.in_flight_workers] if not workers: in_flight = True continue worker = random.choice(list(workers)) to_gather, total_nbytes = self.select_keys_for_gather(worker, dep) self.comm_nbytes += total_nbytes self.in_flight_workers[worker] = to_gather for d in to_gather: self.transition_dep(d, 'flight', worker=worker) self.loop.add_callback(self.gather_dep, worker, dep, to_gather, total_nbytes, cause=key) changed = True if not deps and not in_flight: self.data_needed.popleft() except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def send_task_state_to_scheduler(self, key): if key in self.data: nbytes = self.nbytes[key] or sizeof(self.data[key]) typ = self.types.get(key) or type(self.data[key]) try: typ = dumps_function(typ) except PicklingError: # Some types fail pickling (example: _thread.lock objects), # send their name as a best effort. typ = pickle.dumps(typ.__name__) d = {'op': 'task-finished', 'status': 'OK', 'key': key, 'nbytes': nbytes, 'thread': self.threads.get(key), 'type': typ} elif key in self.exceptions: d = {'op': 'task-erred', 'status': 'error', 'key': key, 'thread': self.threads.get(key), 'exception': self.exceptions[key], 'traceback': self.tracebacks[key]} else: logger.error("Key not ready to send to worker, %s: %s", key, self.task_state[key]) return if key in self.startstops: d['startstops'] = self.startstops[key] self.batched_stream.send(d) def put_key_in_memory(self, key, value, transition=True): if key in self.data: return start = time() self.data[key] = value stop = time() if stop - start > 0.020: self.startstops[key].append(('disk-write', start, stop)) if key not in self.nbytes: self.nbytes[key] = sizeof(value) self.types[key] = type(value) for dep in self.dependents.get(key, ()): if dep in self.waiting_for_data: if key in self.waiting_for_data[dep]: self.waiting_for_data[dep].remove(key) if not self.waiting_for_data[dep]: self.transition(dep, 'ready') if transition and key in self.task_state: self.transition(key, 'memory') self.log.append((key, 'put-in-memory')) def select_keys_for_gather(self, worker, dep): deps = {dep} total_bytes = self.nbytes[dep] L = self.pending_data_per_worker[worker] while L: d = L.popleft() if self.dep_state.get(d) != 'waiting': continue if total_bytes + self.nbytes[d] > self.target_message_size: break deps.add(d) total_bytes += self.nbytes[d] return deps, total_bytes @gen.coroutine def gather_dep(self, worker, dep, deps, total_nbytes, cause=None): if self.status != 'running': return with log_errors(): response = {} try: if self.validate: self.validate_state() deps = tuple(dep for dep in deps if self.dep_state.get(dep) in ('waiting', 'flight')) self.log.append(('request-dep', dep, worker, deps)) logger.debug("Request %d keys", len(deps)) start = time() + self.scheduler_delay response = yield self.rpc(worker).get_data(keys=deps, who=self.address) stop = time() + self.scheduler_delay if cause: self.startstops[cause].append(('transfer', start, stop)) total_bytes = sum(self.nbytes.get(dep, 0) for dep in response) duration = (stop - start) or 0.5 self.incoming_transfer_log.append({ 'start': start, 'stop': stop, 'middle': (start + stop) / 2.0, 'duration': duration, 'keys': {dep: self.nbytes.get(dep, None) for dep in response}, 'total': total_bytes, 'bandwidth': total_bytes / duration, 'who': worker }) if self.digests is not None: self.digests['transfer-bandwidth'].add(total_bytes / duration) self.digests['transfer-duration'].add(duration) self.counters['transfer-count'].add(len(response)) self.incoming_count += 1 self.log.append(('receive-dep', worker, list(response))) if response: self.batched_stream.send({'op': 'add-keys', 'keys': list(response)}) except EnvironmentError as e: logger.exception("Worker stream died during communication: %s", worker) self.log.append(('receive-dep-failed', worker)) for d in self.has_what.pop(worker): self.who_has[d].remove(worker) if not self.who_has[d]: del self.who_has[d] except Exception as e: logger.exception(e) if self.batched_stream and LOG_PDB: import pdb pdb.set_trace() raise finally: self.comm_nbytes -= total_nbytes for d in self.in_flight_workers.pop(worker): if d in response: self.transition_dep(d, 'memory', value=response[d]) elif self.dep_state.get(d) != 'memory': self.transition_dep(d, 'waiting', worker=worker) if d not in response and d in self.dependents: self.log.append(('missing-dep', d)) self.batched_stream.send({'op': 'missing-data', 'errant_worker': worker, 'key': d}) if self.validate: self.validate_state() self.ensure_computing() self.ensure_communicating() def bad_dep(self, dep): exc = ValueError("Could not find dependent %s. Check worker logs" % str(dep)) for key in self.dependents[dep]: msg = error_message(exc) self.exceptions[key] = msg['exception'] self.tracebacks[key] = msg['traceback'] self.transition(key, 'error') self.release_dep(dep) @gen.coroutine def handle_missing_dep(self, *deps, **kwargs): original_deps = list(deps) self.log.append(('handle-missing', deps)) try: deps = {dep for dep in deps if dep in self.dependents} if not deps: return for dep in list(deps): suspicious = self.suspicious_deps[dep] if suspicious > 5: deps.remove(dep) self.bad_dep(dep) if not deps: return for dep in deps: logger.info("Dependent not found: %s %s . Asking scheduler", dep, self.suspicious_deps[dep]) who_has = yield self.scheduler.who_has(keys=list(deps)) who_has = {k: v for k, v in who_has.items() if v} self.update_who_has(who_has) for dep in deps: self.suspicious_deps[dep] += 1 if not who_has.get(dep): self.log.append((dep, 'no workers found', self.dependents.get(dep))) self.release_dep(dep) else: self.log.append((dep, 'new workers found')) for key in self.dependents.get(dep, ()): if key in self.waiting_for_data: self.data_needed.append(key) except Exception: logger.error("Handle missing dep failed, retrying", exc_info=True) retries = kwargs.get('retries', 5) self.log.append(('handle-missing-failed', retries, deps)) if retries > 0: yield self.handle_missing_dep(self, *deps, retries=retries - 1) else: raise finally: try: for dep in original_deps: self._missing_dep_flight.remove(dep) except KeyError: pass self.ensure_communicating() @gen.coroutine def query_who_has(self, *deps): with log_errors(): response = yield self.scheduler.who_has(keys=deps) self.update_who_has(response) raise gen.Return(response) def update_who_has(self, who_has): try: for dep, workers in who_has.items(): if not workers: continue if dep in self.who_has: self.who_has[dep].update(workers) else: self.who_has[dep] = set(workers) for worker in workers: self.has_what[worker].add(dep) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def steal_request(self, key): state = self.task_state.get(key, None) response = {'op': 'steal-response', 'key': key, 'state': state} self.batched_stream.send(response) if state in ('ready', 'waiting'): self.release_key(key) def release_key(self, key, cause=None, reason=None, report=True): try: if key not in self.task_state: return state = self.task_state.pop(key) if cause: self.log.append((key, 'release-key', {'cause': cause})) else: self.log.append((key, 'release-key')) del self.tasks[key] if key in self.data and key not in self.dep_state: del self.data[key] del self.nbytes[key] del self.types[key] if key in self.waiting_for_data: del self.waiting_for_data[key] for dep in self.dependencies.pop(key, ()): self.dependents[dep].remove(key) if not self.dependents[dep] and self.dep_state[dep] in ('waiting', 'flight'): self.release_dep(dep) if key in self.threads: del self.threads[key] del self.priorities[key] del self.durations[key] if key in self.exceptions: del self.exceptions[key] if key in self.tracebacks: del self.tracebacks[key] if key in self.startstops: del self.startstops[key] if key in self.executing: self.executing.remove(key) if key in self.resource_restrictions: del self.resource_restrictions[key] if report and state in PROCESSING: # not finished self.batched_stream.send({'op': 'release', 'key': key, 'cause': cause}) except CommClosedError: pass except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def release_dep(self, dep, report=False): try: if dep not in self.dep_state: return self.log.append((dep, 'release-dep')) state = self.dep_state.pop(dep) if dep in self.suspicious_deps: del self.suspicious_deps[dep] if dep in self.who_has: for worker in self.who_has.pop(dep): self.has_what[worker].remove(dep) if dep not in self.task_state: if dep in self.data: del self.data[dep] del self.types[dep] del self.nbytes[dep] if dep in self.in_flight_tasks: worker = self.in_flight_tasks.pop(dep) self.in_flight_workers[worker].remove(dep) for key in self.dependents.pop(dep, ()): self.dependencies[key].remove(dep) if self.task_state[key] != 'memory': self.release_key(key, cause=dep) if report and state == 'memory': self.batched_stream.send({'op': 'release-worker-data', 'keys': [dep]}) except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def rescind_key(self, key): try: if self.task_state.get(key) not in PENDING: return del self.task_state[key] del self.tasks[key] if key in self.waiting_for_data: del self.waiting_for_data[key] for dep in self.dependencies.pop(key, ()): self.dependents[dep].remove(key) if not self.dependents[dep]: del self.dependents[dep] if key not in self.dependents: # if key in self.nbytes: # del self.nbytes[key] if key in self.priorities: del self.priorities[key] if key in self.durations: del self.durations[key] except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise ################ # Execute Task # ################ def meets_resource_constraints(self, key): if key not in self.resource_restrictions: return True for resource, needed in self.resource_restrictions[key].items(): if self.available_resources[resource] < needed: return False return True def ensure_computing(self): if self.paused: return try: while self.constrained and len(self.executing) < self.ncores: key = self.constrained[0] if self.task_state.get(key) != 'constrained': self.constrained.popleft() continue if self.meets_resource_constraints(key): self.constrained.popleft() self.transition(key, 'executing') else: break while self.ready and len(self.executing) < self.ncores: _, key = heapq.heappop(self.ready) if self.task_state.get(key) in READY: self.transition(key, 'executing') except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise @gen.coroutine def execute(self, key, report=False): executor_error = None try: if key not in self.executing or key not in self.task_state: return if self.validate: assert key not in self.waiting_for_data assert self.task_state[key] == 'executing' function, args, kwargs = self.tasks[key] start = time() data = {k: self.data[k] for k in self.dependencies[key]} args2 = pack_data(args, data, key_types=(bytes, unicode)) kwargs2 = pack_data(kwargs, data, key_types=(bytes, unicode)) stop = time() if stop - start > 0.005: self.startstops[key].append(('disk-read', start, stop)) if self.digests is not None: self.digests['disk-load-duration'].add(stop - start) logger.debug("Execute key: %s worker: %s", key, self.address) # TODO: comment out? try: result = yield self.executor_submit(key, apply_function, function, args2, kwargs2, self.execution_state, key, self.active_threads, self.active_threads_lock, self.scheduler_delay) except RuntimeError as e: executor_error = e raise if self.task_state.get(key) not in ('executing', 'long-running'): return result['key'] = key value = result.pop('result', None) self.startstops[key].append(('compute', result['start'], result['stop'])) self.threads[key] = result['thread'] if result['op'] == 'task-finished': self.nbytes[key] = result['nbytes'] self.types[key] = result['type'] self.transition(key, 'memory', value=value) if self.digests is not None: self.digests['task-duration'].add(result['stop'] - result['start']) else: if isinstance(result.pop('actual-exception'), Reschedule): self.batched_stream.send({'op': 'reschedule', 'key': key}) self.transition(key, 'rescheduled', report=False) self.release_key(key, report=False) else: self.exceptions[key] = result['exception'] self.tracebacks[key] = result['traceback'] logger.warning(" Compute Failed\n" "Function: %s\n" "args: %s\n" "kwargs: %s\n" "Exception: %s\n", str(funcname(function))[:1000], convert_args_to_str(args2, max_len=1000), convert_kwargs_to_str(kwargs2, max_len=1000), repr(pickle.loads(result['exception']))) self.transition(key, 'error') logger.debug("Send compute response to scheduler: %s, %s", key, result) if self.validate: assert key not in self.executing assert key not in self.waiting_for_data self.ensure_computing() self.ensure_communicating() except Exception as e: if executor_error is e: logger.error("Thread Pool Executor error: %s", e) else: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise finally: if key in self.executing: self.executing.remove(key) ################## # Administrative # ################## @gen.coroutine def memory_monitor(self): """ Track this process's memory usage and act accordingly If we rise above 70% memory use, start dumping data to disk. If we rise above 80% memory use, stop execution of new tasks """ if self._memory_monitoring: return self._memory_monitoring = True total = 0 proc = psutil.Process() memory = proc.memory_info().rss frac = memory / self.memory_limit # Pause worker threads if above 80% memory use if self.memory_pause_fraction and frac > self.memory_pause_fraction: # Try to free some memory while in paused state self._throttled_gc.collect() if not self.paused: logger.warning("Worker is at %d%% memory usage. Pausing worker. " "Process memory: %s -- Worker memory limit: %s", int(frac * 100), format_bytes(proc.memory_info().rss), format_bytes(self.memory_limit)) self.paused = True elif self.paused: logger.warning("Worker is at %d%% memory usage. Resuming worker. " "Process memory: %s -- Worker memory limit: %s", int(frac * 100), format_bytes(proc.memory_info().rss), format_bytes(self.memory_limit)) self.paused = False self.ensure_computing() # Dump data to disk if above 70% if self.memory_spill_fraction and frac > self.memory_spill_fraction: target = self.memory_limit * self.memory_target_fraction count = 0 need = memory - target while memory > target: if not self.data.fast: logger.warning("Memory use is high but worker has no data " "to store to disk. Perhaps some other process " "is leaking memory? Process memory: %s -- " "Worker memory limit: %s", format_bytes(proc.memory_info().rss), format_bytes(self.memory_limit)) break k, v, weight = self.data.fast.evict() del k, v total += weight count += 1 yield gen.moment memory = proc.memory_info().rss if total > need and memory > target: # Issue a GC to ensure that the evicted data is actually # freed from memory and taken into account by the monitor # before trying to evict even more data. self._throttled_gc.collect() memory = proc.memory_info().rss if count: logger.debug("Moved %d pieces of data data and %s to disk", count, format_bytes(total)) self._memory_monitoring = False raise gen.Return(total) def cycle_profile(self): now = time() + self.scheduler_delay prof, self.profile_recent = self.profile_recent, profile.create() self.profile_history.append((now, prof)) self.profile_keys_history.append((now, dict(self.profile_keys))) self.profile_keys.clear() def trigger_profile(self): """ Get a frame from all actively computing threads Merge these frames into existing profile counts """ if not self.active_threads: # hope that this is thread-atomic? return start = time() with self.active_threads_lock: active_threads = self.active_threads.copy() frames = sys._current_frames() frames = {ident: frames[ident] for ident in active_threads} for ident, frame in frames.items(): if frame is not None: key = key_split(active_threads[ident]) profile.process(frame, None, self.profile_recent, stop='_concurrent_futures_thread.py') profile.process(frame, None, self.profile_keys[key], stop='_concurrent_futures_thread.py') stop = time() if self.digests is not None: self.digests['profile-duration'].add(stop - start) def get_profile(self, comm=None, start=None, stop=None, key=None): now = time() + self.scheduler_delay if key is None: history = self.profile_history else: history = [(t, d[key]) for t, d in self.profile_keys_history if key in d] if start is None: istart = 0 else: istart = bisect.bisect_left(history, (start,)) if stop is None: istop = None else: istop = bisect.bisect_right(history, (stop,)) + 1 if istop >= len(history): istop = None # include end if istart == 0 and istop is None: history = list(history) else: iistop = len(history) if istop is None else istop history = [history[i] for i in range(istart, iistop)] prof = profile.merge(*pluck(1, history)) if not history: return profile.create() if istop is None and (start is None or start < now): if key is None: recent = self.profile_recent else: recent = self.profile_keys[key] prof = profile.merge(prof, recent) return prof def get_profile_metadata(self, comm=None, start=0, stop=None): if stop is None: add_recent = True now = time() + self.scheduler_delay stop = stop or now start = start or 0 result = {'counts': [(t, d['count']) for t, d in self.profile_history if start < t < stop], 'keys': [(t, {k: d['count'] for k, d in v.items()}) for t, v in self.profile_keys_history if start < t < stop]} if add_recent: result['counts'].append((now, self.profile_recent['count'])) result['keys'].append((now, {k: v['count'] for k, v in self.profile_keys.items()})) return result def get_call_stack(self, comm=None, keys=None): with self.active_threads_lock: frames = sys._current_frames() active_threads = self.active_threads.copy() frames = {k: frames[ident] for ident, k in active_threads.items()} if keys is not None: frames = {k: frame for k, frame in frames.items() if k in keys} result = {k: profile.call_stack(frame) for k, frame in frames.items()} return result 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] def validate_key_memory(self, key): assert key in self.data assert key in self.nbytes assert key not in self.waiting_for_data assert key not in self.executing assert key not in self.ready if key in self.dep_state: assert self.dep_state[key] == 'memory' def validate_key_executing(self, key): assert key in self.executing assert key not in self.data assert key not in self.waiting_for_data assert all(dep in self.data for dep in self.dependencies[key]) def validate_key_ready(self, key): assert key in pluck(1, self.ready) assert key not in self.data assert key not in self.executing assert key not in self.waiting_for_data assert all(dep in self.data for dep in self.dependencies[key]) def validate_key_waiting(self, key): assert key not in self.data assert not all(dep in self.data for dep in self.dependencies[key]) def validate_key(self, key): try: state = self.task_state[key] if state == 'memory': self.validate_key_memory(key) elif state == 'waiting': self.validate_key_waiting(key) elif state == 'ready': self.validate_key_ready(key) elif state == 'executing': self.validate_key_executing(key) except Exception as e: logger.exception(e) import pdb pdb.set_trace() raise def validate_dep_waiting(self, dep): assert dep not in self.data assert dep in self.nbytes assert self.dependents[dep] assert not any(key in self.ready for key in self.dependents[dep]) def validate_dep_flight(self, dep): assert dep not in self.data assert dep in self.nbytes assert not any(key in self.ready for key in self.dependents[dep]) peer = self.in_flight_tasks[dep] assert dep in self.in_flight_workers[peer] def validate_dep_memory(self, dep): assert dep in self.data assert dep in self.nbytes assert dep in self.types if dep in self.task_state: assert self.task_state[dep] == 'memory' def validate_dep(self, dep): try: state = self.dep_state[dep] if state == 'waiting': self.validate_dep_waiting(dep) elif state == 'flight': self.validate_dep_flight(dep) elif state == 'memory': self.validate_dep_memory(dep) else: raise ValueError("Unknown dependent state", state) except Exception as e: logger.exception(e) import pdb pdb.set_trace() raise def validate_state(self): if self.status != 'running': return try: for key, workers in self.who_has.items(): for w in workers: assert key in self.has_what[w] for worker, keys in self.has_what.items(): for k in keys: assert worker in self.who_has[k] for key in self.task_state: self.validate_key(key) for dep in self.dep_state: self.validate_dep(dep) for key, deps in self.waiting_for_data.items(): if key not in self.data_needed: for dep in deps: assert (dep in self.in_flight_tasks or dep in self._missing_dep_flight or self.who_has[dep].issubset(self.in_flight_workers)) for key in self.tasks: if self.task_state[key] == 'memory': assert isinstance(self.nbytes[key], int) assert key not in self.waiting_for_data assert key in self.data except Exception as e: logger.exception(e) if LOG_PDB: import pdb pdb.set_trace() raise def stateof(self, key): return {'executing': key in self.executing, 'waiting_for_data': key in self.waiting_for_data, 'heap': key in pluck(1, self.ready), 'data': key in self.data} def story(self, *keys): return [msg for msg in self.log if any(key in msg for key in keys) or any(key in c for key in keys for c in msg if isinstance(c, (tuple, list, set)))] @property def client(self): with self._lock: if self._client: return self._client else: return self._get_client() def _get_client(self, timeout=3): """ Get local client attached to this worker If no such client exists, create one See Also -------- get_client """ try: from .client import default_client client = default_client() except ValueError: # no clients found, need to make a new one pass else: if (client.scheduler and client.scheduler.address == self.scheduler.address or client._start_arg == self.scheduler.address): self._client = client if not self._client: from .client import Client asynchronous = self.loop is IOLoop.current() self._client = Client(self.scheduler.address, loop=self.loop, security=self.security, set_as_default=True, asynchronous=asynchronous, timeout=timeout) if not asynchronous: assert self._client.status == 'running' return self._client def get_current_task(self): """ Get the key of the task we are currently running This only makes sense to run within a task Examples -------- >>> from dask.distributed import get_worker >>> def f(): ... return get_worker().get_current_task() >>> future = client.submit(f) # doctest: +SKIP >>> future.result() # doctest: +SKIP 'f-1234' See Also -------- get_worker """ return self.active_threads[get_thread_identity()] def get_worker(): """ Get the worker currently running this task Examples -------- >>> def f(): ... worker = get_worker() # The worker on which this task is running ... return worker.address >>> future = client.submit(f) # doctest: +SKIP >>> future.result() # doctest: +SKIP 'tcp://127.0.0.1:47373' See Also -------- get_client worker_client """ try: return thread_state.execution_state['worker'] except AttributeError: for ref in _global_workers[::-1]: worker = ref() if worker: return worker raise ValueError("No workers found") def get_client(address=None, timeout=3): """ Get a client while within a task This client connects to the same scheduler to which the worker is connected Examples -------- >>> def f(): ... client = get_client() ... futures = client.map(lambda x: x + 1, range(10)) # spawn many tasks ... results = client.gather(futures) ... return sum(results) >>> future = client.submit(f) # doctest: +SKIP >>> future.result() # doctest: +SKIP 55 See Also -------- get_worker worker_client secede """ try: worker = get_worker() except ValueError: # could not find worker pass else: if not address or worker.scheduler.address == address: return worker._get_client(timeout=timeout) from .client import _get_global_client client = _get_global_client() # TODO: assumes the same scheduler if client and (not address or client.scheduler.address == address): return client elif address: from .client import Client return Client(address, timeout=timeout) else: raise ValueError("No global client found and no address provided") def secede(): """ Have this task secede from the worker's thread pool This opens up a new scheduling slot and a new thread for a new task. This enables the client to schedule tasks on this node, which is especially useful while waiting for other jobs to finish (e.g., with ``client.gather``). Examples -------- >>> def mytask(x): ... # do some work ... client = get_client() ... futures = client.map(...) # do some remote work ... secede() # while that work happens, remove ourself from the pool ... return client.gather(futures) # return gathered results See Also -------- get_client get_worker """ worker = get_worker() tpe_secede() # have this thread secede from the thread pool duration = time() - thread_state.start_time worker.loop.add_callback(worker.maybe_transition_long_running, thread_state.key, compute_duration=duration) class Reschedule(Exception): """ Reschedule this task Raising this exception will stop the current execution of the task and ask the scheduler to reschedule this task, possibly on a different machine. This does not guarantee that the task will move onto a different machine. The scheduler will proceed through its normal heuristics to determine the optimal machine to accept this task. The machine will likely change if the load across the cluster has significantly changed since first scheduling the task. """ pass def parse_memory_limit(memory_limit, ncores): if memory_limit == 'auto': memory_limit = int(TOTAL_MEMORY * min(1, ncores / _ncores)) with ignoring(ValueError, TypeError): x = float(memory_limit) if isinstance(x, float) and x <= 1: return int(x * TOTAL_MEMORY) if isinstance(memory_limit, (unicode, str)): return parse_bytes(memory_limit) else: return int(memory_limit) distributed-1.20.2/distributed/worker_client.py000066400000000000000000000031511321233345200216570ustar00rootroot00000000000000from __future__ import print_function, division, absolute_import from contextlib import contextmanager import warnings from .threadpoolexecutor import secede from .worker import thread_state, get_client, get_worker @contextmanager def worker_client(timeout=3, separate_thread=True): """ Get client for this thread This context manager is intended to be called within functions that we run on workers. When run as a context manager it delivers a client ``Client`` object that can submit other tasks directly from that worker. Parameters ---------- timeout: Number Timeout after which to err separate_thread: bool, optional Whether to run this function outside of the normal thread pool defaults to True Examples -------- >>> def func(x): ... with worker_client() as c: # connect from worker back to scheduler ... a = c.submit(inc, x) # this task can submit more tasks ... b = c.submit(dec, x) ... result = c.gather([a, b]) # and gather results ... return result >>> future = client.submit(func, 1) # submit func(1) on cluster See Also -------- get_worker get_client secede """ worker = get_worker() client = get_client(timeout=timeout) if separate_thread: secede() # have this thread secede from the thread pool worker.loop.add_callback(worker.transition, thread_state.key, 'long-running') yield client def local_client(*args, **kwargs): warnings.warn("local_client has moved to worker_client") return worker_client(*args, **kwargs) distributed-1.20.2/docs/000077500000000000000000000000001321233345200150445ustar00rootroot00000000000000distributed-1.20.2/docs/Makefile000066400000000000000000000164221321233345200165110ustar00rootroot00000000000000# Makefile for Sphinx documentation # # You can set these variables from the command line. SPHINXOPTS = -j4 SPHINXBUILD = sphinx-build PAPER = BUILDDIR = build # User-friendly check for sphinx-build ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) $(error The '$(SPHINXBUILD)' command was not found. Make sure you have Sphinx installed, then set the SPHINXBUILD environment variable to point to the full path of the '$(SPHINXBUILD)' executable. Alternatively you can add the directory with the executable to your PATH. If you don't have Sphinx installed, grab it from http://sphinx-doc.org/) endif # Internal variables. PAPEROPT_a4 = -D latex_paper_size=a4 PAPEROPT_letter = -D latex_paper_size=letter ALLSPHINXOPTS = -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) source # the i18n builder cannot share the environment and doctrees with the others I18NSPHINXOPTS = $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) source .PHONY: help clean html dirhtml singlehtml pickle json htmlhelp qthelp devhelp epub latex latexpdf text man changes linkcheck doctest coverage gettext help: @echo "Please use \`make ' where is one of" @echo " html to make standalone HTML files" @echo " dirhtml to make HTML files named index.html in directories" @echo " singlehtml to make a single large HTML file" @echo " pickle to make pickle files" @echo " json to make JSON files" @echo " htmlhelp to make HTML files and a HTML help project" @echo " qthelp to make HTML files and a qthelp project" @echo " applehelp to make an Apple Help Book" @echo " devhelp to make HTML files and a Devhelp project" @echo " epub to make an epub" @echo " latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter" @echo " latexpdf to make LaTeX files and run them through pdflatex" @echo " latexpdfja to make LaTeX files and run them through platex/dvipdfmx" @echo " text to make text files" @echo " man to make manual pages" @echo " texinfo to make Texinfo files" @echo " info to make Texinfo files and run them through makeinfo" @echo " gettext to make PO message catalogs" @echo " changes to make an overview of all changed/added/deprecated items" @echo " xml to make Docutils-native XML files" @echo " pseudoxml to make pseudoxml-XML files for display purposes" @echo " linkcheck to check all external links for integrity" @echo " doctest to run all doctests embedded in the documentation (if enabled)" @echo " coverage to run coverage check of the documentation (if enabled)" clean: rm -rf $(BUILDDIR)/* html: $(SPHINXBUILD) -b html $(ALLSPHINXOPTS) $(BUILDDIR)/html @echo @echo "Build finished. The HTML pages are in $(BUILDDIR)/html." dirhtml: $(SPHINXBUILD) -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml @echo @echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml." singlehtml: $(SPHINXBUILD) -b singlehtml $(ALLSPHINXOPTS) $(BUILDDIR)/singlehtml @echo @echo "Build finished. The HTML page is in $(BUILDDIR)/singlehtml." pickle: $(SPHINXBUILD) -b pickle $(ALLSPHINXOPTS) $(BUILDDIR)/pickle @echo @echo "Build finished; now you can process the pickle files." json: $(SPHINXBUILD) -b json $(ALLSPHINXOPTS) $(BUILDDIR)/json @echo @echo "Build finished; now you can process the JSON files." htmlhelp: $(SPHINXBUILD) -b htmlhelp $(ALLSPHINXOPTS) $(BUILDDIR)/htmlhelp @echo @echo "Build finished; now you can run HTML Help Workshop with the" \ ".hhp project file in $(BUILDDIR)/htmlhelp." qthelp: $(SPHINXBUILD) -b qthelp $(ALLSPHINXOPTS) $(BUILDDIR)/qthelp @echo @echo "Build finished; now you can run "qcollectiongenerator" with the" \ ".qhcp project file in $(BUILDDIR)/qthelp, like this:" @echo "# qcollectiongenerator $(BUILDDIR)/qthelp/distributed.qhcp" @echo "To view the help file:" @echo "# assistant -collectionFile $(BUILDDIR)/qthelp/distributed.qhc" applehelp: $(SPHINXBUILD) -b applehelp $(ALLSPHINXOPTS) $(BUILDDIR)/applehelp @echo @echo "Build finished. The help book is in $(BUILDDIR)/applehelp." @echo "N.B. You won't be able to view it unless you put it in" \ "~/Library/Documentation/Help or install it in your application" \ "bundle." devhelp: $(SPHINXBUILD) -b devhelp $(ALLSPHINXOPTS) $(BUILDDIR)/devhelp @echo @echo "Build finished." @echo "To view the help file:" @echo "# mkdir -p $$HOME/.local/share/devhelp/distributed" @echo "# ln -s $(BUILDDIR)/devhelp $$HOME/.local/share/devhelp/distributed" @echo "# devhelp" epub: $(SPHINXBUILD) -b epub $(ALLSPHINXOPTS) $(BUILDDIR)/epub @echo @echo "Build finished. The epub file is in $(BUILDDIR)/epub." latex: $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex @echo @echo "Build finished; the LaTeX files are in $(BUILDDIR)/latex." @echo "Run \`make' in that directory to run these through (pdf)latex" \ "(use \`make latexpdf' here to do that automatically)." latexpdf: $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex @echo "Running LaTeX files through pdflatex..." $(MAKE) -C $(BUILDDIR)/latex all-pdf @echo "pdflatex finished; the PDF files are in $(BUILDDIR)/latex." latexpdfja: $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex @echo "Running LaTeX files through platex and dvipdfmx..." $(MAKE) -C $(BUILDDIR)/latex all-pdf-ja @echo "pdflatex finished; the PDF files are in $(BUILDDIR)/latex." text: $(SPHINXBUILD) -b text $(ALLSPHINXOPTS) $(BUILDDIR)/text @echo @echo "Build finished. The text files are in $(BUILDDIR)/text." man: $(SPHINXBUILD) -b man $(ALLSPHINXOPTS) $(BUILDDIR)/man @echo @echo "Build finished. The manual pages are in $(BUILDDIR)/man." texinfo: $(SPHINXBUILD) -b texinfo $(ALLSPHINXOPTS) $(BUILDDIR)/texinfo @echo @echo "Build finished. The Texinfo files are in $(BUILDDIR)/texinfo." @echo "Run \`make' in that directory to run these through makeinfo" \ "(use \`make info' here to do that automatically)." info: $(SPHINXBUILD) -b texinfo $(ALLSPHINXOPTS) $(BUILDDIR)/texinfo @echo "Running Texinfo files through makeinfo..." make -C $(BUILDDIR)/texinfo info @echo "makeinfo finished; the Info files are in $(BUILDDIR)/texinfo." gettext: $(SPHINXBUILD) -b gettext $(I18NSPHINXOPTS) $(BUILDDIR)/locale @echo @echo "Build finished. The message catalogs are in $(BUILDDIR)/locale." changes: $(SPHINXBUILD) -b changes $(ALLSPHINXOPTS) $(BUILDDIR)/changes @echo @echo "The overview file is in $(BUILDDIR)/changes." linkcheck: $(SPHINXBUILD) -b linkcheck $(ALLSPHINXOPTS) $(BUILDDIR)/linkcheck @echo @echo "Link check complete; look for any errors in the above output " \ "or in $(BUILDDIR)/linkcheck/output.txt." doctest: $(SPHINXBUILD) -b doctest $(ALLSPHINXOPTS) $(BUILDDIR)/doctest @echo "Testing of doctests in the sources finished, look at the " \ "results in $(BUILDDIR)/doctest/output.txt." coverage: $(SPHINXBUILD) -b coverage $(ALLSPHINXOPTS) $(BUILDDIR)/coverage @echo "Testing of coverage in the sources finished, look at the " \ "results in $(BUILDDIR)/coverage/python.txt." xml: $(SPHINXBUILD) -b xml $(ALLSPHINXOPTS) $(BUILDDIR)/xml @echo @echo "Build finished. The XML files are in $(BUILDDIR)/xml." pseudoxml: $(SPHINXBUILD) -b pseudoxml $(ALLSPHINXOPTS) $(BUILDDIR)/pseudoxml @echo @echo "Build finished. The pseudo-XML files are in $(BUILDDIR)/pseudoxml." distributed-1.20.2/docs/make.bat000066400000000000000000000161371321233345200164610ustar00rootroot00000000000000@ECHO OFF REM Command file for Sphinx documentation if "%SPHINXBUILD%" == "" ( set SPHINXBUILD=sphinx-build ) set BUILDDIR=build set ALLSPHINXOPTS=-d %BUILDDIR%/doctrees %SPHINXOPTS% source set I18NSPHINXOPTS=%SPHINXOPTS% source if NOT "%PAPER%" == "" ( set ALLSPHINXOPTS=-D latex_paper_size=%PAPER% %ALLSPHINXOPTS% set I18NSPHINXOPTS=-D latex_paper_size=%PAPER% %I18NSPHINXOPTS% ) if "%1" == "" goto help if "%1" == "help" ( :help echo.Please use `make ^` where ^ is one of echo. html to make standalone HTML files echo. dirhtml to make HTML files named index.html in directories echo. singlehtml to make a single large HTML file echo. pickle to make pickle files echo. json to make JSON files echo. htmlhelp to make HTML files and a HTML help project echo. qthelp to make HTML files and a qthelp project echo. devhelp to make HTML files and a Devhelp project echo. epub to make an epub echo. latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter echo. text to make text files echo. man to make manual pages echo. texinfo to make Texinfo files echo. gettext to make PO message catalogs echo. changes to make an overview over all changed/added/deprecated items echo. xml to make Docutils-native XML files echo. pseudoxml to make pseudoxml-XML files for display purposes echo. linkcheck to check all external links for integrity echo. doctest to run all doctests embedded in the documentation if enabled echo. coverage to run coverage check of the documentation if enabled goto end ) if "%1" == "clean" ( for /d %%i in (%BUILDDIR%\*) do rmdir /q /s %%i del /q /s %BUILDDIR%\* goto end ) REM Check if sphinx-build is available and fallback to Python version if any %SPHINXBUILD% 2> nul if errorlevel 9009 goto sphinx_python goto sphinx_ok :sphinx_python set SPHINXBUILD=python -m sphinx.__init__ %SPHINXBUILD% 2> nul if errorlevel 9009 ( echo. echo.The 'sphinx-build' command was not found. Make sure you have Sphinx echo.installed, then set the SPHINXBUILD environment variable to point echo.to the full path of the 'sphinx-build' executable. Alternatively you echo.may add the Sphinx directory to PATH. echo. echo.If you don't have Sphinx installed, grab it from echo.http://sphinx-doc.org/ exit /b 1 ) :sphinx_ok if "%1" == "html" ( %SPHINXBUILD% -b html %ALLSPHINXOPTS% %BUILDDIR%/html if errorlevel 1 exit /b 1 echo. echo.Build finished. The HTML pages are in %BUILDDIR%/html. goto end ) if "%1" == "dirhtml" ( %SPHINXBUILD% -b dirhtml %ALLSPHINXOPTS% %BUILDDIR%/dirhtml if errorlevel 1 exit /b 1 echo. echo.Build finished. The HTML pages are in %BUILDDIR%/dirhtml. goto end ) if "%1" == "singlehtml" ( %SPHINXBUILD% -b singlehtml %ALLSPHINXOPTS% %BUILDDIR%/singlehtml if errorlevel 1 exit /b 1 echo. echo.Build finished. The HTML pages are in %BUILDDIR%/singlehtml. goto end ) if "%1" == "pickle" ( %SPHINXBUILD% -b pickle %ALLSPHINXOPTS% %BUILDDIR%/pickle if errorlevel 1 exit /b 1 echo. echo.Build finished; now you can process the pickle files. goto end ) if "%1" == "json" ( %SPHINXBUILD% -b json %ALLSPHINXOPTS% %BUILDDIR%/json if errorlevel 1 exit /b 1 echo. echo.Build finished; now you can process the JSON files. goto end ) if "%1" == "htmlhelp" ( %SPHINXBUILD% -b htmlhelp %ALLSPHINXOPTS% %BUILDDIR%/htmlhelp if errorlevel 1 exit /b 1 echo. echo.Build finished; now you can run HTML Help Workshop with the ^ .hhp project file in %BUILDDIR%/htmlhelp. goto end ) if "%1" == "qthelp" ( %SPHINXBUILD% -b qthelp %ALLSPHINXOPTS% %BUILDDIR%/qthelp if errorlevel 1 exit /b 1 echo. echo.Build finished; now you can run "qcollectiongenerator" with the ^ .qhcp project file in %BUILDDIR%/qthelp, like this: echo.^> qcollectiongenerator %BUILDDIR%\qthelp\distributed.qhcp echo.To view the help file: echo.^> assistant -collectionFile %BUILDDIR%\qthelp\distributed.ghc goto end ) if "%1" == "devhelp" ( %SPHINXBUILD% -b devhelp %ALLSPHINXOPTS% %BUILDDIR%/devhelp if errorlevel 1 exit /b 1 echo. echo.Build finished. goto end ) if "%1" == "epub" ( %SPHINXBUILD% -b epub %ALLSPHINXOPTS% %BUILDDIR%/epub if errorlevel 1 exit /b 1 echo. echo.Build finished. The epub file is in %BUILDDIR%/epub. goto end ) if "%1" == "latex" ( %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex if errorlevel 1 exit /b 1 echo. echo.Build finished; the LaTeX files are in %BUILDDIR%/latex. goto end ) if "%1" == "latexpdf" ( %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex cd %BUILDDIR%/latex make all-pdf cd %~dp0 echo. echo.Build finished; the PDF files are in %BUILDDIR%/latex. goto end ) if "%1" == "latexpdfja" ( %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex cd %BUILDDIR%/latex make all-pdf-ja cd %~dp0 echo. echo.Build finished; the PDF files are in %BUILDDIR%/latex. goto end ) if "%1" == "text" ( %SPHINXBUILD% -b text %ALLSPHINXOPTS% %BUILDDIR%/text if errorlevel 1 exit /b 1 echo. echo.Build finished. The text files are in %BUILDDIR%/text. goto end ) if "%1" == "man" ( %SPHINXBUILD% -b man %ALLSPHINXOPTS% %BUILDDIR%/man if errorlevel 1 exit /b 1 echo. echo.Build finished. The manual pages are in %BUILDDIR%/man. goto end ) if "%1" == "texinfo" ( %SPHINXBUILD% -b texinfo %ALLSPHINXOPTS% %BUILDDIR%/texinfo if errorlevel 1 exit /b 1 echo. echo.Build finished. The Texinfo files are in %BUILDDIR%/texinfo. goto end ) if "%1" == "gettext" ( %SPHINXBUILD% -b gettext %I18NSPHINXOPTS% %BUILDDIR%/locale if errorlevel 1 exit /b 1 echo. echo.Build finished. The message catalogs are in %BUILDDIR%/locale. goto end ) if "%1" == "changes" ( %SPHINXBUILD% -b changes %ALLSPHINXOPTS% %BUILDDIR%/changes if errorlevel 1 exit /b 1 echo. echo.The overview file is in %BUILDDIR%/changes. goto end ) if "%1" == "linkcheck" ( %SPHINXBUILD% -b linkcheck %ALLSPHINXOPTS% %BUILDDIR%/linkcheck if errorlevel 1 exit /b 1 echo. echo.Link check complete; look for any errors in the above output ^ or in %BUILDDIR%/linkcheck/output.txt. goto end ) if "%1" == "doctest" ( %SPHINXBUILD% -b doctest %ALLSPHINXOPTS% %BUILDDIR%/doctest if errorlevel 1 exit /b 1 echo. echo.Testing of doctests in the sources finished, look at the ^ results in %BUILDDIR%/doctest/output.txt. goto end ) if "%1" == "coverage" ( %SPHINXBUILD% -b coverage %ALLSPHINXOPTS% %BUILDDIR%/coverage if errorlevel 1 exit /b 1 echo. echo.Testing of coverage in the sources finished, look at the ^ results in %BUILDDIR%/coverage/python.txt. goto end ) if "%1" == "xml" ( %SPHINXBUILD% -b xml %ALLSPHINXOPTS% %BUILDDIR%/xml if errorlevel 1 exit /b 1 echo. echo.Build finished. The XML files are in %BUILDDIR%/xml. goto end ) if "%1" == "pseudoxml" ( %SPHINXBUILD% -b pseudoxml %ALLSPHINXOPTS% %BUILDDIR%/pseudoxml if errorlevel 1 exit /b 1 echo. echo.Build finished. The pseudo-XML files are in %BUILDDIR%/pseudoxml. goto end ) :end distributed-1.20.2/docs/requirements.txt000066400000000000000000000001001321233345200203170ustar00rootroot00000000000000tornado toolz cloudpickle dask numpydoc sphinx sphinx_rtd_theme distributed-1.20.2/docs/source/000077500000000000000000000000001321233345200163445ustar00rootroot00000000000000distributed-1.20.2/docs/source/adaptive.rst000066400000000000000000000161231321233345200206760ustar00rootroot00000000000000Adaptive Deployments ==================== It is possible to grow and shrink Dask clusters based on current use. This allows you to run Dask permanently on your cluster and have it only take up resources when necessary. Dask contains the logic about when to grow and shrink but relies on external cluster managers to launch and kill ``dask-worker`` jobs. This page describes the policies about adaptively resizing Dask clusters based on load, how to connect these policies to a particular job scheduler, and an example implementation. Dynamically scaling a Dask cluster up and down requires tight integration with an external cluster management system that can deploy ``dask-worker`` jobs throughout the cluster. Several systems are in wide use today, including common examples like SGE, SLURM, Torque, Condor, LSF, Yarn, Mesos, Marathon, Kubernetes, etc... These systems can be quite different from each other, but all are used to manage distributed services throughout different kinds of clusters. The large number of relevant systems, the challenges of rigorously testing each, and finite development time precludes the systematic inclusion of all solutions within the dask/distributed repository. Instead, we include a generic interface that can be extended by someone with basic understanding of their cluster management tool. We encourage these as third party modules. Policies -------- We control the number of workers based on current load and memory use. The scheduler checks itself periodically to determine if more or fewer workers are needed. If there are excess unclaimed tasks, or if the memory of the current workers is more nearing full then the scheduler tries to increase the number of workers by a fixed factor, defaulting to 2. This causes exponential growth while growth is useful. If there are idle workers and if the memory of the current workers is nearing empty then we gracefully retire the idle workers with the least amount of data in memory. We first move these results to the surviving workers and then remove the idle workers from the cluster. This shrinks the cluster while gracefully preserving intermediate results, shrinking the cluster when excess size is not useful. Adaptive class interface ------------------------ The ``distributed.deploy.Adaptive`` class contains the logic about when to ask for new workers, and when to close idle ones. This class requires both a scheduler and a cluster object. The cluster object must support two methods, ``scale_up(n, **kwargs)``, which takes in a target number of total workers for the cluster and ``scale_down(workers)``, which takes in a list of addresses to remove from the cluster. The Adaptive class will call these methods with the correct values at the correct times. .. code-block:: python class MyCluster(object): @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. """ raise NotImplementedError() @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. """ from distributed.deploy import Adaptive scheduler = Scheduler() cluster = MyCluster() adapative_cluster = Adaptive(scheduler, cluster) scheduler.start() Implementing these ``scale_up`` and ``scale_down`` functions depends strongly on the cluster management system. See :doc:`LocalCluster ` for an example. Marathon: an example -------------------- We now present an example project that implements this cluster interface backed by the Marathon cluster management tool on Mesos. Full source code and testing apparatus is available here: http://github.com/mrocklin/dask-marathon The implementation is small. It uses the Marathon HTTP API through the `marathon Python client library `_. We reproduce the full body of the implementation below as an example: .. code-block:: python from marathon import MarathonClient, MarathonApp from marathon.models.container import MarathonContainer class MarathonCluster(object): def __init__(self, scheduler, executable='dask-worker', docker_image='mrocklin/dask-distributed', marathon_address='http://localhost:8080', name=None, **kwargs): self.scheduler = scheduler # Create Marathon App to run dask-worker args = [executable, scheduler.address, '--name', '$MESOS_TASK_ID'] # use Mesos task ID as worker name if 'mem' in kwargs: args.extend(['--memory-limit', str(int(kwargs['mem'] * 0.6 * 1e6))]) kwargs['cmd'] = ' '.join(args) container = MarathonContainer({'image': docker_image}) app = MarathonApp(instances=0, container=container, **kwargs) # Connect and register app self.client = MarathonClient(marathon_address) self.app = self.client.create_app(name or 'dask-%s' % uuid.uuid4(), app) def scale_up(self, instances): self.marathon_client.scale_app(self.app.id, instances=instances) def scale_down(self, workers): for w in workers: self.marathon_client.kill_task(self.app.id, self.scheduler.worker_info[w]['name'], scale=True) Subclassing Adaptive -------------------- The default behaviors of ``Adaptive`` controlling when to scale up or down, and by how much, may not be appropriate for your cluster manager or workload. For example, you may have tasks that require a worker with more memory than usual. This means we need to pass through some additional keyword arguments to ``cluster.scale_up`` call. .. code-block:: python from distributed.deploy import Adaptive class MyAdaptive(Adaptive): def get_scale_up_kwargs(self): kwargs = super(Adaptive, self).get_scale_up_kwargs() # resource_restrictions maps task keys to a dict of restrictions restrictions = self.scheduler.resource_restrictions.values() memory_restrictions = [x.get('memory') for x in restrictions if 'memory' in x] if memory_restrictions: kwargs['memory'] = max(memory_restrictions) return kwargs So if there are any tasks that are waiting to be run on a worker with enough memory, the ``kwargs`` dictionary passed to ``cluster.scale_up`` will include a key and value for ``'memory'`` (your ``Cluster.scale_up`` method needs to be able to support this). distributed-1.20.2/docs/source/api.rst000066400000000000000000000067621321233345200176620ustar00rootroot00000000000000.. _api: API === .. currentmodule:: distributed.client **Client** .. autosummary:: Client Client.call_stack Client.cancel Client.close Client.compute Client.gather Client.get Client.get_dataset Client.get_executor Client.get_metadata Client.get_scheduler_logs Client.get_worker_logs Client.has_what Client.list_datasets Client.map Client.ncores Client.persist Client.publish_dataset Client.profile Client.rebalance Client.replicate Client.restart Client.run Client.run_on_scheduler Client.scatter Client.scheduler_info Client.set_metadata Client.start_ipython_workers Client.start_ipython_scheduler Client.submit Client.unpublish_dataset Client.upload_file Client.who_has .. currentmodule:: distributed .. autosummary:: worker_client get_worker get_client secede rejoin Reschedule .. currentmodule:: distributed.recreate_exceptions .. autosummary:: ReplayExceptionClient.get_futures_error ReplayExceptionClient.recreate_error_locally .. currentmodule:: distributed.client **Future** .. autosummary:: Future Future.add_done_callback Future.cancel Future.cancelled Future.done Future.exception Future.result Future.traceback **Client Coordination** .. currentmodule:: distributed .. autosummary:: Lock .. autosummary:: Queue .. autosummary:: Variable **Other** .. autosummary:: as_completed distributed.diagnostics.progress wait fire_and_forget Asynchronous methods -------------------- Most methods and functions can be used equally well within a blocking or asynchronous environment using Tornado coroutines. If used within a Tornado IOLoop then you should yield or await otherwise blocking operations appropriately. You must tell the client that you intend to use it within an asynchronous environment by passing the ``asynchronous=True`` keyword .. code-block:: python # blocking client = Client() future = client.submit(func, *args) # immediate, no blocking/async difference result = client.gather(future) # blocking # asynchronous Python 2/3 client = yield Client(asynchronous=True) future = client.submit(func, *args) # immediate, no blocking/async difference result = yield client.gather(future) # non-blocking/asynchronous # asynchronous Python 3 client = await Client(asynchronous=True) future = client.submit(func, *args) # immediate, no blocking/async difference result = await client.gather(future) # non-blocking/asynchronous The asynchronous variants must be run within a Tornado coroutine. See the :doc:`Asynchronous ` documentation for more information. Client ------ .. currentmodule:: distributed.client .. autoclass:: Client :members: .. autoclass:: distributed.recreate_exceptions.ReplayExceptionClient :members: Future ------ .. autoclass:: Future :members: Other ----- .. autofunction:: as_completed .. autofunction:: distributed.diagnostics.progress .. autofunction:: wait .. currentmodule:: distributed .. autofunction:: distributed.worker_client .. autofunction:: distributed.get_worker .. autofunction:: distributed.get_client .. autofunction:: distributed.secede .. autofunction:: distributed.rejoin .. autoclass:: Queue :members: .. autoclass:: Variable :members: Asyncio Client -------------- .. currentmodule:: distributed.asyncio .. autoclass:: AioClient :members: Adaptive -------- .. currentmodule:: distributed.deploy .. autoclass:: Adaptive :members: distributed-1.20.2/docs/source/asynchronous.rst000066400000000000000000000101411321233345200216260ustar00rootroot00000000000000Asynchronous Operation ====================== Dask.distributed can operate as a fully asynchronous framework and so interoperate with other highly concurrent applications. Internally Dask is built on top of Tornado coroutines but also has a compatibility layer for asyncio (see below). Basic Operation --------------- When starting a client provide the ``asynchronous=True`` keyword to tell Dask that you intend to use this client within an asynchronous context. .. code-block:: python client = await Client(asynchronous=True) Operations that used to block now provide Tornado coroutines on which you can ``await``. Fast functions that only submit work remain fast and don't need to be awaited. This includes all functions that submit work to the cluster, like ``submit``, ``map``, ``compute``, and ``persist``. .. code-block:: python future = client.submit(lambda x: x + 1, 10) You can await futures directly .. code-block:: python result = await future >>> print(result) 11 Or you can use the normal client methods. Any operation that waited until it received information from the scheduler should now be ``await``'ed. .. code-block:: python result = await client.gather(future) If you want to reuse the same client in asynchronous and synchronous environments you can apply the ``asynchronous=True`` keyword at each method call. .. code-block:: python client = Client() # normal blocking client async def f(): futures = client.map(func, L) results = await client.gather(futures, asynchronous=True) return results AsyncIO ------- If you prefer to use the Asyncio event loop over the Tornado event loop you should use the ``AioClient``. .. code-block:: python from distributed.asyncio import AioClient client = await AioClient() All other operations remain the same: .. code-block:: python future = client.submit(lambda x: x + 1, 10) result = await future # or result = await client.gather(future) Python 2 Compatibility ---------------------- Everything here works with Python 2 if you replace ``await`` with ``yield``. See more extensive comparison in the example below. Example ------- This self-contained example starts an asynchronous client, submits a trivial job, waits on the result, and then shuts down the client. You can see implementations for Python 2 and 3 and for Asyncio and Tornado. Python 3 with Tornado +++++++++++++++++++++ .. code-block:: python from dask.distributed import Client async def f(): client = await Client(asynchronous=True) future = client.submit(lambda x: x + 1, 10) result = await future await client.close() return result from tornado.ioloop import IOLoop IOLoop().run_sync(f) Python 2/3 with Tornado +++++++++++++++++++++++ .. code-block:: python from dask.distributed import Client from tornado import gen @gen.coroutine def f(): client = yield Client(asynchronous=True) future = client.submit(lambda x: x + 1, 10) result = yield future yield client.close() raise gen.Result(result) from tornado.ioloop import IOLoop IOLoop().run_sync(f) Python 3 with Asyncio +++++++++++++++++++++ .. code-block:: python from distributed.asyncio import AioClient async def f(): client = await AioClient() future = client.submit(lambda x: x + 1, 10) result = await future await client.close() return result from asyncio import get_event_loop get_event_loop().run_until_complete(f()) Use Cases --------- Historically this has been used in a few kinds of applications: 1. To integrate Dask into other asynchronous services (such as web backends), supplying a computational engine similar to Celery, but while still maintaining a high degree of concurrency and not blocking needlessly. 2. For computations that change or update state very rapidly, such as is common in some advanced machine learning workloads. 3. To develop the internals of Dask's distributed infrastucture, which is written entirely in this style. 4. For complex control and data structures in advanced applications. distributed-1.20.2/docs/source/changelog.rst000066400000000000000000000503651321233345200210360ustar00rootroot00000000000000Changelog ========= 1.20.2 - 2017-12-07 ------------------- - Clear deque handlers after each test (:pr:`1586`) `Antoine Pitrou`_ - Handle deserialization in FutureState.set_error (:pr:`1592`) `Matthew Rocklin`_ - Add process leak checker to tests (:pr:`1596`) `Antoine Pitrou`_ - Customize process title for subprocess (:pr:`1590`) `Antoine Pitrou`_ - Make linting a separate CI job (:pr:`1599`) `Antoine Pitrou`_ - Fix error from get_client() with no global client (:pr:`1595`) `Daniel Li`_ - Remove Worker.host_health, correct WorkerTable metrics (:pr:`1600`) `Matthew Rocklin`_ - Don't mark tasks as suspicious when retire_workers called. Addresses (:pr:`1607`) `Russ Bubley`_ - Do not include processing workers in workers_to_close (:pr:`1609`) `Russ Bubley`_ - Disallow simultaneous scale up and down in Adaptive (:pr:`1608`) `Russ Bubley`_ - Parse bytestrings in --memory-limit (:pr:`1615`) `Matthew Rocklin`_ - Use environment variable for scheduler address if present (:pr:`1610`) `Matthew Rocklin`_ - Fix deprecation warning from logger.warn (:pr:`1616`) `Brett Naul`_ 1.20.1 - 2017-11-26 ------------------- - Wrap ``import ssl`` statements with try-except block for ssl-crippled environments, (:pr:`1570`) `Xander Johnson`_ - Support zero memory-limit in Nanny (:pr:`1571`) `Matthew Rocklin`_ - Avoid PeriodicCallback double starts (:pr:`1573`) `Matthew Rocklin`_ - Add disposable workspace facility (:pr:`1543`) `Antoine Pitrou`_ - Use format_time in task_stream plots (:pr:`1575`) `Matthew Rocklin`_ - Avoid delayed finalize calls in compute (:pr:`1577`) `Matthew Rocklin`_ - Doc fix about secede (:pr:`1583`) `Scott Sievert`_ - Add tracemalloc option when tracking test leaks (:pr:`1585`) `Antoine Pitrou`_ - Add JSON routes to Bokeh server (:pr:`1584`) `Matthew Rocklin`_ - Handle exceptions cleanly in Variables and Queues (:pr:`1580`) `Matthew Rocklin`_ 1.20.0 - 2017-11-17 ------------------- - Drop use of pandas.msgpack (:pr:`1473`) `Matthew Rocklin`_ - Add methods to get/set scheduler metadata `Matthew Rocklin`_ - Add distributed lock `Matthew Rocklin`_ - Add reschedule exception for worker tasks `Matthew Rocklin`_ - Fix ``nbytes()`` for ``bytearrays`` `Matthew Rocklin`_ - Capture scheduler and worker logs `Matthew Rocklin`_ - Garbage collect after data eviction on high worker memory usage (:pr:`1488`) `Olivier Grisel`_ - Add scheduler HTML routes to bokeh server (:pr:`1478`) (:pr:`1514`) `Matthew Rocklin`_ - Add pytest plugin to test for resource leaks (:pr:`1499`) `Antoine Pitrou`_ - Improve documentation for scheduler states (:pr:`1498`) `Antoine Pitrou`_ - Correct warn_if_longer timeout in ThrottledGC (:pr:`1496`) `Fabian Keller`_ - Catch race condition in as_completed on cancelled futures (:pr:`1507`) `Matthew Rocklin`_ - Transactional work stealing (:pr:`1489`) (:pr:`1528`) `Matthew Rocklin`_ - Avoid forkserver in PyPy (:pr:`1509`) `Matthew Rocklin`_ - Add dict access to get/set datasets (:pr:`1508`) `Mike DePalatis`_ - Support Tornado 5 (:pr:`1509`) (:pr:`1512`) (:pr:`1518`) (:pr:`1534`) `Antoine Pitrou`_ - Move thread_state in Dask (:pr:`1523`) `Jim Crist`_ - Use new Dask collections interface (:pr:`1513`) `Matthew Rocklin`_ - Add nanny flag to dask-mpi `Matthew Rocklin`_ - Remove JSON-based HTTP servers `Matthew Rocklin`_ - Avoid doing I/O in repr/str (:pr:`1536`) `Matthew Rocklin`_ - Fix URL for MPI4Py project (:pr:`1546`) `Ian Hopkinson`_ - Allow automatic retries of a failed task (:pr:`1524`) `Antoine Pitrou`_ - Clean and accelerate tests (:pr:`1548`) (:pr:`1549`) (:pr:`1552`) (:pr:`1553`) (:pr:`1560`) (:pr:`1564`) `Antoine Pitrou`_ - Move HDFS functionality to the hdfs3 library (:pr:`1561`) `Jim Crist`_ - Fix bug when using events page with no events (:pr:`1562`) `@rbubley`_ - Improve diagnostic naming of tasks within tuples (:pr:`1566`) `Kelvyn Yang`_ 1.19.3 - 2017-10-16 ------------------- - Handle None case in profile.identity (:pr:`1456`) - Asyncio rewrite (:pr:`1458`) - Add rejoin function partner to secede (:pr:`1462`) - Nested compute (:pr:`1465`) - Use LooseVersion when comparing Bokeh versions (:pr:`1470`) 1.19.2 - 2017-10-06 ------------------- - as_completed doesn't block on cancelled futures (:pr:`1436`) - Notify waiting threads/coroutines on cancellation (:pr:`1438`) - Set Future(inform=True) as default (:pr:`1437`) - Rename Scheduler.transition_story to story (:pr:`1445`) - Future uses default client by default (:pr:`1449`) - Add keys= keyword to Client.call_stack (:pr:`1446`) - Add get_current_task to worker (:pr:`1444`) - Ensure that Client remains asynchornous before ioloop starts (:pr:`1452`) - Remove "click for worker page" in bokeh plot (:pr:`1453`) - Add Client.current() (:pr:`1450`) - Clean handling of restart timeouts (:pr:`1442`) 1.19.1 - September 25th, 2017 ----------------------------- - Fix tool issues with TaskStream plot (:pr:`1425`) - Move profile module to top level (:pr:`1423`) 1.19.0 - September 24th, 2017 ----------------------------- - Avoid storing messages in message log (:pr:`1361`) - fileConfig does not disable existing loggers (:pr:`1380`) - Offload upload_file disk I/O to separate thread (:pr:`1383`) - Add missing SSLContext (:pr:`1385`) - Collect worker thread information from sys._curent_frames (:pr:`1387`) - Add nanny timeout (:pr:`1395`) - Restart worker if memory use goes above 95% (:pr:`1397`) - Track workers memory use with psutil (:pr:`1398`) - Track scheduler delay times in workers (:pr:`1400`) - Add time slider to profile plot (:pr:`1403`) - Change memory-limit keyword to refer to maximum number of bytes (:pr:`1405`) - Add ``cancel(force=)`` keyword (:pr:`1408`) 1.18.2 - September 2nd, 2017 ---------------------------- - Silently pass on cancelled futures in as_completed (:pr:`1366`) - Fix unicode keys error in Python 2 (:pr:`1370`) - Support numeric worker names - Add dask-mpi executable (:pr:`1367`) 1.18.1 - August 25th, 2017 -------------------------- - Clean up forgotten keys in fire-and-forget workloads (:pr:`1250`) - Handle missing extensions (:pr:`1263`) - Allow recreate_exception on persisted collections (:pr:`1253`) - Add asynchronous= keyword to blocking client methods (:pr:`1272`) - Restrict to horizontal panning in bokeh plots (:pr:`1274`) - Rename client.shutdown to client.close (:pr:`1275`) - Avoid blocking on event loop (:pr:`1270`) - Avoid cloudpickle errors for Client.get_versions (:pr:`1279`) - Yield on Tornado IOStream.write futures (:pr:`1289`) - Assume async behavior if inside a sync statement (:pr:`1284`) - Avoid error messages on closing (:pr:`1297`), (:pr:`1296`) (:pr:`1318`) (:pr:`1319`) - Add timeout= keyword to get_client (:pr:`1290`) - Respect timeouts when restarting (:pr:`1304`) - Clean file descriptor and memory leaks in tests (:pr:`1317`) - Deprecate Executor (:pr:`1302`) - Add timeout to ThreadPoolExecutor.shutdown (:pr:`1330`) - Clean up AsyncProcess handling (:pr:`1324`) - Allow unicode keys in Python 2 scheduler (:pr:`1328`) - Avoid leaking stolen data (:pr:`1326`) - Improve error handling on failed nanny starts (:pr:`1337`), (:pr:`1331`) - Make Adaptive more flexible - Support ``--contact-address`` and ``--listen-address`` in worker (:pr:`1278`) - Remove old dworker, dscheduler executables (:pr:`1355`) - Exit workers if nanny process fails (:pr:`1345`) - Auto pep8 and flake (:pr:`1353`) 1.18.0 - July 8th, 2017 ----------------------- - Multi-threading safety (:pr:`1191`), (:pr:`1228`), (:pr:`1229`) - Improve handling of byte counting (:pr:`1198`) (:pr:`1224`) - Add get_client, secede functions, refactor worker-client relationship (:pr:`1201`) - Allow logging configuraiton using logging.dictConfig() (:pr:`1206`) (:pr:`1211`) - Offload serialization and deserialization to separate thread (:pr:`1218`) - Support fire-and-forget tasks (:pr:`1221`) - Support bytestrings as keys (for Julia) (:pr:`1234`) - Resolve testing corner-cases (:pr:`1236`), (:pr:`1237`), (:pr:`1240`), (:pr:`1241`), (:pr:`1242`), (:pr:`1244`) - Automatic use of scatter/gather(direct=True) in more cases (:pr:`1239`) 1.17.1 - June 14th, 2017 ------------------------ - Remove Python 3.4 testing from travis-ci (:pr:`1157`) - Remove ZMQ Support (:pr:`1160`) - Fix memoryview nbytes issue in Python 2.7 (:pr:`1165`) - Re-enable counters (:pr:`1168`) - Improve scheduler.restart (:pr:`1175`) 1.17.0 - June 9th, 2017 ----------------------- - Reevaluate worker occupancy periodically during scheduler downtime (:pr:`1038`) (:pr:`1101`) - Add ``AioClient`` asyncio-compatible client API (:pr:`1029`) (:pr:`1092`) (:pr:`1099`) - Update Keras serializer (:pr:`1067`) - Support TLS/SSL connections for security (:pr:`866`) (:pr:`1034`) - Always create new worker directory when passed ``--local-directory`` (:pr:`1079`) - Support pre-scattering data when using joblib frontent (:pr:`1022`) - Make workers more robust to failure of ``sizeof`` function (:pr:`1108`) and writing to disk (:pr:`1096`) - Add ``is_empty`` and ``update`` methods to ``as_completed`` (:pr:`1113`) - Remove ``_get`` coroutine and replace with ``get(..., sync=False)`` (:pr:`1109`) - Improve API compatibility with async/await syntax (:pr:`1115`) (:pr:`1124`) - Add distributed Queues (:pr:`1117`) and shared Variables (:pr:`1128`) to enable inter-client coordination - Support direct client-to-worker scattering and gathering (:pr:`1130`) as well as performance enhancements when scattering data - Style improvements for bokeh web dashboards (:pr:`1126`) (:pr:`1141`) as well as a removal of the external bokeh process - HTML reprs for Future and Client objects (:pr:`1136`) - Support nested collections in client.compute (:pr:`1144`) - Use normal client API in asynchronous mode (:pr:`1152`) - Remove old distributed.collections submodule (:pr:`1153`) 1.16.3 - May 5th, 2017 ---------------------- - Add bokeh template files to MANIFEST (:pr:`1063`) - Don't set worker_client.get as default get (:pr:`1061`) - Clean up logging on Client().shutdown() (:pr:`1055`) 1.16.2 - May 3rd, 2017 ---------------------- - Support ``async with Client`` syntax (:pr:`1053`) - Use internal bokeh server for default diagnostics server (:pr:`1047`) - Improve styling of bokeh plots when empty (:pr:`1046`) (:pr:`1037`) - Support efficient serialization for sparse arrays (:pr:`1040`) - Prioritize newly arrived work in worker (:pr:`1035`) - Prescatter data with joblib backend (:pr:`1022`) - Make client.restart more robust to worker failure (:pr:`1018`) - Support preloading a module or script in dask-worker or dask-scheduler processes (:pr:`1016`) - Specify network interface in command line interface (:pr:`1007`) - Client.scatter supports a single element (:pr:`1003`) - Use blosc compression on all memoryviews passing through comms (:pr:`998`) - Add concurrent.futures-compatible Executor (:pr:`997`) - Add as_completed.batches method and return results (:pr:`994`) (:pr:`971`) - Allow worker_clients to optionally stay within the thread pool (:pr:`993`) - Add bytes-stored and tasks-processing diagnostic histograms (:pr:`990`) - Run supports non-msgpack-serializable results (:pr:`965`) 1.16.1 - March 22nd, 2017 ------------------------- - Use inproc transport in LocalCluster (:pr:`919`) - Add structured and queryable cluster event logs (:pr:`922`) - Use connection pool for inter-worker communication (:pr:`935`) - Robustly shut down spawned worker processes at shutdown (:pr:`928`) - Worker death timeout (:pr:`940`) - More visual reporting of exceptions in progressbar (:pr:`941`) - Render disk and serialization events to task stream visual (:pr:`943`) - Support async for / await protocol (:pr:`952`) - Ensure random generators are re-seeded in worker processes (:pr:`953`) - Upload sourcecode as zip module (:pr:`886`) - Replay remote exceptions in local process (:pr:`894`) 1.16.0 - February 24th, 2017 ---------------------------- - First come first served priorities on client submissions (:pr:`840`) - Can specify Bokeh internal ports (:pr:`850`) - Allow stolen tasks to return from either worker (:pr:`853`), (:pr:`875`) - Add worker resource constraints during execution (:pr:`857`) - Send small data through Channels (:pr:`858`) - Better estimates for SciPy sparse matrix memory costs (:pr:`863`) - Avoid stealing long running tasks (:pr:`873`) - Maintain fortran ordering of NumPy arrays (:pr:`876`) - Add ``--scheduler-file`` keyword to dask-scheduler (:pr:`877`) - Add serializer for Keras models (:pr:`878`) - Support uploading modules from zip files (:pr:`886`) - Improve titles of Bokeh dashboards (:pr:`895`) 1.15.2 - January 27th, 2017 --------------------------- * Fix a bug where arrays with large dtypes or shapes were being improperly compressed (:pr:`830` :pr:`832` :pr:`833`) * Extend ``as_completed`` to accept new futures during iteration (:pr:`829`) * Add ``--nohost`` keyword to ``dask-ssh`` startup utility (:pr:`827`) * Support scheduler shutdown of remote workers, useful for adaptive clusters (:pr: `811` :pr:`816` :pr:`821`) * Add ``Client.run_on_scheduler`` method for running debug functions on the scheduler (:pr:`808`) 1.15.1 - January 11th, 2017 --------------------------- * Make compatibile with Bokeh 0.12.4 (:pr:`803`) * Avoid compressing arrays if not helpful (:pr:`777`) * Optimize inter-worker data transfer (:pr:`770`) (:pr:`790`) * Add --local-directory keyword to worker (:pr:`788`) * Enable workers to arrive to the cluster with their own data. Useful if a worker leaves and comes back (:pr:`785`) * Resolve thread safety bug when using local_client (:pr:`802`) * Resolve scheduling issues in worker (:pr:`804`) 1.15.0 - January 2nd, 2017 -------------------------- * Major Worker refactor (:pr:`704`) * Major Scheduler refactor (:pr:`717`) (:pr:`722`) (:pr:`724`) (:pr:`742`) (:pr:`743` * Add ``check`` (default is ``False``) option to ``Client.get_versions`` to raise if the versions don't match on client, scheduler & workers (:pr:`664`) * ``Future.add_done_callback`` executes in separate thread (:pr:`656`) * Clean up numpy serialization (:pr:`670`) * Support serialization of Tornado v4.5 coroutines (:pr:`673`) * Use CPickle instead of Pickle in Python 2 (:pr:`684`) * Use Forkserver rather than Fork on Unix in Python 3 (:pr:`687`) * Support abstract resources for per-task constraints (:pr:`694`) (:pr:`720`) (:pr:`737`) * Add TCP timeouts (:pr:`697`) * Add embedded Bokeh server to workers (:pr:`709`) (:pr:`713`) (:pr:`738`) * Add embedded Bokeh server to scheduler (:pr:`724`) (:pr:`736`) (:pr:`738`) * Add more precise timers for Windows (:pr:`713`) * Add Versioneer (:pr:`715`) * Support inter-client channels (:pr:`729`) (:pr:`749`) * Scheduler Performance improvements (:pr:`740`) (:pr:`760`) * Improve load balancing and work stealing (:pr:`747`) (:pr:`754`) (:pr:`757`) * Run Tornado coroutines on workers * Avoid slow sizeof call on Pandas dataframes (:pr:`758`) 1.14.3 - November 13th, 2016 ---------------------------- * Remove custom Bokeh export tool that implicitly relied on nodejs (:pr:`655`) * Clean up scheduler logging (:pr:`657`) 1.14.2 - November 11th, 2016 ---------------------------- * Support more numpy dtypes in custom serialization, (:pr:`627`), (:pr:`630`), (:pr:`636`) * Update Bokeh plots (:pr:`628`) * Improve spill to disk heuristics (:pr:`633`) * Add Export tool to Task Stream plot * Reverse frame order in loads for very many frames (:pr:`651`) * Add timeout when waiting on write (:pr:`653`) 1.14.0 - November 3rd, 2016 --------------------------- * Add ``Client.get_versions()`` function to return software and package information from the scheduler, workers, and client (:pr:`595`) * Improved windows support (:pr:`577`) (:pr:`590`) (:pr:`583`) (:pr:`597`) * Clean up rpc objects explicitly (:pr:`584`) * Normalize collections against known futures (:pr:`587`) * Add key= keyword to map to specify keynames (:pr:`589`) * Custom data serialization (:pr:`606`) * Refactor the web interface (:pr:`608`) (:pr:`615`) (:pr:`621`) * Allow user-supplied Executor in Worker (:pr:`609`) * Pass Worker kwargs through LocalCluster 1.13.3 - October 15th, 2016 --------------------------- * Schedulers can retire workers cleanly * Add ``Future.add_done_callback`` for ``concurrent.futures`` compatibility * Update web interface to be consistent with Bokeh 0.12.3 * Close streams explicitly, avoiding race conditions and supporting more robust restarts on Windows. * Improved shuffled performance for dask.dataframe * Add adaptive allocation cluster manager * Reduce administrative overhead when dealing with many workers * ``dask-ssh --log-directory .`` no longer errors * Microperformance tuning for the scheduler 1.13.2 ------ * Revert dask_worker to use fork rather than subprocess by default * Scatter retains type information * Bokeh always uses subprocess rather than spawn 1.13.1 ------ * Fix critical Windows error with dask_worker executable 1.13.0 ------ * Rename Executor to Client (:pr:`492`) * Add ``--memory-limit`` option to ``dask-worker``, enabling spill-to-disk behavior when running out of memory (:pr:`485`) * Add ``--pid-file`` option to dask-worker and ``--dask-scheduler`` (:pr:`496`) * Add ``upload_environment`` function to distribute conda environments. This is experimental, undocumented, and may change without notice. (:pr:`494`) * Add ``workers=`` keyword argument to ``Client.compute`` and ``Client.persist``, supporting location-restricted workloads with Dask collections (:pr:`484`) * Add ``upload_environment`` function to distribute conda environments. This is experimental, undocumented, and may change without notice. (:pr:`494`) * Add optional ``dask_worker=`` keyword to ``client.run`` functions that gets provided the worker or nanny object * Add ``nanny=False`` keyword to ``Client.run``, allowing for the execution of arbitrary functions on the nannies as well as normal workers 1.12.2 ------ This release adds some new features and removes dead code * Publish and share datasets on the scheduler between many clients (:pr:`453`). See :doc:`publish`. * Launch tasks from other tasks (experimental) (:pr:`471`). See :doc:`task-launch`. * Remove unused code, notably the ``Center`` object and older client functions (:pr:`478`) * ``Executor()`` and ``LocalCluster()`` is now robust to Bokeh's absence (:pr:`481`) * Removed s3fs and boto3 from requirements. These have moved to Dask. 1.12.1 ------ This release is largely a bugfix release, recovering from the previous large refactor. * Fixes from previous refactor * Ensure idempotence across clients * Stress test losing scattered data permanently * IPython fixes * Add ``start_ipython_scheduler`` method to Executor * Add ``%remote`` magic for workers * Clean up code and tests * Pool connects to maintain reuse and reduce number of open file handles * Re-implement work stealing algorithm * Support cancellation of tuple keys, such as occur in dask.arrays * Start synchronizing against worker data that may be superfluous * Improve bokeh plots styling * Add memory plot tracking number of bytes * Make the progress bars more compact and align colors * Add workers/ page with workers table, stacks/processing plot, and memory * Add this release notes document 1.12.0 ------ This release was largely a refactoring release. Internals were changed significantly without many new features. * Major refactor of the scheduler to use transitions system * Tweak protocol to traverse down complex messages in search of large bytestrings * Add dask-submit and dask-remote * Refactor HDFS writing to align with changes in the dask library * Executor reconnects to scheduler on broken connection or failed scheduler * Support sklearn.external.joblib as well as normal joblib .. _`Antoine Pitrou`: https://github.com/pitrou .. _`Olivier Grisel`: https://github.com/ogrisel .. _`Fabian Keller`: https://github.com/bluenote10 .. _`Mike DePalatis`: https://github.com/mivade .. _`Matthew Rocklin`: https://github.com/mrocklin .. _`Jim Crist`: https://github.com/jcrist .. _`Ian Hopkinson`: https://github.com/IanHopkinson .. _`@rbubley`: https://github.com/rbubley .. _`Kelvyn Yang`: https://github.com/kelvynyang .. _`Scott Sievert`: https://github.com/stsievert .. _`Xander Johnson`: https://github.com/metasyn .. _`Daniel Li`: https://github.com/li-dan .. _`Brett Naul`: https://github.com/bnaul distributed-1.20.2/docs/source/client.rst000066400000000000000000000152771321233345200203700ustar00rootroot00000000000000Client ======== The Client is the primary entry point for users of ``dask.distributed``. After we :doc:`setup a cluster `, we initialize a ``Client`` by pointing it to the address of a ``Scheduler``: .. code-block:: python >>> from distributed import Client >>> client = Client('127.0.0.1:8786') There are a few different ways to interact with the cluster through the client: 1. The Client satisfies most of the standard concurrent.futures_ - PEP-3148_ interface with ``.submit``, ``.map`` functions and ``Future`` objects, allowing the immediate and direct submission of tasks. 2. The Client registers itself as the default Dask_ scheduler, and so runs all dask collections like dask.array_, dask.bag_, dask.dataframe_ and dask.delayed_ 3. The Client has additional methods for manipulating data remotely. See the full :doc:`API ` for a thorough list. Concurrent.futures ------------------ We can submit individual function calls with the ``client.submit`` method or many function calls with the ``client.map`` method .. code-block:: python >>> def inc(x): return x + 1 >>> x = client.submit(inc, 10) >>> x >>> L = client.map(inc, range(1000)) >>> L [, , , , ...] These results live on distributed workers. We can submit tasks on futures. The function will go to the machine where the futures are stored and run on the result once it has completed. .. code-block:: python >>> y = client.submit(inc, x) # Submit on x, a Future >>> total = client.submit(sum, L) # Map on L, a list of Futures We gather back the results using either the ``Future.result`` method for single futures or ``client.gather`` method for many futures at once. .. code-block:: python >>> x.result() 11 >>> client.gather(L) [1, 2, 3, 4, 5, ...] But, as always, we want to minimize communicating results back to the local process. It's often best to leave data on the cluster and operate on it remotely with functions like ``submit``, ``map``, ``get`` and ``compute``. See :doc:`efficiency ` for more information on efficient use of distributed. Dask ---- The parent library Dask_ contains objects like dask.array_, dask.dataframe_, dask.bag_, and dask.delayed_, which automatically produce parallel algorithms on larger datasets. All dask collections work smoothly with the distributed scheduler. When we create a ``Client`` object it registers itself as the default Dask scheduler. All ``.compute()`` methods will automatically start using the distributed system. .. code-block:: python client = Client('scheduler:8786') my_dataframe.sum().compute() # Now uses the distributed system by default We can stop this behavior by using the ``set_as_default=False`` keyword argument when starting the Client. Dask's normal ``.compute()`` methods are *synchronous*, meaning that they block the interpreter until they complete. Dask.distributed allows the new ability of *asynchronous* computing, we can trigger computations to occur in the background and persist in memory while we continue doing other work. This is typically handled with the ``Client.persist`` and ``Client.compute`` methods which are used for larger and smaller result sets respectively. .. code-block:: python >>> df = client.persist(df) # trigger all computations, keep df in memory >>> type(df) dask.DataFrame For more information see the page on :doc:`Managing Computation `. Pure Functions by Default ------------------------- By default we assume that all functions are pure_. If this is not the case we should use the ``pure=False`` keyword argument. The client associates a key to all computations. This key is accessible on the Future object. .. code-block:: python >>> from operator import add >>> x = client.submit(add, 1, 2) >>> x.key 'add-ebf39f96ad7174656f97097d658f3fa2' This key should be the same across all computations with the same inputs and across all machines. If we run the computation above on any computer with the same environment then we should get the exact same key. The scheduler avoids redundant computations. If the result is already in memory from a previous call then that old result will be used rather than recomputing it. Calls to submit or map are idempotent in the common case. While convenient, this feature may be undesired for impure functions, like ``random``. In these cases two calls to the same function with the same inputs should produce different results. We accomplish this with the ``pure=False`` keyword argument. In this case keys are randomly generated (by ``uuid4``.) .. code-block:: python >>> import numpy as np >>> client.submit(np.random.random, 1000, pure=False).key 'random_sample-fc814a39-ee00-42f3-8b6f-cac65bcb5556' >>> client.submit(np.random.random, 1000, pure=False).key 'random_sample-a24e7220-a113-47f2-a030-72209439f093' .. _pure: https://toolz.readthedocs.io/en/latest/purity.html Tornado Coroutines ------------------ If we are operating in an asynchronous environment then the blocking functions listed above become asynchronous equivalents. You must start your client with the ``asynchronous=True`` keyword and ``yield`` or ``await`` blocking functions. .. code-block:: python @gen.coroutine def f(): client = yield Client(asynchronous=True) future = client.submit(func, *args) result = yield future return result If you want to reuse the same client in asynchronous and synchronous environments you can apply the ``asynchronous=True`` keyword at each method call. .. code-block:: python client = Client() # normal blocking client @gen.coroutine def f(): futures = client.map(func, L) results = yield client.gather(futures, asynchronous=True) return results See the :doc:`Asynchronous ` documentation for more information. Additional Links ---------------- For more information on how to use dask.distributed you may want to look at the following pages: * :doc:`Managing Memory ` * :doc:`Managing Computation ` * :doc:`Data Locality ` * :doc:`API ` .. _concurrent.futures: https://docs.python.org/3/library/concurrent.futures.html .. _PEP-3148: https://www.python.org/dev/peps/pep-3148/ .. _dask.array: http://dask.pydata.org/en/latest/array.html .. _dask.bag: http://dask.pydata.org/en/latest/bag.html .. _dask.dataframe: http://dask.pydata.org/en/latest/dataframe.html .. _dask.delayed: http://dask.pydata.org/en/latest/delayed.html .. _Dask: http://dask.pydata.org/en/latest/ distributed-1.20.2/docs/source/communications.rst000066400000000000000000000061611321233345200221320ustar00rootroot00000000000000.. _communications: ============== Communications ============== Workers, the Scheduler, and Clients communicate by sending each other Python objects (such as :ref:`protocol` messages or user data). The communication layer handles appropriate encoding and shipping of those Python objects between the distributed endpoints. The communication layer is able to select between different transport implementations, depending on user choice or (possibly) internal optimizations. The communication layer lives in the :mod:`distributed.comm` package. Addresses ========= Communication addresses are canonically represented as URIs, such as ``tcp://127.0.0.1:1234``. For compatibility with existing code, if the URI scheme is omitted, a default scheme of ``tcp`` is assumed (so ``127.0.0.1:456`` is really the same as ``tcp://127.0.0.1:456``). The default scheme may change in the future. The following schemes are currently implemented in the ``distributed`` source tree: * ``tcp`` is the main transport; it uses TCP sockets and allows for IPv4 and IPv6 addresses. * ``tls`` is a secure transport using the well-known `TLS protocol`_ over TCP sockets. Using it requires specifying keys and certificates as outlined in :ref:`tls`. * ``inproc`` is an in-process transport using simple object queues; it eliminates serialization and I/O overhead, providing almost zero-cost communication between endpoints as long as they are situated in the same process. Some URIs may be valid for listening but not for connecting. For example, the URI ``tcp://`` will listen on all IPv4 and IPv6 addresses and on an arbitrary port, but you cannot connect to that address. Higher-level APIs in ``distributed`` may accept other address formats for convenience or compatibility, for example a ``(host, port)`` pair. However, the abstract communications layer always deals with URIs. .. _TLS protocol: https://en.wikipedia.org/wiki/Transport_Layer_Security Functions --------- There are a number of top-level functions in :mod:`distributed.comm` to help deal with addresses: .. autofunction:: distributed.comm.parse_address .. autofunction:: distributed.comm.unparse_address .. autofunction:: distributed.comm.normalize_address .. autofunction:: distributed.comm.resolve_address .. autofunction:: distributed.comm.get_address_host Communications API ================== The basic unit for dealing with established communications is the ``Comm`` object: .. autoclass:: distributed.comm.Comm :members: You don't create ``Comm`` objects directly: you either ``listen`` for incoming communications, or ``connect`` to a peer listening for connections: .. autofunction:: distributed.comm.connect .. autofunction:: distributed.comm.listen Listener objects expose the following interface: .. autoclass:: distributed.comm.core.Listener :members: Extending the Communication Layer ================================= Each transport is represented by a URI scheme (such as ``tcp``) and backed by a dedicated :class:`Backend` implementation, which provides entry points into all transport-specific routines. .. autoclass:: distributed.comm.registry.Backend :members: distributed-1.20.2/docs/source/conf.py000066400000000000000000000277301321233345200176540ustar00rootroot00000000000000# -*- coding: utf-8 -*- # # Dask.distributed documentation build configuration file, created by # sphinx-quickstart on Tue Oct 6 14:42:44 2015. # # This file is execfile()d with the current directory set to its # containing dir. # # Note that not all possible configuration values are present in this # autogenerated file. # # All configuration values have a default; values that are commented out # serve to show the default. import sys import os import shlex # If extensions (or modules to document with autodoc) are in another directory, # add these directories to sys.path here. If the directory is relative to the # documentation root, use os.path.abspath to make it absolute, like shown here. #sys.path.insert(0, os.path.abspath('.')) # -- General configuration ------------------------------------------------ # If your documentation needs a minimal Sphinx version, state it here. #needs_sphinx = '1.0' # Add any Sphinx extension module names here, as strings. They can be # extensions coming with Sphinx (named 'sphinx.ext.*') or your custom # ones. extensions = [ 'sphinx.ext.autodoc', 'sphinx.ext.todo', 'sphinx.ext.ifconfig', 'sphinx.ext.viewcode', 'sphinx.ext.autosummary', 'sphinx.ext.extlinks', 'sphinx.ext.intersphinx', 'numpydoc', ] numpydoc_show_class_members = False # Add any paths that contain templates here, relative to this directory. templates_path = ['_templates'] # The suffix(es) of source filenames. # You can specify multiple suffix as a list of string: # source_suffix = ['.rst', '.md'] source_suffix = '.rst' # The encoding of source files. #source_encoding = 'utf-8-sig' # The master toctree document. master_doc = 'index' # General information about the project. project = u'Dask.distributed' copyright = u'2016, Anaconda, Inc.' author = u'Anaconda, Inc.' # The version info for the project you're documenting, acts as replacement for # |version| and |release|, also used in various other places throughout the # built documents. # # The short X.Y version. import distributed version = distributed.__version__ # The full version, including alpha/beta/rc tags. release = distributed.__version__ # The language for content autogenerated by Sphinx. Refer to documentation # for a list of supported languages. # # This is also used if you do content translation via gettext catalogs. # Usually you set "language" from the command line for these cases. language = None # There are two options for replacing |today|: either, you set today to some # non-false value, then it is used: #today = '' # Else, today_fmt is used as the format for a strftime call. #today_fmt = '%B %d, %Y' # List of patterns, relative to source directory, that match files and # directories to ignore when looking for source files. exclude_patterns = [] # The reST default role (used for this markup: `text`) to use for all # documents. #default_role = None # If true, '()' will be appended to :func: etc. cross-reference text. #add_function_parentheses = True # If true, the current module name will be prepended to all description # unit titles (such as .. function::). #add_module_names = True # If true, sectionauthor and moduleauthor directives will be shown in the # output. They are ignored by default. #show_authors = False # The name of the Pygments (syntax highlighting) style to use. pygments_style = 'sphinx' # A list of ignored prefixes for module index sorting. #modindex_common_prefix = [] # If true, keep warnings as "system message" paragraphs in the built documents. #keep_warnings = False # If true, `todo` and `todoList` produce output, else they produce nothing. todo_include_todos = True # -- Options for HTML output ---------------------------------------------- # Taken from docs.readthedocs.io: # on_rtd is whether we are on readthedocs.io on_rtd = os.environ.get('READTHEDOCS', None) == 'True' if not on_rtd: # only import and set the theme if we're building docs locally import sphinx_rtd_theme html_theme = 'sphinx_rtd_theme' html_theme_path = [sphinx_rtd_theme.get_html_theme_path()] # Theme options are theme-specific and customize the look and feel of a theme # further. For a list of options available for each theme, see the # documentation. #html_theme_options = {} # Add any paths that contain custom themes here, relative to this directory. #html_theme_path = [] # The name for this set of Sphinx documents. If None, it defaults to # " v documentation". #html_title = None # A shorter title for the navigation bar. Default is the same as html_title. #html_short_title = None # The name of an image file (relative to this directory) to place at the top # of the sidebar. #html_logo = None # The name of an image file (within the static path) to use as favicon of the # docs. This file should be a Windows icon file (.ico) being 16x16 or 32x32 # pixels large. #html_favicon = None # Add any paths that contain custom static files (such as style sheets) here, # relative to this directory. They are copied after the builtin static files, # so a file named "default.css" will overwrite the builtin "default.css". html_static_path = ['_static'] # Add any extra paths that contain custom files (such as robots.txt or # .htaccess) here, relative to this directory. These files are copied # directly to the root of the documentation. #html_extra_path = [] # If not '', a 'Last updated on:' timestamp is inserted at every page bottom, # using the given strftime format. #html_last_updated_fmt = '%b %d, %Y' # If true, SmartyPants will be used to convert quotes and dashes to # typographically correct entities. #html_use_smartypants = True # Custom sidebar templates, maps document names to template names. #html_sidebars = {} # Additional templates that should be rendered to pages, maps page names to # template names. #html_additional_pages = {} # If false, no module index is generated. #html_domain_indices = True # If false, no index is generated. #html_use_index = True # If true, the index is split into individual pages for each letter. #html_split_index = False # If true, links to the reST sources are added to the pages. #html_show_sourcelink = True # If true, "Created using Sphinx" is shown in the HTML footer. Default is True. #html_show_sphinx = True # If true, "(C) Copyright ..." is shown in the HTML footer. Default is True. #html_show_copyright = True # If true, an OpenSearch description file will be output, and all pages will # contain a tag referring to it. The value of this option must be the # base URL from which the finished HTML is served. #html_use_opensearch = '' # This is the file name suffix for HTML files (e.g. ".xhtml"). #html_file_suffix = None # Language to be used for generating the HTML full-text search index. # Sphinx supports the following languages: # 'da', 'de', 'en', 'es', 'fi', 'fr', 'hu', 'it', 'ja' # 'nl', 'no', 'pt', 'ro', 'ru', 'sv', 'tr' #html_search_language = 'en' # A dictionary with options for the search language support, empty by default. # Now only 'ja' uses this config value #html_search_options = {'type': 'default'} # The name of a javascript file (relative to the configuration directory) that # implements a search results scorer. If empty, the default will be used. #html_search_scorer = 'scorer.js' # Output file base name for HTML help builder. htmlhelp_basename = 'distributeddoc' # -- Options for LaTeX output --------------------------------------------- latex_elements = { # The paper size ('letterpaper' or 'a4paper'). #'papersize': 'letterpaper', # The font size ('10pt', '11pt' or '12pt'). #'pointsize': '10pt', # Additional stuff for the LaTeX preamble. #'preamble': '', # Latex figure (float) alignment #'figure_align': 'htbp', } # Grouping the document tree into LaTeX files. List of tuples # (source start file, target name, title, # author, documentclass [howto, manual, or own class]). latex_documents = [ (master_doc, 'distributed.tex', u'Dask.distributed Documentation', u'Matthew Rocklin', 'manual'), ] # The name of an image file (relative to this directory) to place at the top of # the title page. #latex_logo = None # For "manual" documents, if this is true, then toplevel headings are parts, # not chapters. #latex_use_parts = False # If true, show page references after internal links. #latex_show_pagerefs = False # If true, show URL addresses after external links. #latex_show_urls = False # Documents to append as an appendix to all manuals. #latex_appendices = [] # If false, no module index is generated. #latex_domain_indices = True # -- Options for manual page output --------------------------------------- # One entry per manual page. List of tuples # (source start file, name, description, authors, manual section). man_pages = [ (master_doc, 'Dask.distributed', u'Dask.distributed Documentation', [author], 1) ] # If true, show URL addresses after external links. #man_show_urls = False # -- Options for Texinfo output ------------------------------------------- # Grouping the document tree into Texinfo files. List of tuples # (source start file, target name, title, author, # dir menu entry, description, category) texinfo_documents = [ (master_doc, 'Dask.distributed', u'Dask.distributed Documentation', author, 'Dask.distributed', 'One line description of project.', 'Miscellaneous'), ] # Documents to append as an appendix to all manuals. #texinfo_appendices = [] # If false, no module index is generated. #texinfo_domain_indices = True # How to display URL addresses: 'footnote', 'no', or 'inline'. #texinfo_show_urls = 'footnote' # If true, do not generate a @detailmenu in the "Top" node's menu. #texinfo_no_detailmenu = False # -- Options for Epub output ---------------------------------------------- # Bibliographic Dublin Core info. epub_title = project epub_author = author epub_publisher = author epub_copyright = copyright # The basename for the epub file. It defaults to the project name. #epub_basename = project # The HTML theme for the epub output. Since the default themes are not optimized # for small screen space, using the same theme for HTML and epub output is # usually not wise. This defaults to 'epub', a theme designed to save visual # space. #epub_theme = 'epub' # The language of the text. It defaults to the language option # or 'en' if the language is not set. #epub_language = '' # The scheme of the identifier. Typical schemes are ISBN or URL. #epub_scheme = '' # The unique identifier of the text. This can be a ISBN number # or the project homepage. #epub_identifier = '' # A unique identification for the text. #epub_uid = '' # A tuple containing the cover image and cover page html template filenames. #epub_cover = () # A sequence of (type, uri, title) tuples for the guide element of content.opf. #epub_guide = () # HTML files that should be inserted before the pages created by sphinx. # The format is a list of tuples containing the path and title. #epub_pre_files = [] # HTML files shat should be inserted after the pages created by sphinx. # The format is a list of tuples containing the path and title. #epub_post_files = [] # A list of files that should not be packed into the epub file. epub_exclude_files = ['search.html'] # The depth of the table of contents in toc.ncx. #epub_tocdepth = 3 # Allow duplicate toc entries. #epub_tocdup = True # Choose between 'default' and 'includehidden'. #epub_tocscope = 'default' # Fix unsupported image types using the Pillow. #epub_fix_images = False # Scale large images. #epub_max_image_width = 0 # How to display URL addresses: 'footnote', 'no', or 'inline'. #epub_show_urls = 'inline' # If false, no index is generated. #epub_use_index = True # Link to GitHub issues and pull requests using :pr:`1234` and :issue:`1234` # syntax extlinks = { 'issue': ('https://github.com/dask/distributed/issues/%s', 'GH#'), 'pr': ('https://github.com/dask/distributed/pull/%s', 'GH#') } # Configuration for intersphinx: refer to the Python standard library # and the Numpy documentation. intersphinx_mapping = { 'python': ('https://docs.python.org/3', None), 'numpy': ('http://docs.scipy.org/doc/numpy', None), } distributed-1.20.2/docs/source/configuration.rst000066400000000000000000000127461321233345200217570ustar00rootroot00000000000000.. _configuration: ============= Configuration ============= As with any distributed computation system, taking full advantage of Dask distributed sometimes requires configuration. Some options can be passed as :ref:`API ` parameters and/or command line options to the various Dask executables. However, some options can also be entered in the Dask configuration file. User-wide configuration ======================= Dask accepts some configuration options in a configuration file, which by default is a ``.dask/config.yaml`` file located in your home directory. The file path can be overriden using the ``DASK_CONFIG`` environment variable. The file is written in the YAML format, which allows for a human-readable hierarchical key-value configuration. All keys in the configuration file are optional, though Dask will create a default configuration file for you on its first launch. Here is a synopsis of the configuration file: .. code-block:: yaml logging: distributed: info distributed.client: warning bokeh: critical # Scheduler options bandwidth: 100000000 # 100 MB/s estimated worker-worker bandwidth allowed-failures: 3 # number of retries before a task is considered bad pdb-on-err: False # enter debug mode on scheduling error transition-log-length: 100000 # Worker options multiprocessing-method: forkserver # Communication options compression: auto tcp-timeout: 30 # seconds delay before calling an unresponsive connection dead default-scheme: tcp require-encryption: False # whether to require encryption on non-local comms tls: ca-file: myca.pem scheduler: cert: mycert.pem key: mykey.pem worker: cert: mycert.pem key: mykey.pem client: cert: mycert.pem key: mykey.pem #ciphers: #ECDHE-ECDSA-AES128-GCM-SHA256 # Bokeh web dashboard bokeh-export-tool: False We will review some of those options hereafter. Communication options --------------------- ``compression`` """"""""""""""" This key configures the desired compression scheme when transferring data over the network. The default value, "auto", applies heuristics to try and select the best compression scheme for each piece of data. ``default-scheme`` """""""""""""""""" The :ref:`communication ` scheme used by default. You can override the default ("tcp") here, but it is recommended to use explicit URIs for the various endpoints instead (for example ``tls://`` if you want to enable :ref:`TLS ` communications). ``require-encryption`` """""""""""""""""""""" Whether to require that all non-local communications be encrypted. If true, then Dask will refuse establishing any clear-text communications (for example over TCP without TLS), forcing you to use a secure transport such as :ref:`TLS `. ``tcp-timeout`` """"""""""""""" The default "timeout" on TCP sockets. If a remote endpoint is unresponsive (at the TCP layer, not at the distributed layer) for at least the specified number of seconds, the communication is considered closed. This helps detect endpoints that have been killed or have disconnected abruptly. ``tls`` """"""" This key configures :ref:`TLS ` communications. Several sub-keys are recognized: * ``ca-file`` configures the CA certificate file used to authenticate and authorize all endpoints. * ``ciphers`` restricts allowed ciphers on TLS communications. Each kind of endpoint has a dedicated endpoint sub-key: ``scheduler``, ``worker`` and ``client``. Each endpoint sub-key also supports several sub-keys: * ``cert`` configures the certificate file for the endpoint. * ``key`` configures the private key file for the endpoint. Scheduler options ----------------- ``allowed-failures`` """""""""""""""""""" The number of retries before a "suspicious" task is considered bad. A task is considered "suspicious" if the worker died while executing it. ``bandwidth`` """"""""""""" The estimated network bandwidth, in bytes per second, from worker to worker. This value is used to estimate the time it takes to ship data from one node to another, and balance tasks and data accordingly. Misc options ------------ ``logging`` """"""""""" This key configures the logging settings. There are two possible formats. The simple, recommended format configures the desired verbosity level for each logger. It also sets default values for several loggers such as ``distributed`` unless explicitly configured. A more extended format is possible following the :mod:`logging` module's `Configuration dictionary schema `_. To enable this extended format, there must be a ``version`` sub-key as mandated by the schema. The extended format does not set any default values. .. note:: Python's :mod:`logging` module uses a hierarchical logger tree. For example, configuring the logging level for the ``distributed`` logger will also affect its children such as ``distributed.scheduler``, unless explicitly overriden. ``logging-file-config`` """"""""""""""""""""""" As an alternative to the two logging settings formats discussed above, you can specify a logging config file. Its format adheres to the :mod:`logging` module's `Configuration file format `_. .. note:: The configuration options `logging-file-config` and `logging` are mutually exclusive.distributed-1.20.2/docs/source/develop.rst000066400000000000000000000127601321233345200205420ustar00rootroot00000000000000Development Guidelines ====================== This repository is part of the Dask_ projects. General development guidelines including where to ask for help, a layout of repositories, testing practices, and documentation and style standards are available at the `Dask developer guidelines`_ in the main documentation. .. _Dask: http://dask.pydata.org/en/latest/ .. _`Dask developer guidelines`: http://dask.pydata.org/en/latest/develop.html Install ------- After setting up an environment as described in the `Dask developer guidelines`_ you can clone this repository with git:: git clone git@github.com:dask/distributed.git and install it from source:: cd distributed python setup.py install Test ---- Test using ``py.test``:: py.test distributed --verbose Tornado ------- Dask.distributed is a Tornado TCP application. Tornado provides us with both a communication layer on top of sockets, as well as a syntax for writing asynchronous coroutines, similar to asyncio. You can make modest changes to the policies within this library without understanding much about Tornado, however moderate changes will probably require you to understand Tornado IOLoops, coroutines, and a little about non-blocking communication.. The Tornado API documentation is quite good and we recommend that you read the following resources: * http://www.tornadoweb.org/en/stable/gen.html * http://www.tornadoweb.org/en/stable/ioloop.html Additionally, if you want to interact at a low level with the communication between workers and scheduler then you should understand the Tornado ``TCPServer`` and ``IOStream`` available here: * http://www.tornadoweb.org/en/stable/networking.html Dask.distributed wraps a bit of logic around Tornado. See :doc:`Foundations` for more information. Writing Tests ------------- Testing distributed systems is normally quite difficult because it is difficult to inspect the state of all components when something goes wrong. Fortunately, the non-blocking asynchronous model within Tornado allows us to run a scheduler, multiple workers, and multiple clients all within a single thread. This gives us predictable performance, clean shutdowns, and the ability to drop into any point of the code during execution. At the same time, sometimes we want everything to run in different processes in order to simulate a more realistic setting. The test suite contains three kinds of tests 1. ``@gen_cluster``: Fully asynchronous tests where all components live in the same event loop in the main thread. These are good for testing complex logic and inspecting the state of the system directly. They are also easier to debug and cause the fewest problems with shutdowns. 2. ``with cluster()``: Tests with multiple processes forked from the master process. These are good for testing the synchronous (normal user) API and when triggering hard failures for resilience tests. 3. ``popen``: Tests that call out to the command line to start the system. These are rare and mostly for testing the command line interface. If you are comfortable with the Tornado interface then you will be happiest using the ``@gen_cluster`` style of test .. code-block:: python @gen_cluster(client=True) def test_submit(c, s, a, b): assert isinstance(c, Client) assert isinstance(s, Scheduler) assert isinstance(a, Worker) assert isinstance(b, Worker) future = c.submit(inc, 1) assert future.key in c.futures # result = future.result() # This synchronous API call would block result = yield future assert result == 2 assert future.key in s.tasks assert future.key in a.data or future.key in b.data The ``@gen_cluster`` decorator sets up a scheduler, client, and workers for you and cleans them up after the test. It also allows you to directly inspect the state of every element of the cluster directly. However, you can not use the normal synchronous API (doing so will cause the test to wait forever) and instead you need to use the coroutine API, where all blocking functions are prepended with an underscore (``_``). Beware, it is a common mistake to use the blocking interface within these tests. If you want to test the normal synchronous API you can use a ``with cluster`` style test, which sets up a scheduler and workers for you in different forked processes: .. code-block:: python def test_submit_sync(loop): with cluster() as (s, [a, b]): with Client(('127.0.0.1', s['port']), loop=loop) as c: future = c.submit(inc, 1) assert future.key in c.futures result = future.result() # use the synchronous/blocking API here assert result == 2 a['proc'].terminate() # kill one of the workers result = future.result() # test that future remains valid assert result == 2 In this style of test you do not have access to the scheduler or workers. The variables ``s, a, b`` are now dictionaries holding a ``multiprocessing.Process`` object and a port integer. However, you can now use the normal synchronous API (never use yield in this style of test) and you can close processes easily by terminating them. Typically for most user-facing functions you will find both kinds of tests. The ``@gen_cluster`` tests test particular logic while the ``with cluster`` tests test basic interface and resilience. You should avoid ``popen`` style tests unless absolutely necessary, such as if you need to test the command line interface. distributed-1.20.2/docs/source/diagnosing-performance.rst000066400000000000000000000114631321233345200235240ustar00rootroot00000000000000Diagnosing Performance ====================== Understanding the performance of a distributed computation can be difficult. This is due in part to the many components of a distributed computer that may impact performance: 1. Compute time 2. Memory bandwidth 3. Network bandwidth 4. Disk bandwidth 5. Scheduler overhead 6. Serialization costs This difficulty is compounded because the information about these costs is spread among many machines and so there is no central place to collect data to identify performance issues. Fortunately, Dask collects a variety of diagnostic information during execution. It does this both to provide performance feedback to users, but also for its own internal scheduling decisions. The primary place to observe this feedback is the :doc:`diagnostic dashboard `. This document describes the various pieces of performance information available and how to access them. Task start and stop times ------------------------- Workers capture durations associated to tasks. For each task that passes through a worker we record start and stop times for each of the following: 1. Serialization (gray) 2. Dependency gathering from peers (red) 3. Disk I/O to collect local data (orange) 4. Execution times (colored by task) The main way to observe these times is with the task stream plot on the scheduler's ``/status`` page where the colors of the bars correspond to the colors listed above. .. image:: https://raw.githubusercontent.com/dask/dask-org/master/images/bokeh-task-stream.gif :alt: Dask task stream :width: 50% Alternatively if you want to do your own diagnostics on every task event you might want to create a :doc:`Scheduler plugin `. All of this information will be available when a task transitions from processing to memory or erred. Statistical Profiling --------------------- For single-threaded profiling Python users typically depend on the CProfile module in the standard library (Dask developers recommend the `snakeviz `_ tool for single-threaded profiling). Unfortunately the standard CProfile module does not work with multi-threaded or distributed computations. To address this Dask implements its own distributed `statistical profiler `_. Every 10ms each worker process checks what each of its worker threads are doing. It captures the call stack and adds this stack to a counting data structure. This counting data structure is recorded and cleared every second in order to establish a record of performance over time. Users typically observe this data through the ``/profile`` plot on either the worker or scheduler diagnostic dashboards. On the scheduler page they observe the total profile aggregated over all workers over all threads. Clicking on any of the bars in the profile will zoom the user into just that section, as is typical with most profiling tools. There is a timeline at the bottom of the page to allow users to select different periods in time. .. image:: https://raw.githubusercontent.com/dask/dask-org/master/images/daskboard-profile.gif :alt: Dask profiler :width: 70% Profiles are also grouped by the task that was being run at the time. You can select a task name from the selection menu at the top of the page. You can also click on the rectangle corresponding to the task in the main task stream plot on the ``/status`` page. Users can also query this data directly using the :doc:`Client.profile ` function. This will deliver the raw data structure used to produce these plots. The 10ms and 1s parameters can be controlled by the ``profile-interval`` and ``profile-cycle-interval`` entries in the config.yaml file. Bandwidth --------- Dask workers track every incoming and outgoing transfer in the ``Worker.outgoing_transfer_log`` and ``Worker.incoming_transfer_log`` attributes including 1. Total bytes transferred 2. Compressed bytes transferred 3. Start/stop times 4. Keys moved 5. Peer These are made available to users through the ``/main`` page of the Worker's diagnostic dashboard. You can capture their state explicitly by running a command on the workers: .. code-block:: python client.run(lambda dask_worker: dask_worker.outgoing_transfer_log) client.run(lambda dask_worker: dask_worker.incoming_transfer_log) A note about times ------------------ Different computers maintain different clocks which may not match perfectly. To address this the Dask scheduler sends its current time in response to every worker heartbeat. Workers compare their local time against this time to obtain an estimate of differences. All times recorded in workers take this estimated delay into account. This helps, but still, imprecise measurements may exist. All times are intended to be from the scheduler's perspective. distributed-1.20.2/docs/source/ec2.rst000066400000000000000000000051051321233345200175500ustar00rootroot00000000000000EC2 Startup Script ================== First, add your AWS credentials to ``~/.aws/credentials`` like this:: [default] aws_access_key_id = YOUR_ACCESS_KEY aws_secret_access_key = YOUR_SECRET_KEY For other ways to manage or troubleshoot credentials, see the `boto3 docs `_. Now, you can quickly deploy a scheduler and workers on EC2 using the ``dask-ec2`` quickstart application:: pip install dask-ec2 dask-ec2 up --keyname YOUR-AWS-KEY --keypair ~/.ssh/YOUR-AWS-SSH-KEY.pem This provisions a cluster on Amazon's EC2 cloud service, installs Anaconda, and sets up a scheduler and workers. In then prints out instructions on how to connect to the cluster. Options ------- The ``dask-ec2`` startup script comes with the following options for creating a cluster:: $ dask-ec2 up --help Usage: dask-ec2 up [OPTIONS] Options: --keyname TEXT Keyname on EC2 console [required] --keypair PATH Path to the keypair that matches the keyname [required] --name TEXT Tag name on EC2 --region-name TEXT AWS region [default: us-east-1] --ami TEXT EC2 AMI [default: ami-d05e75b8] --username TEXT User to SSH to the AMI [default: ubuntu] --type TEXT EC2 Instance Type [default: m3.2xlarge] --count INTEGER Number of nodes [default: 4] --security-group TEXT Security Group Name [default: dask-ec2-default] --volume-type TEXT Root volume type [default: gp2] --volume-size INTEGER Root volume size (GB) [default: 500] --file PATH File to save the metadata [default: cluster.yaml] --provision / --no-provision Provision salt on the nodes [default: True] --dask / --no-dask Install Dask.Distributed in the cluster [default: True] --nprocs INTEGER Number of processes per worker [default: 1] -h, --help Show this message and exit. Connect ------- Connection instructions follow successful completion of the ``dask-ec2 up`` command. The involve the following:: dask-ec2 ssh 0 # SSH into head node ipython # Start IPython console on head node .. code-block:: python >>> from distributed import Client >>> c = Client('127.0.0.1:8786') This client now has access to all the cores of your cluster. Destroy ------- You can destroy your cluster from your local machine with the destroy command:: dask-ec2 destroy distributed-1.20.2/docs/source/efficiency.rst000066400000000000000000000065231321233345200212100ustar00rootroot00000000000000Efficiency ========== Parallel computing done well is responsive and rewarding. However, several speed-bumps can get in the way. This section describes common ways to ensure performance. Leave data on the cluster ------------------------- Wait as long as possible to gather data locally. If you want to ask a question of a large piece of data on the cluster it is often faster to submit a function onto that data then to bring the data down to your local computer. For example if we have a numpy array on the cluster and we want to know its shape we might choose one of the following options: 1. **Slow:** Gather the numpy array to the local process, access the ``.shape`` attribute 2. **Fast:** Send a lambda function up to the cluster to compute the shape .. code-block:: python >>> x = client.submit(np.random.random, (1000, 1000)) >>> type(x) Future **Slow** .. code-block:: python >>> x.result().shape() # Slow from lots of data transfer (1000, 1000) **Fast** .. code-block:: python >>> client.submit(lambda a: a.shape, x).result() # fast (1000, 1000) Use larger tasks ---------------- The scheduler adds about *one millisecond* of overhead per task or Future object. While this may sound fast it's quite slow if you run a billion tasks. If your functions run faster than 100ms or so then you might not see any speedup from using distributed computing. A common solution is to batch your input into larger chunks. **Slow** .. code-block:: python >>> futures = client.map(f, seq) >>> len(futures) # avoid large numbers of futures 1000000000 **Fast** .. code-block:: python >>> def f_many(chunk): ... return [f(x) for x in chunk] >>> from toolz import partition_all >>> chunks = partition_all(1000000, seq) # Collect into groups of size 1000 >>> futures = client.map(f_many, chunks) >>> len(futures) # Compute on larger pieces of your data at once 1000 Adjust between Threads and Processes ------------------------------------ By default a single ``Worker`` runs many computations in parallel using as many threads as your compute node has cores. When using pure Python functions this may not be optimal and you may instead want to run several separate worker processes on each node, each using one thread. When configuring your cluster you may want to use the options to the ``dask-worker`` executable as follows:: $ dask-worker ip:port --nprocs 8 --nthreads 1 Note that if you're primarily using NumPy, Pandas, SciPy, Scikit Learn, Numba, or other C/Fortran/LLVM/Cython-accelerated libraries then this is not an issue for you. Your code is likely optimal for use with multi-threading. Don't go distributed -------------------- Consider the dask_ and concurrent.futures_ modules, which have similar APIs to distributed but operate on a single machine. It may be that your problem performs well enough on a laptop or large workstation. Consider accelerating your code through other means than parallelism. Better algorithms, data structures, storage formats, or just a little bit of C/Fortran/Numba code might be enough to give you the 10x speed boost that you're looking for. Parallelism and distributed computing are expensive ways to accelerate your application. .. _dask: http://dask.pydata.org/en/latest/ .. _concurrent.futures: https://docs.python.org/3/library/concurrent.futures.html distributed-1.20.2/docs/source/examples-overview.rst000066400000000000000000000001131321233345200225530ustar00rootroot00000000000000Examples ======== .. toctree:: :maxdepth: 1 examples/word-count distributed-1.20.2/docs/source/examples/000077500000000000000000000000001321233345200201625ustar00rootroot00000000000000distributed-1.20.2/docs/source/examples/word-count.rst000066400000000000000000000213051321233345200230160ustar00rootroot00000000000000Word count in HDFS ================== Setup ----- In this example, we'll use ``distributed`` with the ``hdfs3`` library to count the number of words in text files (Enron email dataset, 6.4 GB) stored in HDFS. Copy the text data from Amazon S3 into HDFS on the cluster: .. code-block:: bash $ hadoop distcp s3n://AWS_SECRET_ID:AWS_SECRET_KEY@blaze-data/enron-email hdfs:///tmp/enron where ``AWS_SECRET_ID`` and ``AWS_SECRET_KEY`` are valid AWS credentials. Start the ``distributed`` scheduler and workers on the cluster. Code example ------------ Import ``distributed``, ``hdfs3``, and other standard libraries used in this example: .. code-block:: python >>> import hdfs3 >>> from collections import defaultdict, Counter >>> from distributed import Client, progress Initialize a connection to HDFS, replacing ``NAMENODE_HOSTNAME`` and ``NAMENODE_PORT`` with the hostname and port (default: 8020) of the HDFS namenode. .. code-block:: python >>> hdfs = hdfs3.HDFileSystem('NAMENODE_HOSTNAME', port=NAMENODE_PORT) Initialize a connection to the ``distributed`` client, replacing ``SCHEDULER_IP`` and ``SCHEDULER_PORT`` with the IP address and port of the ``distributed`` scheduler. .. code-block:: python >>> client = Client('SCHEDULER_IP:SCHEDULER_PORT') Generate a list of filenames from the text data in HDFS: .. code-block:: python >>> filenames = hdfs.glob('/tmp/enron/*/*') >>> print(filenames[:5]) ['/tmp/enron/edrm-enron-v2_nemec-g_xml.zip/merged.txt', '/tmp/enron/edrm-enron-v2_ring-r_xml.zip/merged.txt', '/tmp/enron/edrm-enron-v2_bailey-s_xml.zip/merged.txt', '/tmp/enron/edrm-enron-v2_fischer-m_xml.zip/merged.txt', '/tmp/enron/edrm-enron-v2_geaccone-t_xml.zip/merged.txt'] Print the first 1024 bytes of the first text file: .. code-block:: python >>> print(hdfs.head(filenames[0])) b'Date: Wed, 29 Nov 2000 09:33:00 -0800 (PST)\r\nFrom: Xochitl-Alexis Velasc o\r\nTo: Mark Knippa, Mike D Smith, Gerald Nemec, Dave S Laipple, Bo Barnwel l\r\nCc: Melissa Jones, Iris Waser, Pat Radford, Bonnie Shumaker\r\nSubject: Finalize ECS/EES Master Agreement\r\nX-SDOC: 161476\r\nX-ZLID: zl-edrm-enro n-v2-nemec-g-2802.eml\r\n\r\nPlease plan to attend a meeting to finalize the ECS/EES Master Agreement \r\ntomorrow 11/30/00 at 1:30 pm CST.\r\n\r\nI wi ll email everyone tomorrow with location.\r\n\r\nDave-I will also email you the call in number tomorrow.\r\n\r\nThanks\r\nXochitl\r\n\r\n***********\r\n EDRM Enron Email Data Set has been produced in EML, PST and NSF format by ZL Technologies, Inc. This Data Set is licensed under a Creative Commons Attri bution 3.0 United States License . To provide attribution, please cite to "ZL Technologies, Inc. (http: //www.zlti.com)."\r\n***********\r\nDate: Wed, 29 Nov 2000 09:40:00 -0800 (P ST)\r\nFrom: Jill T Zivley\r\nTo: Robert Cook, Robert Crockett, John Handley , Shawna' Create a function to count words in each file: .. code-block:: python >>> def count_words(fn): ... word_counts = defaultdict(int) ... with hdfs.open(fn) as f: ... for line in f: ... for word in line.split(): ... word_counts[word] += 1 ... return word_counts Before we process all of the text files using the distributed workers, let's test our function locally by counting the number of words in the first text file: .. code-block:: python >>> counts = count_words(filenames[0]) >>> print(sorted(counts.items(), key=lambda k_v: k_v[1], reverse=True)[:10]) [(b'the', 144873), (b'of', 98122), (b'to', 97202), (b'and', 90575), (b'or', 60305), (b'in', 53869), (b'a', 43300), (b'any', 31632), (b'by', 31515), (b'is', 30055)] We can perform the same operation of counting the words in the first text file, except we will use ``client.submit`` to execute the computation on a ``distributed`` worker: .. code-block:: python >>> future = client.submit(count_words, filenames[0]) >>> counts = future.result() >>> print(sorted(counts.items(), key=lambda k_v: k_v[1], reverse=True)[:10]) [(b'the', 144873), (b'of', 98122), (b'to', 97202), (b'and', 90575), (b'or', 60305), (b'in', 53869), (b'a', 43300), (b'any', 31632), (b'by', 31515), (b'is', 30055)] We are ready to count the number of words in all of the text files using ``distributed`` workers. Note that the ``map`` operation is non-blocking, and you can continue to work in the Python shell/notebook while the computations are running. .. code-block:: python >>> futures = client.map(count_words, filenames) We can check the status of some ``futures`` while all of the text files are being processed: .. code-block:: python >>> len(futures) 161 >>> futures[:5] [, , , , ] >>> progress(futures) [########################################] | 100% Completed | 3min 0.2s When the ``futures`` finish reading in all of the text files and counting words, the results will exist on each worker. This operation required about 3 minutes to run on a cluster with three worker machines, each with 4 cores and 16 GB RAM. Note that because the previous computation is bound by the GIL in Python, we can speed it up by starting the ``distributed`` workers with the ``--nprocs 4`` option. To sum the word counts for all of the text files, we need to gather some information from the ``distributed`` workers. To reduce the amount of data that we gather from the workers, we can define a function that only returns the top 10,000 words from each text file. .. code-block:: python >>> def top_items(d): ... items = sorted(d.items(), key=lambda kv: kv[1], reverse=True)[:10000] ... return dict(items) We can then ``map`` the futures from the previous step to this culling function. This is a convenient way to construct a pipeline of computations using futures: .. code-block:: python >>> futures2 = client.map(top_items, futures) We can ``gather`` the resulting culled word count data for each text file to the local process: .. code-block:: python >>> results = client.gather(iter(futures2)) To sum the word counts for all of the text files, we can iterate over the results in ``futures2`` and update a local dictionary that contains all of the word counts. .. code-block:: python >>> all_counts = Counter() >>> for result in results: ... all_counts.update(result) Finally, we print the total number of words in the results and the words with the highest frequency from all of the text files: .. code-block:: python >>> print(len(all_counts)) 8797842 >>> print(sorted(all_counts.items(), key=lambda k_v: k_v[1], reverse=True)[:10]) [(b'0', 67218380), (b'the', 19586868), (b'-', 14123768), (b'to', 11893464), (b'N/A', 11814665), (b'of', 11724827), (b'and', 10253753), (b'in', 6684937), (b'a', 5470371), (b'or', 5227805)] The complete Python script for this example is shown below: .. code-block:: python # word-count.py import hdfs3 from collections import defaultdict, Counter from distributed import Client, progress hdfs = hdfs3.HDFileSystem('NAMENODE_HOSTNAME', port=NAMENODE_PORT) client = Client('SCHEDULER_IP:SCHEDULER:PORT') filenames = hdfs.glob('/tmp/enron/*/*') print(filenames[:5]) print(hdfs.head(filenames[0])) def count_words(fn): word_counts = defaultdict(int) with hdfs.open(fn) as f: for line in f: for word in line.split(): word_counts[word] += 1 return word_counts counts = count_words(filenames[0]) print(sorted(counts.items(), key=lambda k_v: k_v[1], reverse=True)[:10]) future = client.submit(count_words, filenames[0]) counts = future.result() print(sorted(counts.items(), key=lambda k_v: k_v[1], reverse=True)[:10]) futures = client.map(count_words, filenames) len(futures) futures[:5] progress(futures) def top_items(d): items = sorted(d.items(), key=lambda kv: kv[1], reverse=True)[:10000] return dict(items) futures2 = client.map(top_items, futures) results = client.gather(iter(futures2)) all_counts = Counter() for result in results: all_counts.update(result) print(len(all_counts)) print(sorted(all_counts.items(), key=lambda k_v: k_v[1], reverse=True)[:10]) distributed-1.20.2/docs/source/faq.rst000066400000000000000000000075621321233345200176570ustar00rootroot00000000000000Frequently Asked Questions ========================== More questions can be found on StackOverflow at http://stackoverflow.com/search?tab=votes&q=dask%20distributed How do I use external modules? ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ Use ``client.upload_file``. For more detail, see the `API docs`_ and a StackOverflow question `"Can I use functions imported from .py files in Dask/Distributed?"`__ This function supports both standalone file and setuptools's ``.egg`` files for larger modules. __ http://stackoverflow.com/questions/39295200/can-i-use-functions-imported-from-py-files-in-dask-distributed .. _API docs: https://distributed.readthedocs.io/en/latest/api.html#distributed.executor.Executor.upload_file Too many open file descriptors? ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ Your operating system imposes a limit to how many open files or open network connections any user can have at once. Depending on the scale of your cluster the ``dask-scheduler`` may run into this limit. By default most Linux distributions set this limit at 1024 open files/connections and OS-X at 128 or 256. Each worker adds a few open connections to a running scheduler (somewhere between one and ten, depending on how contentious things get.) If you are on a managed cluster you can usually ask whoever manages your cluster to increase this limit. If you have root access and know what you are doing you can change the limits on Linux by editing ``/etc/security/limits.conf``. Instructions are here under the heading "User Level FD Limits": http://www.cyberciti.biz/faq/linux-increase-the-maximum-number-of-open-files/ Error when running dask-worker about ``OMP_NUM_THREADS`` ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ For more problems with OMP_NUM_THREADS, see http://stackoverflow.com/questions/39422092/error-with-omp-num-threads-when-using-dask-distributed Does Dask handle Data Locality? ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ Yes, both data locality in memory and data locality on disk. Often it's *much* cheaper to move computations to where data lives. If one of your tasks creates a large array and a future task computes the sum of that array, you want to be sure that the sum runs on the same worker that has the array in the first place, otherwise you'll wait for a long while as the data moves between workers. Needless communication can easily dominate costs if we're sloppy. The Dask Scheduler tracks the location and size of every intermediate value produced by every worker and uses this information when assigning future tasks to workers. Dask tries to make computations more efficient by minimizing data movement. Sometimes your data is on a hard drive or other remote storage that isn't controlled by Dask. In this case the scheduler is unaware of exactly where your data lives, so you have to do a bit more work. You can tell Dask to preferentially run a task on a particular worker or set of workers. For example Dask developers use this ability to build in data locality when we communicate to data-local storage systems like the Hadoop File System. When users use high-level functions like ``dask.dataframe.read_csv('hdfs:///path/to/files.*.csv'`` Dask talks to the HDFS name node, finds the locations of all of the blocks of data, and sends that information to the scheduler so that it can make smarter decisions and improve load times for users. PermissionError [Errno 13] Permission Denied: \`/root/.dask\` ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ This error can be seen when starting distributed through the standard process control tool ``supervisor`` and running as a non-root user. This is caused by ``supervisor`` not passing the shell environment variables through to the subprocess, head to `this section`_ of the supervisor documentation to see how to pass the ``$HOME`` and ``$USER`` variables through. .. _this section: http://supervisord.org/subprocess.html#subprocess-environment distributed-1.20.2/docs/source/foundations.rst000066400000000000000000000111401321233345200214240ustar00rootroot00000000000000=========== Foundations =========== You should read through the :doc:`quickstart ` before reading this document. .. _quickstart: quickstart.html Distributed computing is hard for two reasons: 1. Consistent coordination of distributed systems requires sophistication 2. Concurrent network programming is tricky and error prone The foundations of ``dask.distributed`` provide abstractions to hide some complexity of concurrent network programming (#2). These abstractions ease the construction of sophisticated parallel systems (#1) in a safer environment. However, as with all layered abstractions, ours has flaws. Critical feedback is welcome. Concurrency with Tornado Coroutines =================================== Worker and Scheduler nodes operate concurrently. They serve several overlapping requests and perform several overlapping computations at the same time without blocking. There are several approaches for concurrent programming, we've chosen to use Tornado for the following reasons: 1. Developing and debugging is more comfortable without threads 2. `Tornado's documentation`_ is excellent 3. Stackoverflow coverage is excellent 4. Performance is satisfactory .. _`Tornado's documentation`: https://tornado.readthedocs.io/en/latest/coroutine.html Endpoint-to-endpoint Communication ================================== The various distributed endpoints (Client, Scheduler, Worker) communicate by sending each other arbitrary Python objects. Encoding, sending and then decoding those objects is the job of the :ref:`communication layer `. Ancillary services such as a Bokeh-based Web interface, however, have their own implementation and semantics. Protocol Handling ================= While the abstract communication layer can transfer arbitrary Python objects (as long as they are serializable), participants in a ``distributed`` cluster concretely obey the distributed :ref:`protocol`, which specifies request-response semantics using a well-defined message format. Dedicated infrastructure in ``distributed`` handles the various aspects of the protocol, such as dispatching the various operations supported by an endpoint. Servers ------- Worker, Scheduler, and Nanny objects all inherit from a ``Server`` class. .. autoclass:: distributed.core.Server RPC --- To interact with remote servers we typically use ``rpc`` objects which expose a familiar method call interface to invoke remote operations. .. autoclass:: distributed.core.rpc Examples ======== Here is a small example using distributed.core to create and interact with a custom server. Server Side ----------- .. code-block:: python from tornado import gen from tornado.ioloop import IOLoop from distributed.core import Server def add(comm, x=None, y=None): # simple handler, just a function return x + y @gen.coroutine def stream_data(comm, interval=1): # complex handler, multiple responses data = 0 while True: yield gen.sleep(interval) data += 1 yield comm.write(data) s = Server({'add': add, 'stream_data': stream_data}) s.listen('tcp://:8888') # listen on TCP port 8888 IOLoop.current().start() Client Side ----------- .. code-block:: python from tornado import gen from tornado.ioloop import IOLoop from distributed.core import connect @gen.coroutine def f(): comm = yield connect('tcp://127.0.0.1:8888') yield comm.write({'op': 'add', 'x': 1, 'y': 2}) result = yield comm.read() yield comm.close() print(result) >>> IOLoop().run_sync(f) 3 @gen.coroutine def g(): comm = yield connect('tcp://127.0.0.1:8888') yield comm.write({'op': 'stream_data', 'interval': 1}) while True: result = yield comm.read() print(result) >>> IOLoop().run_sync(g) 1 2 3 ... Client Side with ``rpc`` ------------------------ RPC provides a more pythonic interface. It also provides other benefits, such as using multiple streams in concurrent cases. Most distributed code uses ``rpc``. The exception is when we need to perform multiple reads or writes, as with the stream data case above. .. code-block:: python from tornado import gen from tornado.ioloop import IOLoop from distributed.core import rpc @gen.coroutine def f(): # comm = yield connect('tcp://127.0.0.1', 8888) # yield comm.write({'op': 'add', 'x': 1, 'y': 2}) # result = yield comm.read() r = rpc('tcp://127.0.0.1:8888') result = yield r.add(x=1, y=2) r.close_comms() print(result) >>> IOLoop().run_sync(f) 3 distributed-1.20.2/docs/source/images/000077500000000000000000000000001321233345200176115ustar00rootroot00000000000000distributed-1.20.2/docs/source/images/network.png000066400000000000000000004737421321233345200220310ustar00rootroot00000000000000PNG  IHDR ó bKGD IDATxw癎edMdUHv*) "BB${{gy~|ιxonޯsK_=fX,={V׭[}z ʛ7orR۽{vؑm<ʗ/ *$wܓJЮ]tn`)cߟ-}7cg*f|r9 4IуPwssS*U4n ϝ۩^CRﶲX,Z}XV͛ʚ)Cg>yRo衣O;t$,\XsƎUjQQ2~xC+[6MP$ܿ_65vjU55jBB+6mR 5k"{II"<"BϚkv۶8{vRI~CNJbfZySu\͛2a+Wo, i,pRB ׏>蘘$1L^*Z@w<ٖY;cW{z:8}ܝAGNа0{zu= 4Tm?P/vIĖ-.[kl7sm[i|}^I]to;[}SuۯJ\)^I`)̂11:Զb6]P|Jn'׈ɓմkWEDF&y ж} TZ57N/׍ݻuvF-:U]ZOTPy`Ѻ~_#>o˂%ܼsG :vԛzիհwb3gԢGɑCuӦsuzz]ߵKSƍSW^u11jޭn޹c QzS'>%̝a'VbII2H>,X*/~Ja,-Ypw5J`:{~j5SGVUR9Cѝ<~\+7ou'$Ȩ(}0tϟ`/I-9ܿ_\kUP!,\X% R 6;{v`jM{'](2Ob? ЉjӻLϾ|NՓ+|:r:dM…6M]ySwӵa+RRٺRڽ K_ڿU5 I Kx_Uz4 ЦsB'GmW/Yp6Mj̒1& b&۶i1j۴ #i2ʕ5Ouz:5SZ~}$zfVn6Piuz 5K7wֺٳ5_?լTe˪xOsء?7n4q0 *T_ ԦsUX1)SK;ӧkݬYiHըXQ驜YM&ڵp>ژXu_qqqپR R'eϜYҥ33jŢk-Y'Ξ5}mE<٣rMg%ymý#G`޼6 1cj/:&FMu^WOm4uTѱի5gO Htmx5]5s,ĖﻙͻwN0,p'U*}ޭ.n٢)_}*5y`y-_}U7N/Tõh*{tDa ß^^s4UPZwww [ul1`zuTJ@@=|XM&-vA4K,l۷OMtQ7о#Gg2\jX]έOt1ܛhBMŠ5~ ̚U6M3gM6tZmj)}7ry5EQQuTի5wou1d&v'oΜjk{n95&}hc}/n7rY,{tFJ@A77T]8],X7gN;Ux,\J5K͛kզMZ? h},mPw|S^V´b&jikL~Od,~ĔRw#11jѽn~m9mrgϞ(=uJ![͙SŞޡ׫g~q9b̙빲eӇ:\C>p_WKJ@By0̒kM,[T}`i)kOnڶoVo٢EWkMsi8 {׎x_SD ^H4H>+:&j=TP2^ƍ6ɚ)C?5Lߺt}{Ȑ|G;>s#M./5(`~Mk#"2R\iף];v|U`/?g{,  .Vt/իZա\QxjӲC%iI²ԊMfV93/*}/mZȒEEW ԰V-Ȓũ/mܵK;ui8sF<@(TH/..Zp.qw3 9Իkh:p.\TȡREfJzF N/O][Rw}էiRS]eׁ T*ooV-_^ &$=|ᚎo;wI)uႆMh׸vm7{i~x|z\\5nvϫf;TY_v5z/`) ˕KiÇV{΄k S___qt \x²~FNS޻Љuru4HUSnTT)}d޽h[K׮}}ڴk6ڥWkLԀ㝐UjL'Ϟ Ӧs =}Z_}-_n8266V~v߯Is/MumZtt-I}_m7Se׀=TR%G*U7gNx@A7nȓǩJ6\?r򤢢T}[Mk$=<.hY2fԴ#Lddb2wMpM{޲pT"ʓ#C5_[We~=9tHKtR R1XO)eZb2YqZyC`bфV5͏?5,..NׯWfz4z/Ԥ9s 7W)''2*JMt1 ɓUP!jEĨ*٠f/^l63 4T_ԭkEb_Ruڷצ]а0mܹSuڷ) zU+I=|7kP^I2l5\ёSmOC|Rǝ8sFs-3 L>2=;а0j:qpDճצ] k5 ͫ3u^bH2HH% ^]ÇڸsM]z%)QsL W=} = uGf-ZNqVάY!]DmXyn=7775J*T֭wUeK}NMsvt쨟.t+}#ހŽU$ݢ6̙k֨ONL,f(#. ts眪mO=HgqgϖbZ/?6id}7 {FĘ_~vI08:&FO4+icL… םG{LK\$G:YzaX(dBe&}UxkN4?>̹KTU$<.1K{cSW^έwUeKvI_1zo_rp+}߷ۧn^s :p 1G@C,mϔ`3fS=xtm[M-Sd̘d}<.%GDhuoVIT@ýa'*,<ܮzBCV^zaCzDZӧMCN6cvcǜkt/3xZ,sj]~5VpaAƍjv!ڿkʕ(+[LJ+ׯȩSZi#" {u.xsTbEU(YR TlXvHN;ǎiΝzti42xx9pSl];S&MT-S&{@ϟעիfVÉj:jU߼};sn𚧅O9t&K*Xzu-\pzn-!bo//n8iٰmׅ q0k ;kkݸ2OTӽy:Upg.^b1D Hlٔ9CɊ{zuaZP{>wR7х+W *,eR͌j?W\ i܌ '^~]oE'O|LiP*oo PܹUZ5}ҥ4gc׹eK Ck}wڲg^_6ګSkF[U>^sm [8~̸;{ɓLHOf>z/Mk1\O`7,KFf-[p֋/*/kZ_OV{SSj}7`@+VhVi|}5ĎFgtp=]@Ry{;U;K vfT}WJg<ʕ(anotUZoZ2K*eZ-]Naa6a6K/)W_WM'|Ol*7Ӯ]wG\/^F~-~RCL׫I\ٳ>i^,YkBP^*SF2wn޹csO\$ϫ ,_.uOTv>Ӥ9s zo L}ظpnժj`lZyVURSҕ*5jvȑZu'(0P>t̙ުakתNQneϜ٩mu u?_p͛Np%.T /\bף]j-_\*Q$5GBkڵ6a,}H&>tן|ڴ hՌL`5I|zW_1cn_=Z1ߏ>>Z5cmSl3kw,FoHg͔I 4=u""#]~r'$Dmz6=+[D"v.^jᆱ eʨYn;}ء/+}EFE)}ulݪ&v픫reM;שڴQM7ڥbӧ[,2EMv5Tۣ];ulu'$p=sN׎/jv.@r!X BJ6uM5nݽu۶Y?yn] el:C2TLPpz sgxxdž{g/^Ԃ+쮙ݶMͺu3@R2Z-[ -թR%~:q"QkhZ+6mrTZ4i>9 ̇O԰aN=o//}7p䓈͂ծ_8;1 8`) U GToƯSG˙SeךcMY>•+v-KzUΉsww{mܿ_\qvRstа0 h̙vOַ5sxdE3kV8HZijAWU_km~떢cb |z 0A[RW_3%{Pwýk䧟&ʹO WNL ן|C8CjԶIwFTt+&OH.K\E`W_]֛Hկo8mΒ% h̓#f!N;mk b:qZѣӾTsk?+}`uoܾm:Q2GPۢlGOv.OƍժQ#;!!1yjm,*Ȼ`Ay(%F ک;w&l |.85dqޜ9AgūWKFj& IDATzL߼Y!+IٷOWִ#Ԩvm1ǺP~Y=¡k%iɚ5W~^23l cwF&~4/K[Z6lh#K֯۾]_&Pz!M >!iӪT"{t~b;pjmY̚Uk~YY2ftnKlx+[6S9U맑#պqD?'{,.٨Q9e^Ơ 1CҧwOI Ԝcus-]ewYOp_ڴ˜!feifS[ ԖyYRݪU5s(]ڶM{V@֭SCmYfAWIFI`)@"^x-2\/X9C08ɓ 6JҾÇ eΐA3=$'xjݺseg$Xb?rt{a$?ɝ=K[jjӤJ- )<<<6rgϮ+W;jIw~8tb…]߰5pXý>>Z:e^,]%g=Q*T0VbA~kG+gPEM Sntlj`$9S6oNg\޿tmI֏H= /Y>^7nVnX __r٪Q#ٺj}Ƃ*UպYZ YyF̂Qъ{"cr۴k^j\KL1}-#I}HZY2fTVԥU+զݻ ̂5e>3[ 'N4 RJeۿ6e!xFbQ/4 Տs)V̦3iӑ#5|$|$Hʓ#u^yzxqԨvm?k&C? unyM?n޹czfl1>~մٽ<^5\ONp5nirʆ6_GDhZAjۤaf+tziݸ}u3gVm:ˬmf2ᡡ~2ŊMކS޸mh˼yʗ+]\NP]p;\wjМcմN?&^qéK&نGDh֭VnnnNAjը͜i=8Ặ(/ޡ'TBk<~p`޼H,)w{ի xPQVr2 ۪I0i϶ʙ5OCmu Ŋ9U 1,H$5+Uz\\ܿ3~5ҧ+5j|Z7nlI3-{fMܥK6׉ϾÇ ׋(I|ɒ/?{v`*6c?s"϶cO3 6mjz&c߾vwZtT"'Ϟu٤he`+~9G? SDdSD 9\3:&FLew.@b!X HҤQR m&E7zpۦM C`,M,ussS^ŋpoIp>Tѣ{Kt~R*?6͝Y>yR-zPLlMȨ(mر>lT>} Cɞ׽CuZeתQ# CB*UKtM랿|p$eɘpAhoHtL✮/hdž mۦа0ýZ&$'nժ+6m$haPf>x*m:}𠎞>'OV% Vm>'(0tLu2ӉtuTq~R˟;VϜ &A6_T+}`mx{ij'..N)zKys活nhX>2pT"6bi0 +/ipsjׁҥ3Lw;$>cR#K1sݻڲgC5͂E7D =Թs:sf<6Mr%J8w@ 'JR>EvԑSH\\&ΞmWxq1y'5ܟ8{̝kS*wݼs숊V>/KȓG?ڣMӥk׬jJP]ժe7É`+ ׫+gzY cOm$j6=ՕJ*B\zaZ`}W$REoO[wlZ5j$o//Y3%^Ys:l)4i-hwG~;WG~meo1/L>mӦa0fS=wYUMZp?>>v׎3 #igMtE]zUKL&p$*ZT>RYGD+q/6\LڪkZHgO3G޵ZwwwW&M ,Hdf͟ JNL0 Tڵ֯ݼ ҩBɒvw|poΝ9e58Op/OSR$Mр= "{%e˪Lb{1 d= .\ Q-TE }0d.\m=޻ Qz=85nnn9j/-ҍ۷֋(nm8TYO=SJeJj=..N]GÆ)"2j=]@Z5jdz]*ooӠoUhXݽH;tZý6Nv}eK10.bTܥK ׮9sx/`)@"khb#׫@{E _[zu;WD;vڴ{ ?ls#Nvt~ݺá>S={/]vMM޽M:|Lp)Uѣ*+;}ݫ{j*հ;HQZ~0>hVlڤo>L?kj܌r8EFEq.nnnyhU|59ujɚ57szkgZ~jjޠ~X:T VЇ:QZ6rcbc}>-\JQQ\\|B߷ڻv}herssKO\\^zŋ:rꔎ:G+~5S^"iný???z* k{go:z|!!ѷ->֩{N'Ϟ[f~+*R$:i?xYI.?+~ݻk…v?okL~? [ IzY3.Z4^)RD*Uj:sz N[c}]*ooճa[|Z5j6i…֭ӲuTX1U)WN2gOTzㆎ:7;.[̚:bS=Di}}hdzUGÆ֋/H5~:T_.?j&ʗ,RE*}`Ȩ(ݽwO]Ӊ3gA= $Ȓ5_>x0@|Að0w".I/~kծnd ͫW/P6m >zT-]V[Tr+[6Y,]zUۃ5ql0#?|]3FT7PLlCTQ#կVM kRbŔ1(Hi|}u'$D^պ5or2I*?aG\IÇN ?-X+VQzpa[qNmعӴng%x#O R&Iy K;6oc7ח) H^Jp-& W{>lWtZ:e:b?wn}7p:k}hr UM]vZmnfso]]r˗|͹K`UB)OP`wnm&L=SbE N;޽ՠzu__T) c6pb dž v"I*Uuޠzu}{2aP^X/oԚ7nMG{<5 xzx믻Y^ժ浺u]roZ>uI~>(VfjVޞC4|Ҥxϗ++ގ:ʘ>}dk6^c3g?_rITnܾ:3zG;ޠ^ԯ[7S'}էǞ3kV9}˙Ӧ'>wouaZjB|S‰UdI{Ve,Hi}} z6QCY3er陝[4P_vYSk n@uVW_%KTpauM Le:a\e~=e}VTxSY3e;ejilԵuk>]g6lgァ@q'$ĥ,3_򉦍N|f =i׮ZO6@Ҩvm_̮|b?w39i\թRk{qt$yZz^-\~+U޸aW|y?bM=ڵSz4bdM?_>ڗ+WVUlYx7uu<ٳ;U7dJ^>TڋTA|xk֠^ԹeK:U3]u"Y_]%'U*Mxe2YlgĄ  z]f\ߧMxD_u۷+QpAmp-ׯw^7&bIs~]\K֬I'թs4o fHfL,ăPٺU[Gue]uKaSj(cʟ;9/YRʗW>ȩSط5+EϚGPXx2[۶Har9 8Q,]HׅGD(<"BCBtYwssW}.]!4,LͻwWXx$ v6M1B_zիطlڇ_}JLbHiܓ)66VGVѺu5}xCX,ʑ%St0@NūWU}{2ũnϡ]m IDATC*^BݻރHX [Dd^Unz^xkZ.4RU < J*<9rcV{g.^ԇ_} Kت{ qoXlz-`N~PiP` е>TALGoJ)V]B`n`mpn (Zmm6I˹}MJۜw1 #J'lmmB8k۵ o4rUDDDDDDDdXJDDDDDDDDDD:z8 z˔ ݇cM^(=)Y07NcLVĉHLJ2rUDDDDDDDdlL]QQ)WZ5n+WFYJ1qq ƣgpM:sU˖իXuƘ͞ BDѐnݰqqӀ\7AeDDDDDDDdNXJDDDDDDDDDDZ[s'"lY`˂hXA[[8:8 Kƌ(_$~jIɈ֫Q3f@Ri>lrf˦׺DْɓQ JwW_N|Y8+S@DDDDDDDDDDiJ²͛qw77ݺUsS 2\gq] cxϞ:Id J-]jceF KH+OeP0>}(,\8P՘~ 7"2b<2gkxQR""""""""""5PȞ5+tl:v= Ǐ5ԨT"K KJNE\~dpg^ETtK1BLnU˕3u9iƵ{[Q꘾t06i@GZBCm9pG{1VR%ԭZU\Ax5d[ؠ@<( _c'$FԪT 6Z6x;9:bd^0ow1J[q.6Y0/M]IT*<Z ʕ(al(Mz۷CVnݴ>~p׮z5##ݹS%Q`etT|_J lWydΜ߽3g4 ˇƵk˖ٳT 3Й3i9in<0wsqAool9p@҈( ac)i%2:Z6Fjr6(J^:@PȖ+"*JҢظ8$&% Ǜ>>M - S@DDDDDDDDDA(WHgGWL]F%j괶6J wo(57KD%w1?<| 5AeDDDDDDDd*l,%""""""""""w{1ִN̖-kYVVs#~RinN몕oWgwS*n?Ȩ0ݡN116Y6u0ֶIPbc%{|-6Wٲ2u4iDXQL>DDDDDDDv u8qB@uMDȕ-ʖ(O|{x 2"""""""26Aj5޼\{'UPX%CDRK, """"""""""r-aRkR˷o"""""""256Yz0Vre#VB~U)[V8A/j+""""""""Sac)5ҊF(A5B4 VȧOx1/W.dvw7rEDWҥ1o"""""""J_lL]QZlU yЪaC`og'ORe; cRMpD;m??1b5DDDDDDDd*l,%"""""""""ґZFDT$&%T0Y+!Jʖ(!L&""""""􇍥DDDDDDDDDDd^J4˕ˈysƤ^DDEHGza2Y G{899"˅9:#|@|||>݇Xh=J.}G/#V+K洆mDDDDDDDDDDdކcl,%_ޜ91# %œKj*yyBFhwoɶ6i/,4/1b$4y̷a̟RDÿk4/a04'lP, şsWAR|v}eGkTIBB"vlݏ*ea3#]EG-cș,ʗh~=`gܺዠ7sbc!l}>DmѺYɦRD~˷@?7)JL45+z:?|MK*U)/K洊Kt`ou@˰ѱ!J[F""yjZS&CdDT:~,N=gAW3WTGع ƏŪ֨h(=HLLBñ1s: ^+&&ƉcޕpϘ5@eoT*X4o n^="gӴ#]ܺcHJJ>˒1zHHH4x-Z-vغ{*W-'C򋉉ECqY:y&kp*It:ϟB ZśZYNv6vG||BHm~i{hT~;ԩ>}4to,MmZ4K˺бu?Y,&7肀)'K洎DDDDDDDDDDdbJpZ)Q*zmI&M@Ԯ-k|~leD$[ Nh>C6z3f@)b(W4+[>~-{#..^?~—ϩ3lrI ~i/y8Uoo!-ZפxB@QDax-lٳ:bbbӺ=5]g*?|F-چkNlL]&7;X pX*$"2''統FPj(W )Y3 * 1~/1n 746~) %%v#HOJJFFH6G٢гo~ř|z\pCƭ,_2E_"11Ikkk4h\ A+wvd䎄|wo?cw'_7to5+\y3'/OQmcoee'gG"::..N:q55]򘒥bhU9rf*cGbmSI##at llRnFMm7֍{$ɛ/M <ŇOv֮چ+n }!cK.]oxK,I4ٳ(U8H^DDmP0vתY&{ EV1;J. /FD\8{wƁ=T*!w\ E;r5wOz,/a3\π.$88|Y|7_.t7Ӈw8N?A3ծ/K"9DDDDDDDDDDD&`og'%&&"ˑ ڒzM8i?Hggg_'1|L_ɦR{{;4nVo[Xr ODFӖl&o|y y٫{M0])lS՘0fͅR(Xq>2~x.TlYZI3'/![4&szIQT=wL8?#3k lٵ~MQH~:T [6O5Ƭm K}G/?9/ <,BK,Z1oص v vvcƎdk%^szR""""""""""2KN7L$"Hmw/$"2FM$ɛ/.< OA[..Nݿ3<>ʲ&Z]B 0 ƴrFa3]lٳfV 1z@aέ8sNk@9ҩM n˟ȖE`ъ\1kd#)Nʥғ_=eS4 f1eʕLz.\'&B%O,Z1C{w꼮\Ƶ;M`ZC50"ԩFY5'l,%""""""""""("!ݽkNISʳ.:ϲ%Rk89{%Ú[PX}q޼<>QꊋdžčWCFFEZ{QTQa|ҍzݿ3ڂkY[[c_%~A9d#) X7Kӫߏw"ϟ#?1 y SkPZ?}kj٢HLLsttU3upWTV^U:'7K􄍥DDDDDDDDDDdaeRoD?mN,%"s#JSP>9Yf2RUDdNEAo KVL>qacv=vk`Ȉ^Sx Ԭ]YҾRf%(W,y e#.c¸=6KV̈́[Wt aEѵgaջ/Rr鼮BoF >u^Wx Kl&$&$&"!1'kV<);8c۞وUYX'/{!nw{!= @dDK$#<, .E6<Ԫ2SXn.6h/.5i^>[ HNV3/j co9?4 @?CVk8a h˨u?y!UQ0x}};}?|Db轶!,S@DDDDDDDDD$$&"W.~\leٲF!!(W<Նt,̖͈i'9YsVH,[HW8w2nŋgg..ț//5*^pϘ!j#?ͼRU۰1]fdt/߽ #gqe\p_>kXOwWMB^C\5s, oG|\<ȃ0~pvv2zѨi](^qBVm`ܤ(SIj{X{ eQFEac˷ea(ɬj* F{<݇ؾy0>iH4i^ψeߓ-><>Ν,n[7Y |^Hх%^szR""""""""""2[%G߼7ҋaa˓#lLF>&kVl2a0lmmqZaIphIm;4%S-A5 7ޠܽ-{˻5dHDGunLL^۽W-n^d,l*]b F= )pY?IKg@,-Zk' FE^Ͽ"#$ʭ+_|:˒K_R$LT Hcg:+`Fc6Gٲg5x}]{xf?|EOdȌO5OpQ,-#+S@DDDDDDDDDD$R xM6K4~-w!v0=|:{"Eǻ/vW#:pK]BmUK9KJgO0fda3~4vh[(W4f/mL 9nSWHo1>q ԑVcHP|R*V.pт''$'+ϟB2 =µ+wpMRJU^J6-{Vd˞c5uK8|R 9IWg:#o\^ƌGb؀_mX:jL=V4I`3HhҼiE[.SW.ݒkkkb ]wzTx!<y7A<X||,9qM8zNy`tKޑ>^FDȟ K`IpusA;Zv6Lk-T8kx*H˖=lyRZ #6Z5G4*cW5b%{CQ {DO1V\aLE [.SHყXwo&iLz >GG6rVE͊-1~Ll߼=BRDtt,޿ wy. 3ՄWp%]ldɚIWT1y!jT#gV5r&qOuMӉT%xd"[.gg'8 _d%91uDDDDDDDDDd^\ݏ3W Y"ӧbCXD2f`H^2n[[-40b%DD zN2+wv#U߻%re˗k_~=;Kh,-Z5Dp\߆*^j6j* ~PTȖ;z#?i6ō./9-V<w=5] !iVmX\\bל^̚6i[|S*8q:y<+ElT4/[d'8:V#yKx8nܿ1?wnTm1DFDIfUN+lƊ/{꽶'j9xsj&:v6>~R̻mSxm Rd8i1-^ލ1qO0XZlmo7:~Am9dKm?aL4]Y^E-jԥ5Ǝq+ׯrfP94jZW >˺Ux/cmm-:qŸvlݏop4/YDuJ%ΝMv Ɉ3o^. '`[fO+&&]}c9ڳAkUZoDcpy&qdĊsѢUCCRRdIߒ>(k.K1%""""""""`5u &P(P OSAZS2e\)GF"+=Dt/4%u`c)8RsxDEFk٢}pquFr%75O6INˆY.y̥ 7Pٳ)|b͊-?|FRR2sıghtl3wسdow 9 jT4J=MЛ:- ʖ/Zua:R< +->V1x?]3hU4+Rj4$YsX5WXJDDDDDDDDd*.ڵq5$'K7qG&M'GSBZBͱtӦb zw*[>'GG$Xd ;;;}~L\d<^wnnx^7S Bw77yKniȓUʖ4""Y[K7ȹ\^-ժ[..NP7flEn{g#WKJ p놯J#OӲΙ+޺VM!2"Jx=pT%=CXhxU䉆M~Ú,[3h<.2" _NJ%%GJ%z4Fw~O_18ykk8z &N!#z-0IZGzd|?;-zn\^ލp9<\x>|)=w_ FRE_DG`ܨ8v,6l[d߲fB?$HR-rkNXJDDDDDDDDDDiB…ѠF zؽǏqU˕%Wڵ93^ɲ^Z;{vT.SƠ5߽J)kk)7//_ZYp|: {h99:KE۔sXREe,&B,({I%S8uq'vCO%5˕j߹ Ŧkcͦ.hh_~wsep.ԯѱYz;NL3kwѪI7 *V.A288hׅWѨv:ySy.bk+ݚ "$ֳ5%^szR""""""""""Ҋ:m]cc)XW-W%QdIֳ$RWL3d@S5Gzn.<'ood̠y[L""s1dC'.SHդ,9Y)9M`7{]/FȢW7A9ཤ悿m0ql{bz66Ȗi^p)6ClUnر5fxɸ!@)bd Yc<66>i#?x7.l*3 XNfpwC֍Ѳuc,\> 'kpmLw uEDx𘚵+cRՒħXImԣ=,"?rN뵢"ѹ1aъ+lѲuc gg^|?Ӻ>dJU Req,+N,%"""""""t?~g7Q%D0ae˰kR#WEzRVVVRy>ap Q|yrJ~ƚG`bS ammm{Idggn|Э"#q=<~ o"K8aeeWgd̘GREQBio s$O=C].Zj3AP!!d%9XZ-+6QҫCz5޼.q}J2Ae3a_cfŊȚ)+o%$&b[P`ȑFHwE?|Hh'Am_lIi%Jw74jZmP0ްqCKʳhV3|Sr8..NdS ..Aw5p< E\[]c,_Xju`1Ȗ?"qQc,:*wn=@j\f69'ɛ3swT*lٰ[oڢl"#ݴtԲK͒Y2J#"$?-B#"Q*~?*Y OmfW*ȖU2m0 g}wpsO;di..NڳfO[1.jxL-%J>-'Xdˣ KI: |)S\ӧc~зv;&l͕-fc䊈 Ҥ] W3 lL Zi-K*Uɒk qa8rYe7k|r2Wz' KL+""2\EPha<y؈IF$}oԚZCl\Sr2wexL8p|#%+^H{ly$fɚI/]ӷ -(lrUT9T\)XZCTDeƝmלҬlY`ر{#Vd) /Zc 4\5BF\<(5uL-,[B;}+I_y&}]7z0^`^teʉ'4^zW< cYSh,,Й]FakkY\##d˕Y2x[NЅ%^szR""""""""""Jt&uhGFb"`oKphh"jrlZ4뾾xXLl#WDD$=;VV,^oĊĪ׬$ݽ퇏>QId΢cѱu?DEF 8e$ll 5~A_q>j^H^%ַo܃RiK70w]$&&p5WJ7-w=.+Q9se78,6QP(y|ʖ]fLL5DY{;;xկs"M޼g0uYzw/J.VVVز`r~!"JKd͔)㑟*װ䶰kVl1rE郋0Fb!x/\ơS~ASks=NJ%CO ֫.. X-RWo[ai ":ֺ]Sa,EA[ëT*hԔʝ,6Q5S&x1lSZ5;>M(X[A(SK!KC0OXYЄѸx QZ1v8HLOw_O̞Crr҅$fyƞLdB?e8w Xv,fO[,YʳfU<=lYPFEa\cVMb c^ի&MN7ɛSPz(W_pksކ㝺HK􆍥DDDDDDDDHxw+d&:z'a#.>>cǰjv/L_ ֮Cp}ʚҾZ*aqX8ilق|MDƏmнm[L>%"Gl3~1_E.Mеga,2" #nj҇BE '}jjy/SVl2,‘Q_#< k7\  ثyZ5V ;;[Œx cǏÓG^Kq06)&q?U۷/ U)`CG(z@rҰHbbN/׬]E%yIkL_cc)$+p+SB]bzWu~ >WR}LZQ4 +YϞ5v,MCQ ykˆi+>!Μ)SPys8.-ZÐ!0q"̚I `̙}!s-Ȓwuixb21%>;wk cJ}Ar{'%66&(LqF!q57bY IDATk֦u;q놯0Hz.cJzD֜<=:I+LlQA%%'㶟k\u+c kOF:\ARSAXa4h˖ARi]_JЪ_?,ݴ ~P)Ru) z,Nm̙;~ԩXu+#>!._F= K=DDDDD[[lضHؼBЬ~g +BuP&q{~4J4DllX;;[L2Rr ?Ĩ!#11I,AUK67l5Woݻ!X< jPl\jw@ʭpϙ2g ]J;G'KCYs CGØaۿT*zv =n RnCsϘ-Z;pJxg֫`]F >}&kahۢd31`՚<5v)qzKN NJJc Oi7zݘ54S@DDDDDDDd)45^ycO\'Oc* WA:uR<_ Tu [R{;;+YٳdAL\~wwɓh=`[Q@-PݡVsDFGB:_oF!CO;:8К5(S\6BB 6"""""2Lv>:ŭ~ulݸNT-|sHJJFBB""g~M;D֍y m]q_b؀_/PzT^%JEƌn''G$&&!::_-wkШ4W/jԴ.\\×_jn AZh˸yj5ը(9ֺR:m^]TU41i'Mȃ6/@d7?$Zk]#ꧏ_{a̛\r·-V kkkRBims9y ]{vw#w Jwn=fPd !|+5GXn.Z6&|:~ ՅOVhؤ. [[[|kWnc]x$7ȡrrZzݘ546gb+w@R8Qܵktv7o~5*V?~ E#k*usqQУ];|ΙW|1{J80qس|FJ'OP`AOOtkk^ *t)7/L?twxTe{ @$4 %$*Ųb]_t %-ބ 5e?\YBf$L$纼d9Ϲ'gIOWow z~Y閯ϷdTW}A}GYmÕibZW<[ߺs/KieŚ3>nsa8sЁv$X\]]u93zUZ;$+쀍u9)Cq([},ǜgf UWIrE74lay1Xx+0?:{|zڱh3HS+BόWo+ԩ^M__MZݴ!ך#Ǐ߃*ߐ0L7տW/z .Q~W!Z}Z6s/'%k;l?}ujiCw3!7klscb˽,)08B,|LJ%h{򮲿_:QRcϟ*>noJ'M~Z?6TټݏUv^^.C~:}*M>>Z'e5Ai^G&?Q{!ZcZYX @Q:o\F99ZwÊضgOHK~Zh+<{V_/^\hUK>H7yIzsDYh,/?_㛞(ORK?!a]kBե&=Ͳ8ڷKjql1*_M1v"$X*}nY۶VrWmڹTzt.,+I3xxz _͹eڶouc<==24i*oRWfu=1ϊ?y>*+iWt~{;ɤs zyjky8$RF0_=ºܲ~ Ku%ӊ5ߨ_RϛaԷ{"!ܰ9 m~|ٲ7_? @mCB,;cȑغ[Z?y<= oJjϝSQ:}r_|Q3lΞ5_o6WXGs?q_@ff̙vj{iƜ9FG؁&M~Z;jizGt/yAu/ꃏ~`]Hԅn~`%ZaV.ݶ%D;|rssy̘Fȧ w~jּԃZJHܦiL-a.*F_...YW?k}{KټP5|4{۪[.sJ7}Oj_GzWUVR!~gX;WK=#>osm @5WW @1.NY78ڱukwqƍEV CNn~ٺ޽[n]G[=ޒ WsXfVn٢%˞Ξ?>#GSc J㕑)Ij&}2u"u38]u*1Qc{*uپ}~P~5jw>&M~Z~H .d6oiZ~5uW Ӏn&4L7jP_~@7ƕ:dRP5TM/2⸛} 5zj{Vݷ>Ҍw?Qҥo2鎶~?jh_北4oTo~eϷ}6ԉ$\]]-$O/OxmU`P=5iPڷB2oI9zX! C}>K;zD ք?/ýܭ糿_kån=7j}b4K:oPEFik̯uJzsqq!znxp߷:H)rlRJtT7ˋĮ`rŌ_hlƒꝗ^xiU5F WZIuB\?Lgќ/2n24q*vݫQ,z[ڹdWf@ 1iaǗ+G)q%]J֖Mqڶ%^N:xA̖{jTz fM24Xmچip2˗x6ۦ[u1.%+#=C99R>VjTM2$X!]VRNN֮ڨ7iw.\,L&V5|UwlwU.vYzv/׮ڨfj]VAg@=GU }qx-Z}֮ڨvǕxUԩf-c]94GnmߺK GO3JM̌LͿ\WpԶ]uQNm=#_GiV^JKM~;vϟ":T-iQyxx_ V``6mG}TK,QVVih,0Ȉ =!ܰо} ?a[%5&jpնX6ϱTbbJ-(4zygӛ}d@*u?^M;wZkŸD[ĨQkYYzm '_^zS թUbEXԎrsso>͝;W ĉR{j޼yrwwcZ83WWW}%n.^ffbcc5qDiӦoo>1-0C7TAA$@.XPa mԼI^ͦ{*VF>S~~Bc Sb.46f0լ^X;̘ܣ_OIK=O< &+gd}VZc5klѾ#G,ֺwꤰr_Qx{c[o胯қ'r8n٢'O*11ۃGц :/ I=.\+W\ JHHƍzaÆNCZ(K T? @}w׊u뮍ʸnaj[k=TM~B?x\]Io }5ܹjڰM}bE1WWW=3n\HU}|hL1t.%'_ux͞2깿<~ks͞2E (ذ!??_knՉ3gw]ste`ѶQ@VkxI*'FҴYV6{$xt;xPUezUViժUez t):uJ7n_ZjޢZQ͌X `G(XqölC\k,t{-Qb7nԹ9^kOVJsiE{u5kܸDIZ0c"G.'[ǎc1itDՕ1G,?S~9 IDAT=-Res]ZGS-aJܰjSp-9˃GkPxxl d*IVUCQAA1 et87KXwuu՘aÊ=ϮB:t2oy>4TBC->}GE бc:s"GV4G4k\J2v<>b?Z40Mo>xh*S}'ȑrsu-2=?[8.vm 0ԙ´oKt^~I}6.ofO"5 ۧ3gkAAEwqqQ.]4#l߳G'ΜXS=׍Ψ8$-Xܞqʽ|mرC/_8xBiq2238,}%ӎ ojyftH }jPҮ]5kRrGW]h.\o,ѬsqY~~@=uoFSO1 $I&IV(}զE fm^ۣxMvN~^NZy6:qR\QFVLժV [Wڬڴh-[ڵ~eVku_Cӓ'[_LNDuEXNE+7lPrjёXa6gu@oUxdwEFot/խotD/cd-v󺐔$ooխSG͛4QLR]|\Coz\Dn:rZLY];t(uQ6\]ԼIVܭ(22Rҧ hyڹc󕙕))J8uJ;-TwuuUhp:~lN:uRHӦrM&J5'/ϊtI,XKf`@چhEjGR_RҔ4efe@ def)%%M))iJMI@X bY]*Fzrhc}zu!).sk{>?_44bϵa?ub]HzPZu])\\JH^,6J+WVҖMeӦzzXedfj͖-Z/Z/:vܠqKQa sT1: R;y})m/kR~v{6a@z-6ъ呏}v%~~\Z+7lPJZ ["Ppu6krJQlڴsZΝѶ<=ZҦ ^Ү'777*#+K._֩D;CÇ;0Mj]zUtq?~\v^ǎkӮ^zI۷o뼞jҤ4i=:jvJRpbfYWCkM=f$M_2LJeݱ'ڴh$ΫmHZdaYU-I*((Й3gta%%%iӦMi\\OEIRVV bRɤ:( @ZR``IMw`6m(::ZQQQ޽ LeRsZҭV*J}c+,,Laaa2ezjT߾}l2UZ(2335d߿_<"""ԧO RSә;i4ŗe͖ccK[ SLLbbb͛7+..NaaaFGC9c֭x+s!Sڵ2m4,zXP^6>޶777(x}bt!=3ffJR2Vk999Llo[ "l}dXefg;0 lmwo>T$4m%)%)*6^3n +v@EBc))lE))JR2^rq|[?[X * KNv͚rwsZ?wӔ0sr4+U5V,*u1$INAe6S,7ܩ~@'$X>wNM6tp"t*1j~@Kf%--ZXm,]uk_nkjcDKZc:8 @p1:rG۶Vkԡcn8zY&q^e֠^=:8@٠4{Z3͚:s”@ 5KjPX pCCmnբEڹOF9$kVkVhh,8'FZv\[g>4ԁIʖ1c8#F?˗-֏<ѣhLխS'^}v^EqP+…"_MiWʷjU^j,ڱ-V,8jUh_jmw뎡C5oRb{y22fbiuҼh6Ϫe@X[pp"Ê}iҥZucN%&jSO3tOt"wW@ծUKgjZ}Voڤ2ܵC-XbmΝ ҥ̮v:k]ڵKNE_6LϝyÇa;JgK%iڵzƹ,]j}\OgtbԦ֩c9>^ɩN]ͻՑb_Z~]/ZRhk:WVAA=NPh,P)=F@9Ro]Z94h11z4uL-g2Jg ][/]*RaJo_b\LsRVk14JR#Q#͘rRI=o-5:OϗlXȑe ps?NUw{K[Wdit PXrZZ[ت^B޵ZںkW[-U>>V99,|e PX R}jָ_dtkĊ#Gւջ3F6mX>Yn:GL/Pڕ+!Cl6 2/\X{`լ^RFPݺ [pr.^T©SkU||T1O5i@-\gɪU}چ؜aJxQ0L ][Vԙ3֟3, 4`G߭XaֶeK5\\\ܣI,_1C󸺺Ҍm׉3g,GDuN8+ I׿f϶Zuvؘ5mb+s~]˙d5+M&I|i'p,Kznk ,fxf]YL5K V-h,84ΧjڬYVmZPvz-x}wv^ewYM-\]?V iug:xhٹ_{q{װEo=kN+W~ﯿLGTpFNxnFђRRtI;yR%IգsgjLjޤjVjUZ*QիJIK_Uܾ}yzwuQ T&ϥ_Ϟw@" IIzrd}[erdZ|Z`@g4={%^yKZjWo-dK/iźuJIK+RлF PfׯN;G'MZdURŁbtRciY뫟>L:kksY?6iN=[**٬1<))wGFH0͛k…ԦM_K1BzXKIKGs:$GEe.۶bNZ1yRٱwo]WS-^Mund24_R`@@jU_?7eHj^[.Thppq}9}h,8-}'k@ުY-g2ԵC:q{Qy!iԩUKs{Onwީ˗#"غv.Y?\\~W= NxnF֯<2:m{pBNT©S gf*+;[..T __qPZ6mNmڨGQ%IީM(=5fL&ё*O `KNd2eӦjٴQcPaY=:w6:a\RH܌t]zU999FSUVUftDc)t!;w(.^Ǐ^zjٲ\]]\fvMS)\bbv-lt@c)TN8dc|N:et @c)Th.lXJ(Æ +\t-YDK,$\k2e5SPl7[Jh֚KwAYsjܹ>]*:,Laajht<J#++KYYYJJJ*RT:uTNիWOAAAR)h,J"';[6Ӫ˵e:a#NjFjql$)48Xѽz):,L;u *l>}ZOV||$f͚ TÆ ըQ#yyyR$223]ftIrE)iiJNMUfVd۷O&IuU&MԤIrR$jԨCl6k]ZBbWhM3:"4ԭcGE)gO 1:$f%&&*11Q6mRժUլY35kLF\*!ɤ6W{^Yںu~YV,CT:[VDnM:v7[{W^U||6m(44TFG04VX˓h޵S;wPff Z* Lz[FG0j֬)|qAAj׮}O>V$a̘1 믿nرcf̘!WcĈ2d6mj Ѻukn刍۷Z=BJJ t邐BdLDDDDDD_,,%""""""hڴ)NSϟlj'p ǣ3f̀;N8pofuQ aÆO>2pDDDDdl,,,Ю];kiii}v/Ÿr bbb0h 0c"jHdTh޼S!"""jPXXJDDDDDDZgeeU]}ȓ'Oѣ1c6lΥU;vE0qD9}"""";QTT7oݻjoa޽h׮ SSٳ8{0p3!!!NIR: "ؼaODDDDDDs...LNꁲ2h|NQڢO>ڵ+ܹ7nTwAZZgggfJD D"ADD0Tբ| Snnn7oBS!=`a)镥%o4D~ Ҳp]ffia!B!^^h S#z.W ͅ5\77huҲ2\}5i'NDTT""" R|ٳ/2<==%w&&BoiDEE K DDDDDDDDz2?QXT~ݺiSB!9=wbbp/>VV03SPa|d$[C6pmťx4>ѡ8};ZW 7!JQRZt{EEض¹Νô%Kn$УcGrs矈OJOB%23)%#CGLtJ]σalxŘCp$'': GA,4R""""""""=C2fMbǺuнyYYsz߶5k0qr JmmɢE>nmmenFGcڵrTD|͝v/6ZD"_أh\簾}JACOcֲex_O?k++ʍH1?ooM`4}^z)K"""""""HN_(5.9SWi)ϑ%ssq9q&, K֬[Ȍ+6b|ZP+FJJ;u T8/_P {klCSǼuXT;u˗irsтxoRJs'n I''L;Vy q10$@Ν;ǏGnnns]2 8G""""""2:,,%"""""""҃ 5{ǎȤf21P߶lQZT/8|LuL\Yu[⛏?v ݏeb݃ZO.Ģ>U;tKfR8gɬYpqvcPRZ^!!XZ@YYZbĉN0.<9;;^ӧZ{. """""""j&/äef }c*bؼɓѩm[\&((*JٲgmhW/YP'J&v!29Ǝűmpnn|hQV277Lj#Я_?E޽Ǐ}BDDDDDDdH,,%"""""""bmTjl;!{{Tpoc^zI&VZVW)7Uz1x4k~]<۵݈JnDDD0cذa05qqq8u}_HDDDDDD  K`0ykc[o!#;@Yz2~~V{qC~sSUzV)*M̜)7"""jX1~xX<&&.\CVDDDDDrdff"-- %%%NHoj$Y 77&?xN\c`Ih u h-E_\5JubO qnDDD𸺺bҤIؿ?>}Z㵷n݂wﮧ숈TH$ ,--뼦X,s޽{ΆH$%\]]>}_~077^eeep.]h䠤ְC-Ю];:tc]!deeٳ|2J4jԩz䘝={ܹsHLLz & )S_#dXXJDDDDDDD'_._d?^&^ZV?? 0eh4rpk~9yyr1-4ZˣY3B՜Ǐ5MU CFvV>-,DDDcؿ?k~UXZZ"88XOڵk?^hNZ5###' 55Un HHHѣGꊥKjSRR~{AAAxaa! HlٲXhjp]")) 7nݻwt OkoÇQQQ!7LDGG#,, Xp!u͇T#T~}D"k222o>۷cǎ{gzPeR%DDDDDDDD ffjBX)ynl,Z={bʕH[~<9i mmeb 6]QT\~øq"""C=dEDDDT{iW_}9s(,*U$33 .G}$96.]ȑ#eEbܹؼystIgP<|=z4FUVСC~:QTTXj<::'Nľ}*P|wrwYDKbÆ JJwA̘1ֻ蚾#0HB!>xm;ޛ=Jn^RZowrxkx :[Ԕ7!"""c„ pT_DǏz""""mJؾ};>sn:O jxÇj*^cھ};ϟT~-ݫ|m(SPP/CŦMX~n1{ld*y%llli&> OV{bXYLjXj84kk.#=ǎ;p˱d>z-W4'Fb]"H&X[ci.`}DDDDފJEE>_VVVz̐+V ""B;w;v쐋wo&affrܻw;v.sÇѮ];;V>%%%رc`Z q==zTa!_~~E\:e.]>OT@ @~~~׉D";(~N@5kF&Y\\/ᅲuVk׮*>"9sӧOG۶majjxݻo`䧈#c)YZX`ر!D<1/{`} Vri4Q[=/%#C{Ek`SRt/lmm1n8X[[x]AA?^/:ыܹs0a֋JKK'ŧO]tX~=}]96l|Lo߾صk~L:=z@PPz_8x BCCD|u3PfxO>Xx16mڄ'Nʕ+y&ZڻvBs/vѣJCŞ={0h 9RTg]K_'NM6K.9ڴi>+W~ǎHJJIs&.o3Eg**?_.*߽+kZhR._:ߗH;v,,,,j.99W\SVDDDԐ%$$ 44 .Tzߐ'۷/y4o`2|ڵ #F}k*0a]SuEg]\\b ?6l)SУGZ7ٳGc'''|5v733ի룷oFrrJΖu9cǎɓebb[nz~2s.SC'@DDDDDDDDʾ}0bR22RRO(vUmU@__D?xPC|RZyzSbwW;'uy5o7d]3hdd$<<>3gT8fnn9sk׮x7O~~\[pp2d~}tܾ}:tP:NR:teYYY:),q13""ëtE4u)NմUXUX?Zj@DGG{{rcرcG֭La)Z K55Oj}@g,D+Wte˗kbUUcH$SNϻt\l*4hLai՚,,5s`a)RTڸ\^^}Pf7C[#꘸#/\IJ/'ebB걣YΝ$UT`th(~{tjV=HL7ă0Z[cۚ50|Vi(*1[j>'NĘ1c0eʔ:tk׮H$QFXn <}T.qrrRim K[p0|4D?xL<)(@H lml-[MVC:^+HNOG_!'/073C#S۶*O>տBzV?\47:@ si u%0apL>%%uⓒrX.hӪ|abA={"z,JJKѻsgi5k;ƍJEPPZḧflŮuĪnkm]b YPf;W_ŪUdbk֬'ڶm[|DbV R)V\8x@@O^ųU>||X]vظqJ&MBYY6l )X}a߾}CF)3yꫯ OD.faa:*Maa)y]x4kfʹf# Wkꛍ5zWHtwSQ=CRϺp&O\7U ^ZPPTZ]l*JQPPBTHJ!QQQQ]VՉXպUU1(--EQQn`駟T+,,̙31qD+hٲ5ŕ+WpIdeeɬiYnN:%_|aźR&Oyd̟?N9bڟYŋUW {u]v !!A5j KKKأG^Gu'ba"wt*jcLlڴ C>}:-""""""""""""޽{QQQx!"""ҽ۷O.+F^^,Y AbŊ:puueffJeee!##C&^пX||("亖Xud> hѢE]#2870prrB^^k5knjy?3/^,ZoCDDDDDDDDDDD"jٲ%4i7n[eY?YYYiȈ[2Ɗ+jڷoիWC h%šK.*NNNr1E߯6nX}^G;"&&F&vM[HKK\Kd,Vbرغu1c1z^ZZ¢R"""""""""""gϞ8|hCDTTVqv "ҩgrzK5N<+W_ܼ9HZn-ux xO31X,Vy*l`mm:w={VRT&*ыH_1>3_ 88K.sFDDDDDDDDDDDDy{{MX,FTT3""""Ҿ˖-3*QQQLMMpttJ/???4iD&vEƹsb:uŞe*>"$$vvv2guUD*СCrqR}R%\\\fffᅬ/)QTJܹ#9^\0333tecccHO***}vD2cxwU^Ζ͘1ZyѸquG]`21H]v4͛sL,((Ha<<s~JDDDDDDDDDDDD'''O>EBB3""]211A>} iD":u *6lD"3쌍7Ug0sLd3'Oƛo:٫.Ǝ P(۶mkWPP?X.+3x`E"-[ZJXf 222d]tQ fΜ O?˗/+իWzzQ'D`jH=8{[nUVz̈tih޼9 yxxiD" 11}6.\7o~-7oO@iؔ|3g4=gggL4 ?cu.}9&&wcdd$V\uv튐%G"C`a) M6|2"''M4sfD+7tDDۻw/֮]J 4+WZ{ݸqRT&v1;vLuPxnn-ⁿ/_1ch.̛7ΝCjjjuХK111pܙYZZ?)|^ƍtRZJ&7xmڴANФIGƍ+ܫg0w\DDDÇձR|ذaڴi333$$$-V\Y/󄈅DDDDDDDDDDDDD/SSSi7oTz͝;w0p@=fEDDDP(Te˖X`^ӄ76f(VVV1sLS\\joaa7ǧֽƎtlݺU&.Jhrի/_tgPOǏŋJ皛㫯{{h"eDDDDDDDDDDDDDd:vXcb=fDDDDe4h[߿_Rxsf͚MLLj54k ۶mS;?OOOl۶ ;wVyo> 5R7MBbÆ  J#,, *-[x'԰c)i(((@ii)***PYY KKKX[[ N޲D"$$$DDDH$BJJ [n!55NM)+++vvvpqq;:L+{5~!ϟHT7<ݱuV8q_lƍkaڴi077W{Ç Çk(ꊗ^z 'OFӦMk]_[g憝;wb޽عs'^qaƌUiyB ^ KSSS/"66=ӧOQZZ +++ANн{wA #EɞqqqGdd$␕T===ѺukԨ_U%%%{.޽(Źsp9|' ~#Fh%We6oތk4w˖-߿3"""""""""""" OOOۣ@xb8 IDATb"acc̈6Fvv\hzz]7UiFI&: 9::W^zZ_WgbyB N Kϟ?#**Nddd믿Faa!mۦ쌇T*Ŏ;tZ[<+"pa>|ǏƍUj\ٳgi~M_#GbԨQغu+7n}󑟟b-gCDDDDDDDDDDD[JI$ƢcǎzΊ]GP^^JJJ H&5LZ-,… i&J51h e,ȑ#qEm- 8pV4d <qqqZ]7==Dxx8tյ->>C իWdt^XpqqAvvlnrDDDD"AVV\hUhnnA255;Zh!>>VVVɇbСz6k(۷/u~aa!F۷oM'{h˃0g۷Щ@\\ K0Ʈ  e GT} s}TTvm\saT*ŴiT**uvvK/ M4 "::ΝÇ kF#G 'OѢE //III8~8vڅsrr0}t:uJk7ڷo???xzzvvv.jCjj*"""pQ\ruߏSNaȐ!Z˕ K JǣGzΊ8 !!Aahff&D"A򲵵Q777$/"":nܸ|駘]iq駟~RM >;틌 \zwEPP#"""""""""""?@BBÇFu G5kS:|#""һ:rJ+?>[7dK,QZjiip2e DqUtM|}]Eba)Qx{{8CbQ"""qaÇsN{fQ?n޼ӧO+5!?݋X KW^A(?{,V\笈[[[4mYYY SSSQ^^ =gFDڐ$ue4 yN=RX< S,ʮDDi?JcVVVؾ};JqFc[Ƃ 4^Νp__צɓ'+-,s6DDDDDDDD8BЉbtܩ qyIIIsVDTWgϞٳg Ac̘1 1t*J]QKKK*,uqqAҕr: "' UOQaT*EXXɓ'UV'UD"@ 0u DDDDDDDDDDDDf x{Ai@R}zW~1$ADdiv]G]]]ѩS'рۛ."""2ZKquuչkS[[k ,vuuVd2AAA1b}޽#-A "25T۷o]Q@۳(D`S̥K9hX%u7"""˥h~~>bIrssCpphv&@DDDGgg0էm̑iӦut{yS,Cyy9;]GuG|iiX6mڨej5CrrBDuݻwQ__oĮDDDdVqttD6mp=III3fL FII K ΫzI$ܹs]G@ ^tEY0+KII1r5^\CC+!"""""""""""2OpssCeeb#WDD22E%"""kSw8sҹg϶ kӻwoҸ8_2rEE\UU+!"""""""""""2_*%%%d1rUDDIOOGii)|HRacc???i۶mk I`СC*s0pf-u)x" } )R,DF|y{{#;;[\CC*++n䪈4"ccc:(,3f lll ɔoذ=z?+Lo˗/7~aU\VV !"""""""""""2c>>>j狋,%"ꪷ`3BBBGࠗ}NR___DGGرcJgѣGKj Xl0l0䄚%%%FHsKuW """"1cO/9sgφD"u YXXRנ3<@[lʕ+XsrrBT3KDDDDDDDDDDDtʹb#VBDmb7n;W⯿ڵk駟СCѱcGJH`>l޼O_~#"""""""""""2 ꂥPVVfJ<7nDqmǔ᭷1vX 23!PZZ4$%%ԩS̞=3gT[СCU޽Hk}Ø0a޽%夤4{ȑ#˗/7{lMM Q 8<(**2>qo7nzuuu8<yGDDDDDDDDDDDdj...*FH3"HANNN>pGSi``џlJDDDd,K0sL;vL_)ɜtQq?f WyZ}R"""""""""""*nnn y8Xڜr\~ׯ_W W: &"""xz @@@9 6`ɒ%K{۷c֬Y(//W{)ǏcÆ XlܹPPP???kzSNaʔ)z\̇;ΉD"HRv#"mTXnDDDDklll0w\<3ذa~G\tI"##1c :Ts&N+WwݻQ__p[!Uch5k6oތOL&yHL8f)Tl2vvT rG}Tuq5\_5ݻ666̙31k, """""""""""2Gnnn*R)D"ݍXQ*ա`{v8m`uރpB,\8z(q dee 􄇇ڵk0!"" {cǎ(((/ʕ+GEEJu xjK(bܹ;w.p #55YYYw WWWݺuCǧE,_\?_ Æ ða #,YwyƑ#Gpedee555<==~!** h׮A"""""""boo:A>Ƞaccc2yK.]: 77hNNrssQZZR\vM鼺n:tqDDDdq ,}PHHBBB /c&~~~7o͛g}[]v1cf̘aR,P(DLL bbbL] Y ???䘺 """V%XBlADf)쩧MAӇ^_]SPӐ8ìDDDd,%"""""""}a"##M]sqq-R #WDD@DDœn7n@mmAjM{>}ZivJDDDR"""""""_~HHH0u)DDDDBdd$:ud2@ *++Ϋ'"~~~JC]t"""֍R""""""" 2r `ꒈЧOtnnn*FȼcӺ:v;%"""`HN:sYgggsUUUFȲ4J;.]`ucP,T*+""~~~S:-// >|8>lRLѦ.ު 5L&kuTTMX Y"$%%DDDDDDDDDDDDD& qq_NN˕ BܼyhF@ر#M]iHn7oDUa>hFDDM6.DZZZSZKZ ;;oG׮]ѵkWSAt;}6RAҵiNjLR"""""""""""""#5u jk0T`ANDDDDDDDDDDDDDFϛuLTj=:wlȺYbS{{{)Y$K`ذa(,,ٳgMyggg۫wppP;+WWWxuS@:xa \`2Ν\=Lsjkkqq:u III(**Buu5:W^:t( BAA>ӧO#;;%%%MM<XXX۷ѣjs 6 ӧOGhhQ벆,DDDDDDDDDDDDDyv,%""2DDD ""BaN]ӌ (]PJWX-³>ۢ5ϝ;_}\UU233{.F>駟}vTTT(WVVyyy8w{t-u &ׯ#77yyy@mm-8ooo*++ڵk{n+#??IIIشiv7| 痤L&O?N}ݻwsNܹSN;c𐯵\dY,%""""""""""""j%`)v*qmiNN!^ˎ;_bƍ7|'OҥKakkyN²eP\\U}iiix饗0|̟?_s :%%%߰o>dee(|}}[P*N‡~RAYYYljb _Dtt4UUU8y$֯_7o?{+2gFϞ=agg ر)իXv-^ud 1Y6KZKHGbŊZ=\555׿0>{lrHH/Xr9Wȑ#t/ggg|XhQXtt4͛p ׯw}'7'?wm C[~9PAa AN'''Meܾ}[nlꫯ$7cԨQxqС9Lo Z| 3f{'7֣G,_z,7yfL<AAAzүc| " YLdggkxbkP"?YϞ=())oȐYVV-[}4ȑ#??8p-[{L>X`JJJc#G%g@/6mRży0a8;;~?\!T {{{|駸y\ҫW"'';vԹe~wʍEEE}ͧNLlٲi?,:&`)Q+`)9|0>l2LСC2e oRҥKi&?~\#}~̸ /^ѹo&bccQWW4c…jf>}:zq .Į]4;w} c_Jy̟?-Z?55UiBgҥKϝ;`kS=3,X*++bccxbtua)1Y "JHHH)"jZ\t?$"""]7iR""Jq Sa22UDǎS4 B >])####5zxx`ܸqrcyyyzܴ '0^PP:2kTZJa> ZC0{ג$&&"''Gnlڵk纹'khhZ%\d,%2X~=&Ndeej"pttDFF|||0qD_o6uYDDDԊ~ DDDDDQ,H$F4!Q__o2LJ$"ԥK+6 Ν#B}jA)5,U@@XyyA2k TKu>Ǐ>%%% cNNN=Xܢv)#Gj|ѣ5Zyu3uDUcX۷/^D"@ ƍ>@RӧOGUU^xݻQQQ0abbbl+w""""M~DDDDDSwoDDDdM$Û ,P%Z#""Ba,55Uy8p p5xО={:N<'O>ʂwk+--Unq]xСC5>?<<r͋/B&=-c_d},2XZ^^F 2go>)<53ϘBvs H$8{,Ξ=?~~~7nbbb0vXߐ.w4ÎD}jQ͒H$$$2uD㑗'jhzmUWW+iYj׮ܽH}y1^F2 6lsttkڷo0v)ht~bb˜d2RRRܔvTrժ*lc>v6iiiHLLlO"`޽.]+11ppp@mm#R.MpMv2lJMشi6m@"&&111۷?MCDDDdxKDDDD}o߾ի֏) """Ѐk׮!>>/_FVVV~L%=zĉ1zhDGG% DDDdL=z􀍍\ٳJjKpT,^o?Ķm1k,ojR'O DHHn޼ӧO#%%Dptt7ڶmAaذaJϪ`;vLkiZZ̙> ]tQ8… XdXhh(~inν{|||^G940kx JOOkB*&MիѾ}{ClIdTŞ={pQؚdjO>$hfҐ۷ {ƠAg׶*HDJTVTB!Gښ .U(**+WVuxKnrr2bŊOO8&L@``%""""2KȘѽ{wGs!66=hBV]]}$2ؾƔR1P'|7nP{ݻaggz ̓F{`ѢEXbxFFf̘)S`֬YM?b cj*7(//WzeOR1똌GҌ <%%%Mc صk.\ .6ȔZڕT'''̟?zDrrrBeeN >>0 jz鋞={A `7ScH$H~W._+rTV 돐Pt CXԹÔyyq!ğ=iξr ..AǠN}a@`t KmE8~ş+ yee B^"1p0Ϥh4̌t?v !3#E^W77tаn0pGwG3'p#=99Y@}F[_?0CFcАGn:uWiHG~~D"TWWɽ ݺG^/ ;NkЀkp>.ģ ^.D"ݻ{nDDDDdj"""2iӦ?={6{T*Uhekkd2,]rݻwٳ[x0HKCClق}a͚5nOWZЀ;wbΝ Ν;'wn`` ˦>UWW+988hP(hmCk-1>\AOf}oKwJNEE$2lfc脞5m}a[k} nY0aԧF/ud2w÷8u7 Buu"=-En?LлO?k44`c"I-bܺ[73{o`qX·8DU[S_7CRUǦ&X|KB;D 6xǏ7kV*\23$pptDL$y% Yz%}ҹxK?{ujbsypqdnU^mǎęQ\U**ʑxIWo)ЎOGݻwp o?w%Ud͔1tJDDDcaƍn?3f`ҤI )++CZZΜ9@nMG==lժUsww@=́_ĺuЫW/Ι3gwKPa0$p?n:i̓]QTVF ZZc2Kߢy"S1TWR֬/x|+tOD2ݝA,/Z_MUT#.vb (FN:~)ɉ-Z .}5DJ|>_"̩:/p~>z BL6e _b1ݍ}{wOנdakpAί?v`;0pP+RƑ6m}*ՀX,3'}/xKRfJDDDDJ]Ԑ%""~)̙#X,ƦMi&8::2 v}l۶mZz5~1@>>H 6 }App0aoo "..{D"i:|v '''CYY?ӔTr/lϭ:&kT,C$=Xy'"S(..Ɓ8 "IKMFZj2Vm}0jx#ǚޖM.u~Q-:K~FeR]D?:Zokj r9;~-++ C\8w46oݍȾ&oxmt{ş:^W@}())Ʊ#p`,cf#&&cǎw|;ooJjkkUiio Ɨ.]!C $CcwO r[lAxx㽽1d 2'O믿.۷o_~/v_De˖5}@Qn0|p\xW^ 6:t_ 22R/Sc4shmCim1^B())QyLv%VR).]X۷.\P^Qam݈m[7B A5& W$xٌ{2q$uݠNiDğ9Ǫ0{x(ī>u<Ɲ[pvx| 23w0e(0, ߯[}]aa2KMR)]Ccq>\]"S)((ƍqA!&&111.;Vyy9Z /JQ__" TWW?T۷믿Vxl*С;us=֯_0h"<:kuUs###|r,ZHnܹsC.Yrcfo28p[lQxKKKK/~@Ϟ=5W cK5ZZuLƣ`)L:?)S{K"!!! HLLDfB" )ڴE۶h߾|ڴ7_{@b"Q%Oon=zDM[_kxW-Z b"U궭* r]S~.l++ z%)ɼ%2׮]/|}}ѡC3͹$ჱHK.ٳKa%""""DD=8-=io{c߇ӧ{n۶ HS} IDATVR;w.͛zN1&V__Pcƌ @bb`iII V^-76tPPlmm1m4 Giii\aa!6n܈^zIzPZ<==uIS:&{ΝG}]v۷nj3d{K"B5 Fʕ+QTTcǎaϞ=uǂ=b4=2E\o69ww|7xə sptrBGlr" SM߯[O0m3j8=z섦_gf?bt u_Mʁc\]СC7qc B|᯿TקW._T9w[.V?|ѯ@|gXc?,8zb1iz,x7&_c`L >d%2nM?v!J.\z^(P //ot wB`mB!GQ(WS!.vR[@@&NѣGcĈhӆDDDDd}R""t89{!55UnZ+V(?3xW[T%pssS*Xڽ{w۷믿uŋkOXX>S]0|ܳ+iν{Ƃt)I#:&{իRDm۶x'OB"ʕ+8tكgZ'{́ uQē31xh4(B"Q%9r~æxt}"T9⋯QCaG7ڰU*Ue̸ǰ{LJ9s8n"4Vk'%]7k?A>}W KU{O~njxi%+>| ^=|ZDYuo jΧM[<>I<>'뷭p,RSLLDlmm1x0idd$w"{ƈm'nݪ3ѣ{aBB.]O<~5ZeKKKNUQ։S٣`޽rGΝ5k8p Ν;4VTTLC4ùu(;GSɘ ,%tC~ﲛ)1 j0z鋈pw>gDɮ)ɉ*GS>RTWWqt!XwӪs{V._oC՚gXR:E\i^]m-V:kcc|h?m Cz#)Nki$())ozѣGcɒ%-R+6Gǎ5^NYAAޕo>F!, Kmll, 7//UUUZ=;==]:thq}uL`)fzQ8qqqqxΟqrr/GyDi3gbNؠk׮߿?zЮ},2@h) w=znb_TņJ0 JO;UKj~ikb;q#Mo8c/WTr9T ;{As]rС1tpHRd_ŵkpE͔dZzkkkcܸqƈ#F;YVu1鈊x e}VSHy]>߾}{2Qo߾rRtkΝ;rcp똌R"- =Gܢn5555kك#Gj":^xA7}}}1`{G\UU4~֘6D霣֬`7Ӧ?2X|]_F\hX7u{ |I}ߟ}Z۷mR9צ/]V2,K2"22xP^^WٳHHH@iik~LM]v%%""""k.XR9.0USUhXN2ÇW5^c8sX_C(FKҭ[7˗/k<1etV^[ʍ>|X`#Gs1Bo5uL`)Q =ʹ'Oo>4{~uu5LcF输'Obʔ){쉘?=z@bb+$e}T%\7b%.|gЩs#Vd8FW9W\|Ӧmbk,nqwgD[ȿvj(*,Tο[puukQ?r]1|pDDD%ВaÆqDDDDd=R"2';ǥ T+&ˑ#G+O2=_|ܘzӒm۶rDO< kѣG00#5,ڶmjpss{a,^nnd?,uL`Amm-xppphfrJdeeaÇQUU ?qqq4%;y$&LH7775;6iәK]p+;vlی3f˗. YϽh:!S|QXt>=-58~JB!NѢ: }pQERyL֩ͅǧNoQ_r]u.YYzsHDR8L7o.:u[oZCaa؜9sZ&JZt ...GWx?7blٲ/ut]&74D퍎;"''iԩS(,,F?C޽Uk 0i$lٲi?^{5!--Mn,22ٟ5k[cF *d666ppp0Ėٳgqq>}yyy(..Fii)lmm@tC qfZ#-- ˃H$BUU?FN^z!** %Hpu\|unݺTWW555 pvvgc:?3& "b-%ľ+j+KUK.h}k-Ric+" }_&̙LfL^Wss瞙31squ|||йsgt Zz&5"00%%%z*N<Çܹs/=֬,AAAZiMLpp0XsD"A1p@>>> T5n ;;{/3;N ;lj?cu=Uk_   &$ ؾHQR^ҧ``LL^u|`E֨]ڹU'~%}ϻDDDDDS H$hѢ `bb333OoH$066~4%  H/nnncc`ffULt!!!Xf }ZaGgd2d2pM=zK,-&O9sȽ@b<F*&M 22R#00Pnm۶xܘ!Zn֭[c޼yr;=[&XDτ%z.MU mC[o`p"k-+Z~jr*o/𴫣_*$E0{NGވiNWT.~%""""Xqqj!鏺unݺJXp!" ),,DVV߿;w ""gΜA^^ܰvZ _{Uzz:^/XTY~KyyyXx1ZltN1c矟o+..Ɣ)Sdtڵ1qqqz{{O>s`ǎ/l^|ƍg}&XC| -:u|xkII Νc/Օ]p .DV+ M6󨓣1ы_%%%6l߯.\1c|rlܸ־}{?/T222waXlO^z>#XXzM$$$hPSquwٳ'>,l"u:u  B˖-٥;3+rs3búuV8&MC:u5낱-[k|>]sppd+5Ɲ[1ƚ[J0(uEwy5jVN=U弢?yH9XJDDDڳg,_nٳ'.\kkk|ri9#G4΋}rKO OO٘1c._7}!NC oBS/ܹs(((72d2qMvijժgG~YXofߩFT*tP0^X*}Qqq1ƍ$2SMjJxaa!Ro锨yEϻmKDDDD"q*]TZ^={ܹZs%$C͚^Y̰aL8tmܾ-܄xbʕ+Y\CE||<6m2DFF*?>cxe_giHN~WjjZ!6 T'G1cWmǿJJJ`L:ULai2O?_+cKQfMx{{ U%|(Zzq*C&͚=[wRgFF:6_ v0|po VcE:mW)CEyyʤr e|qEˡyEDDDDDDDKHW`ll yׯfϞ7̙3о}{Tjb%h-ZWL4 4-UժU 6mҥKѺukC1c ݻ͚5Sy7xGA`` \]]+= WZkPF رS؁&MѢE V'G1cWU[k׮}pqq h (( &~bԨQͯ~h߾=ڶmMaÆ9dܽ{ ChhJK Ѽysq􄧧'`nnsss@FFܹӧO㧟~;w{_~KUǀwɍ=~׮]C+={DD0p(:_m+REC?ѮC'|hOc+*"TzE,sQQwBqLP%U󊈈GEٱ^ԴiS\zU#cuFƪ{g{Oӧ!ЪU+ +@~_~7T 6DF*]knn@"..111HHH@~~>$ agg&MF*@LL R١~h֬ZTQEU/CD#Gbܯ[n6mҰpBYF2SNE׮]W)bbb&&&=пO wY wF^j'''os7|ŋ .~ #Gpleh#RRRѣRcw5f??m\7!--Uϟ zua·VxPzEQ sGNN&E?UuU""""""""DDDDQF : _q54h@]gyB:`ݺuV{{{Z Æ Ñ&w,̟?[nPQ#F5 >Sc;wXa=0`kv̛7O-7fooTgY""mrssϗaނm߯شaE\QR,znGͿTX\jff0tޯ Ns 4ss9ddk2Q*V_U󊈈 K"Ujv0=|0z|b;v`…LU;#3gয়~?}4*Z "XXzYQϜ9#֭R`*b =v.^8MV{+\  gW,Yƫb9+g{UkbwpD"AiixFrtD'yy"f"r^{BQ300P:`͚5XRsT5K,|^._zss]JJJ=z=.6uͿw0?3`bbR1֭K 1J \r՝T ;;{x;"f#.57o^1YZZ rsrDD\U""""""""QQR#*ۗ4jʸq.a믿*᭺puuE׮]%333=Ҟ={=.j׮߭¹KxcaْY~Cص+*:Pݾ%b&yy5s񜈙(Z0QWDDDDDDDd2¸27Q?'%%!44TČtcǎx3QO@@`ĉU}9܇ vkmР4hDDPN]l۱+V@`*شY ꚢTq3QO?΢Dlcmm#{綈WDDDDDDDXX }􁍍p1NQƍc"f  RyLEATHF%~gV.ܬ IDATNF{wp=gW/!9)QluU0#AKFIu Նr^~PTX*JĒ Kc0=XFFCݻW18P.z@0έҵsVVV-׊|J0a Q'yEDDDDDDDDDDDDDT]vk׮YZZ }٠OF-;z(rssUԩSr_􅁁;w$+<:ve+nET;ŵ n͏M&&&xs[g‘׵ISoر U>WDDDDDDDXXJDDDDDD1uiӦt$%%YČ4͛7Ldee!''RVVVpssCzдiSԮ][a{Y&q!ŧ/;ʕHlm*uw&!rcxt/H$c&ٿƲ21oL۸Cʛ ~w2Zk"fX6~p^~c{wcYYgbb"Tk̪r^z())ŋs]@DDDca5j0_e Kqa:t'OD^^:99N:ZNǏ_-7m6 KؠAʏH_V\08d(,[ʍ_p35[X#`ْ$7oNx{ČYsEe[BpXxzzF]\EN>bͪƊӹ!*V0tyEDDDDDDDWPP ca)QQRRP]ADzʬHKDH)_ ɊXl֭[*X1qD,]eeeb'OǏQV 9r&&&ӧO$"RUnn,,,+=΃|\0^۽ g_a13UaoVՉ }9ޛ9IpE}vԶE/GHq?n\~:`ǮѸI3+}ΨS`󱣿cG"g2kkH$ʍ :c'4fU:Hر۱Qp]*JO]عs'<==?իW]t+--+#;;ҥ +#:`蠞e;}zr6h_5{(.!mYkVA,VEоcgl޸;4Ƕ-!WyԔdeF 틐S'ӫQ|8}HJLPkMH$2=l\CDd5*  Ɨ| =Syܪr^X0R"dbbccc]SNH}BD&H`ff4H$X333?ҥKu^4iN>-7m6,X@'N6hРʦGD޻{wz&s?NKמhu=W>eeea}_ǃp())g015U:붡G6p@ff~:˗.FŵQ\\dfN-DGG{ `x {k93ߴ;FJr~wbna@,#m6ѨqSff(**DNNRwoE?y#yz[jq|N>زy-o݀6mۡg NΰGIi <~1q2"KȄذnܿ'O`tͼ[[hаammr'xpnF^GFF(8 o M!T|?XU:^w:iS5/m4[H$,Z%DΝ;#88Xש脁`C!"uE@@5w=E5kd KUVV 5bZz5J6lf͚%ӭ[pykN  KH\υ>y '&&O>\]˞2txJs܊fM7lfggTq >\ob=:'.r·0,SS\. vΝ[~:uL&C^^6Qu///]ADDDD|MT EL(\&;bbbtR 2Z!H`nnGGGx{{cXx1֯_/оӧem_{noȑˏHu cccY#Q{f-qet調G4>m]NZCص(ޤ+ZR33jTyEDDDDDDwNftmSRRhu""""""<.:t[wʘ [np 8PĬSzзo_ 8p@n,((Hp#Fh$7""UEGigm31JZNǕ⪁_u-dY +7FZ?//I_I"`GLX裁1DYf ww$aM-am]rT,,yEDDDDDD:)--k=E+~vS"9w`VZV~Q19rD0fffÇVČ̙3_v-m߻w۶muj4?""e}>qIĘ'baiAw&csOر}3E\Q{,>:lun7PvstԵ?{Fz{v]lŅa(++S{<>o FzNΚը ?qJ :Ji``zu?눎]Q~Jکsw' ۶n+{N422WlZoXXX5V~q:*>#=rPKK+wSU:^WƦZ]ELbXXXitBktL"""""">+44T0ֶm[3QիWcǏG5DFǝ;I/\v ͛7-""rbR"%kk1!Dž ax,nߊB܃X$##= |-kFIڭ{ㄩTc{{7Gff 14-mZ7D Ѹ7|Z]s}`VKh_sO ~Ǿ3 cߙ'8w69ш墨R,,-aii{{4h QhZ͵EXv+ ,ٳx> ׯ_Ń99(,(D"1,-,aemG@uP7n-v>J8 Sgj%+/"6.W)UW}/6" AAq%IDDDDDDRQQQ wMlcX˖-E̤jH$={6fΜ)7zj_ϻwFT%53C=ѥkO]Q7YשT5jbȰQ2lSQ]BntJZAKq.ԨwO^^^9rY)AXl5j`رrcرc#7>h X[[k3=""""""""""""W733)""""""wD1$$$G8|j:t _-b6U˼y tϝ;>0a6""""""""""""$7 ~'1_| ܹs 3f  $Rf֭lmm-œ9s-bVUCÆ [vm[iiME׌,,,DʄQǎ |||0k, <ނVPPzM4A:u [[[HJJBll,?T8vfͰfR)O3-[୷B߾} ݿ  Vz?\^ &͹ODDDDDDDDDDD, K9#GbÆ ;aXx10w\|"(޽{wFѣGammqO?ůX}ӱaÆ^cSSSHҗ+++CNNJKK>iѢx 9r}% &N(BVDDDDDDDDDDDDڡc9 ĔQ wϯƱcPfM]333l޼S3*_]-\PҞ={N:"dDDDDDDDDDDDD999|:{.vu <Fjj*;ܸqCff&T sssXXXprr+jԨZjN:􄇇څ~>1zh4^k[/ ?Rn!,XH?8 """d2?~Ǐ~~~NZSTXjll SSS!"MH ѢE]EE2D L&F닜HD/^DllӨ222cVJDDDDDDba)fΜ;wnbb  #""""""W^u DDDDp]P)*,1"""""" XXJe˖arcgFÆ EΈEbbS """z$%%:j+''łqѫtrss1|ZJnf͚DΊIQQQm(++A6DDDDՔK? :JRԭ`a)RKڢWKDEEaظq#=z$w D۷R aQ)UU ,,%"""""WM6!..f͂`ll,#//1/^=z-8c1"""""" XXJU^||zL pGF`an=[kK|xxvn.ss!N\ ^X0m˗QXT$!5ln!phxyzj,]uWXzWnGpc$"/?2r##aT#%%%q\w<Ν/fƘ7Dޱ{Er144Dm!0167I%O}TTT,W&$W IDAT R`O_ض *{=Cݹ;+KjF[\劖JJJ(oV +/F:u*.+ ΝoǎHHNVrj*j}y̝nތ_~MW+m((,WV aOr<Xߵ8VuǷS&!&&F0쌨 ID K9.)) IIIJcjjjE]QkԨ+++r#7uUnz^XJU1<=)}DT2RϵhXƍĒz9C+,@ӄ\7NK.\ !Cpd&Qk va7 =3Se`wMySR*,{(&̛ܼ<}QDT: =+WwNK1hsIє)XG000Byߟ އfff"eBDT9;=R"""""""JjިҨ;wPT\ c# &BM49'1r,DW ?gS齟w5`9s```1oqE]Ѳex&|P̞9Ց'̮]hf&$ȑAfgeaÒ%Ly|D"<t%yXTEDPTjff;;;888vvv-v]ޔBDDD7ADDDDDDD7j_.QwyF:J5BUիcc!_N 3.+W зsgL>^ܼ<ܼsAf)Sq,+!#+ ǍS2&͟zjwQد"ii7w.8sF1SRTֽ{EnnݩjIsoǎ=o͘ 8۫X*&OtQ3wB]77,6M#)OӰaC\xQ(*,J:+466N&ӧOGff&Ddff>Ygff&RSS5vË#??JîDDD+,,%"""""""$EQQUE"gBT5%b:uJ9brcC_do77N7n.wݸ8|w…!+(@Çq3CzFΝ.0Jk8_زo22_PX,}*Elxp*N EFТqc4o.ԪҼrrm700Y0?0&&/ŜЬaCտ?ߺ|krg?NLĒիg&,[\~]nL"1n0lHHNƙ -|k'Zsiٲ7\4olllD>CCCEc233O_,H}ViGծbT8;;ypuu+ӈUˈh9}K7R%%%r % K*--O{CrZ\Hnvpnnkn?#"܉wǍC:uu_}3fs&OƬq`'LIr8#!!x[7EW"'0$09yy`X߾8]4$wQ7-]oU^^ݵ =Ǝ[qn|2c]K#-rc8nv;x0F?%W;> W~]%_300a0l0qM?~\0޿xyyim~""}RY2yirrŽHIIы*((@\\ŮBQ_,ReWT"" K* 쐒^.& Teee8,7ɓ.,+w9liڴRcUg_W"#EHH \(>ckm?#_&{)VT\nڄ_~Y8A@N娥dG);] æOǡ'E._w!LBq)Lp{>jJžUдOr&ϨQUs.[(bbl?~I_} ƍQQo*=?:?Ǐ+R+1B*B* 6TW"մ4kQŮ111JƆ]Q)iOӦmDa7X`YC'Obi_QXXJT+W 8:a͎r{Piu`~` c؎݂J#Zg.\}ߏ?V!6.].|Ji>yCJ{w K2KJp$$DnW:d*óvm:\ϰ0 KDFdXg+,*}?Dtzط7b000TʪOz٣Gc1"Mݮv?}(YmY~R+(݉H K4@airZ$%c. .FDTz|RڵakmD́'͆ 8p?s33駘'Z#/?_ko5Ԁ|j^頔cַocXY}doqCbb))zj}y)*.ƈ3!7Iv|]r=kF3%|jX߾r KEG5:V9fM̙Z_vzir'$'=*dػc#.tCMq%2\lCI* i)\\\.D)bjBBǮ'== HJY\\SN!$$7oDRR !Jƍsڵ+LLL*5L&Ù3g#22˃9Pvm4o;vD˖-+شA.$&&"88aaaxҞm:88uԩڴi#*JJJݻ?FLj#@oXmd CXX.^Ǐy\"^zA>}кukrꉅDDDDDDDDЪY3Xxd$vָq CYҫ+*UBCq2, ᑑ!2Eoi 7WW4_]0GVj>yřq><7{YE3[kk4o-7Fh\zQzf&SRl*J'r<"\ nni۷ݺ̬%UyWǴ1cpX_ϝ)FA")#"ni%.u[Xz T%SD)}y_u7.K׮{gO,7O9$ F*_HԹs֘[X{p0JKKTA?(Jpww7|dEYYY:+*˗?>;S1/\/=*ݻwq]CnT'//[lݻ%oldgg#>>.\ƍ>~~~j=6mk˸~:>|xdeeA&ĉWz ^DQQQxBB۷ >ڵkɇPYYlقu֡P>O<޽{w^ :}V|?СC o(--ELL bbbg4m/i@=wwX[Z"+'\2EY)zթs琛 5az$bYESRƍؼg`Uzf&33q->r3/F.]hּτ]")OwE[}:ugڸmxox̞0֢$)I}VrG޾¢"tR0V ᅲ>wurO˗E9^2pTbeer4i5 郏L\~-Z5.^VZZ6'T TRWL$&&>|jGT/ʕX+f^Ĥt 2ϝy ?[ʔX *TJǀ$/G bep{D5EĠСرlt\`30f\=Z(üѸNޯ*p]ar\|z.3ު*gEHs891K$IjR%B)9V^6UQ+NRW(J䭊jee3+$*jrr2f͚7n(uޫW2J7naÆ ZZZīW+W<]6z437UVppp!RRR+>}ѲeKXTz ܺu gFBB $&&,oI>|8Zh͛زe ޿/m۶Ah/^đ#G$}&"'7CM5k9eyQDDG~P<4P&&%\ȕB\VV*IbTKx}WWX/_l(L^}(sl܀[B*Cy338:%ZӒѷK/%5XV&vZ ekk$՝:IK,Ԅm-BQ,UQD!.. GBBbccwʇ!,,,`jj MMMƢ^z իXlL-HOO… %ĉEdutt777ٳ˗/f͚5hݺ5L8c٘|\6u0q` 6M$sׯ[۶2̸,myGϞ]dRiڵѼQ#XZBOWQQx-߸ 6 II:|8Ξ:8:e&h\\kԀ t%&Ǐ @+x!uϏ96sL<<99>jԤU[7VKK$ 'xnfVŀǫQ{L\\񘲂j0h]^\QQ8vgŲuw-fKO.rD]X&,XmKk_`in:x9VjUvrp`|TR6:>[zz:bbb8mmmޖRrssp^JkQ_ߓDbb"8$x&%%!..N-ߛ)))HIIB76lP޽{___SÅ:u "-Z~BCCE*D&&&" @jk֭[?ss7jhh`ԨQEƮ\R,~ XV\ ?y*KKK1:u>ǹ>8 o333J$ŋ"K;;;rdFI}{w! {,''۶mc&P+ϟ2o޼'ܸqKL$!B!B‚YYg2͕nIĻk.m@(J\L1 g5X);9+?T8-- kw5kmFEaشi8y3Hn|dT5 Scc8ۣc>jcٴw/Zާ;Y[2Lzl do׬G>R0njѣ QiĥKqU١g1#Qg$C5T5[ָw/hմ~}\߷aVۺ?Y85]NZ:xg \;u #{n] rםǏi^A`2י͛}[pCHJIAzFtuanb**AZh?$LLd79^K9?N]&1/32\WD8GQex8Q KT޽:vBȏHWWvvv222'&''KTDUEUTYBCC1tP昶6Fƍç;EILLHtss466FqpnݺʔT &&&[.z Ľ~+V߂ AI >~(sss=}8Mz΋OQFgϞصkXNN.^^%| "  ) %B!B!(Quaሎ3jvv]&Gi8y2vk\uyUf[s'gÿO/20GG{7͜1GBs3WI˓kUdJ욵r%rtuE&5kиGefeaב#2|8u/##XQ*H+KKKԪQ߼[};vΙ */E5%>1>'3;U.m"cb9"9_FĹkט7&׉xKocM 0W/?cZ` ;q>\;tOnpbi67l޷9ֲqckըS2~GΟw2[.߹c^Mlj͛eJ ^[op/ajah^& ;|z4V؁GÆ^EȾ}"߼ɋaÂr͛+;Q\8>fo]H,4ר(ۖ,A#Y̛+Z<µzutiw?IXCUKLNfV *@ t|߿g&poB@q^JhѢEI/ball ccc^E||<O޽{2{b&M0j('k0:TXQ9jժ%{k}v) ݺuCnݔVbhDD疐 S SiiӦwqqHÇ_7Agdd$w K !B!Bq: kOJ,=r<eNGDsu$а byT~ NN^[/gB[ÓW$VQKo?z_FDFfX=پ]愥3_7p\ݿ?3OšrŊŞhtY+W26 #3ϙ;+6!~G۷q=^@ut\dT߆ AfT҆UVsG6l@e^G[7m __ؾ]jۋY@[K gx9?XҒY#ChhqJ,%lhӦ *V^JCʕKzPˣ $ZzѡCK.[*--M")[ʒ97HeamllT/x^رcHLWW&LPl7ߺu  ykOKĪT"1%Kq 򒺏pD2E*<{oդ 9Sb,SN'4t(RO\񏟟\ GΟgƇxy]U *U b;//OR220y3y2s|8qp^f&&jh[Zb;<} oƝG8ۧ@yPhv=kި|gP~KpΟ.;v B5}:WC*9|eqVF^ h G[,'qcccB[5PF^!رcUXJJLsB#::{URQ111x%v^JWx JJ ---ds9Wis%_HMM9::ݼħOǏEX ]==9S(crm '/]BVsȋwPIC---f< JǧOZ TZWC!j]Tu===V???:"Tڝe,/d_hٲH,''SL)2A``HlJ~_|sSDׯrIQɘ4iފ?6l*UT"k"U,%B!BQcD<7|Tw3O.pWOj镻w5* RNr$hs䛖}Q ;K Qx $1VB\md3f\[K ?)Ίۏ.__82.(bݺ9V vRYpUUfsm7561jT!^^x9z iQQcX0zllٿ9ӥKocҥВEZqfee)eLyEg"޺S\Ջi%B!ܜ!D6޽?,X'0upU,mި6^0&&-r{>깺2K.~k0+YYX= ޼ό;VpbarxDsDhij 8V6!#3S pPb˿9Vaƍ1#ݱC+K2{wiP9Fݻ#,<\b|ɓktX9W~~>~1;ƅ 1迊Yٸz.Nı --s^c7K6heW997ep$YÇPB1B!NNN"-q9tܙ<$*333fIÇÔݻwHHHikkW^.?~ݻ7Fccc^V&Oeb7.O>޽; mcǰl2LMMb %Hbn"G!333^s+KFFzccc+8PAa̮WEuB!B!?֨PqwĒ@ {N,ԫS'f (i-sX 8Z9Pb՚sEH*531E??8V\۷T\9w~N/o.2DvУ}{x*U8`jڵXns>܉j)e_eQ:Gc XYXorXfVv:kE60JMLpII7rN s)N_ɋZ qJ6G97:>K΄eߋtB!K"tRxyyyHMMihh(%0??sA NNN}BTT3~~~0a}' rRB!B!h٤ 3.^]=Ǐ3K,,Њ!!x8Y _`V_@j8U8 &FFEn\Jۇ*NkRǫB'zիml\>ݺ ,LM P#ڢ& c6!#ݸ39RַdF]9S[½^=쫬RE+sV>z{[" w Ɔ5v,/^Llu~~G]*q$&$'+<7uiŅ҉PB!ubSRR0tP\OLLD`` VZ;ӧO"㺺JYۊ+p|}}eyS# >Ϟ=͐!CqFXZZ2CBB(srr޽{zd1c˓JFɑkMʒqa۶m"Bc!B!B~@-5%⡟>!:."|8!uY_׮tD|#psvs%hX~ p%Y1+4+;ѴW/ںv66rSIʨGuկF{\x9ƕ(/[b&GeLmm۸$ODqTNVՠ=p}ϑn2??,`&կPՕ˛Ȇ 닥613u2k֌|hoܑ11c%V)>>氲*B!HŋcȐ!" }YYY󃟟tuuabb|Y| k͚5سgHL(bҥXkժԭ[+Wׯ_… 8}wt̜9#QMw₃`̘1bRUɓ4lSn::u +VCI/2&B!B)%b;߿ls>["ճC1*4;u 9dRu%ABEaܾIZׯP(^ Rv|b;)K2Ǵ45qpZo\}( qErscSRS~fv j89aD>2cɔ)ؽ;s,(8XjoeT3)p(:>K`NJ6M!L...ظq# 9ddd 22QQQZ+XaرC">g+4:e| ?Fƍ zzzԄ1|l۶ Fb7E")%77f‚ Dk֬#F^z7_t ^^^x䉌϶hXVӢd2:CM>}G֭[8z(VZJ<ӧO>|D_BBB!B!*PB8* DLKS}t)r_FҦD<:.]s%JJboxmGRJ 3U`Dze9f g4roQUG[&&*_ela}*+Pŋ7osLS(ākS4\*4oU9?b9qq ArV553g䚓/zzFd%"771Bsˋ+??_QF1B!h߾=m444`8VJױuVlٲE">yd/rϫ%b|ntss<#G //O궳fӧEb @@@ƌ;v˘9s&uNHHѣ ^kK]Ll^s+P(VVZ?SpB턄L:9 v}GQ&͕ #1^7ThI--  ;԰vmfrϙ jՒ{^yY%''"~mmm8BO!RZ^p<ѹsl2x1~xXGULaž}Dk?իWŋ%Gev^Gʕ%b1112+gIJ.QӧO OXJ!B!"ʡsʝ;mj%<{wfR49ZV:r* x6m{? jՂ@ `]H\EoxsQ: _pc~Tf 5)3+ ;)bָqr{9X]E$"Ņ "Uׯ`]<3ʌ+ݟXU”uKkwwfkY-ʝ;HZ T/((ZZZŸ"B!ۻwJt IDAT@2#̙3Gb={bʔ) 4(WD,!!A9X8Yt݁VZ*}OhܸH,&&8:"rܸFJ??IڄpRB!B!j߼93~ D!^^2VH:z^EH"%89ܜ~LL8+1Ӝ7k`(u L?\&&pxp%H~KPdddpWXfw!eڻw'd1*ΗU:::pttDR]+V %%E$֠Aptt`5kEbuŋXmCBBШQ#sھeཟD%X^z<?~fݺ)NYY69VbExw${v3~޻SI3yVVuѧ1.Tfegc̛X>~k0~) X礎'͇ա 0/Bʆ<`|Qϟxrssܾ}GΝ;zj̟?'Oڵkƍf͚R [nʕ+1cD{pss5Gvv6Vu_4^:ʋuy&yqUX}wCkF |Yş"4hCCC˗%رcquI,btpp(Ր҈*B!B!X6mgϞ:H sx=c˖ |L:[GocS..koߢH8i;kXB֩nN`xQbKd$N7Kc4t(sl)$ɫWhݿ? ]Õ={8+Ȳq-C||HrʨXJGJ,%B!BQ?r Vx<<`gcë Aoce IqX1s&W|LJj*ލk"3+_<=sQ+WIxK䜣cթ=+1ɋ_ѵM^I91b|lOL3iHRUD <=}VWr|KMEJj*}Çϟ[|O_LZ>W@v^KfVnph664W[jQg& UhV@\b"MMc?//sWƢ VV֒۵C*U؉KЬwolYnRŮ#G…rQ020:2_Y1DHe j, _f>0 x^VsR.<<?BzqEBȏ+''qqqFtt L(})[VV^ 6ӧOXvLIx1Fܰ0?1o+WpwuE ace]DDG۷p&s++l_L5#}}߼n?Ld ڸYJk-xWCa=G`Xթ7ZZBBRDFxRRS&J҃gϘ t~'^N*+Aj\\9t7GJyըZ +-MM?m||mWnߎS/cx>hް!ll 60A lYbY444zFN>]cҦ 껺#>1#"p]6.\p4^/dnKTKX3w.2)gfea#Mpan_"#9w0UsoߔPO__$Ayզ\37c ⧉Kiդ Myۏ1uR]8y2:1.$ri憥S%K8{^&Zl";l?ׯg" f䢯kJ-n'M\\޿9.Z)!?( !QFAuADn)))C||</66 "8İLGGB%NIJeˤ9s-@GI$v9sFy 5jF%h4xmt]}˜1cpU|{,55WFiiix5_.b" ,,,0m4,X@$kG! tK6nMu Uҟ`inΙURdBOW)RcgcέZOOZMD^oI3M'L?-ofwrchPFΚ?=Oعld]}ܹZf[R%Z]]eVIݎO_dd$޽{'1 6,B!IIIADDC F OJJ*%+LMMaeeˋ$N-H 511n7={I& u)-lll.(5zI1w\<|vXh\e~3n8TR˗/GBBL >|8F!|2^[[[āNKKKrVRUdQQQB4443fT9ȏK !B!BIN8KV:-1ּQ#'n 8^:`y0}m[7 pѷ/£$╋PR,ͱ| 6Mb,#3ÙۥΡy'bx>X};E' 8;8cGtl![[MUuutqB7yyy"cB.) ]mm QnΨ+|uG{ BfͰpZ?t^iki[7,6 J.\}zN|,V tm ˼UE1._UѣQN__*9I :nggj!X$%%!&&JeP(DPd E%SHmmmQ\\\Pn]h)ZNNRC `ܹ?~HaXl됕-mۆsx)gEY  zΝѪU+EVFvпXYY9^ *`Ϟ=8x ك_2gbb={bȐ!0005k˗q-ܼy=߳ѬY3xv!E̾4B~x1s,E:alBwB!"1!/coӠAjH~ׯH8//&32pU\{O^0$ '7hm gGG4o]==agcSK.3rrsq=07, 11HMKCNn.uuah`cCC8j@FPB^>Kocux+N?T+#G|$˸zy_ ۷6VVpqtD~Bv`|~., HKOG~~>L`jlfff YÆJY_? o"6!YY0G P 5/mˆƢÇ%qɶƍW&Dy(22RъRTTڵkg;::Zkaaan@K?ټy.hQG|XVY,R``&u9r䈶l٢ edd~ffpp:v쨔IKHOOÇ뽦k׮JLLQG9Ѷmd6^ǎnTTF%&&gϞ2^p X 4{*33ktW`0+МTUUCfY[nUyyuu릔L&L&z[`)Є;+VѣG+22核"MOOW^^jjj4!!AڵR`)jjjl2{]v4vXu|eJOOWFF233~k*11Q&IIII2d¼-`)eVU˖-Ç.$$D]w|h VU9992_m۶I/腎R֮]]y啊AG-٬4egg㺝;wÕDٓ )^۷#FW^>xRJKKlVnn=۫W/4RɤX/t g^vZeggj>4ɓ'qFedd(33Sf@@ D%%%d2C^!X x  vYcƌi:͛7l6l6k޽Z H֭[{cR6Mk֬ymttz2b?8m_PPqp%%%ä PV1W͛5b\=X,Ztkt뭷0ǬVrrr m߾]׍҈#d2d2ԿF/t ,4)ͦ,^ZK.Uvv/^L^JkN 4iƍD \YY-[Cڰ0] AglEȐlֶmTVVqݘ :TIIIJNNVll!X JmذAZ|v-I g}B3~x}۔.I۷M0A]vafx|rUTT6m[nQTT:)))w}',٬`P\\L&RRR;zc7׊+|r[NC&MS|ZpLjܹS;w/֭[/ׄ 4a TXXedd(==]6׶mVr"""|F}m4Z@QQ6mڤ eff*77;JHHk?x`oKp~!X h3j׮]u^KrM79KO+++ҥKtRIR~!S_vڥ˗+55~TWWkݺuqpx㍄Jx]mm7 g84{t H4dy[Kn;TRW )//OyyyiW\q&L[KYY֭[gQt饗j>;^>??_WVYY[wQ7pfiǎcf <a&''_~ $6+ SYY233i&} nV_+ln&}WUYYYeeeꫯW_Iڵk>}O>ׯBCC}2pT 6aߗҚk֭ln믿"88XAAAudKкuk&TSSm۶l6+##C:~u5|p(11Q={hB\l6~ai&ܹSE6~x}nKtqmܸQ7nT@@z>}'.Nbbcbhٲe~:UfiΝڸqݾoРA3f ^QFi͚5n/ F6EUUUl֭[:\bccl?>66 2,L[f*Mڶ-K'NxnEE*j*??_(K}zRjO6MԎ;cϲZoZr"""{j*,,t@;VM+P׮]uQsѣۀN:e?>==]a 㕘$%&&cǎ^,fil-Y2ۤݻ?hiNiwiwi…IW\qJ$@P SN^~AS%)..Nolٲ&*))i=4iMO>ӧڴi=L4x`L&hjӦ:g#X -@q)[V.uTp[{5JTzf$G׮*%E㒓uQ l6ehUڑfkTݻ;TqqqXM0Aaaanؽ{233L߿㚡JLLTJJL&ߵR@֬JUJ۰Nen t^Po/\6aa|HM=ZFVENiڵZvoެ'<ٽ{w͘1C^{BCCuWwnZɱO#5*((nddF)dF/t `)\ ,UUڸq֬JժkbիՒqq0f&I!~{vkZaٲEUjwUw}w SW\qZnVhRΖlVZZUZZq.]hذaJJJRbb7, DyEN,ҩS'URҼZS%%:Y\STQYI\0jkkUtWuMW7ieZȑ#5h @(--Ֆ-[HsssU^^QM84RɤX/u fr eee+p}jՄ]3l6ؾMkVК+ej/*91QGQ4_?Oo uY᧟/̃6rݴI֮U7ԵkWs=jժǵ+88XCՐ!Cx믿.VkpFoI~DQQ6mڤ eff*77W Ѐ,ɤA颋.Ry}gE6篵GWp2 8hf=ʊ mْo֭Kkg[.8qJNLTfv\+@*7߬?چ-Ylzܹ*EѺoiIW\IW\!I?p@viϏ?6*rA~ (>>^ h ?}"izzq`%%%d2)))I jݺR@1,-de)+ˬ ώZ֭5~h{neJFc_ֹޝ;W㒓֝?`/ZxddX{^SvNTK.O?퓠J@@ԯ_?&I1@Sl4fx\7<<\III0:Yzu(Jyy9ޖmYڻ4[bb'.N}zҋ?TR83gCT%LwkѻSgo׸SuabNYKzmoppS|~_ھ}rrrgϞsN3׫UV)*blVU999JOOWFF{;vQQQ1bL&L&/腎@sEZ |T]*>yXYYYʒlVyy㦙F͛7Ov;j:rUBCCշo_b[gBCB&}ך.4~(w4U]紗yy?7ɳ۵k1ch̘1*))Q׮]b }:~:effզM& EUUf222d6uF/L]ve]fz[p>!X pйsgM8Q'NTmmyfڵK۶msNmmM&@~јP`PnԻwoǫG7eQ\Zy?}!  GNUUN.Zi7ިQC6i:uMZeddhJMMUVV=X*I~¥"MOOW^^jjjh~^N{6M3fV c45|p >\> rJڧ~8q.\N8͛7+--MfY>@%$$ا&$$]v^\(֯_?OfRYY֭[g综b[ng}F>dM2Eu^g3FaaaիWMlٶMo.XrMX} w4O6MW-[vݫxC<::t蠩SjԩNLZ-[p)p)##Cڿ5CBB!Ҥ$ 2 ,4J֭uk$=zT+WԲe_ԩSk-nF͛7O̟?_ӧOw+R^{jtbs9P>3rq1 /oNWL>^z/fiǎ2_m۶Xd_F `)+:viӦiڴi땚TٳG>}Ù\&xbu]Pi޽5aM0AFRV!]YY.}N+WQY?xY7V:Cn|b޽ڷo,[񙐐K]vw+Vvvf222m68q㺝;wÕDٓ )R׵j蓤XB˖-ꫯ2^5|͚5KSLѤI4zhЂl6=չ9 +x!}t w{Ownlk֬њ5k݆_^Z7xL&[iJeff*--MfY*//n^GۛL&z[\(l6*++ Zshr:tO3ԦMTUU @eeu릃{ Itrmuwȑ㎀Gp~Lz9=+CgrYVmذmfӆ }ɓڸq222\*؏7L -T\\_]%%%nsκ{B,ThhƎ6p ՘1cp^KٸQ%ng ##uqve?{wa'imoB~zZ >dXT]]6~r!m޼YfYfYI4x`L&hjӦ:ƅJ4ÇC Ah_ l:^T/Vl.7/6o֎ݻ]L=l۵>~E9uqn< ٷOrsUYUV|h4sJ11aZ;ԓ.&ZV`^=~ݻw+33>t UbbRRRd2Կy[D6C),,TNN$?kpa#X `݂_YVOK/ }A{:[mjz`unrrڥMJN_~['Ã*U+< pAFUfj`ZcFBFEEiĈ2L ) ) 8P۷w.vKr,Z0k _/T~Y_Yr/{w]uenZvm JLYf.vd-6T*I;vұ"VD6N҇_~Ns`Fyb(##C2ڶm<C*))IɊBM`)Gޝڠ䩒ou^*Z!nA**+݂ߕWTH^J]' J?#X 8oᄈO$QEEG5 d2DuKRywMb/5'm|tuѨo!+mu۾kvޭ޽ STTM6)##Cͭ(99Y&Iqp X 8e|˽FER@@u#oezh;Vm&|z8yIq׭?gKIӵ~kd2d2)))I jݺh~dٴg>e)wٷO֑ctD%%TWFAAjn߻F&&jhӦI\|y{7]}W12*_Gûkk 6X:v6Ćk`W_]gtzfӎ;ڛfx\7"""5LWhh:h Wzf6oݪﲳRXTeɓڹw_/I քѣuϯ~.Il AFng-ALǎܯrQޞ=-UMMm&٬ }:vu4b(11Q={hBCzfŢ/VVeII^}.x˽~ԹC= h3eTX`)IYV:tHe6uV{\[nJIIon. K>QSSO=V^8Nی:&q}8v.fwh)jjjBhʕՋСC$7TUUHRXX[|`)^P}R}W.ȣZdT:xBUe8m޶M6M.d:ujjj|oPPr( IDATlFv5A@QTT$*IjݺBC555Zv֭[\b(44T:uR~4j(3F=R|ҔB+,,L֭ d%$$x5ѣGfk߾}:q=ٮ];%&&.dOn:OԹsghի_; ="X mJEiѢE馛c5#Gtwȑ#۶mSG8_,nd8P}.X]}tZj`)=vL?Vnؠlzkݿ_K~[npOG .Ĩ]ddkp2X*&X h ]t`iddFiHڿF7~C>p#G 7ܠ9s( O o׮]{4c ͘1A6ĉϴ|zǏw;XGy)X]vi̙9sf̘KV~믿v%K(//OoڶmۄݹH3fhg.K,U:6$IW]7nPHpׯwyD 7[}z0<_|}eQ^b||p'kdvh):t #*%%%)11Q={$H 4ӓ_{5}饗4|{AmڴQII/BO3d^;;j7TTT&ߺM1Kqq~m}uN<: ~Ϟ=o22u[}wgykP<=׿@E***t}i߾}~pͦSukÝ/:wW~Z~nؽk?XD^s&6hý׹7OPչ_`< &cǎzbkڰaW[etذa뮻4dJ?֮]p_|A馛n9?P!C4vX)<<\%%%ղe\N_1c؃T驧RQQ[ JMMUUU}z?&OAޫ%K振Y9Y1M]_} k޼yNwyf̘ܹ) {-nppɠ[lQllG}ߛk9s+RSS#(22ҫ=zno^oOz`)g 7߬wz }{3Ņ`)]]9%#K^WTTdNO?䙧Cgku;5Nnth*{ѝwr/88X3fаa_ڣRXV?a*bsI8aWW{(''a=%%E=bcc#''i祗^ڠÇwJ:g\#~)?GLVUsq*IO>Bt V :TQOgKKKsZݾܸq5},<111;w9'"X }hĐ!ڴuu7,XT[[r^=}^UT@K`ist!]z+ڦMF_[n\>|fΜfaaZ׮]TcNk;wltO9;p){)cpK:Mt 7x9v}+**rZ3=Vݾj۶Cl6f5͛7k…/m۶ML<K>7j:V:JbZ)ƪ뇈-I8L ?&?^ҤI>][10a.]h4:pv8[>/>l6kz4M0w*`X6a篫u1 իC@Lheqq̙0Syh|nD=8|X%ee oݺ啕ur1gT#N:)((HW]u_;555QYY$TX,bl6{fXTQQ!ߗ睃W5)AAA $EDD`0CA~9<88X $nZ2jӦMuu=Rjժ>o.ꩤA5Fڵk*,,TnnZNԽ{w?4_| N۲e6ydOxھrJ͘1íen߾_AAz:qqqNOd?&I~{< -R ۷i9^Щ'= G+::Z mgVVV̵FW{ ];3 yug^?FQ'OҡC={4N&7y9Lk˗;]M6mh̘1Zj}F>y{ڕZ[lqX6mF;xZǎ\SNNk?sz:4ߎs=w^MI`)##?z9!uY,FnU|^$44~y۶m/Nuו;LD\e`plnڴIVA=,X,VPPWܻw.\SzYV}k]vUϞ=t*--UhhuEiJIIiDܙ3gjvڵKӧO /8{2224{l^zy ;֮]quOaaazOEE?9oTpRυn-ѨZe??Rt';Gat z2)u~M5`iDDpC'])߿*=N[QQQM\_ڻwւu뭷~/B/uϣ={j֬Y;w={4ex㍚:u?\suaѨ(K^ v)s+n믿#G:t5MR  өnC[5L kw_#4'Xz=הئ˛DŽ{-ܢg}a^PV6l3gvޯ_?qo. 555Z`/_^{ͭ$vmԫ0F-ҢEԻwoEEEi˖-vUSo*w!!! W=g}3,E}!5W/̜-4$DZ q l3頫$]eaSͦbIQ-jkkUZZ*I*++SMMaӓX\;]t٥fĉo>ZIINM2E]wz)`?yvڥ7jŊ:zCsOKZrZDD^|fmHSTT[?5h >}9s樠iPK"""<ٕ3֘0jMMMzܹs3q޽W`),ղuZ9%uV*9kj$UY,XF@{~p8uuǎ裏|̆ ?ӧ;,͛y)44Tl*))9Nj.^}U}kFQ/vqc4p@(!!A=zPDDT\\;wjʕZlCBO>-ZVn~-p!Du Pnݴo>/}~mYfot欢iO>n;d=35kŋu]w;sJMMuX:uz!FIɓ'_kN碢"s=zwV sZkLi,j7_|o߾꫽R8 9w׹ݪsI=5/+*REe˽n;+?/ҥK͝;W_|~a 6LݻwWxxFڴiK.DW^y|I/wӻwF=zם^M6Q5&k/\sX;rvY= .t Μ9S<ó###5yd}={S0RfRyygiii94oL,?O{Gڀ;X ۶չ7͉)tq׮u,iR@]z=zo?絛7ovZKHHh3.\^zio;:ĪtW^<;vNמ8qBZrrfΜYgr-4hf̘"^AA>s=n[mۺ칡?٨δl2ݻFq]]K>U]SkԹl2]gu7@صν&:vTvکW^ s:WQpKϞX*Ity4aGq9{?"]df̘ѣZaaa;vi{=߫pѣGu׻uSξ>J.ZaNԹ?.9Z .^:-+'A}[u q1>H6amܸq 9s8չ裏zuNkEEE.uqDtu3 0@\r1b -[ w]ԩBCCUYYi_\= y?o>O:20z`)%4$D{fL,ѵ:wgw?REiӨ^#} Zr-ZisƉ'裏a}ܸq={=/:uv6@fGÇsN;vCT``P=:tHeee.Cu}i)FQݺu?|%K$zۤI2 RSm6ha ZږrMX5nI0A ^z%udҀܮ1{l8qamze01wСCݮh5WGʷovש˥^,,]ven_PPamnpslU`Cgڴu3}ѨnuG^z`$m"X mե&չ_;nه>;wްQvMj͚5Zx}v+Vhƍk111zWq瓾}:mݺUݮqvSzVSSsNsaaa SѣG룏>rX[fҵkf9;+m۶[z뭷V-g1gTW/}E$w|e&77ܠq\\r=p/_"ڴw#չWXk׮O<~7jȐ!nը//J޽{뢋.rXoU\\vu.6Ht5AN>2g0L wX[fӔ\Wl6>suoKcb)k6~5wh6oݪ!&&NCӧkdqyNS<6<\/^)DP=kjţN53<-.zt*Mt5j֬YV`0hn@X,Znx ۷i]vzOn۶N{uegUW]$knIw8b_FUoමX{J_mXO7P9~f^ÇևZ~ w߁QǟIJR0H-&`D@E.\A`Ae v`-"UBNzH(\L䜔篛7=亄'Yf)))_ y9|ƍa}.UPNb)PY-#Ɉ9vI}7orӃյb$]-5~KM`Hq__kΝn۫(y 'E]wO.} ӧ͎b0vmfFɊÇ~m۶M+WT_J G},… [kѢ|w3ƮW_}U F Ӏ4mڴk)))zoM69z{ :uSNNտM>ݮyf 4H/"c4i$EGGg 2rv'jȑ5jzaT_V\\>M4QF'73P,bXlg"55|0-[X,  5;F}^>7VP`N;a%R#Slszz-3΢(Wիrlƌ0ai/,l}nJOO[?ϟ}n6tP :4s2<"^4f; ?b 7֮^zJժUӝwީ+>>^wʕ+3g>>>;v]Vzk٭޽["""԰aC-[VE.\жm۴uV_#}Uݺu+aXKHH+_ԁ秗_~= @`,<`kH˖ZoJI)Zxޝ:)BI6M?FVvSf%&%sjdybJJNvKK^yՄ~_'rtma`XZVmp;.frr1bZj{:u*GDbbbffՔ)S4x {j޽._~ޫO>:q>3tܹS;-Z顇ٳgfϟתU^{O.QоN@fXԷsg(&FM^[TL .ԬYD@O Wխ[7mVSNՏ?鉲˗Wǎ5p@eރ *h1cONZT)W?}ER@ծVMKZeV_[:.\$ժŋ+BUW ձE ը\9s RIzӔ)Yޫ^DED+nqޥKTs:3g%KfUbbbܲWvqF앙#FhĈK4is_Ubb4hS,#]tQ.]tEرCϟeXTLUVM5jS-+CjС:zۧSN)!!AVU*]"""TBlf߿ۧX;wNiii*]Tڵk˖_˯]<AJ ڱw٣{(p/ookT65;)<4T4:o2mUݚ5MH:m>*RRS5OΟ44@PB9VT)h{),,=ZO-_'(:(o"ycolH&%`ժ 1cߘc^1HȆŋ@!gZkv b)RRSU~}]zƚ.m&_,1q=?aOkT^=ש^]/Xg.X)}ZJׄV>:!CT\,{]bR^ūWkڱw]^cZ5O}q$7ޘ1C Æjf)_`F^RnAWgg(V(WNKNP*>/YZ]~X;-[2dG:u̎($}Zf2ݲ%Kצivk;Pu}2+.YZ-7Kқ]Kᆱ{WbRRɉj躛_BmV_RdxW=.uĉqT,ްA1pZZZnެNZez>P;t_ׯWGq{o//ՋP@]MHоÇu x:T B%ӌlj*S#'Nh׾}rE j7[ixB)7|}|4OuG9~Ύ<<b)AT 37lެLڴ)k >nְsڳO?TןyF+??ٖ; Zf/=lXoֱeKCߞ}o-6V}T 7,kr1~>wn^^s=vڦI>\sLɋ#ϟ.5;FaK ԱE l6%WfvkYjjV[[5k:|it-%na.qƎڗf)ʕ,W8uJx@:e/jkP>Z7zg\ Fڹo?z L/==]MS55y4#b)J, ڭټYIINY|ˉ<]x|V_oֽm[_atzGԻcGDI68}Zԡc_==xI+' Ijl%ӱ!;yR zqڼcE`=ڵ81)I6ntO[=t~[J|۷w/j=vkUSp;ӷs k-3gv@;|n}a3|9  {eov09OMJmھa7)9Y>"b);CO;vYÆj״J,ycmCNN}}|ԾY3tFu8Cdu19+N;jCv#zHF4%9kڜ9g^ܪ ;jm!!gӳy%%%EVʕ+͎|GJJQ4K VUreWp+Vؕ=;j%ͦͳte>ܪ|}|fX  ūU(b9+~]^{[)y /]'_}|ϫN*`<=Ϳ-oo/fT ޽[&LPw}W͜ ЬY3}'W4aŚ R`&өDZk#^vwN-[| k;uyߛp!Z%*??s7ˢ+YfyuiNZ韃REFr8KOOא1c|O%}ԠAEDDhԨQZtڴioVfD>eW_)22RK.ըQTV-իWOGVtt4((o.YbqRr^} իYSԵMA]NWiR<=գ}\gϩO^5m!!*k|?#S2;GÙ_bb Hñ'|Pm4q8}}U'Oԗ_~{We˖UV[oi֭77H.]4g89KfR=$cmۦǫu TTT>s8q~KLРV-~cEF1Br^k״x*n)F6oҀҥ3]|9?ciiirn-L[MTʕ3^xm 7TZZ+*`﫯fs88eN89JLLԼy4dU\Y+V5sL]x1{sۄ(4g 0 ҥK5k}Qr2d͛D{Q,0I>^yTw{wh-wS1¥KZq[w@g޺ߪN6{$mھ=Zݚ5s.YR R=##35~9ӄTszeyzxS*pбc'*RRRzjjР]JH׮]?pR)rK3g̴\zLy~=zi=(5TjUo~I>&Foznv2}?R*Eyyy~u]zzbbb4~xnZС&L?P|w,ԪUӦeN9X6jMRRt:pI {x$ӐULsrj\vM;vw}*Uf͚5j.]{}iƌtcZe6M_u˥7KHHҥK5j(EDDroc]v͍i wxK'MRZZ$)--M_ΚeW,-﯎-[:z /k7=f\@i}Td,]o/OO͚Xo$b4Otg:w]4WNw=J1 IDAT>|cǏ׽ݺul_ׯבǕrS9TD*n]@k1Çp8e.e}>cƎb&$+XRRRwvj:yze˖iٲeyzRRRt1;vLk֬Q@`U^UjTU͎X `G+={~`^7^5ktkOVÞ=3hP߾*Q5[wҨxժ :bv[2%<4T&OV;3ռaC rxݠ#oJ%J|5J5RpPP^dž~Qmbv Se.9=ˑ\Ju_VҮ}\ᡰճ}LK~~ث&Oav%M;WgE4wR-]F61ΟS[֬_vZ[HUYx 9&@>w.ìbPu衑9ItW3䭤dTfGA7fӠ>}OLl"HPBoGD^D={c^wG+0v8 |>2 kwQ*ءGŔoQ׻>֞fGPH͖a{;x]P`-Ǚnwܑ|}|OjGeP;4OCBt[HHV4i`g[RŠ,ukq('"I.tW|/55U7mҋC={*S'8QK׬Q|B")!!Q+uiZմ~7$YQ)y9avH*)Mc{oҥu…kw<=s^D7OSNk6mR/W R5jP/l2zw 9n}܇j@Ϟ=4TdXqcJÆNfk^>2k&)9QW2;FU7j͊ٵKӉӧu5!AI)bTD PH=,LUNꊨZ/ t{ ޝ:ɱcf.XWG08qN=EZr^ . =f~٥P%K}й:viʙ@A 9Sg7mrcgX4gO S.]Һ8}ZgΟW1__/[VUfMY,#^{u͛k%i2'$ej֠AayZxN-(۲sךx.]uiiJvMWu9=t(k|UfM55__4pxqVZbQ֭s͞zutv端t5>>Gn͚ak~߯Yeb\.\qqJHL4;lHLHŋqx1N.Q,eKeKtKq0'|,v~=Z=>&FcbԩVժg=:tY5U\ C/G\۵MRIqɒB[,QjTV|BV[+Wjʕ:pPuF:VvMUx1#((4GKOO56nt:ֶmn m>luUmo|>wNZ/גիu1.@_";TվcK/kv$R@'Ђ_5;Kklq:kٸI;֕3(==]ńd ԃ{޽X[L?/_-;w@ 5Ţz jmdsu^Mp+"j|:䲴_ذNMF m ?/^TJ6M kVڵ:sEGJHH0;"P`sڣ";R@#((4,Jk6~*R$I׮)>1Q^?NҾÇeڿm'6:{Vs8|# VڵKEXzr7N3} qvjGl8qL,BB**vMժ z{eZ͎X ETRros)lYU=jT;v4W4]-@\}7m߮G4jUhpB%2R\͛u!:tHIIIn_Æ oۭ^zI7nt Wxxl$~nb)Qt1$u꧑v8Wo ?T J܁GTn`n͚Ng(JK.:9ڳgΟ?k4͛7wܹsȀ$)11QzrKbA RZ|DTKBy "gĉ.ˬ۬YT*I]K+h X{VBCCUVӚ1cx)S&.YD;wիWݘEYBBzŋxҥKЌ3t)mڴI]vUhh(pb)HK:iї_*<4T: 3Bm鰔|qQ˥e***JQQQJJJիh"-\P;w^Ѻ{?;(HNN}ݗRi͚5եKuYZk@@PL>_WRiKcXRâﵔ;yd[۷W5i$;wN+VҥKO?ԩS`uQϗQ$$$rL2֭z6mڨlٲy܏_RR4c|׀= L5(Q,5VXŊNg'^``'ȑ#Z|{9թSuׯ J"99Y˴TZV-9RK.ɓ'5uTEEEQ*P`qb)Q>J?pްA._v:CƅɆ+׽<=U4E[)+WhR۶mնm[M8Q.\Eiҥ{ Իwo͙3G#Ku=h޼yf%JPdd:wΝ;+,,̄w(9)UJ};w60M$%'+554p9``,$$D={1]vMk׮Q4ݶm$i…ݻ~Jԩ.]sjѢ<==MLyb)NgMחGU'JRz`OOOnZ[uy-_\K.?;j3;*򡄄K;w?HkNfG俟Gbw:kZI.>1̗Gz.]W0O@@oڼyZnmv4C6mҋ/͛# T8sn͚ɺeEoq9w^!P˖-͎|fSNK&뼽Β LIJr9w^KEBf)/UAIxbNg II&{PPP, I0k,>!$\?{PPP, .OooY䈼ˉ0X (|}|\/_bPsvfRr NSŹ:RPx`d7mjv Iҗ'Gyzeb(==KQ1(H{t8)U 38QuIAA&KJOO͎!Iҥu?kWRId 4jUT׮` Ĥ$o\ ɓW(?rD6nt8  ՝u o_;P񑏷1LP {t CnٯyÆ P-{jNgϘ',,KbL{PPQ,Pl6ݱcJbzu`v/<4TZKNUVd#}կ[77Z=cIΈ*UԾY3vi}LL)XN ??+5Ijlbz{"<Tt9N,uټ>_abБw劮]%JLR*_ԫwީVw)??C2s -[?OSdGT׮ԩ|=/// +23g3g%KE<SQ,Ph/͎B"8(H48*(toNᡡ:tXټe˴-6VuktGXŊs:yRSS*jcX(o//KIMՄO>q:r wG(PXV=?dpɚc(c;1m9~gdjWfp"cXopT*9^H[v48Qᑔ7s-^ib)^1,==]_yE*&Nǎ9׽QDơX @5gOd_u=D'41ct ,4rxtw0R GSV.38Q.&F68-SFǎ58y(PX,M{٭(^\ןyƤd_QݺjwU[>R eߗ&Ij{]ھpd_ڵe<=裲Z Sǖ-MN`,iѨ&%ժ$bvKo^| :bّ G1C3;hpTn 5; )?0pEdiNΞ?)~4 #P,oؾ=o n7$-7d^6~:yrwbȑjߧJ-Sȍrs͋kn)6UFIDAT}KR* T, ""n)[UR;7^6߼9RTo&F=="/J6n/Ιk֭UAAߩS).kнʶmE˖¥Kݍc֭QV^-ZD8c@ѪN떬_/ΙKJbMѮMܡC 80zvޠ>mK;|0.XPؖ-ZCS?> jY]g]vǽ?İ8w>w=`<3{vl߱1tF}[ѥֽ~h[yyzxٳ c׮]5͍F_pCvSg̈;?^~jݳg\4zt;hѼy.۶-㡙3ckYY=|xL81zYY屭46=={@C,hD z+z%…_m[Vcnju4h/tTY=GqFB4%Ķ:ͫ!T3 c*TQ+׬/}v<5ۼukC%+V#O>Y= |?3־NFwϞqGDHRy˖Xzu1o∨{e~R*OϞ'~Uyf1ؿSZV-Veqǟ~ɩSJRѷ8Gvb׮]QZVWyŦ-[b!T9w_*圜ܐ!qX~vdxbi㶖Ƙ/7<:wP֮U+׬'.R|/14.̸kQQq#FT;ouIIL936N3iR>zΆ qWdJ۴n?8ӣMin?lL㎸z:qGo-ZT"̟+ҕ+W^1/8G93x".xwݵw6l+~M7UC-,}k|TOqq즢sǎѡmۤ K~~4RF_Q'?1T+V[U_KJj] ={V9gȀ1k?iR8eJڳn9Ԍ1dHlq#RT)ՆJ?g~,^,0kڎ;w'Nlv4xGj#[v{Ǐ"Ow׶Sg̈^X^;C3gFiYY5lX|?iZht|?h I7r3 j:{;vվ}1Cjƭ'Ffj>=6m =ɔiӢ"vJj5ѣe˴3g8Qb=_?~|5޴ L=n]v%M{ٌڸsϭ݊NHmO'ꭶg8k8k=UAA ?д_Azl'X N>ح͜_Xn]B]{wƘ`AZ>}Oqq:- /ԻZ]R/N%TtIy;Ć?QoitM]%_zjԴÏ?z8⢳Ί!$a?ϸAuZysϯZu`Ҍڬ矏a_JYjUFMCv > Kɍ&Ē+ѧNmw>@1o߸3SOm6jlؐQ+>zu`׮T}w[m3YAn}Od=!4ϏG+~tѲE*] .яGƷ.^{2jmۯ^kF֭jX=lݺl!X ЈRT\;n\,~꩸ꢋc5V/-+/zw\8eJ+?3ܹs?6 ]}/&^~yL93&{ok㶕Ibҥ1jO۷Ϩi˖ZV]o6vwW}KԢy8oԨxiƌq 'T9i׿]rrrNUKvZfM U:Tq[+W}`_!X %>{aߦOze@~}iqX~U92jgUkƣO?&;VSN[N[fyu޼y1;22{۔믏fi%%q׾No,ZTK/~8^}hٲ?o)-o~^@n)xW//y{1oѥ0 Z[ƲߎΆ ksiѲrǝ&vܹ$:S\}{ظiS,Z,VYc*1yy#뽿ys\:qb4)w[bѲeGƎӟUCԿ8aB|gĴzĭSƭSFvѮMh[JKߏ7,:(۶?5 QRZz35A_y_IWɩyƤۨ]Gvq@늊k==bUWU>}g:5¥KcҥU{W7bD<2eJ|mܸZZl[yyXEN>x["77\zѼYzw/׸FEEE_$ PjԾ(ڵt{E*7t u<.$[}&ҺGߦO6[8k3Ox䮻:qN^*rKwql~^^|#7[vܠc}7~x䥪kQQ\r9_6~8nĈ?ÇZh_:꨸?SjĔ믏͛կ;6{ƖtM/6Zo,Z֮6m>Tnnn*:k9ѲE:cN9%ƜrJX:fϙl7mfѾm۫W =Z5{xqΜX]R}7 Z;w>1_ɩsڵdIINNNqqƉ'xu޼X|y߰!ˣyf_VQرc?]\\jD<2ޜ5+󟣴,><>7d^,7 4ݺEn>vvڠko6|1 fckUPG G `k-.=[tdR"BJD`)KR* T, "K$X @DPI,`)!X @%R"BJD`)KR* T, "K$X @DP)/j'Tn*6[O:syd K~Ú[#Gvqyܤ_rs68<};'BZ4om/6vs#8m4R k92ڵitˋ1yyIpydl<ڵkWMdɓ'GEEEz4cܸ~5rrr+hZKJnuh.k*628<[OUP1nܸq %J~^^-6<.&A,`)!X @%R"BJD`)KR* T, "K$X @DP)/>n>=KR* TKlłdh"#'''6pc)@[hr n$X  J&gIT,=zСCn%///bРAѫW%@[wIENDB`distributed-1.20.2/docs/source/images/network.svg000066400000000000000000001106441321233345200220310ustar00rootroot00000000000000 image/svg+xml = f( , ) = g( ) Worker: 192.168.0.1 = p( , ) = q( ) Worker: 192.168.0.2 192.168.0.1 192.168.0.2 192.168.0.3 192.168.0.4 Center: 192.168.0.0 work: store: work: store: scheduler clientlibrary distributed-1.20.2/docs/source/images/scheduler-detailed.png000066400000000000000000007104621321233345200240600ustar00rootroot00000000000000PNG  IHDR m}TbKGD IDATxwx? a! {d(Q=:Z[mm}{O CL#̀A"w/!~u}wBeee(]~}~~~YYYKIIi֬Ynڴi *++W^vں^ NիG۶mիW'[77ׯOܹsg]ovؑ\^^_UT2 [bsyyybmԳ,@$1kgg)aYPz@ Գ,@gB= YPz@ Գ,@gB= YPz@ Գ,@gB= YPz@ Գ,@gB= YPz@ Գ,@gB= YPz@ Գ,@gB= YPz@ Գ,@gB= YPz@ Գ,@gB= YPz@ Գ,@gB= YPz@ Գ,@gB= YPz@ Գ,@gB= YPz@ Գ,@gAmصk׊z ꅢ畕" ֭[7nP[@%JJJ-[aQFOlf]Fm۽{U+3/s23׬Y]QQ9'M6u]s=defff.ܶmM7|]Գ 6VX+/geef.^zވ:SRRrS3Z|Yiii]oTM6&yPG= bcڴwuG]oAk׮[oM]o0<ԓ/B]oAIz@ Գ,@$/;vXp%YK,ٰqÖl-/...))ILLlGmkӮ]uҵO}۷_˖-;B=Pee9yiӦ.X0⇒eeePhMjBBB߾F5jcЩs`M=k#=׭e˖.[' 8Ng :I?^=}1fA0dΝAz>8ϣ>\\\\ '.[_Gο]EEEp"^noٺutʔOk]vu7?jz>$3s 'ys]-x?yb9YݬP(4~]u6gV\q'^xq.I=+ܹ߾}{ݮѮ]N>nwڔ\ 믻v͚UT^ԓ;vty3 \l2i777?^͆[Գmee˖p5q8}g5̳&Ϝ9{vڵ竇vx>}1ĺ^_3gv4̛]nޒ<ǟxr-^zB /,9Y,3:wӛ4irW.\#~Əc;?/7[vXN^xf͚Frr_p<=+n͜9(b쩧mӦM-5cg3ƍ3fl-, 5IέM`ߤVZ1sQcja7%bÆ3M4ԹsS'v=eKZrUy۷BM6mڴIӦ[ޣG=ѻw_[7xkW\M~~~aaaF4iҬiN;ޣcNIIIu&D}ԳS~~~yyyL-kgZ+/sʧfSƊ^nG=j;|!mow\jUW^jO#??qiiiM4iӶmݺv떑砃nڴi[UCYYٔ)N?g̘`wq#F8juԘ[hǎ3`UVZr֭eeeiiiiiim۶ܹs=>1.vݺ=gj֬Ylݻu{ @sAA_~1w ,^hժؾ} vA(O?cƌuM7%|q;vǃ{cA wN<䈱ꫮ3a±oz5ߪW&[WO䓏OO9ᄃO5VU_}vTշGΝ/蒋/}۶mءm6mTgwޤ^|!LছnVcr4\XX8ciS5mԅ ޽;ÓݾI&1 ǦN*$$$DZzqqka@YǞ~u9*&V\Ĥ .]r% _]N>cce BoƈCҘ̌ʕ+8ѣ~wk B6nd5W}֘lH-hX977=I'?믿m7+ +Y)g̘1=M ?`ȴiS;bРmFx R^{gqi[R:d%%%w}縱G|TTT_F ;ovrqq>I煅4n\x}衇,Y)&}W ;wCF,^(#c'_4Xiy{?-ÇSYqW3}4믿eb+//o#_zZ8klCgBqs4+/4o999Nt \CG5꿸Q!5thb]zqIIIЎ;?/JII9#j᠘عsir劘Ooƍ=/|͝p̸͛c;veGyիb;%ns@ ;Yq#&hK.2}b'~rȦMY1m۶SO9)o+,,<2c83[l4m{tnnn}N^z \ ƧFLҋ)))>DJJJ-,V\}9;yQGѧgϞ-[jڴiBB®]vڹs5W/]t%_ՊlCyV\~ɴsW_}e>11Ïpq0o-[JKK+,,ܶ-wŊ?/\ i3fLz]| 6nǛ5kּy&M5F#{# Գ~wɧkф{k<]v zj?晧 '%%5 /ÏHHH@֭[n rJ=a)wcǎߓثW~g蝑ѫWN:kZZZrrrqq]6o޴.;믿1cܹs***N#FX=/v4ċ.oѣԼy͛s„cu^4cx=nݺWu5WUI{z9G3ӧo6mjqqqvE Μ9O>^fu ףv}(==Yf?kڴiӦM۷o?|~tz(y~;wF<[~OjmVVMrȐ?X?[_~I^^^pqq 7\+G3ګ>t/k~ֽ{4nܸ_?󬉡P/xy k'FNn펑#G~'%%5nܸE]v:tؿ~Νfɇ~D@] =;>|~~Map[UW]M%--}w[ΝoXG>t/oαcDžfW:t{.Loʋ/$bˮ3VNMթsɷzsٲU뮿6ر-^?5L7?t=썷r[-&^n/Vb~qǼ Sϊs_Ҫr :r+++ZJ|O>8bl?ZXNt'OsO9?C1_,zwy{Xzz~#ja:'jܸqXAAzG!--w?Ι?W]ѽ[k+Wd:SڴiS86h\iӦVcbK/O_u>3OJKK4wg7nȣ[qshM4y/䲚ڹs>0h`UVV|yODZjSnj>jj3q9͛7 1l*">pkTif9dd_̍~gӦ޽;|ēNСC3ip>pg틺vֻ ̜[n)w#f19!ׯķ:]J3?hr@zzGUQ:C-BlʔO#f~:}RwpI/z8`xf~wvK^|Oi-[ Rq_kV>L86%%J34lk֬@xy5j4f(@}Rڧ%&&5_|=N<܇[O9z =lѢ3zXvn>u˖deeό5:UI`eeekkX<|fԨM4v94ht;#|%KWxo:wӦM1b;oUgeeό> ;wo`AG?J#frrrYQQ>3x֢ @ٳ]w߳jU?x9iii5w晧sE"fTcN۷Wjʈ~͛G|uk4">3) 80Q4\qshВz?;w<ܹs=;nۼiӃ=׭_1ӯ_WŽ6 (,,~#f:w?1M7֯|yPu>Рyz?(==/1s̥ڶm[QO!R"!!K^)>4Ej[۷o&iӦ7M=6{ /|Qc7Gzoڴ1|UV)))[&nGgD?mȭUݷiԨQVbWg4,3͙i?:=1ʗ/ ر#|M?+nD,UTTD?f.ʶYć5o<P}ASϢʆ+/\z 'Veee]zݻ UpZ IDAT4k>РgQM}z_~CǎjE>p4ɈFEnJNNᴈ?:e/b=+K!qshԳ~KO9?~=P+^<˧4|I& u(n3 5o\w Qsrr^|NJX7nM,7RVVu>Рg 7وhEHͣE|8V|_^^ @\ ?K-"&g͚yӦ:Dgm~Dg>x_|];WIII"~#;v:g7M=IHHx :p۶mᓕӧ6sdڴm~HnnneeeBBB~[$'7_ZغeK< yXZRR qshмܐXj׮ݍ7Mr֬:{V(ڽ{wnF:Fo۶Akeee[neb+Q~6X>Рgc\rY#Ɩ, =-_ZD!ϼryCw .]R \jjjL^^^-lR}AS"Ҏ?ĈkzqHĎUҽ[{D,^6MF|0 sqshԳѣټiSyyy!C"3gv"h~g^ a"f"^~JMMtu2\g4,boA3a}7n~Hm[OII 1cz,S{gj*|f/[mڴ ^t4hY^֭g%%% H6n\`~6#Ԉպ%K֮]S;ā>7}gLlu1|1>pg{M4&,|`qQDtƌ1&54&s&1׫WEVVV2\g.,b XM}L!O??QEƎ1?ON9Ԉ7^m֭Ll.\P;sqshԳM6Ek֬YȑZl>e˖_~1͈ÏիW}OedׯLQQџt_C"/ Mi-lR}R"2#fڶm>rgFuEa5i(;u7WTT>51F@g{S>$b&#O4=6{L#&En5ef.~䑇ja5j>SRRr٥7jNt>`>5>_qshԳMF3mQ gӢHDƍ֭{ͿqŊS}m۶gE͜9X qGGMj.%9Srd~g(,b쮻YDcƌr71SQQ13-[LHJJ_Gr~m-TmMc}Z'VN<񤈙7x}ڴL ,ʪx4DYg打'sD%&&;.ʙ?:hN0~ѢQ%λcNcW:v[Jӷo .(ϮzX7􈱋.:o ~iݺuիmV#>gŭgѯo?}]-s93M6QMHH&yG+/G9ҨQ{~ի.\paΝJvmwDd Bꗗ\|a-7I&&1aO:7"˟ٚOG=+nM-[rrrn;_q_~Epǽ cǎh—\rYs̄I4ɂI>S׬Y]#sLi/^{ԑ]˟]v=W_uEGo/[gyjA}4+JHH79sZXz1~SS>gŭ^x\PPO:aC[.;-\䓎?Ir‰'U?}g^xϊi^SPP]v]tTZZZ Gk֬Y4|oW_u_U7o3O~;Es=SN2v횓O:G[W_P_Fo~gM<;dvqc<眳:譪aȐ3rb q}aIOs3V!F;wڿC =㎙pԤIje˖ɓ?x?lZ>OƏ?'Κ5ǖUS]u?QG2[[^|˗/[tɒ}{2oAG< tѣ*->SZVڿguFU?աcǣsA 2Gbioْf͚UV~WsYb2k֬ܥK5K^^ᇍjN<:={rOeew}y+WdefΟ?/++<9ѿ^;h`*u{r>}v-===)){m6oZ6{Ιr{ V{BЈC/^1֢EK/‹.իwjN~ԩSc^oJHHPϊO=EM211Wރ ѧCǎ;tlߡC5jԸQFJKwos6o^vɒy_|j,s{ɧ=xⱫBM6m֬YZZĂ]v)/yĈ㲞 nޚLHJJjݺu㴴F***vE+z9fBMN쵇:oVZ5m,qZeeeyyyqIqQaΝ;+**1P(?[~S ))){JKKPYUc]v=ЁqGYz@\P+~ax+Wh۷ ۶mބ,+_~糪= ¹_Ԇ_Y> yQUg}/ohۼiӴiSzԹLnڴi pÍ1Y)JYKjzV'G/]n .]pfJLL|׺vV׋H'|J5>8pj|*n3 zVzt㏧t=&ӮM(iTBoZ>wIq}1C=,cP>ࣶm"5r=kԨQU?5`7O=+|o62r/Ùg5q}zZ8+5uz=k ,ٱS)vM7'&/}͘9;#O]/R}UxT @='vŕWՓ%%%]sZc>|Р_}[n 8w^byG4|EY7ts=vhΝ6nؤ;9*DϞf̜}i?EK/GjR }>SϊC\rٲe~a5F=s֜{}5YFv,x9III VXqw 3=ٳWqIq@G8鳞y~ti׮]z[~7^|;vrD 1;|YZn]>P?gŧs~2/͞套^ުUsN;1o>7w?r۬5:b[gꡄO?t?WT/3V!w1cGW^)m۶=/CtDD%%%?_9czNNN5&2>#3v?W]&0z>YX|ATZV>PCӧMٲeK++y\˖->Ï8rq#FhSlݺxΝ_Q ]v:lСÆ0|ā0˧NK/0mVi=z4xC6lAAu֬<Իᄑ}暟/| M:e?&&?'$$gsr6o1c̙3ϟ&Ӛ4i2bāGz'9N*5?d͚sf^|ٚk֮Y7%%%5jԨyzz.]3wFzHNNn9eg[~;w'%%6jԨE_6m{=tذ={<3s_]b벳}$)))999%%q͚5oѢE-ڶkױC;vڭO߾{g4iҤog.{]v7'&&&'''''6i4yzzzzVڵۿ};tޭ{={ .3g%KVZm[]JKK]tI} PY5kVg.^6{ 6lܰqÆ⢢==MԦ͚mӶ]v߾g^37`@e&YY5޽3zΨE գW,@gB= YPz@ Գ,@gB= YPz@ Գ,@gB= YPz@ ԳQVVlٲKZ\ @ðvڬ̬˗'&&ڵ[Y}~o5jT'c*ϰྡ(⾗ijy},e43Դr)Ӭ\2+ AQ\asa o߾o߾~~~bXMJJMLLLLLLJJxYAAD"ݽM6]v޽R^^^^^.[XXXX褤GYZZ6lݽcǎ 6lXFt rJJJ"""Ο?Ք KKK{{{-Zm۶sݺuܹ3`j yW^rJFFFnnnnncÆ ]]]۷oѩSvW2(v9iii)))yyyEEE 4pvv~^xOOOoo:[{GøHuӧO˾H}sc\e0a<**J~߿`tt(*++[xq@@YUTjH^i,09666>>>¸YN%^ӳTPǏo>Տ%tooK.iӉt[ZTQQ1k, A~wcǎ4(00P222FiȷQY'OOty-R2?~8 ..NõiӦ}Laܞ;rNyXHR?Ga\ ʳ jur9aD_GFF T7W-v_ɓ'cƌվ+yzz갷7xѣ5kΝ[c +ԕ?iҤ -QnCHTPP0z'NZ^b_hэ79V"1}Laܞ?nܸmޭ[7mz#0.҆蜿'|"LMMMKKkѢ*GDD8U5a ǪTTT̜9S{9eʔΝ;YYY߻wرc!!!sssLrWWWj׮w߾}thggW^^ٳǏ'''EEEpe˖{キtL}lݺUvoåK߿_0==}ǎS=sY¸ѣE"QIIfGv 6s/Taʕ111¸˗,Ybnn^tuuuuuׯ߇~v5kH$O>z9c<:uʕ+;vXc?ކMLz0<ß/ :::nڴiAKKK[[[mV-/ 8PswSoٲ 7jjժU3f03S4\S$%%%OU/u.]f Va\%7Sdff6ba\aB!!!mPW]]]{Y] 7{JUruuݽ{ofii)zԩ{6tUWsNg"hĉ}*=\~}ƍ¸sHH/DGGϱttHYo^jM͚5 ݷo*Y]t𧟟q}AVO>~m}%.}La~˗+4k֬7oΜ9SM\VrDU瑼BGy#G gϞUyu߃Dyyy.]Rs .k|a/{!kʔ)vR+JKKko Æ ۳g*oH$ZlK666'O̙3?#a<&&F1&?Vw!<<\e:kz~=o޼)U5|pR@m¸}ʕ[o?؃1b\=ʳD1BXJs\Jd>}ZTX7V)>>}||֯_+M:߿{n1Çk1 8x3K]rE᛿z^zɲe<<<]ڵkG۴ic_F9 ]vUphwSA6lؠ !H^!Eڣ< LTӦM~Zʪ7nddd(AY̔|Iz۶mRT.haaeu| omۦV?və3g===9ҠAٲe0رcǷ~[dya\ain͛7ן,Qr`QQG@]¸}ǎ™zmmmon,Fa\=ʳt3FT<}.^3ܽ{799Y.ոqcWTTo+Wmmm?a<..͛fӦM.ҭ[Ǐ;::OiiC%KjƌEaRSSW?S߿ X]XP{c vTga|ƌڵS71Wq(ӥ<+44T܆`ٗr9”wr)apر-ϙ3G3uT{{{aȑ#uW'O...{xx8qYΞ='6mY5lذgϞK~S_z~=܄Խ{{\Td)/__,m0Wq()SxNNNllVᲑ?^n7.!Sp * nzРAT L8Q ѬC?~|AA\M6AAA͚5S VЫF}4Ν;' :;;O<Q=u={T, `wS+Fv4AckĸH{g gRv e#ÇMyyVEE𱍻N._, 0@Uu.""b„ "[ly-Zۡ/0@\\\AᲕ&Hmiiid_:} 'N?pa)Я_? #yU0.Y`g).J0h ;;;[Ҫ$&&&''G.8n8%ǽy0驤I6˻w6PxxOt\]]O:AotI~"QÆ Gi֛! 7|&Uϯ#/]H$͛7{xx|GeB¸ƍ:A]UǸHKgI+Dϟ˫ɱcd_vfr{Vɉ'A%Y¸ٶTQ]s)*++0ap%&Mo^>3223fXiAnTУG'cDz0lr喗W_uiԨQ+..4b)%ɃΝ;kZɫq(fkk;rH`YYYu$$$F^LɈ#v>~xuVn9;;2?~\QQ!;;;WD666666ÇVW$H$Z~ϖ#׃O޼yk֬QV"={vΜ9K,u/¸eeexFA-ŸHg0n_^H^c4`+)S:-9}tYYldĉ/'MFl" ;ѣmlldhkk0*>}* j~;w_hZ+W,,,\z}VNN2Egݺuk]u'սD!ì\#2ޞ~V֬Y|oݺʕ+JvH$V߽{3&4)γTH^KBy&L*-- :t?lD(Id_dffVEzU9yp_~YyM6533~:;;[yC劋ΖM5s̮]N81##CuڵVVV+VP꾀{yyyiemӤIX,I+J |~z z积͜9s̙u(Yc?puuꫯ ! ;& 0"ǒ4uwܹ~z'NЭ[^xA6bff6~x~2<|#GT0~- y¸6\^zpիkQ*99Yj!++&M㉉ΤaÆg st_0`ݻx ++vۼysLL!a}Lanccpea$< jWU?=yDvY>i8887Xƽ髢u|͍7"J?3g۴¸]-^ySF򖖖 tqQu(aܸqvvvrW_mU7aÆ9::Fn޼Tɓ'L6M + FEEҶ: tĉC*o={V~~wԩSVu]xPXTO{=W]6m۷[XX={`S>0nWxp69hΌ4h leJL %sΥKD"D"[ݽgϞ~ƌ#@?JC ݻJsC =514hpSYYU꼏0#X{ 8P1M"]v`vv>^5gΜJ/-- 3|>@)ⓓ5A{uc$<(PdQƬY >|(饗G8VeUVAApe3f(ҳ0R r@= Gz?f9D w50.ByÇwss ۷LX}TgYf͚)`ƍ̤[nѣG st_%e\wS 0nׄ҅ J$rЭZ=o߾p7nB*Qٳ%Kȭ/+gr;w;wN)ѭ[' ~:t/[h1w\R2Ν;ڵL׭[wa%mg̘ѢE a|߾}6lY&oƌ͚5ׯ_o[YYfi;V,//4P]ƽk 99Yl޼3}La>mڴ6mQQQImmm[l)>B*QI.]3dddEWr "7}Qj̙fzj+++a믿OT#Gf͚y̪UMɈƎg)ܴ`{UO?Uiٲe/.--Mfҥ¸T*}UȔK/]~]4|}}>;nXjM{6mWP eq(ZfwիWǎcffVc ׄ 6m^r"Qǎ?c֮];d%%2YYY{>>|y?pڴixNNάY˫k8g___;OO۷CVVݻǍ鈉{7:w,͝;^z扉nΝOropӖ-[Fqx=5j/b&Lstt8p3C}La>}=ڹs7Ojju4]UZ:}7o~WWWUN`mm)ۢ!C 7}ᇟyu Yڵ]iigϲSSS.^xڵʩT6Dz{5[RRpň#tҴiSss󬬬̸۷o+ʕ+]v1=zT7X,ٳ#:vظqcGGGX, SRRnܸ&";;޾DF?~jAkkCѣ}...U{޽{7&&&00000P "sپ}* 1]$=z;-->`:ٕ={ӧo߾uVddd z;vRF򅅅ZRxv&M999I͛7Ϝ9sET:`U"$K3,)gѢE?MwqssS>}TmWXh ''ƍ5V%WPP 777 ŋnj(՟TyH$ڵkWiF,HB?jQ>IdATN&MneMEEE ;rppkٲƩ };ntՑ#Gfggkyt//HH^$[_f~ڔT< YpauUDfͪnӂ 4D5 ٳ=Tiܸq``xLM6l WTT̛7OɊ- `Iuҏ6l׮]kDF_|YI;={V3HH_g3={ R1IHHвjH~۷@%%%.]0.YPtmȐ! 7͞=[~f̘aaa!͛7OiӦΝ>}6+**_~Z&c ^{ &+44h9,HhѢ}>Ӓ>;֭333#^?O?Yƺ¸gϞ.\8*Ͽ6=T֍䭬ٱΟ?/q(ZiѢE ĉUuȑ̙35S,;;]v|CAVlӦM69:: [lz-Z޶m[͵IѣG`\SL>|m}||tMŋ8q]4f͛7\$/88x̙Z?x1q۩Smq'n&%jH{͡uEٗGyJƍkݺ\^Qs oƉ M4)!!a=zPe͛GIII˖-a&F_-JJڊ^{-11q۶mTk?Xܻw˗ߺuYfjmJڴis'N2K/t̙P0lذׯoذ]v4޽ܹsl|`Fرcرjfee5zGDDx{{Pcq{-oݻZ {_hBq$ߥKؕ+W6iDcoÆ ;v쐍ʬħO}-y6L/Э[%I \\\<<<<== cj/޾};55ɓ'7رcNtݸqcc{N OHHw^^^^YYcF\]]Pwll\'W\ڵ9Ą^tΝ;b¢prrrqqqssk۶mtҹsgkkkO[c]􋊊Ο?STTTVVfnnnooߠA'':tڵ{U1)ܹzſ;55577y:tСCnۨQ#e[={6444..***Ŏnnn^^^vssSҏ)\&+$K3}رcǎۨQ#ç (@-ygeeʳkM+W-) O*]n,96իsss%dff8p־FԩBgNH$5NSljlll \+@vΝ{mҨ67g/xYa b>6X,޼yڵkkܹ̙3gΜ6m:xcǎ3BY#v#J_yݻpW+33s{լss=7nB*ee.=ٳ@mVuV^-TЪU9yϞ=nj3`333P:ԩS/_6vݹs5k p8+@ٳCݻ744Bjt ##0$d˞=w+-+ӸH4ϯ|Ï?0=rrdf sse 6zggC۰"66666vݺu 4С Ǝ榇4(BbRRR޾}{ P[:ujϞ=Ǐ/**Ҿ×ƌy|*' <{60$$) IoP ..7433իߐ!C,,(%=SZ~Qx&N*?~޽ד'Otm~XF,C۟}&}ӦIJ޿ }OAF٨[tKU%JGܽ{={ѿۯ޲>:j7`\yVԕ+g]yPfY{<}ɩƆd_ n犊|O? kw߾=~n[Zن{M,JkSV]_dofIn~+VyxK??tO>Y0cFMDdgMnTQQq#1QImVZ7 rPk )Rk̘1͙gN#g8?믿_cw||l$*6ӧP8Oի +Wd_zvpς™K OYg"#M~t۶5*P̬gNj5SQQ1"cbdܹvmu˛OO=o^bJG)+/_rìSÞxD"QIi>8pZU_W~AEEEkA_uK";#MՔZFٳ' F,TEEEHHȾ}>õuw6Vzxݺ!ض;/H^y]aKcƼ5{޽~=|Ӂo[mʕ\7V=I} "u넵YƎӧK"Ya;w.]z:2s>> ;EWQ>Ӻ$` ƍwv!.X[[= BDy _ڵkBB͛7oݺ@&bmmЉɖgD+iT0ܥkJJK4<|QN;}w;pď.VnY̘?kzR;kBVr/i>JQB :ylD" R ~+**"66m߫Woݺsld66k r)ѽ qqmhaZgjZ4{v+U(0(s1\rÆ kذ;cx}|lKJ"gΟ_8sOWlme5gʔ;'D" paȑV.F &MX6s$N̞UGUװaC??CuYA1ʳ`4ɳgϾ|Nz7nd!&;[[VE]XQQann.LM>}& dۙHY_Xk">*ui^^U$ T*6=;}bi,}M*cC/u6}YPX4/IA8R'?ڨGWf*,*r%٣gMKSĔI צMg%;[_S<^Lrr 43fL,o7cǎ~a~~:Ξ='[}ڀ޽vK?1%___//;[¢HZN??QF$Ezw)6 +ٳG뻊BɰAAY֯"MWDkR^~Kyn߹#UkR'?ZMvJϟ 9Z sBINN>|xʵn#F1bвOo6kџ}zaBi ϝ}0p߸}[XpUzK;[['\$=3SN۱cY"hʕ/zi6\BӘ܇kkcPZvhWKLZ}7:yr3.LHN`Aӧݽ{wԨQf/Xwݷo2.ʳ`P^^^aڵVõiS<_PUxHHn..;uDsLfJi hj~C0–AP QAU Vlk*kk[mkVjmqU[8QGd! HH/II)$WCs*2#@$p|GQ~ɓ,7<c75)C4*ʽ4f[5>ff c~'N|Ǎk# O<}(͛pu?%cccǎYXX(O%)e2 ^^<ڝ;>'?>Xy֙+LLVՕ&s}Uxy킈6>_ȓ2PP-Zj14%bٲ+G<8G!!}y7;kŠj'CzzzJ7nܚ5kRRRjkkϟ?raÆ>C,tt IOO_$i#G/;;cg}'9w;6ohim؜=cF;;0wʕk߬ohzvYeMMf}CڠBE0̺ՃS]q'N˫CB.Pg{03+~cҲnrM_#'Uf $On״!HF7bDIJe/o޼p+kj?p7Uh=yǧ GGGiӦaB͂@X, 6`t۴i| [3g o޺{j/\ֱk“ n( IDAT<]Vk\nǦߔ)%0_?Ͷ=)($RQc~O?WXKuݼ+ry GhB5 xmmү(f[5,ݼƍǗ.~vv~WU;5 tss۵kW^^^AAΝ;gΜ,,PT77xٚS}řׯ_LL L;Mz_x:>*) ᙊ7tuq8E#̘ʆA:T$QBCSS5:ΎFDQ"-E&* hnff*>?qfg+Þn9[o׋/cVw9r{GԝYx\.wݺu%%%ᑙLDiii]]^##SN* Wղ&';dri"m̘9w AMM\>Ld.cNJDξΥT ~YÎY\`Z]]E"^UJ~*4q8r2ЬwV |ABEƌCR՝(ʳ@Lז-[dhN&ׯ_1QDVV1ѣ@]͞-R 4ZAL03i8? @x 8wJSssG4:O8QxD \zB?.9}͕!xΜ/ x?2&t"S.U=TC{=_v+}W"/뿲 D,[&<>{E"qq"q> z޲ewU=A+s{lv}##\bEMmAӦ 8P8|r"1QY*!C#`ΝCzhk Y .n:__ߒz W g:u^\z /wAGZ܎yKȡCG *A.X e?񅽭paE_nl9JJr93])*IU_~UT,3>/r΁-[R%O>aX!d5s8^]vg=Dڿy U\+g~fVVg ymm#~Q|[5MC:d2CBBekN&####""tu;ݪ ]vʘ(@C"f-LԠӻh 5ݢ)h4gbbblli4Z\\N, >Ev"s7l.2A (.2fΌܺMhMv g262>/w?ڵ"9pZZ._N|!slsOJtWׯW_~ Y 5Kzx?'/^\9rf vMT?Ϟmv~wwf[5ͅ X, 6`6 OOϟAt6VVstfƤIe/a\oiw~{7o%PeV-Y=qpڵ# #HQ_}%"B_OOeR//߷y O*;?_b\-BytOjj[||lͩT}̤9z2}xHd`_{ CL]!/w΅Ο#G|7+Wp.U"HmD}y:\&sJ<~EhhƩSStDuu-2fds<:~A` DUnaag4ݢ)h",nr6lضmjRh7Y|y>}d;].w9#CNaؠABSx)lE031nʤIOps;x#=^Ҭxc>}{+$ }g9>o@ZYXXuk-B-Z"mmm՛hK›:BH$̞Ζ4K4h4Z\\NWlVp$ޡɥz3SS3SSug`6}=a{hk =ʳ+,+,,,>>^ Ftt(N֜JFEE+6.A"*;5h4,>QYQYߦcr6lضm[[_zxxh4 $I AH*K `< ӿ;qL&3$$$==]d29222""BWWտF888<,xILLLxx8͖9FJzt:R] ՝8q,d2YI=wl2DIs*)D7V mffKOOoUUU,HB@݉VAyhggg+;}*AJRSSCCCe^JFEE+6+ِH$[[[[[[u'ʳ@Kx{{fdd(&(bOaff`0TV~\.wÆ ۶mkkkXм4ʳ@K2 >|XZZڱL>]gQ%c2!!!5'ɑ @>%^111l6[4-..N+6+,ވb633S`VZYNjjjhhhIIlͩTjTTTppb>:NT[W,LfHgL&3$$$==]d29222""BW1Hzp6-[sGvCycXaaa2`0@o,mZRR"[s*جz u'Jr׭[+smGff&jdٳ IOO9LIimfekNtb腰!Ac#IsS6,`\`0P= TEʫW.|]]]?ngWyi'OZZ8N{t +\%%%5RQQQ 7Cyh'Ojy^g \bSju10V mff@ʳgWusI3UH"N˵A77u#?%=#}}} K s'1::dug EE6ug+55544DT*5***88XYg@R[^xN=,Pge"LNy(pCIQ3ӼC(#ug bV4; D\.wÆ ۶mkkV Xм@u'?Oʨ"b-tWĺoGrԒa2^^^[l6L&_>99YٳG~U]1qSF4pU>>GHyZsspC4{ޛJ~ >6nsgOT3ﭕ# ՝ gٲ5hqqqt:]Y@PY 6Wc#WB176|_, Z[Z_O9uX™ A'렼[,,6~Vw"b633S`V=@ y H腨T,P3NsӢfUKܫ?^/g } eto78ݿCqZjjjhhhIIlͩTjTTTppbɸ\k*+@̉':8`Pg}Śܜ,cŻ-[FԐK.zm#Ar6lضmS@yxxh4ݹsY w@SSSuZBG @r9`w w|KڬFrvzL&k˖-f''',S\\,@uggx%@x,S!bbblli4Z\\NWlV^ oC x<# թ7&(; OW/~ZT\4YwJ8y⤩zzzZVG4D"Sin~qnVaaAQaAeEyccCgǛ=v}RЬ=8ީ,/\TTPS]r. ]]>C qvldLUY=Jz+*//! 55544DT*5***88XYPqeqAC_rG=g}ŋڂ/$_(<Siif=zpAiwnݹKɞ}uk<}R{ී8횛rd?:tp? 5Uwnȸ}7#჻lv}ֲ '''BB[^Sgtv͆E}fݗՈ!,s )KL~q)޶׉v79\o^EO ֲ$''ff Y,Sk&/j<} i+5=tpoڝ[?nkka}}ݍWn\m1?[ol9 M̸SR\,}+WQ^^Q^~U ;:>v2U?0Vebb8{lGGGugD\.wÆ ۶m֟<<ns*)/V.nqegଟﱴ#6JʥnUIqqIqsg<&M:Gi0VX'V^zj'''oo "&9LfHHHzzldrdddDD :::llԝ,즦]M= ?|F5(vM񸭝 ӎܴzжuN7h4wYC~{!J YG%t`<|pwάoqO2LhNenXɃ[UGGGY_A\8w&7'i@qߺq*ۖ+n^:s;W0Vjammѣ###R+&&&<<͖qniG3FY@ϒI&ʳ@Pjpĉg>ޓ{Zj8~ս )厢V?bf9zF] @4ArpccmSH>j,AO7].b @ HO\Ɇ.&HT sksEOWO m6ֿ~PlJ֔鷃f;΍D")(Jz+NHH1cF^^^{)99999>;v;LVobe`DGGwm IDAT)0+P g\O,1>*Τ]]]s|sFnj7u"b՞ (M֣srui㴬q[D~O/ YBSDU`9W9x@ǔZWvrr P(*N\ m&ixxxh4?( +++.--{{{{ u`@6Uq uttTΨ^$q[5zp}wUWU|Ś7,/=+[Jzz?G.'H斃}r%'ߕ8AS(o/^AاYY82{΂̌;˖?=ѻ)ۙy]o*ˣw7.W Dc^rP>YM d^S:57Vڹuoſ-.~p9 dKCHf=zVMM'LxExrҹ5u(_fƝG=eܘ df/nrZ֗>xQgB5e]0V=*)**QM]c2!!!5'ɑ} @SUUuݼ<&ohh>,??cfȐ!Ç5jTKcr*dJe3!mww}:d26v;4kn!E)/}o.Rd/:wgT^ɸ: +֖}e]jck]|9Y{T^oHy"s\ѻⓓ}X, O%}ݻ\dWFG]F*5v%⹳D>tH<]_Fקڝ7o=>#noЬ^Sg(\ʃ glΰXG=z 0VLLLxx8͖9F q7oJ\`N__c'~ۤIcRRTb|a*΄ 7omi/_}/M3g,[,~/|}B=_(љ{z[[iWj4xϿڰ~x'w3&xLV@]r8+oܼqU|7"N$ZV׎: y@ h_zOhKA?neq<]ϝDyn^pH;aMQɇ.474w'( ?`^ag:222 ={Jbe`DGG)0+P65!C;v,<|Yo cUVZ"1޷+$=!w0$&/++9AΒgroߐfɕLL>Asr񲶆vQaZv\A|M-ΩRTpvqc" VF MNo ًO]W+azU "[󦦦իW2믿oQT777kT}PqPQ[[[rr_pyaaݻwQ^.n&X2I[H;ۓ#^ y/ݝ[hHGGx=2'}K*zx_NLL̦N?yR({fddLA===]VYn$/ÇTm7l՜@2yPs>|DelF}=lӈA>O}XIc)W99R|~FFƦMLbee]\\$ r֭-)|+yxxdff+0+P,nNAA#GRhv}Su:ArZ\V9YMg3?}wSN5kIR(Issĸ*hkk;qO6|Dw{Zŧ'^nN v[E,;\)`i"|%ױ9tpHee%We2}kEԒ<%bxz^S]Vg&Çt2lˆiotT/:x[:Ptc%=fyCڧjU)(((((h E>LfHHHzzldrdddDD)iz@ti>/>}O>`~]|6pn^lnh.-(ͿϼrEkw*77wҥ666zjnn7{.s%g,\,[ϞODN+cҒeh`IRٻK?/""%8-0x0`^I.X UUVrQ =@Ḻ/cI(i-wYѿx;}Uf0V}3f)ۢ;wܹƌ2&&&<w^ @C)d<zwjժ䖖[X,&smHKKCm@y ۷oHD"o(d&ZKg[mnn޳gOjj TPbo] SʿA"3;[SQ%TV4Pll$\ڜN:S*x ^Ȉ*p8DNt+A֯zFDž#TsoLAT@W_o,CK:y* @r SjDGG/YT}@;UTW'߼̬W `qC~555"qxۿ|/v}ϭ‘n.Ýԕ0уaH2kz?XIcTYա}J;wx "Ln\]]WZ5{lww oocj=f<޹,d A555޽[ވbH|G2:XY!ㆈϞ= cԷoRP˪x <P$WYU!O#:V'$I_R@{=efI_Mu3Qj+qcWx?Wf lZvӗPoe kݸ~UŐB5Y4K\.kJc%=P*577\ERRR֯_YAH~)$$֭͛FZfMRRӽ4ʇ^K ߿ٳ&021<6*H@n!Y>>3g677Ww^mXSqqӧ#CsUYokg" To_>$_  EbIYs$o/tY<׽@A{ym ^qGxsp}CUϴJ +a@-h4ٳg{gmVU,Y"qoEEšC.\HΝ믿>yD :=3BBMϟOW;5i{UVV40`Yv܉5 f->,\A":ڪ+%uee{=x`J e)/&iCCCqxX9~zVk,tN/͒;Ksr;[XZXXQ[[8MM%OٕZI6|F;qNJˉ\1 hc@YZ2r_ܶKi+.XP';I邝S?>.|U& ԫhΝ%%%NJ+>#i۱cG``ԩSlْ)8h@p?;{=&0Vߨ^tRxaa]fϞmgg Pچ?^8b`衮|:Mkpʕ+jJ 0VV>'r*H@WW^<-O9Yަ ĸ}ΚP&ƆE.Gwj է.`gEHgue" ڈ›y<;Jz+!W^ٳ={+**:t!2_560VX@p233333wڥ\4@[[[VVVVVVLLL>}F9bĈ..N +8jBJ۷oH"DD6Dڹs?ʃ[ZZ.]tҥkZ[[O2%((}h "w;hLṋu:'1q݇ G^*T Ԏhx*H`xuxIdϩu;qܤ]@Hm-Ztar%16P(ă#GO),Zx@)zcc?VxZVM+Jz+mUSSsʕ_~O> IIE%%@eZ7m:&0+/RRP7n\vmTWW?~|ɒ%֭qeP%̞Z&\:~җ.w_KWSSSXX,@0VvV kڌ.#y] Gb'xL3%.K~Քu)^ӻhտS6^$~9⛌E ccx<}67KN>g$\'w/]_r"Hz*cc=^-gǼ)Hkjjb]*0VZ-(8s겷ĉ3<=M9'ŭ\"htm^'QummiU6d_ۚܿ/arԩ"eI}r-[< W-ʳ@{>~X82vXu%ss aaG֭[*.9XAruGb[8x rk>?u[?PH<J9{L&wvxwijEN),N8; "H"?~$C›#G644R)x(.`z֗nУ(~BR7[NQ0Vffu1t//_OOO77>f# ʫN^x5--'?oin>F8n\i%s |(898PQ8>477sSbW/$V-#{qs݈{$w_K88L8XPmg$9ruuuuuuY3EFv@G鍄/Ns ,XqSIIiִNrlq\gDOyz%N$O23Gl+QQbOP&#]p~| R'dή_GށxD?]uYq!+  {ܹcmVGif׻e^S>,Rk߾}^^^Ç_lّ#G^ < DEEn2TFfF$#3v|vj d_O+)u$@p䯎Sl6m׽;---=NX"kVaw㚚c#;rG>~vk7FU+>\>ϸΚw{VߺwNU뎏jii8SRR z[ ?SZ񙿰 iBgXX__~9cv'mu}];zIG?#`n|]t͜vT9/r%> LVW dډvvT*@Ѹ;9XMǚǏ9~/1ct:ݓ=tHȄK>ޖ<*))MhwnMHI!Ξ1طv ^ՌŋvMxr?'IP(f-[vЛܺ䓯wˀAkk/~[:R>twwo=zr^↠ ˉfJ2]}hlnL,--ک+A9~9@N?wT0b#54UUx.[[[]Q^^X,_=t:MK,'M?hk[#c:QV 捫wouU.fb2cɁb_j:{)FFJʵU/^d'߹ujc#%VwN{;ޞ:e XԦP ]___Z ?73#oO}fChTAAu:UVfȔ`m"PU]u+`~;9o;nو ʊĻ}{:~a3Fn7oԮ*#JldzOg99|ĜUUT8E*r')IdÇ^UZQSP@ؾM3wժ'OJlp:m]_RVb'}+3#c>x@siii8󗁡qϻ^:qԉý:uZ3TT~Ws;'Ҟ>J{CBUy'{v+/\|?~ܸіO/^8WPq39um撌@ `Y$_XL udE">vMbwu._ȕ+AeyܝP ҥ2c䨻on-*zUV6ԨS|~|z+Q3:s_w zE6F1Z[[KKFG 54,߼9%mM`k=k֔ql-, fvSS~qqVN݇YYOw?#֣F]9rDt>|u<׿IKK_ݯ43N5?ȁNBl߾Y4ew||滻;Mlvݤ1Ǐo%$\sա[9z*3g鍳{ѣWFEEyܹ^^^d#P 9`YTQSr뵺h*ۿ+Y^G Xs{ުz|?a Nyzφ`/}Ϩ_`7 nV4մP:Eć\(Kp0*EYYyaVxoٲO۵xD|#L55q}}şG~wq$1!]^6 &=kȞÌw_G``IYpz#c$P(qqNt:ݻF`2';yصkjtt:N(_R^SEBR'Mdڨqq7J+* 4bOOOOOYfԾyL IZ"W F[܊Jܺ}ܻn3;Bb츉';H2 (++}!} ާD9 ȉߵ;*N"t9sɺ~ Q/,फg@3]i&1q؜F&FёRkJ|9rώ;~f0';(@,P-Ch*rfwt PU%WwΞr8N4- s<|SUzԼW:YL-[#D<<]zٍJrm;vGWX/<}`g7.jLʷq*%bf8}=O6M#q6Ϩ_`hzzm4x9Qu$o` +!W 455O>~xmmmX~mtttg`2SLQ>A6fl\\Z*(2>wMQٹđa 叫W ^"23;muc߃O=K/3. ∎ֵcǬ-,9յđ/6m>iR`gUTTU];ˑ/}=>}.wB2,hɒ%K޾{rTT˗dޞby{{;::hrPQ DP":"O(_|Me SMrrz۶¨wFGG&$pe1tOKKkΜ9,kΜ9#\Q< Hf߮[K'nJrrDW`׼A9ϳsKlJ kddb2t( K+k1cƏ4n1[gw${"3V&Ϝ6IrJ4`$͟H> oݍKNWRR\]]U[SMUR46fii5a,9CU5 Amx(%!.&ar^W \Jh4UU5uu m}}C#c#GeSoݎI{ ueEۛVKK[SKdIS;wX{Q7ӟde彬kpTTTLS70066616jj6|ĈQf#|C>x& q i3e*Ae\juz˕xٮc4ώ\5o<Ñڬ󄚕!)YG&$tUu')FkNjFFum.lߔLdyy23##FxvV)mM͕  7nx=1g_WVnf5UK9N32^Nl|5UUqN:fԨ̜7fd&RP\|#Q,'>O(_|MeHKoGF <5s]VW+Gs='ZYحyϫ`3f8'L,2 w;˳2Do=_X`5J\]VU^Pj8ܘܨ:U[\Nda1Z WCDQ>qF1kF1k,uu ߱TUT8EB"]y[qqŸUU'ϏGǣZ[[c'sَ*$߻ n(JTB)YGE-۴Gb8wEF<]{Q?ƆX(3瓾zK ݢ ٹ3=;;&);1Ç =t:FFFyHSnn__uT|frwwРs(!Ri8Tj[r) Qs^d GdfMVr˭m՘R\̍DxUV죭={lٳgcO~-&F8r'1['Ν{d[dBBY>%c:Md^ѣujԎ *ګy$t6یє\ggqO鐇)S$nktutğa t|q{" eݢ<ce5j–ZWn/.$ٹke45 &!uĮkxNqpJ"}\޴i'|a oذڬQF5ruue2edIsw'g'=y2}$ s,j]۾,ݽ{(by?cǎ\]]ΝiiiIVlj/‘ow,Ϻzqs* C?^ee9(u.[Qi)q{M EF^Ko{{?CqYYFD @USUnJ eݢ073 Z$hɒȄ7*+?YN[q8(J~~8Y#FtssÒвY I'4W#W ӜDG GƦ9)+_5Wץ?&8pҏJnxWN-._iRBJprTԞ[Ev-p BA(;:Zm$ IDATolyUВxi6z(kxhk/^-ef^sFllǭ ٳgw̙3Νecc#ؠGJ####b/r^ ijj*S#W RMܸ?o;ji~;Y WCF[mVAAAϻ/pXҥKgϞ-** @m "%Srr^_<16)I9zz:;w#pҾu&˩<Iӊ* Ȉȴҗ9].h%1S"OJUEE ݢR>].))ǏoxCv35#uٲҊ )#={imۣ#"">#f t`0F#).1Y#+++8mF|\p$r>>,KII髯Ȑd߿ѱY&M"; 0aB7+_i_7ʊ6gL¹s3g>rѢͺ;-*]mJf]CWRKՄ"6zڽABÇǝ?~׮S/KY{k g]p~~}o kt?z@ I|[5lT1c|n]UMMݻܹ_VY(3s'SÇ{zzzzzy@q77TVaˈYѣGϐ+k UU/﷈#FNԙ֖VB5np[|2J|\BmFcX_sssCCC.\/III...ľ̛GܼZQU3Ͽ+|h̙&Cs$˺--1lqynnȈYgK (7|yxEUg``AqqBl =rJPBZ[_SnQt0Y}-LHȊ ݽ{ǪU%d&%SN ~Ł͛,,PM@~?xk={82uT)ǀ\`c;8*85_ `|Ǎ8adr%> @N̛7oj|}}<󈈈۷p 6Hxs@  p8mOO$|>?֭RR|g&8ΎX+s3._7o3 )8kk^=5:Ç3z5ItݺJATqiՕ8Ê`w>㋛*YGz(k m̂,x͓'k:cSTIIIgϞ 9r$@ߡ< J6mq$%*dII|VQQ~'g J]|v r?+i|7r_ ,yWatr 򜝝qNbk;DnHHHjjggvvvgΜH()014tiG74eeo77c}\P(ƈn" DLpZt4~NiZtoY.`./M8Qd$ΝOEFe|[5::dG!%&L ; (2m4eee&лk=nWmooOJ;z 䝺r[ ,1ɿwhxVr ƊUA*$ur%> @&&&AAAgϞ-,,QV#""$ (00911Q©@6nU^ ѢP(nӧ9Я}ׯS(k11MMFFnoD\3B\?.sw]DRR쳮֡H(<2Nw(Ϻt$ݢ)(gB2dӉ#ki +GgײJJJD\e1,b߇.4ֳ:k.!HII忆 3r%> @^KmQVnnnhh$sXP}}}IP(III...NKn˗㒓 ;k8aC11lvXx8q .e-?8mnObundBqЇ, ޲e_m,2xʕOwH[၊O* fϘAip>O(_|M@< ܹsnrcI*%*82eAT6_\OvP0XOv]XdEey 2s;]WBR~ٍJr JJJ͛,mmE:t(''oGӃ6l@$J s8Hd)P('/\8wpAΝ@*Gr={3.ϻ=-(H߽յBdpU*ʧoI|7\)2өS";^-2r$YO- Cy(}}i dL{R# n{FyuvӃ WfZ|vr腈_/U˫߇.654Ǖ._Կ_ȕ+Y윝ݫtQVYYٹsLLL(H *כl6;88̙3_ҭ#nf枺xQ⢣"{^뛫*&XZxvˑ#n_n]U[_?/((}u{KxF9L;8-ݴ>nn֣FG|?xF9Y&'Ν;mdJ=tjni裷{azzWZ[$G|M@IP@61onѫ7/I )AVjt*qD__ng i+@@R+Stt9VWnKʭħq6cFKW٩Ϟ%p[:>3} ]+!WF5 T <<<<<۷CZ>鰴_W tuO}ǪUĢ斖CC=ʚ1@Oի%'ٸr<K3J0c^Xv+>?sz'ŦM;_7606i6SMnl|UVr)ҮH1?۷ok;w"114tupfdK/R32(5k !(_|M@` ̜93;;‘?s4=v4F ɩі7n?~ȩQl,Dj4?N1̬G 'nKgy/ne+2-@ȕ+c߬Cz{{X,777]]]i&:hѢ={Všm͟dv^}?DIӴ#FPg/WW)5kJJ?VT\ysex!aaʕC%c3?wq*_|M@~< ֒%Krrr kH\9re~|iVg_?y].z4f̘m۶544ũ"W 5׼AkW/5wܡJj 1EWt kii4TT4T HK[gIST@=\]jk( ر#""Blvppw7䑡"2+|}?pΌccE6ކׯvS]z{:iupcGŋ"oT*/1X-ףtY-ٹ~a'iyE-O)}?477 sr/tw Kփ맮ݸ0~x)]L~~~  GFFzyy!W TTMyq-]|/i_ 4ul/w+Ać\c,o#kk𨨨[fddH2UQQQ``_l\J ,:cggGv\b%''yy}ۿ:E_-,,tkxy݊RZ^̺::cml&N>yrӧ|׊3fDܾ}/%ԙLKKݝO-"5EDvn"n_U6[ 8qBvF\0s$RWTVz"[ʼn,,';7Q;!W$/AAJ.[``^UCCDUU՞={p@ (.--.)ᶴД454 G1Y3ں⒒ʪ*vcc3GSRRUQQSSfl!W++_jp| SMMo#C!ڽm w|M?5X,$n+T*ʳ`Pxѱc:~?kDM"F[E;y5jݺu2Ro$vܹo> &ݻȕ\CyHJć\6ٹ/*+*N ,,,,,,F3 +ȕ;W(ҕ*++4deeر#""BLMMw?8(@yʳP@P(YYYn-t首i~x9N0!00NKrҲvӧOϙ3@ʳ@Pr$> @ʊF6mP(*jjjLaJ|+gH_TT֭[322$aO%_P},˳h$֟|ѣG D**,*423fg3FMCW3(ʸ<9Qee ̞=[FnQjllz;sر JJJd"+!W 9|]vUVVJ2URRkll_ʾvT*eBUV65[k@ơ< m۶]t)&&]P,B0C33s!Ch _WU/:UIEEorr-[K,BSܦŅeUUUUUNv{22677:J N Zhў={<aaa.]ڲeUGJ BRP(sZZZ Y0h~ܹswmGmT*UEMBW)S(nkinDt:cΜ9H_NN˗/T*۸qcs@DmmuJ'EJۭX\TT\TԶIRM7e%rEWW7$$߱cGDD$SǏ޽_W0PUU555-,,$;WJJJdGz-6l{yyyL&SAJJJDDDIIICѦMᡧ׿3KѣG ,(--%CXc[@ 8{,qg*`2iI df}v^^^WWJKK\pȨL&aٺz֭[K,ihh jjj?b98J1< $$݉UZٺzXSg(LL ʳdGKKɓ'wUYY)TT*u{566d@ (//;m008hhhxҥ)Sanr#ϕ@yH0?+5 rB:+Px(5UUU{9x Ǔp*uu-[l߾W$,PX\.722RV9lsssʪo,sss###SSSkkaÆQn$QHHȧ~*RDeaaq5sss 'Z***+E,賖kW/ݍGSWkpcm!* UUJPf.uqEšnNg?aѢ%*jjLrʳdSVV֎;"""$tP +==ŋm?h9sjeCq477{{{ G===ec*߶mہD'NxhX񚚚ڊ FSSS\)g@TV9}G tڍl#چYY%H IDATbUDL gȲ[fddH>% `< b/_nZ[[{mBa0ٯ7jiiYfo&2>wsihh I `219 'gNp8YĚjj5ӪkkL5y¬e%ɹi9"6_-uptcR\`X'OܵkWee$S%%%,_|޽q/`Dv}3:2***/է0ϟ߱6kɒ%f銋 oGF @@.%ދ?ypz#C3#{oFDF#L<`+YZn4ҤϥkֆwX=UTDqjx\l IMMp*6lggw7(.YGGT gg甔⠒+k2@o-a>H___Gv, ⢯^$2?`]JOV~@_ W 㬭#""$(00911_bP򧤤Zikk# C77\ N?vƍɊ dٳDG؎ݸdZ][c^0xw@O7 /TIII...,3O<n~:*++#jii]redEuٰ"8 eJcAde2sGK!!W _tzPPPZZچ h4$S 0[[`_ *(9s'@**yfpٷ~f oܸNVT ˚;H2}T?#8pv"xS )]]ݐT/// bvvvgΜ,'ϟ?nZXXhjj,^zuss3q2..ndE2捿KKJ#mFxk)L;Md徚K'%+k\TT윘/ (yc2jjj666C.>e˖`{ dܫWEqwnGt<H2È#RS22rb%''K8URRK@@@iiiPrq)++8βe8 2iddDJT ._SXNPht]/ ĐJT /UuUx<Y!A@a 6h4Iaaa"PT(雖FFF&&&$C \xQd|ҥ.\ %* 2s^dGzM7fHVWB1 W0ꆄzyyI8;skAY Ӛ\322222"1=zϏfFVT _?ImiinkkIb<]143205n )IRA:<<<""N©%6,iO88z蘘cǒȝgAbKUL!nf\ +X,VrrrhhS%%%Klr Y ʊVVVL&xoڴ)88XibccGIR\ A G&kUUTXH rt:=(((--mÆ 4MAXXmpp0OVrrr$ @< dTKKKjjpSGGghٽ*6y_~Ed3""BOO@Njhhn2TÍI{::M>Ӿ^j@!W EWW7$$$55K©lvppݙ3gDFѡC]&aByȨt.ĉT]Pf ~":)QH  ` ]_Z*/7_b(*, n 4#n*.j~\Y[[GDDI8UQQQ``sbbG,[,::Z³HA9 뚚T*NWSU2Dwa#mmlǎ;^UM5g_<aNN/s|>m--mS#Fc;f$ff#Yf'^rܸӜf:͘E%| E 椧?}S򪸼`3 Fec;f3]54$?u*+bNJܗ/jj2ƍ0m&&( )eM}S#"SfYYY7;/ @JNN>y]*++%*))e{56W+[paxx$ ʳ@洶>zHammMbO%.]e˖۷vc||˗{u>G iʳ@<{g*:i$% g||_uu'N%+*PT|>?offef ;AP {wqGtNyЋ" ;͢P(׮ߺqr _?:vioOGF\q񓂿8}$ K@p;WOS]_T[ "~kkkvj +!!!;"""$f?~|f/^|ƍ &3KKKll, @<|T+ADEUUUNNp\WWx鯿־|2j3k}@ ^fQ('uHQ}n^7'ݷoOu2)Ά=]A>q9\&C]Qe79RAgmmagg'TEEEΉ]~k4gJKKg椤$f@xNg =sdJN^>|ߟmw-pС111f")(PXϙ5A}qݕgYJ8ٰ_&$}v/r2$Hl|PL%G "ASR~iEj8X,VrrrhhS%%%vԩS7rWVV?| 9NAAQ`_xAv8!Ȑzq㔕IGj>} ZYY]zuĈ ,~w56/O!ss ?r+kSAz޳.]R71ki!n*+e2M""WbAAA-ڳgy<^aaa.]ڲeUUt2VVVvsspBfMMo޼9f̘f N( UIIOBmm-"+ 0Hdff 7̌IG:x<ڵkO:%2>}wM/a%Ce" .նV::ۺ-**ʖldI~kz$a'9W>O>."$=o9 ;Ĥ_fj{0n0M--MMm:]SVև䤄woD6Aj?vY?e9|}}}UUŧŒo߄zyH3ݳA- ַE}MTUUrrAxtla8gWivddf` 5e4)ҒDqX 2#=uw*((/ h4Z?sAC455326QWWWVQ)..ea/~yُsb0duu쫯sƐP++6QWW߿ʕ+7nÎLUYY}3gl۶m4!>~x^yAO<:O\.Uu d6<2 B(ςNFGGf0mTUU͟?LJ/~%Jn~r"e{zk\UV2/edddUUM9HD-L3O]3YEEm55 _zC{fL{Y{-"?n9g5 .6f:t `0"S83 %a{'6|&''Kpݻk_io;&UKU&T[SK.d}(/n8W`nnud?.Yرc>|8A&L&??ܼ#) HNN.)) mmmej,...YK.y&j@3&:RSߋfQŅ&OLmVdd}'V0ܺq=ގ ^| Oor,Aoz?|Eh%/W~pMU ]Ip9<<ȑ#*,,lы/522255 //oҤI?~ ݳzeee}'$333!!/O?m޼@JJ[SݿlZ ̚yKgϟs\q4cbvE^3 >mc\n.v=kS>o}$LM]###7,+31.5j*Eq:`,[l;v8vX}}}4^tΝ;|SRR6HOO=zt@@@޽[3T/mvRyUUIYY@7,É-*##cccCmJ=y䜜rPJJj_|UY@wڬ??8(OK}obWGWRRpءi-&%ܿ=yLq9d|@gu3ZZ:aIA1URIAY:Mm)r8W+WnܸÇrMZ!-ݿJHODy770q 0a_mիWQbu |21oH&yţ/0 Μ:*Ϝ+L'h5.)̥*Vo˳pDÇVVV233>xʰܜl%0vgs>|A 89 ?}*,T &sskۿd%gFF}$vh+rvv?rHS:ǏGtN(ʈcYCS%j|+^Xͳ{n///EM__w![D(`PFVv'g!(hۥ˅/nD[GW0 ĨkhWɮggQO*K+yCah[bGǹ9lٲwޭ]VZZZHNNvqq!߀Z#%%dee=[]]݅ """\(Orr2y߻/SRRʣF:y򤊊ʼy СD:L08|Hu -'٨ KKKcie=d|!O,ҷffy˷wݠ#^ " SO3C&B+qPWW߿ʕ+7nCϟc (++KH'AG&|ׯE^%*:::䈮)S (--=}tUUE@.F08tg + kjj$G`Tk]O`B2GOżk%G5jfiտ- @|ͽ>|hee%߾}F9rrѼe ;8*--y gaaѫ_Z+))cii &ܿ CCCgggrgmfff m |2p23ҒSS*++de{cbn__߀FQ)AĐ쏕 M444>.HXk&Y]]0dU,[S>F*p9<<ܹs[l),, ,Xb,@JJJQQo8pf;8CSLMME2{F)X =\ѧB^E,䓑O|| ~P]BcB M-#$<+B?|0|UU0 A_xƋ}o%]NaV^?jЧslٲӧررc"[w 6ɉjNvCyHZee%yYC---CCCEFFNJӭ)''?sg Q[pPbWUU)4&Lĭ _{Ν=j҄ >5VRS~9 $PŐaO>bԫˋʒ";US9䈽?% @2߿rʍ7>|PTӦ$RtMwȔp&buƕ֯n]VD;)VUIѥ1eOa``d@^wpokSBX]~9bdlleU @ͽ۸8{֮ŗa@N^ph, nM-+-p&ێ 5 IDAT֬Zukh Vk;+yXSU9U()(&GƎs* @✝Ï9! cbbwA#GH>>sՠ+NrZ(009998z6{ʕ8I7mMN.;dzw< ,n5$+gADR߇ '' K8!ƌ266)({.-d^}!1 {7^s3\#\20< Eܹs/]$ڬ'/]]ΧҠBR?Y v8ڔz[ؘJ' o++J>HH khI> \njJrTD}" >$fzV]Y-4kGDѧ ?&5MAP4K544Aj¢% %%~!+++88xΝ'NTVVџ~u+˕gbYCj99 \{$ž Ξ@[GOtPWAK+!m:/=}ars͊r QSYsVA?kx#sTTI ֐c_TVV\HII͙HUU]  92| q6@ԩЈ{=z455DPUYYI^P[[HAƽMn\y%|D%7'[0h=FtP]]_OKfߺqOJLhx\1)Q11f%l!o,$pLBzK}!--MRrhYX$O>˖-;sLJJJbbӧ-[fll,Ýu-[C ?yum'MGLuR陜laaahhx7oPt˛ˍ`0 6%k[[d0EE*3頠eec$ to9f{ǽ㋛[ 67dU0 ={đ2r2́]ǎxNeA FTd8ׯ&w|Y?>&}g]sl‹?WAhIS"t}ӧA999ϟ? % q_ۿyz($qpȺ/}hyIF۷o_MMɓ'/]t%&9zhWWW777333ByQzz:yYÁbYCx-Xrݟ 驟MxMU5$NJ!Jqq3.mq-ma$tc)sgydf7o9¹,r1AQ#wPRa2)?Wf{ 2\$Gf̘1c RϟGEEuա n$L@4JCC}CB|CBݶ~V$1 hÇqI }}}}}}777WW1cS)4Y .qqqp J _ >6'MA5YUs飇(//gm)LH,ct B6}6N|>]pO䠴?͞ Uk֟;wII1y'>uoL;|`׳bG||VAAaג~斃 WSS_l>7^qqќO]r޾Ug8W"j={ZΝ黾^@pܘG!!CB^DF֓/ 뙵Y *h4ډ'#G9rDNNѱV~b= , JKKcYC}p;7 >qyZY{N>zx##c5uʒ⬏aa_>,$01+|p?픖.M~+֊.3._2d2>yiG6]|Np҂Wyj:_?; >ZY{Įgg漏JJHM&,QUUoNpyS*zصϗ%U'9W=T+???<]|ܻ=뜆  ffo,|jkkz۽Հl24x5ՂOlQo=ե a-_yK# }=} Xu b^ҥ:FZF:*ZjZrL4Q__ǪbTTM˩#`0]O"m< 3#=-d"7'ۍ?\N'9WΝ;srrO>VΰAeEů-g@rzFY|MٱcGiiǛ&??͛7oޔutttqqqssX ʳ@cbbxC---ccc ]=ϚZX dͬ.gȱ**wT}[yYw$"^0v䩳VnhSݵCBoTVVټe8ӄd{]vo?!hQN޹uhs٩Y٩ =[ɪ)f Y &w}͞^ݽ_w*NrGOO+77Ν;IIIqqYw i/$*@0('00իHjxh4ڡCTUUw,oƍZZZGpwwWW @y^tt4yYAQYX0xiITQ2rr:v. z÷}5tD@Mߜ>y/ջ:z쫢pʴ qm]y}YLpqcbfZں.4 9~䑜G]e2EUus:tPa%%jk6rof]lںħf5eǎABNqvvvww1bDg@yXvvv^^ohii)//Oa> ȸc\KK(&S^OWjJrkp?O_< CYY9VrG|eg<5rS}&稭Y>|#GpCYY511[jb"48MDU4EEI+'J +dZd?bqV ysnfkCsLj 4CVF\VFf!!aa!_G{G^Sǽ{5UUΘAa]ǜ/ffֶK(2Ax89?uJ UP\1egthx-|ʩ;ӧmݑfGFFFFFٳGAAo߾+Vkn8g(X7o[F]]O|2?o۽pʿuw12=kт{OhbI%Lϝ=j~Zm(`pP奋f> O:2mMOs'M훸ط))Ie~b$'גn;8h0߳JYZ`i5ߴ'##s^G'_Ov{}}})>Ԟ+h %%cǎ%yDijׯ^}[[Wrf)SN+ihH}" |_|!%%5h ggg''1cHs Bmmmi}[ѻoco!1ёii)y55 CUM]MMwG9omSJQc/sw׌ r8:_~=Չ@53τXo݌` JD^n· >i`hm`h׫h-[ w qS^c׼# DQ^ё z6;&!!4"ӧ223&N:;y8(<<>99+/ZVFF]U_vv 0ݓs܄wIIiiIUUuJ z} kۿss:ζ*(*J i~B_OI)(*bZjjf{Opppqt4o_J? ~lyFe+Qtttt_{Xt+W%5R]]cWK~9io )#% AN߮/, x *3g~{ڼlFFO_/#.>-__vUUsr^qs_7lputlM rhP3{8tyd$_<'//'//&!֣G4mĉׯ712je2T[뒷7ܽAmiiqi'Ϟ8wNfZ1]ӞU[k6ʂNӧO7o޼y&ѣC< Df%NljjJa>ͷo)-R&tZ²M'ܽpDp_-b0k1fv.E#޷+23))v?EjS9ݎB^.W@`0u|gJf̚dgXKJ?v9I) ih4b_y֋֔gEjHTR_/߼!71_9#X#mbeƏ6?t4.di{A5,֌/͚v)-/_eˍZ^l޼b$p<ѣknf%%5Sյ-⻦=MfVQQJЍ m>s, KIIٵkf ZoNaud7ş$ǦUT3LyyU MM--]]]=]=}C#ccC/ma;v yHx؋Ғb^/3~C;nBeV}ۏ͇X:',J_sLUV"7m_S yZkղk 'Ǐ#/"#ϙ⎂~¢'/CUnYYU5O8'_h9s@K淬cZm#%%ecaѦ]Ͽ64"?rݻ?Cv%KZ[l$]q͇۴׸+ֻE״'`a11AA!!Qqqꌀ2Z...nnn666w|2gW311QQQ0 dtt:jjgΝ>ʻ[dF:8p"}ACJLl y2gGZ{^f?nIanMqaxʡE^DFX[PI8!t cbJQ^FhYFN^ףg |>mկOygƫ@vt ?<]Jmj**tt܂l v>lݯt7$,'U[;iٲ^5FSVT#ǻǻ^.7$[ϟn}I9={ٳ-[꺸,_|T< :YQQѲg߻-o=Msۏ 11纏qJR7#>w}ϙCmDKNVvϟ"i?f4y|u y)XmϦY3l6;+7'0-**[~玊RYXZ;dKSS--y99VmmeuuFVVbJfVkM{~171w򤢼9μuxf0Ygn{nkn5tFI bӞ=YtgРATTU%GG? } t.nv'.Fikh 07 ~ ߽...'N4oB@< :.--77>vȖ;io+ |VsuuZ:w/_29sş(YU&E.""8,lΤI&>5\VmL3;5.Q9xf7>./^+hScmKWw53]]], <[XOACB]t:mLL򃭭[[?SI]B5Nh4V*.- -(:5رcNA tHUUU%8y_S ڥlǡCCZi: 3ǎx%)pm[$gO fZյ-RHMEe~|,ȶm5_! ݉39|AfʳCbbbx  5@ڿFc+<I@Θ=s\)Ѷ]̡g_<2"lSVJ~tuW6xLhYYIii`eA}|2Ców7v׌3f3/ĉ IDATwgrDUY榦uڵRr篿aklZ)(*zƔ実 r¾}.ZnyM;>fϾ}X^X3g6,_nRtQvvv6l ͽrʲeˌNkCy_NNN d:7'IG`rRBz5gϛ_oQv*YYMrxoگ6>v>FѾ_󭯝:*]kafv6jKVu]O\OEܲENVyv|:9r?q=x`M~QNde'|ZQ˗/w9rH]pIm'555(ZT\)#=5#=uK ttiniĴR]-411.M'1M66a|=]llٓrsA)),_%UYQnيz֬ZȪrsfLpM{!Hر]#GBCzqPXya5t]}}㨸R5;>n\eajАoo3cc7aU\B()-6mݸbE+g{ AL9;=|РwKШIrr9h/]] ^$*.-YKfV7osgvOص-vr -͞]U]"*' CNN{B`gg>|pc,SBBjx,%%eccCm>Т7Qüܜ7O~||V);|윇4,<QGl=è>O}摓_,֚ >.^g2Fx)/ͦ?!Čdٛ:5%y}htr"gČ,Ç$R C3U _Ro-'i +INOZY%w $u*+,w٣[,:~~I*҄Jk AÇw$rKySS"vwd®n״K#Js@VJHIqYPZ]HJJZ]리AAȑ22cHefgKLk170󭱬}|J^hD̙Μ1m]6#$Evp®n״hVVVV?^]U]"*' ிfv6ߖQqqc}r|]0!!%q7vvvÇn׿P\h?򆖖\L=Fu F3knלDV->E ϙ&5_|Kw=]DC?wvrMkwWǫ7cJUnГM0_%Gt)oʪ)&4<05645OIiixĉc0Bs`ɕUTcD7m;i֮&>r=s̬KxC%uw$%S#.|/Mֻ[^BtvppvpQqqoz:yGgϪReddxzzkdeeG2qD sVByALL mxܧOjRHOo6lhdho7msiS<]=`m30h^^*+y&s¨Q4Gs8O.>aXVQ@ݛA]E%7,p8RRRmz!Ť"**mLB][e۷|?xܴTTMzjX׬iylr.ۑֻ[^ӞFo׿W?((gyo-zxFOJOO?~|ff&AFFFnnnnnnƍê] ~ Ak}𡸸78p`j/:7S/|w6f.ZTHj?!h`##Ed ^ܨ8Nؙ-B>fϾ}sJuWaaaW\YlY޽NY*III.qMӭGѣ ==+WIv5nlܺ`޴ rPEE?&zL*ncP[w~r{ z&=mC#WoxpחǍw 55>%ӧtakx= p8)I9|kإdfZ﷜ < K~`g!/]-)QWU43: QTT: i."%%7433cRT ii9s̙3D};mr䠾k-ʪ1z8tgqqoyA.}ۦ_v@|O+0<=m"b܈j**|;Θ8gyrwiUu5/3޾;980kjx{~~E%%ꪪ͛|O'V+&^SVUWݺ Xiӹoϳ(I#&#ʳ? Wo~|.E"._]-)tu(ςp8wz0lI7_{N~#5J dde` S\ ( z3fIiio ]Gnj5:VF[ Xxt޽g6p^^˖df+is_[[#usXpcqML3端n7Kqrut׏r֬A&ZnyMCy4'==1Fn{ [aA̩!|V;yIFVz:k[vc#=\SS) z1Æ#uu`Y07AKgjelެCUT8͟_WVmp<ԉ_UVTlA%)'wa]--r0+7wwq8Ð&+f՜|d55lHn{h3vu~+"ccڱ~w >Յ-@W'&SFFFm$ϝ呙߼e皯6PQ Z:߭_U__ ffO8ܥCSt{4mgmڳG)eEE'f:]~MRW?w…䢥ںG;uyH+SS- Ϟ Nέ<սvx//rړg~=zt5|7ѭGtmmM UUTʪʪ켼wII))5,As.կ6|ćAA KGGC]!-]^Y_X}T\\YEA._֨ztk ]ʳIIII-,,{; z{3o!UYX.PWXl~uu/XTi'N_v?}:hi[A3m>s޽lҚ N / va߾7Pb~ԩN ezFP>NZWBjjSOuwKaqCӧOYYYu+s۽}ߞ\.u4 7{rh^ݽh}: !,Hz##[KQMMEI=rdGsf>t]_ׯHHKKXZ996Fk屒 }ۑg|̣ϓ#O055U&O$8EDDn~~33Z[X 4hoߊ#yQY5fȇ/""JJLA^lؠANͧ-|qM;:hOUp89YYy&SCMMG[[MEMM.nD ̆:::Ԧ]H`y(EiJ= j߿?tu55_\s/nxMTt6ʪ\j;>sgO:{񆒒 UHF33hTLu")ٺܻ[^S<4ryC K͙,X5y̫6s=z5C&Pdff60F6汪3X0ץ'Mkede%&cF+(*p~iظ(lQmM͜ iqȟ\ 'Nsן혶[+NHJԞF#:T=4:p(:Y***R@R'9ubQpaBk UQQ=wo:>gAC$U]]-yϕ 222=jpWZyy9?8gv@礬L9_]qI&j? t%4N[E!:è167 ly,Od{wT6pX= KguAbiQX(v u#Gݽkkq Bq1!a4VdVcP@"*$혱wwV[?fߓ_4hN;z܆ 'v"h>h]|Yf+nnn (QπCZncyQ rD"֪u6wЁ_ e}Z@P(G/**ba`;ww0cўD"taǎl\Me2YXѣ0Tj:8c0Ԉ Jժw;ϱi"&E*^ IDATZ:ڳ H$/_c777PhTcbgq^JV@@CNtiiia.] Ydddw'R)j:@3A{VKW]]-\:+::͍-/=cR١Cq ڳZ4^/\:SNr\|-33tP(X: p ڳZ.nFEEОr]zU՚:v(n>@3C{V e4:( ڳZrӱ\.g,hjRSS"&4KgD%%%cTکS'q%ڳZ4qppY-NIII~~yY@#=4nZ\E]vn۶-Y͍F)//7d`qZ,ڳΆ BdP3Z8ڳVVMR4$$D|f%??|RDLhhjVrss-v鬬۷/]TDV{)F#v".M0< 1'[lYRRRRR҅ JKK%y>}:))IIIԩSݻw;t)))O æMN hg5EEEztT*Ǚ^xSh&LrIk׮M0!%%D"1% YI^^888X* L.\; db'՞%b&$g5t,JYč7Ǿ BdHhj6 l4g5@3`B{VsPQQQYYi:@B{Vp 󱟟\.1&g5,v6ڳBq֭EY.OՖe2Y@@0=}}}r0= ۳EY.|/b&hrmFQVgM AJKKkjjL*J|8Fh*++JG֭RIj=˵tVMM͙3gO:Vj\.mժUHHHN:wܵk׎;:);;{޽LII(-- СC7hРhf۷oÇO<&R"(vgС&M k*55u޽ǎKMM˫T(ZҥGH+]~}tҕ+WJKKu:pp>l0ƘpLMM=ydJJJJJJjjjNNNYYYyy`0ٽ{={EDD8jv@g6a!W^fMNN-6lر=P```}3?+o4kZ^^^^^lO2eJ~lE׏3K:.=====}ƍ_Ǝk {ٳg7oldffm־@=˵ W:u:W[nݺ522r_7o^RRRO4n%555SƝ;wܹ'XxqFu^{\[t7o޼y>t~ʕgyfϞ=6j~Ȑ! p>|ѣGT"HiWSShUXX8lذ~^YBqqq\]]=wܱcַ7,>>޾ /9r YU\\L>70ĉv?,Ç_v ?fI$}6rxhra}<== ;viΆj,w3T*5j}ٳg'LPZZjǵYYYÆ ۺuݳu]Ǐ˗'$$P_>AО@E{ v{ 4̈́ <ؐzԐ;˹s͛W߫rssG5_^x8H~vׯs&&vpyzgܹ~RLL#<2bĈh???^hʮ\rܹ3g۷/;;[bs{h>}z{nz̘1#G iӦF)..NIIp322$vl(Juֿ^zu)*** __v۶m;r׮]O1骪&MtFDD?~|vڴiSTTt;v_˖̘15N>ު7K.w޽M6nnnO޺ukJJ4o>AK;v4hPcbb:w O>}ȑzMY.L؞ժU+/?ˠ'|# #(((**ʼ`bbW_}5tP[Z|O?dvڽ[ 2UƏo:HMM/C)/jժ}C yWm6{lSY-o֞={l7LLL{xxϗ`HHHHHȀ^y>w5Si{-[{キzԩUUUV_0a’%K""",_JHHxSRR>Zs5c">AK/^+ʀΝ;rwwc g0憶gi4-G|͘1clClllll-gfeeV_zG?SJuۛDGGe:!Lf7w߽{x˭~K.u79wܒ%K,۶mc9ww7x#**'?iKs˭(+V}:u|x~W_,X`ٚRvѧO^>eʔw}V<11ѣZVpB˸T*]zuжm 6ƞ;wK @Djuqߓ^ʲ-+**]˾v4Jmvֳ6WLv ў媄Kgyyyꔛk޽cX~e|ذaM7d„  k۶g:ujݖwyǖ |w)))~m=ׯϷ⋶f)^ g4/^8w௩)F9}Xvfx=;zv=;:DF 2Omp je;j=U ۳|||lPV[+++VZUSSS+rʦ֛e`ٞu^|r`.]ygwww3gs=W+e˖Eqڵk-AAA ,}N'p&p2دͭq-ѝxόwq6^gje;j ;ة|\֭[[=ꀜFն;:v.G뮻,u\6md?~}T*U`fffN-Ηw!jժ^X>A3ee]7|=4v(*.YnZ%g2qqBCC-֭2e2H$ɉ'<䓎ᢢBBB,7LNNnӦͭڷoUDTN< jժwޖrǏ^/X:硇-O۶n:|WpsZ>įoJT)2D"UU K Jnddf[IAA>;vhӘY|e{(**={v``g`ǎrFҥK,ˈÇ-#F#X[5X_~۷cI'p‚_ξv꫾mDvmw;{{ aN]tBaU6O&yѩ3]Odje;jlD{K2aڳ&MzZ񊊊'>_', hmXePPǏ[l_AAA[Y훽e ۅ߬RZԮKqΘz=2ؼ+98^k=7 p҃ 8ԝZَZў咴Zy R)u믿aˮ#lٲ5k7ұ J['NZَZaΆfӧOw8`0۷oԩϿp7/,,;QR([ƍ:^ua7PaaanL7e!4C~ kKc{h̺ ydmkؿcƘ!Y.gK~m[/_ާO{׮]MO׷Y:W}5*[nu=X%4IX+-j&f/_}K\MrqF>v ؇,TQQa>o,ٳfͲjT-{0aѣ;VjF'&L:a^o5=<<3fE'h E7 Y`>~J )bX+de]qBVVng$e'''K[1b;SSBnnnV®VԱnոFitleuŦF'p8NſW#NK&wށ paDׯ&ǴZ 8pGgkw}76l駟:u [ouůZ&[VjruxjVKoUbGM7}۵ܜa}׈GL%_0GJJ7Ǟ%Zhڳ\p@J۪TSN:ȑ#~ƍؔ YhKV/i"mCt5쁁2rƞx xk}Ǻ~ځ_ #!v7pH}==9}*o쀘bdBlG@멮6w丹5ƾi믓yRyӖ-[hWT .X]JڋBBB,ϟwBZ 99 ;y®e7[c SʤWyLOuu; D{jc9J?O8 F՗-u袬dcߝ3e0++iG'pIi)Hm‚ǬIGZ G{lبY&}ʕ+ܬlo>$2!'Nwccc-{MMs0ڞ&8hܹc0&9QY}yyyN}ĈŸ~ u2XTT"4Oi)n^q=C2XXxp|JJO:!J&v 8YG؞R6Yڵkgζ Vd_:<1q=? qZ/[ EDDX]@%֠ ᛚC;m߭íNE>7m{Q2VV!hr=%HRȑ#- ;YŋMDC%%%=R|-tZ=z nٲř9Ԣl9M'hʋ #}FYWt}G Wdݰ%QQ+Q+(gg IDATVk>vf{D"iӦYu:ݬY ԩS-jzܹNg9sfNN00`eNKKsJ2XZZj>A@9{dMMcsQBBBXXeoYxqc޷o{2^TTt}6v&Vs/f rO@oZ(7MAś3wje;j,#=~Ba*##cݖ[=޲Һu&O| ;hJw/-X`ܹ6gV? gΜ1bDRRRn2|px?{v___6ol?A݌FcVj+Zdn+M/] VEmlv 8YGכC\\ܸq+Zm%/^,T|}}zy䑱cZ}i˖-ݺu[dIEEmgؘM:uV_rJ[nUPP_O0j3Uݺu_j̸ђd?՗/_>nܸs5rk׮oYYYA'[nua߹R j"b>uk<4 闝 dOs%lϲc}=s#G=zt^:ua>SV_r%11q֭[n_W4hPvvK%%%o=f̘GGGxzzVWW\t… 2m?#}!J׬Y3p|WSRRfϞꫯ4hݻwteeeEEEG={hH$wy GٱcKz~ŊV8pqtҦMNWZZuҥ'O^p"<˖-Ȱ|i߾}ݻw٥K֭[JRDR^^h_qy#4dsrrfϞ=q=zX]aD'ϵahMv:N%bz8gjje;j,ճ;f>ڹsΝ;Bajڏ%s-̂7mt]wY=lӦM6mm/^9" ۸qw߭hPZZk׮]vxzuJ׏7N|jjj>|aY_+W;v`|h4>}Ӎ4C=j*xQQoo{xx u|b} beb6%GXQ2//ZَZbsC#욲=ĉujZ춽Ydҥ=w;w G$''7m[nmӦCVv4oo;vj7I1b| ԣFӧO'TVVjժv(ȿ ůX? wV6Vhr18%Jry}pՕEP|>;2Fj`РAǎ6lXoeGs͛ѷ(sFZJ;D'/]q|oPV;mjje;j,#\ʎ,D w)S]vX"$$]b{Xm۶ Oyyd_=11сhIz?w۷GDD8*k:{G'*mpRT7uk++65$;OuuؾN>O?OnJe||O<1i$;H$Rtڴi _~_|q)/ӧɓ b7xVľ޵CZjĈN2eݺuk֬9|h>}L0aʔ)ڤ[n?ɓ'/_y J 0dȐ;#:: 5ܹsVZ|yzzz}/wss֭[lll\\m{9qĂ 6o,E"jժ]v6G'Q6RLlteiX]]m0$D{cڳ|}}GINN,(((..^^^~~~ݻwٳg||O߂J5k֬Y._Geffꊊ \Ӷm=z߿{ZTgСCuȭ$R6mڴirrr8pK.eff޸q\yxxx{{{{{nݺK.]v4hP֭@~V^]UUu#G>}:###77J&) oooΝ;w޽W^ *{˗/_~]hZTf_PPPhhhTTTNbbbun߼|M^^ޖ-[Ϝ9hlٰ=ANW&qrr+wh$ZizBlGO $о,3'eӧӄNf̘1cƈ2D"u3f̘1cC&APJ$桾Z"iutKRI+3Q+Q+@lxb;IR3JuSӌG_JӾQ+Q+@gڳjm[U^)V&6VܔiSS+Q+@gڳZ ybeb"0/iSS+Q+@ga{@ צMpXRh J[pv 8Y.ճ@ 6beb3 vbD{asCH073GӋMS wpv 8Y.,m|}Mnfԭ8\]n*v#6;D{= @S.aJbXVaDdBpfv 8 Y.,ݺ gӚ>tnjVg6je;j,#ljL{0*ju4W/fCLַ_sVVQhr12`0 @SP(zFNF_#V>rba1Z:9je;j,C{@-0)*MI(b>g^F .J&v 8Y.F.kjܯ8_xx=#?VY^)V>Ta}DDLL[ߨpڳ\ gXp~Xdj9kbIA02rXPVNC{8x9L;1+D}憞:W|$Ԫ>4јn$v*JDЬОbh Ο2G'("[*a;L&?U]]o剝p='O2dHxx؉`sCC{UcƎF**ܤDɧF_-japmۊ9je;j.رcfڳ\\.7Ӟ t;;w&\[߰;Ua$,< Uv \Vz&vh>ŰzHrݸQhf]Ӳo?us{vMmzb,BnZَZ)--5^tːd-V>Y.M*RFT*;)B9'W|gLk h˵;ܜzMd ?ƞ^~hTV p%FB_e87?t:4AEL+D")5.5.]ϿC#J<ѵ[F>v ؇,#ܻZL'<.&UUl;/:kk8?Vh*qTzZَZ;Q* gXէoϯZ)܍h43'ԯTM3mU]PR+.z2K*)鬮W~rke4////7keW!‚4-R@3G{Q*cV.]cf͞տW e%en;pqc)J\8vjkzyy靻tNfՇS'Õ˃ mx*Qp"DkoV\+R [,@Ӓy,42ڳ\(< /.^8or7uA m߱OtF_5'TJjbBk  T*w٤Q~|_Q i{y H4{lӣ_@DZI$O<^SSc>umXX# ؎,C{<<=̡ض0o";vǷ:hR$<}Hm==< J[VQRPRRP|#;?7#ZwKIii P(FǏ=fL&];{<4 RxBke2fI$ׯ~ޝ6=77t"fR{oӥw/Hj5/g_mm.&KKOWWK$L8顀z'왓K'vvԪQJΚ#Y3n}{:+` ~WϢ= FFz%44 So(ee$V3-ޣ+ts戻J~Ös^ڭyg=G^X6[jucr>Wngqee/Vf0Wdv-88400gٟx>YBVdW:w!ӓO͞k l,q??g)(8,YD ԁ ]J2Ӟ`#Fvja^=f܅s)Roǧsn1{8S훯d^rXҍr[}byN7Vbz WH$=m?m9[u=z֯Lczzz=l?$I^nNV֕E7ZmR_]-wsPy|mў咄VTT KcEuvȝ}}'?0r'y}[C'# ^~^.v~& Y.I؞YuKWg$a{Vee4}8ޣxQnaex;s:%Fz3oXH$ɏw_RѤ5Xԍ43ek7m'ΝpCرbhrI,VFvja3JecS7_ɼ5DPU8sF^u߾y+cGлI=%T*Ba:4_eYF|8…]ƌ3oj'sG;/PEWJL;'ٲeg}v2hrU4MgLz~ՊO?MH)i3nYҒF-~ink(w #y9{E \> '~r嵜Sg.+yGU96lH7A}M.+vRN2a„Ԯ]knڴi7n,)߇]Y ck׮2l3suǟx<,+ӬFvgN'~}aW~Mg[C!-۰v֟ʧ>W9x FuѢT~~/_)VzҹW:q̛7'NyREf̘nݺ{wٲeiiibgz=Uz@>#0QuF__ɏ<&^\Xnk䓕Mj[Cޯ᫯i"[RYY_]^oҌFcs|qvD5v#FN…;V}kWyF O*._|ƌaeeΝ;_xᅘ]Λ7o&ۢ=Uy{ "f%8zad9Nd|Ǿv֟t¼upWڶB!o0D8s.`0?~ʕR32,ޜ7oL3.bEܹ_|Z5YDPtzV˟~ĉCBB瞏?8%%E q[g*___qii`1G=z:|i{ϖ-v¼uNx nq 2u|W^ .|رƍڵӒUTQQ4)_Fаxi:OH%飥\(4m nqk+>W\|ݻQ-[Vެ@o?CjIvn &͚պ_&mɒCfI̙҅NH%He˖˶\UUgϞ_~W^aaa%- uR3< d>hok ͯK ݮr/$߱cv@@7Sl=qBC@//Qન>|!-FV׵'MQaa䑬%w{Z ?'. ۶iS7.snΡP(oOիidyVYSOGNjkoL6nNw )zfd eAD.#t%%2J:BԔUYYYmmmׯ_qƽ{ +**t>j(GGQF)XɷoϯjiiQWWӳwww:uÔ>yCCɧ_=C~=QZZx7ߖu-97 U, Z|WYCWQYYo>|877WSg,_ww%=ΣGͤ/kkM {x]!J,+槟~!--1cL8ʪ Ʈ[oO?ݼysoc,`0{ZZZw}'?~ Zf[׿wEŷ aɇNc=&OS,ACZ8x7ggh5XQɗFdT$%%=z… "|/NFIo}zx|w/yCBz(X{}|zxѣ ܼp$>9$/,&l*,ܹO%GiuH5U,^v''f;"/_&$$$$$9;;aFZVWW1{Y\]]fuۻw^h]NN4*sa{{+VtD~iooЪ*D(U*}S\| G\K)w20IV{kvttF444bU?lаS-+kkһ"+)aX/_~ zf 3}Z̡\}onqUtn.̖րŋɶP?S &# ˿T.ܩeU/^ 120p1yhc޵}V? m"#͢P(zz ˔n釯p>!!ٓ!rUdd[bPiTyMM 窪Ç7`}_d2%foիWnbbsnܸ0qD&*}t=voAXX ~s|95n;{~"ݍ?kkH&w9(+cǎ(Tw0! HaiiyU snf|33{\,?OOg~ysw #+(h:,21%e_~>8y@OǨxG 2e(sSS-'7'[Og::Bg|RyǍHYe偄5w0&fIf&&)' [ Mhg̘1a F#ɳg]JO?nkL&O631qpͻw0}t{,%`ff˗,knn ="J$((]cTTT?~ҤI L6T| H䑈./\eyN& ݽsҴ5$ yg ɗZŸGg9[} #FX`ѣ?~ѣR%HSSX@"MLN"!fu3nnnm}=`mL9)GWGsΝqY~~{_rV F:xaaKCC{ڵk{YrEP"{Rr E߈`e#>oߐo) :X7VP;D>y__II>$وΛ`dt_w$%/3?I<|d4/i~,>BsnQT __}5c3DAϝ;w;v1crĀ,e`NzZVV&Hf?ݜcggrʝ;w?~˗/_NJJJHHطo_|pB`6gΜ"ZYYEGG_v7neMMMdd40>b~An`dL]]#G~P'٫(}[C2nCHuU%{W2c``{mݺ5))h֭:] I'}=b#B:p+p).++,$fgbbqy9yd!F=]7e߾g'yP_?i~{nf?ӛ*_Z>n\rvpXx1yHJIer姝?M>do? vJb-JÆ \}܁}O Ə駟TVV;v,22JA)6g)K7_QQĝ_nݚգs}Gnݺtٳg{{{{{{Ϛ5k6l8pݻwo%87<~/tss{mڴJKΜ9#%CŷB?wTe+uuƆ#*֐,~A䑄?&=)K f͚>.8w\D )AϟwurSssWU$\ϵ45YO>156&IN.WRדEut7qN>40`ŢE\EJB\}[x+_Fv}U_ڼ*ݢiy͚Ρ ʺy͛===G\ =KvttTz*'O|WB277OMM=v옽l֌8%%e͚5]kjjnڴ_7H8PP@W|8-0xp[y#a;wޝF}Rm ɤP+HKK4i҉'Ň~(=a9M۷ɍm"y͑66Æ ;wss#VVA²2[4gyf'\ t;hr m-:vl<+)!Κ 8pdAwrrDYn޽K>CnQ״+axV((%JP(O>4H!Ie[f jii?~lO_p§On޼o<++֭[ T:T| Tv۷:rϲU K?XupϼwJK%«Rm Z|wZܰn}qW2d2cbbbcc%!DOOoʕZ} K~Zmm7n,[Pɗx?kjh,~뭽kԛ7 bX7oGfN*x`ZٳE 3%?[lrrUPWsNo^ N#Gl'^(7b-J* nI-?؍޾}ԛ7ۄ}/Æ޻l)(n~077#Kf*uTtzzzSL5jDCᐞ$KϪf2 I߿/fݹsҥK_(I\֭?Ə?T|Е?o'7ݳ?4tu`̤S{~{ 5$ ]xD1sN޽=$H++((̔j e۶mJ_U{Rj*o[,Kh{K f%l.4=+Z371~H{xP_7! 'mWZE.3F//Qg <c{`G>ljn764}i|q{"ݢ"=b#5sN+g\)*+"y-x@SS)ɥoNRzNSPPo5~x??`WWW$cHT:::ܟY,Vee|㑸={߹sfk.A{{+Vʕ+Džf+*}$L~K?W0=ZڹRf̌], ֐L-U8IfgHj5K>kh}iKJ yӼ&M\ "Թ##7a9ϟsp!t|S^"}qڵ+Wsh4Z֭e Uع A]]5E9a/;'Ϟ=/ *ݢBԊݸ1ʕ'/Sjjh8qAAŕ55R R릯?wݻw߼ys͞*9B?,aii? d_㮮!!!G.O<)8z敇 t~yEEIo]$ o̞'x>yI䑽{~˂?|m ɸ-#}YS+a0!!!QQQ {_~V|\<-Ua[_qy'O'MNr\k=hk~'ff|#ս]/~?{EĮE_J!TW[KˠsE>.|Mư!C",矋_?mۢ9sEh\ͥK2+** %7jҥ'N(**:vXdd " !NyXYYs8 jkk_|ibb"$#--sV^MRtW\'И?~osvvuxFFo!~ҤPL1[ȋC#acu0Y̮4D1xdgO=:Z߻kV򈣣4w3/9o?}Ԕ شĸWQ .---""DI_fSE)_Scd`0/8x^p0ù{ ׮ݺwb =NN%K8&:ASSsӧO9r,c!=Ky 8WPiҳv~9sdܸqCp[f&LOzqOR/_T~=*9s5^pӆ5]gEC[#bfқ,k&vtor'*je=8Wʋϳf{{odx[dfT @LfLLLlllWGWW?422R2'0ּko=U^^ҢMjAwssSWW}0򒑑!8.jfC (((o5P ., :::;^W/6FCC7*+*~?ru_>ERTרT*[7 (rp22)5_ʋϳθoL ._|_W _BEWP(8::-:3mښo!NNxmV~s46VVllr_q\)L>}|44M[Kw(^# eXGl>Lr\`QYoX؅F>^Kߢђ%} fоLM5Hb-Jmm??6ls5ZO={<5fΜIB,,2`}}}n8STTvUg IDAT⭦'h;1Ko+/d'9:9PqQNb/U]]ڪ!}Q4NpT*ux+aܸEɀc'>^=v#F{z $[|_W Gqqqk׮m -kkѣG'&& >zٳg-:2h{x#g:g%eYhkOloc F"=mm/WƝ7Up(J_TE)_SUCPƏ=~>`&߸qՋׯWxq777("&sb 4uT]]]y,e3lذ{+,,h :::x%ƌ#`䥫'!44TWjPG.USWikkJ_O̳96ԗjhh/iFAܿc9d/j37qXcU﫾P `DEE8qBBBBv1sL7o܇`̙b]=4mdr߼N"̕+ ׮N~wWX K+b {Lf)ȐAzBijh}?Ha0}ѣCfQ%VRUV bgb-J2cCyd’7SR/>oI'77wܹ&MZnu[Bz|So{{gTxBhy(/_ }$ϥMDŷ45e,1 :SZ$UZ&ryr+/=>ZZ:^*W"R d ---""D̚t:}aaaA<~)7AkfO]`AYYJE:8X \TV=ܬCĉgΞ6ΆA8ۓzW 7yCuE_A?`ч11w**+#"Ҏ!3rdيdo9="*((ٲexY4mӦMɼ&>=;3fL[23˫8ߗ.M;}: HJMm&}}T7nAHę˗0!odӥ!l7t $ |K'C󍜼t^nQҳ-ﰠ@uUK6);=M355YosqqouJJʦM:pX"$$OQӹM lӧ22*+y:gbq:;4wx.5ȩSs:ڡfGۿڱCokgaf6CYA, 9߄oafƗںe^i'f&&䑖/~QnQ*aÆ3L :::>}:zhy%&.7448dw'᭷ՕU)*}`0N8! !!!;v0cľ@@љ=m9PBBaq1PC]}v@D `ƌ-q[~9vbZٳ{ `Mdr˹}+0ť> w@PƢ? '?Do* -Jy<#>~^^ TVFDeD|htTE)_SPtHRNjjj#F-((KF! 0BVib02DSSSpI611:'L4I*}QZZZDDDIIxt„> }x\rzV^aRaЉfҳRQg瑤N;]~{4&6$լ42 L jv{iQQÇfGGȪU'O02;yԩÇ?y7fCVo(bI r1a^o[ cAź[5EJښOp>uؙ33{h7"^Tڵk)yr5O\T*-jʚ467/X2d*͒z>I/ wUjg2^ҥٯk|`gfFC t(k ճژ1c222/_,..Vm#F1/¾ ߙeggw-Ar%giS@W 333śN֯_VeeeA޽{dB;-[ҧgv37l}'t3D 8~ŋIKLݻW?OOSׯWTW .bѢ~~"^Qfx|N0rڥ׿ٳg|'Optbժ?7~45u7fCm򪪇yymmA 06:N.O6o?w깫W-|\\545Ux靜F >y}Jcu(k YlРAUUUÇ8P:99wL` Oϗnj#/&>vچ[[[ǻuڣGXl… Ż@hΜ+Eܠ }r 9s֭,ROCYtutD~Xg|1%^xJgn:/'Knc~z5_!ov3ёOGש9yb}JÊEھ]ϞfnN~ ߠb-JBsC%7zh*_e&J`VVJ7q2 ^Tyxxfeeo0(sBBB222z""77:(@ٳhaf6M'h>q^pp¾߽u˖wj4Zo'3kgq#Q(_|%:ZC]] =ꯩ</wbl* t(k YJNOOΎwX\\\ZZ*x3e "8~qYaee%Şr%WIp8{ >M4ĉM700G;@?bbCL8ljsLuss }2(:IIs>to#GZZkBٻy3s9WEDd: kko3J) =s󂃅~ڕϞub-J"BsCgggWZZ=w|ssɓ'o۶mҥzzz2ƍ|gΜ -,,<==_7}+VJ;BA0̘Xj"rww?t萵S/_>`.踒EacR/7gэ H==r̝:i 33USSSs5*wK;"}r?#N =da!-{cK׮UY]Ɔ#G>aB^R L΢ )++?v옳,T|*...nڵ ª:>>MQQ(K .K}}}}yG!afI"K|MSIǏ1O9{wQ)++7oq Qzyy -|_}hݺuVjrc>b0!!!QQQbfddd 7 TgcǦRX޽kdd$ߨD믹|L&3""޶meWv޽VcR+VOڵkWnn-[^G/<gϞUVikkTMMUVM6MTSZZZDDDIIxt$U7TRy Ғ٥@Qh4;F7HR!ZZZl6 ?nϩ:t˫MSN%&&N6ԔFTWW߽{711ɓ' cҥ;w,,,|ʕ+'Ntvv6mMMM 充>|xWP(񮮮Ղ-_<::uц:::퍍 ?u9A~~~'T ɌeX~!G__Bp&(BCCW!> 9HR-cƌ{.d޾}{ LDcǎݵkWddG;::Ν;w9Ơo߾ns]3, NHHt}}ŋ/^(999_]ŷ: 333śN֯_&O,--.@Rmll(bS9VVVvvvׯ:-doo^&&&Jd5~S8sSRR6m$,.777sss\ɓ (TREeee7nxzz*J ?pРAaʕgڵBHɀ۷ï_ǥOR+yF;M |C[G11'NBHHȎ; $UoVWW3r MMkii;P*HRQcǎmjjݺuݝF70[Ǐ|rozxx1UV3fҥEEE}\J<NNN>x_~Y^^.:MMMUUUo_Aڍeώ_<ŝJE%E HKK- N߾}{XXdB177W?G*JMMÃ\˗ɔcTbmmtܹQK͛wT3330uƊtVMM)""b׮]t:+P(%K/HPƍiӦ:U+&EJ/{-gN0d黡q!@nL rww'Yg.uuuOOt^ /_(#/~~~~~~/^LKK{qqqq}}=4000228p?ìԌgϞ744R(555mmm:nhhhfffaa1|p[[[QFijj= %K,Y"--֭[OF 3EI-d27lPRR" aaa @)!= 5|cǒK4663g{o"fhqKjeqU |||l"^ iӦdrSҳCF:::H(]3>8jX۷g޾5'#sqqouJJʦM$Cz$2|#@zzP(&~=_0zJj+ #$$$**ABBB222$×@55 &l ɩttt422o q3gE%3>.`F\EDD7No߾=,,LQTς. ƵknܸQ__/@JP(6}^[5eP'{L&sÆ bfggg#7 @l(dxG:::x555W^6l#z{B-mmSQQ~G~OEI-(((o:F[~}tt4 > =8pÇxl6C Օc Ar()S0h!k6447:>>MQ gH&N8|G1 8zyQQmll $H3xgN'9@..(1u W ٱcPYHς^066򪮮~Q}}=oᔖaÆYYYh49 }' -.P2-#(dG+/<~@R"""JJJěNӷo&٨Tҳ PRRL~O<2d͛7qA#GDI-s8~C@j IDAT)Uۯjc2111,K:dmm-ѸTҳ@T*ʪɓ'G󵵵---啧UZZjii)5Y,Ǐ%&q ڑGHU q#߰i_`{$w-'Lrug-%~ <<<33S4mjj@K6XQQQXXXSShKK 7O\___f577͝;wfffYիWƍRrUyku)RS.m+臨טJTmjϳ$(..nڵ MwsslTEw ,,,<<Fg *൴۷I6*r|H%eP(gUTTz<- Ԍ^366p8aaa'oDCCc֬YM Cz@K8ot @!= b2111bpww?t萵DPQH9kii|((jCJ}}Wnn.߸sdddhhK!= @@!= *((o:F[~}ttdPYHςbUWWWUU4778KSSNtmm^ݽF3gҳJjYM\\ڵkěnmm&٨Tҳ?jnn~eMMMMMMkkk檩tnNlhgwu]DDĒ%K 9HPZn^ |0'NBHHȎ; $Upn *N[ZZjii;P0HςիW `gJzMWW.ٮ]V^:Bz2Z@z@?QRR"t:}0F՟1k׮UUU;P}QS}r5566#llG7\HUEEu UQRR"t:}0FbCz(9Ft?K]]?`Hy uwdfg.).}VGGGeEEeEEA :tq'I-Rs jjj>>>zzzoСC1̘X% h\'HܼtǏ1B:;;XXXIt{TK^_Www2CB%l{wUq|GD45TDgUL[4W{oW\-{K#+2{Z)n". 0. "0 _s=gAΛϗ:77Zj:=====>t-ϟ]noog͞=[U/={4UpߠT*|O?S3ۦ<``4:Y2]_R\K֖ yx^a ig9]]NNg͉߽ 3K_+Zϛ4G-x- gΜiе7Lh5r<44Juu{]SSӶƏwOu}~R JKk_ZYZN5ZYC:J XXllj_K.,t6{54:b BY0Aɷo~Gߦ8ijNG$yOQhJ?7 XXY<"5|hk;koQ(HH;sDuvH$͜5taVijݺuKs:dee͙3ѣmz7E"Hi,rijdz&%02e@S'kZ]4fa=lT1HGSvJvJroB{ 蘚)7o޼ycZvoq޽]ޯ_ 6xyy*xD( +۶j 9g?8!Φ',/-Wտ򳕥T7 zk׮'dgg]vڵǏ=c.{?Įsnl;yCV t)٫W^zu=\\\Z5ڵkx{~mذ˫m иlVޝF{/Ws#Z_?2|,hcGSΟ;9{'8oTKh& Ms'3|1 V@']rΝ?>… *++pƌ/rY3fHKK#>4(,H$󑿳] GOft61](~e戽}0s sc$ELcEyJZZ.ЪSR+$$W^!!!V~zӒƌO?.֭۰aniOHNoϜtij ͢Q=v( ͑IZvWRXb0 Zeee -rwwlU]]o6jǧϞ=[w({쩨0v!:+ׯ̙W߉CK]-‡z ͢QݻwvHscgcգ#9r8i{ƪL!ZuZjX 󘛛oKHHׯkXFcXx09=Zk:Q2ߗ***֮][O```pp] ZxtuJA&vsmSRR9X4oXf eR[;m,Z0+WL4RYLLP(o:GPԵ4hPHHHPPĉ X)0{ޕ+)fQ 3"PXmtڵ|Ν;w\p|||lmm])C< 6233{tW`P(t1SV]{u5b=pٱPRee]HZ0իW^kSNӧ2ڄի{rfffff槟~*Ǝ*G) P*hjs;v'%iͪU _˗cbbbbbe2YmKAN4xt]mfru  MO#;_m]\xAPwcU 8y byƳX+BHHHHHHXhQWkUk@ hIBRLNNNNN~]]]ryXXX@@C{V ;O~+,*jOz/6bŊ_ؽ{ݻ׿H_&wnR(YEjOCOP?`?1&XXXwm][z'jF/D"8qj_?͛7^knn>xLJjZjѨ|p\>y>C< ziZ,e)s'_Щ*i9Ggȑ@ ((0T<SWSSs1;bv`۶=zRKܺ=1/_VU]yAR=רws~?07EnN8woXFQZZyhXS7C3.%Ν7ݼYdn>vP??HD(4:FY۞9 &$e64JP?~_6331b\.k ]R"泜WgύQ C NsaUEESgk]&9==9=mc@T:fe0[yy[w>vo+˭,-NN5jÆyrZ}>+tf˗ -U(Jʪnvvn..|p2dN~ak״GH$ܥKq{>z\VV~aaMMMg J<@J2"),*ږp %ŋWo޼WZ* lm{89k/bq '\{]7ݻ#1㊲2cSTu-z1i$\NK-"]WC>vNQQK.-E &͜yWf7bD3W=|xZn߾uڹS(>ma1ZZ}il\\eU֗_zuϡC}׿?M`;׻ IDATiTYUuQ6BrΝ-[lٲEL&ԑw@@nhnIݒj5 ?^+[FFKYgeAֵkwz89#ǏkMj䚚>_74t…)&||yK vLs„uT͛!-KRr/9 */Y%>UW6\v}]AMMMɓؕqJ2vE0Z"+(((((h[x`R,u+UIU׋YZ*Z@dok#?^ݿOc.{(k+ CM95[z}YY׮i楝ݞ=q%ϾZ֭ݺݷ:=='OazR_qBVVrzz3;[+VG]kok*^kyjW?rd]{l믵6w[{njqѣE"Vh?:{h=AiYم˗NsލN؉-Fy$pD=yx:x555ƮT* hFLsOûywUI/aZɳ9d GkuLSSgL%ϝ+*юYVsA7u뜻wo7n n?'OufoWw^yɠ[o׍ܽrڏxu"II^kP:f˖ 0@kvС}q֬;EE=Nع-FyD(HFJ$wyCr]_~AAA&M3v9 O=w?EQiUE빯 EEٽCH$6̭Y+0U ]dɪO?}%<|҄ .. ]]T5̮s>p*9= f4n9ʚ7>XkpZpUdnnOrCcG͗dk7Sݻ̓nnM]RTR|j hu s><3eedl߷o۾}Wo0v]пQFɓ'7N$"Y,"`j k9rt=;k133 Ç'ߦATΟ?m٬:`Ls{k7o^re`~(+kKÇ.l.{ds:-]jmeԴ/ZӮ]u#^u衖\cso/v_Hpys ΍^NSS;H'Z#cTelml;Gy'L@kt| VkցT͍4]8{V9޿Os.]r9͑=XQݺ} Je׮^w555u#6[r܈_xu'9euKnup3ΞmqڍG0@ ;׻$iWR5knƯ_`Μ>Mo'hԨQ/NLLݴiSdd$٬yxP(ﻷ u5nU;qNIݡ{T{cQlERna6_~AAA&Mk{{[[YUhl79Y=yORRk+KyӦ}@*j#O5f##a Ok 5uj˾y≕[4%>[5^0W,K W{ki>Uwx9Jew7h6us[Lv-dfOMy#GRkP|~l(/~`a!_W^>yRvv)Snuu7i$???OOOL.e֍\8zcF.CA6`xb8$$$,,L&R7oޏ?93r66~ؿn@JJMMM= GO Jc˶7:zVc0ӴFGj?vrp(*#;~iZ- [57W5ی7}iTڒˏ5X)8uy(++*);9|޷Xk'|ѐA eedl߷kefgO}}ο/YF<޾]o6YYY=XY 3vf3z%Sh]Wڌ`0y@ PT ,h*5`FY>>>g20??xV{'O9R+ׯgfqT*3ƦvdFo-M{7:rݫh+WFFJ$-4/IURZzSv<Ӯ],ss |2DZ'\Ǎӥ{pȨ!Cts[Lvj-_OHNޞ:8oZHVV7ZxC@@\.`B# :g+mS]U}*ؐ9?F64= _+Ph;mZ p^QÆ k@***-[sΕ+WNR֭ 8pjU߂STu#k~a V>co͛/\u Zkt_\C['V~+7yyP_F>''On 7nkE{;ah=\kkt[L򙚌}D>L3-.{w%%.(hYY?|իW[z衇}}}#  Kkk/oĽu#f 8ܶ[Gg{wK:9=Z ¿mVտIsڲ3gl@r势fIMM}.]ڣGcc*꥗^Z~}ked27מ=Ǐyر_/Zuڎĺn..2YFlDx֍3m6@ P Ww^uOh#֣TWXŰVɨf]v؅܇R/UQQar JTΞ={֭fڴiSNNٳgfЊLʺTҲu޿7 J5OӌpVucnr*JC6jei55>v'.hhcf2)k++'\|&L(2͗^ڿiӝcׯ_0gNhxfٳ>3gRl֨Q/^x^{lVD,4u5PxE.khݪX+4tlVB~e˖͚5K("C8uTxxu묭] `O4G~IHX?&>\YUUwx@@ tw?U{zd~afmC||,hicm]RZZwض>F3Mۆyo lKH8~7f޵~`o_TRRwp]JrWB*չIUz3"lmlroo]Rk'OֵԺpc=o`Wj4Z(+000$$0V@< 0Y!!:undO-ݭW7vX# 0Xbΐ,77P\nffٳ̖f͚+W7N_EvX#G4v yÇcmYjlml&N;|"0_ԾVTϝ6@J捞<Ď KJKU*Yv^ѱU3Ƚoߤ͛_Yt?k˓ϙoFwgxͺÊʷ_yшSj]bϴmUU(!9y]PpܹGWJhi۷o`___;;h3`,C&?9RQV1X%)+5'Ml݉V@flH$#GcbbOO8;;8jjL& T=y[FF~aakJ/Ldhj`慚vvLST GH$YII?ޭyԀv6<ݪo,~Fǽo_[VM/oVAܽ̅4,9=]VÇ[n1gZ|晟׬wE M6EFF>C.mG< 0eBpƳsoubv8uVmڕ}栳sgc2Zѕ+WRiffly戈{h!!!:]PDEEI$7F07٘1#?U7(27uԀ;5OjХe|mF}HLTTiH͘?]ֵkOJeUUSWy5@g@ޣFihnbZNN >|L]ggkO9_ sJ iprH+#uYhkt.-22rӦMׯ_oQ}yxxK$ɉJ))):Ncx]yZm.^?ߝ;˫y{k([XTݲEk$ϯ嗷S~tcǞ뭦.jۻ7g@ij٣Kn1g L1Gk×rK=E{]Q\K̏ZfS}Z^YcbbOޣG]iii111κ#RSSe2YxxxnnShT_~IJKS57|x;߿Byάk>]jkc3s͑ʪeV佇k;%9ﯽ5i۶n..Z9_}^K&hWTyn1g QN})r~\1'jڹ+ϽrKsP(NYdXWݺu+,,,GGiӦ}YYYmk< ӧO/X@$2Z􌊊WVsB@ݖ-޶bj`` B3u_nڴ;)I9S޷FFj[SS{ᆱ5y,-{KS#F_0wȪ ֳZ#k~A=:׻$)ΎxUxO4yZ(JlzkIa~ow7p˟K5g<;o8oNX+#rT*lUun߾;<<ͭb8:::###7P($ƍj^`tO̙.T:z͏?nNn1g ө.NgNNc7+)؞SI yx^웙ϧnU'D :i( z!ҾK2UjjL&{g?CWWW}UBaObE/9ۇ5sSF/|~M)o䓠y4CKUĬ w=z+-vCn5s-vuݼzٚٵ=}ɣ}WYRk;ڹӵgO#qrtQ)n޾}:3\VVEe@ pg43/z}psq;w^=b BWPpҥgKJKs]n1g S30/[߯y^KUU'dWd?$VWV߸tW\WXi3fε@Z:z&O,}}}b!kk iӦ-_|͚5JSح[.\pz!}/W`zpU3 {Oj445s;[ۖ3v+W/YUCEe}sN3$#Z>˹uk?4'7?ܥK:Vn_ŋ=\;׻$)΋xu{?M;c{PRP\= ݻ9t`ae!һ%EEwfQcBa{}?퉵:f6  7n\Gh>^ ETT-[6k,~nKӂefOn..&}ܘ1uђ:Dឞ~~ǎ [x >g\iCmgΓO~a_[⋏9xcrY4t#F=SQil ر¢N} wmL3<^+Y0[nCp’%Ku\l٬YZ)i ڈxڪxfgmG* ?LQQ^^QY.lmllm{trdavS&#...!!aѢEgϞe5k֬\rܸq@333Y,v6v!k@wr<--[tiAA.Sdg}?tuuW_ P(<V9ǟ@gt ӦM[|5k5yj:66v֭ .\xu*988BZ-B~WZ㵃!茌R(QQQdƍL>h*@'fffn*`:gtPqqqDǩrrr"""RiJJ^jȼze*@$&NdB`:Ciii}ҥK t*55U&=~*h,,, B0/VVV{"[B`gttӦM[|5kJeRձ[n]pŋMGPثW/zhs!!!:NP($ƍ/YG\\\||D"qTh!򴴴gggJMMdṹz @YEddӧ,X tJVzzzFEEUTTBij:1XT "**J"lܸQV<#yxxK$ɉJ)))z ʈg\T2,<<<77W/],aaay D"]RձQQQRgXT "**J"lܸQV< `<<<%SDDDHҔtijL\.OKKqvvqTLG< YXXDFF>}z"HjullgTTTEE*L,.A,GGGgdd8BH$7nTz)0IijxDT999R4%%E/x@#bbbu*55U&6,,,"##O>`HTj:663**B_&x@%322BBBtJPDEEI$7jtvij:xDT999R4%%E/,\.OKKqvvqTLNx~gaay D"]RձQQQm!++Kx!!!:NP($ƍju/wءcq@#<<<%SDDDHҔ_%fΜ#"&򴴴gggJMMdṹ-9GT>e8LEddiӖ/_fRjull֭[.\xbkkfN֭۸q:4u_~E*Fyyyyyy555mt5>`E< '_x%K2BZ~ef͚% :СC ⩧ڵk[2uuo߾KkJOO~]L<<<%SDDDHҔ }QXXr̜J6 R^/Mw-RI$իW aA233>߿_~=ztЇx4qT+>>رc%j}O֋?T+ţV/FF=zѿϝU* lf44 -D?oݺf͚5kdݻkjԨijPnTZUJHH8} D#Zlɦa|#G;^6k֔cSX٤Ih.]4 裏T5pTxʥ*Uhq…6my+|rƙL VGFpAYFY:g<`^Ϙ۷o߾]*zyyijҤيDgܫ^QFk׮ׯ߸q#՘5s"[FT)1qqBC~h#V U `HBKѣG-[jۥK+++>=3Sjg纶m۫W-^\V_|۷oܺӴ\'UԫUϷi޼e%9oݻٺY37o ?s&2:QbRpwt˫vƕd788+QQw>LMKH$jnڠA֭{t\bg̺ᮠ!!'ΟOȰt90JnѳgO*]C*u{mr%Vm&QdY"s'Zgg QD"1̉;|? JHK| իJM!Av%qϹ={Z4n\\Rvf:u"nao?Vҷ)ww*E~ײqfeH {=~쓤$CF{^%?.YĤ$_sG9iԨo>\ϬV?wND2l/M_abrѼoݲ{wVvֽ:w +o=gSVv3gJw,M-wn(LHHi%D"a,d` ]ij0$q6%B{O<\]xyqʪg΅ V*vEfwAt`?ޞ,0d^~YV9RM"+kĉ:٬z¦$7g_Zs]~3{ꖺ Te6,Z6Ͱ͍J7dbg;8xH(cKee"Uohԅ L<!nmٹC'N\~!inÇ_,rdRYܴU&M5ER=}zٳޞ,H$N端͝gR;"bފ]mԻ 'lz'NؠAooŋ?m}ڶL*$ zN!GxwRLر)l)G{{+L96>{(]JEz0iyOlעV|T>G߸};Wn..-5تUB6ZzT[߄~~(lD"qqrw`Yx[3-G9mZͮ]{ ڵg/_VT. eBnnc>;֭[ɓ.|pC0!Tٮ]ڞ>kRJS{FM1bŏ?~lx;7goSiǿK{^.޾eˑCT~7n\aV6mVؔտhso|{YZٓ& ׯߛok?}|Ælx& ̖Zx'N,\&r0,0g Z:%>XM|Zի:̻f ծ3;##= _P@.NNc^~YܣR8 GJR#v_ߩuk/\z}qψ2!YZF\jp&t/&tjmϴ˿VEFYA LΧkWT?ȑ>tn+|@*ȉ/RӿG/ݲIhm|dJ)m6J`z[*3-/4[jivպrƑSBC8m܂U*U*:q¿yU(cNo$xk7oEG:sZ|| ӵm[~ÿ=sRJjZl8Sw>o3fcmozy2lк81W''q3=##)%L!-r.Xߖ Lˣ5ohҨQGD;|xOpbfnܾ=wSʕMT\axL&stt4饍/j۶oN8̸`S/viFg؝oܾm^.;wJnn DݹӦY3xJhǏM  (fY[[li}|||é.|`uJwpaBC;R;=|x_NA\i/:;P~իTyPEJ7q6K 5|-)95t4Õd.yw*emϴ¨W\ޱc[f/W5k/Md ˾wߐlVݺu'Lcǎwn۶ϏlesV^Y{4h?WRe& ]<_4F#&.ܕ+@!]4䟕n:]%&>>#}(Ԍ]iysf+_oK|˫-w؉;OڵmyՈzu$޽ۻww6@*,YŋQQQV4h9DagDEG{x\ZFFةSf]5{yڛ7\⦭!_Ϟ:lXtJʤXΏI%\|-V`m柾ޑm۞={`ɣGשQ#ȈW{,#]U`6m۶3f yNڤIWK'/W_{[fpxxVvR> mڠAdtyGݵ#='q@v RҴM1]ֈu{GF;xys \1)%Etrp:~|IJjPNI VMR!3^.էk}qj`XX@h詋Jf[^|twulwAʼnNJݺu۷c ijLjרѱUS.h{gRVrynݴ͗\VYR8rd+h)ii'O/oa5?~mT*+❼Tpq) %ԠN1iΜw&$}_7h7{j&M*}(kTZ-HJ`9z[*3}h쪕|x@hc?>ܸ?,%ܹӻw{ PN߿W^ZX!0 7xR´_MА}eehw/5kJ2AySvAGnâEN7^Mͽ/JSD T~cR!U_ߍ/jVo؄Kf&:tXfMTT͛7WZ5h Y,0aQT|?{VSѐ޽[7k& vX[3wH>ϙ #GAH8px]:iwzT*kÇo\H {^4)+;Y[9j̨k۶:=2Pޖ LյiÆLZjh*P gY|%q3jW.n..: n #G2235Vչ{w*>3QAAII:= n ng'̚UؔBB<)̨Kx֮CJPz[*3P:`Rs阸8myXZSyȑ-wnj|E}t#degr?.^@KdzA0bSu:ݻ_ _J[BMUYT)T(Xam@@< ^|L ?m޽ڦ퐾}OH$m*ovQCYG~~:u[:Uħɓ|X]s-:y|'ӳug3e8u[䌿Tg #6vPq?ܩmٹ:[["W6;jդa"ݝKR{.\3+95u_{NW ÇyEڵh!>?x`^TS 4m˫EFZIo܂m@yG< ̭Gǎ dh?7%֢q㖍8K^}=zժ➔4#mjz`JСփxQ\&kj+;c~Jleeҥ66δקLXBq 4͙3۞d:;Te䌜6 ]R\ؙ3.YrTg )zTI$7_{mҥrvtg!C>^;گ0^رRvN5ko٬Aii>8Ej3/\ 4@hh*UztXjUww[Ǐ#oތz5%-Mo]M<|-(׈gyy+Vl%eR lRtƈ /Gǎ,_>~LYYw飇 Y'_Μ:n=4Zz@:G 0Ʈۺ5:q:xP"o,a1ɩ_|]b~3jTTu:R!) xxү_QCXJ>95o&WgNw7.^lcm]܉fot*U*ϜIL"|7wO?5BWjgI/6G]*-(ge?vNOڵo_qCܹQC/?nP++]lb4#eN n'v-:zu`ܸw޽zmd&6jȐ+H u֭¾*_oK|#7@ŗ{YKWIÆD[xur\\SS===K:S{ϜI-eccӪi:v̖H VT;wuF=K~A5W/pw}qCaa=gewWMtlݺKv.֨Q=<=>ٳII soҰa֭WxH223-]*>ɡCm:׽z6k9S۶m(D~D#](jy9d92'..4T*!-͢Z}cceXJj׶+z~811=##;7J&rwժ9:8X@c$<~0>IbbBRdR\VJ7bm%÷B>S&[[A.|ZeA9$nZj*H= DRzZl.LqqvvqvtJJU*U*R!)Ϡ`Y8DH$LfZ@y"V9Q%C< D"QՂ `gg@}vL~8d2YZ,](Ǭ4h`*Pq@ҹsUZ P.t҅pCT( r`TZfMLfBP@E#HVZ87 &A< Lx,0 Y`ij$gI &A< Lx,0 Y`ij$gI &A< Lx,0 Y`ij$gI &A< Lx,0 Y`ij$gI &A< Lx,0 Y`ij$gI &A< Lx,0 Y`ij$gI Dbbb,]@E< @%''o߾}?իW-]@ecj:"""88x߾}'OT*rӧK(g'999((?(((>>^/˷o޻wo PP[nfdd|kgguVY6Y"U溻yT\}.={ةD"1u)([7U*FZ{5qha#Pӆ_|r+<}{k.K|Vۿ6|T o޴N'ծ}'5n䔒r;q xZѿ@%E d'W'zd+&3FU;CիG߼~N]|/SVij ~.9}m6ojZMi+CGkge ,ҥKǏ7nܖ-[fqMzwl -;#mOf~^`hܛ }uiϛ/l>|_T*]|fiXYYMow*ҺE:oN6j\u?2=j֬?%–_6j?W\,-[;%4hHq[ s|\ٹW׆3nn /DO2v3e˖;w=yϝ;g@Y~Rsf`>D*^.ZYY;9; hckkkc  ..Df'-(Ξ>qڐ@7nvoj?K$Of3x'R(4=2mfa!szcԸm/`ʇ疜M366{k)oȧtoد?mؼag;S&M.ɊTK^R`;oOgNsIY"|rqLt#iӦM66mZzzzHHH@@@@@{,]0Jee ^Df N4Q0qD4NͶafQϔFhڞ={ջ|񼞉tx;9ų[my,nTOG-M#f>;P!44$,=(g 4h A޽{СCX4`nJ%s4Q0`A Ii` \\ʈ׆Ѥ[<4壒Z9Uv+V($\n_KixWx7Jf<1שTq=y2\[ڸ%%;}Z.DKC@@< Чnݺ~~~~~~'Oܿpps,]0ijɂ 9&Rț+D~镇1sp)Efigg[䔚5kY[[kׯ_-a ܻw[Lx4jj2|GKN.H_3qVmZ%ꜝ]j,nK$F7ԕzt s…lڦ{A4i|&J2-5Eܜ4ARSSTJevNVfF IjZT(Z,BFfFAlmmApttwU"hB`U{{{;; 6NN 899Y[[;ӤX'˧NT~\kJjO&˗.hի׬נA ,[wѻwnioQR l٪]vv=s*U;t($bR75d󍚜u捒ijJ@yA< (6aeeUn=Azw`)V(2YY A233 A EBRGBiff)I~YY,"--577~fhNyӞVZlݬY S\U7n\3djr%yZm~qh 4=7_~extrrfNN֝0}ߨͨG_|;))2GUzzdߟ@Z呯^b}MNxdmQ㦣[J2#ﰄx8Y@`K-Pfi` 4BܙOLVd,1yY...͚p?g ߳!*2e,iiI L|C7RYEݸ䱶)J_yzdvߟ5_|&ֵ/:qo\ ۖL{"w6KJs!=l#28 9&Rț+1i999Vfg 1s>m.?^-[)lRLOO6 Y3\vEӬ oM4b)S4o?>''߻c깓)Kh0mOmqf*LZ^&etqSj'5dTgXFzx`Bl`j9&2;++S!BrrVkB`j:%%Ǐ̬,R"BZZ27''7'=-]R!BJJZdjurrRjJT&3!6mX}-M3%%yԈ˯ QcDiB.5w߾ڦ뷫6X۔g]D@O_lݦ~LnO?/ \~U_Y6K^lqΰhN<+;'$53,Cp)pK})*Љ<Tj' Vꋯn;K}Y2ٲ׾1EyYYY7ݼaL&wssSթlTr#.?nZm,_vF,eq1tzZn۽g6m;֫&9)KڳORѴ{/"\rqgΞRFmaTZIJ8Kx̭n[o^ʰ?AYVm&QKH$/<5 7wƦ$LsV^}͒'n{w n3 uxTգW^^{c1Iy{27ӏi<~!Lnn}:ʵ۵do,;;yeY` rhZp'֬Y3tPgggKճqI&T*K3k߱ˎU{oX ]͚U=3?;2Fդis=۵߈{v?fN,fݿvL6so}iEs5:CY9CE" -m2cƌmAb$5j^g|vr#qNmsϚ]Q RqJ|қ! 88IV2K#,0efK-ͮZw=tPppCR ۸q r89ƱQ =p2VYM̴ao435sk 9PTTبl8λ8}>xݐ ^R4jԠQwg|X҄˗.(۴r6 }>=9V6SaTjjfyɐ#Bˇwj޲Ҝg~+gϾ}nedd4ͷZm[RRI [~_?ez" ;(%?fUw&-Q~T]jE-(zɃaZ6f#V\ykڌ s~ksJ]=KmnΒ-BÆM5_#ۡ_4T>{>yaۋ-DyP Zzkׯ_?f]T+d2Ybf0 ^YGq7,]eW1m)ϞF49G%E67Ѽgt)>IJ|Vj&Ju17? f㈇Æ KIY@5#o%H<8sLww8h͋/Pdggbd7J]\RT/ͽA0)} *jذaÆ A"fddk׮\~}%ܨxƆ:R$$<2t @WgdˇޝhզUK~|ض*yT>߹KwK &h,ȸ wyUT7+CSSӵCEdX<5]I>p/==]|A|O*;t4AB^]۩tN>˓4lxR9?)1A>|\Yaadee鹕F+mc7_tC>lӦ}ݺ.Jsjtu/^OJL4ʗJ[>455mӮR_"c'ȇ6CO;|ؾC' 4[ YOD"ٲe˘1cJ_`|Zݴmh,tXM ܼVjmNkү۵5f~gJȈ5hʛzviu=7رLLLÍs/ڙ>>Vܥ껠K4cGfM.^8:-ѣ/>_)FK5P>Kiɓ'O>ӧO:͛7~#%dai3w|s)3m;os KC'Tin^zFzd䣇anݺql6Hms_}WRwKMN,B /#(eK>n׾c'].++ L iSƮa]I"}fOGZi7djY8'==MKמ;uU\ @5By6lذaAHLL;~_zzU?SFFu$*_\\L؃{asyYZYwY[W\VPPy!ÿXWo῏[fp'Oo  ǿ1Im1žgNHYY|˳i.];9<rg_FssX통j^|gzrw[عkR% ~F`aaoLi[i2Ds>ZxGÜKjMMM˭خOo Tgϝ;=T}6~XА Affgc#c͵Y{.Xww^/Q J+1"!!^ z_|3s6lԸ=ǿ19.6_#29X?x^"3s vNړDy'lfˆ뻪o TFN@bii9lذ9<<<::z˖-cƌQO?͚5K/T)˗۹}׊kjp!7ow2f Z^}W###NRrW%\ȣA;v\zzs|u5Id o76Q/կ^[i>5<@@,[j8q?((*k͂ p!n=H%IG<<U~bX,6RիY۶km*gaa)W%H$f7)//82}ϦRկ}cGٽ-?Jw Pv7'ʹsȱ cJHPuٹ#ߚ6ɹ}tx\\u[n-66Z5O296666"o T}"_0(zN>+5U[bb}]* L{nH߯E"шQ+;'r9T*I2 W\FFZTdD-HW֧M<@iVuqqNK IDATԶaIMIɓԔg"fӦ^͚oŒȈ qϟ?762ؼEkzZKgOɊj;4lڪU[mruˆؘϦ^g6`RdGssD's9?šE"ݳjKM6M> -+((H"$QHITD-LK?.cť ppٻ_ͣ>;5mմW ʳĤ[nݺu3t"-==UYjC2ʤ/'ʳ@JeXII y~Vfݳx9QU<["*.DJ$訊*fgg â :Y^9C+!V1kk y7pspspskTWlf&"+(`Xgxd($B)DEH$ђȴ :إ{W7wwWnnnvv5*DP-PyaRCxi^^hccѠrC,gX\A',@QQQlXz< @Ւleggb9׭gjjZA'WY 9ϳo''?5t"eVfVژ[X:P7211RXP <{ߤ$:;%Th#++##X qRNQb: ***2t Y  xbC'Uշ6KgnvЉTqѕ¢IfeX899׭ "#KCgY{.Z(!!A1إKCUYVVSWVVSr23##WfC,W7^BK+kC$m (< tTz1-,,jpDN b*!e&M%t,67 k̸ 5j6@Ow2t,x4ⅳ^ s+2ReX,Re6;w(gg|s̢̬Bkkj7hبM{/Y"6646&YTX qH$R6*[[: P!(ʾ^*mllC?122*VfL9t`bd݆?GXMvn߼`,`N^r‚ѯzICsT8*#?[6O߁_ݱ|n,k?--GRR= ;spq?qҴwg|hoP.WANN ۶d,JQPPgU~C,KKF=b9UЉUY@Y^}T˳233BCy(V\ uY7_S O ^|LvY'M[ՏVگUuO?|aڏ-\DK"ʰ=-((㌌w!@1~ujfnqjʳbcnS|sU5أW;޹}k8<7[YYV.;|_Myt/aRRbeeݰQbh@yP&67رkųJׯM޽;RT,kIZZJJA3s]D!'~9kk2-LIy6a{wos:sm;o}D&DGU潄4Y@g]PM._ѱUW/d2`ǎ]eJF!!w.u[_:v!?=%%C:xUv%yÇa֣A#X.ڗT@Y@ _~JHsvvr˗ΫsY׮ ٫D"QfmvjުaFn ljHq|OP{ :wY3ߝ6y֯>qfu83] 8WPP:?##}ѧժER ] QYNNNnny%6nAZjX39R勣NfĄjvlm3rjw߁ڮALMMoyʦ^=׎Y{bTgYa?m5ΫٺM~\߲U[`wg=}٧ ث0!!g]iY+5JLUЉv i`g%z[>|]TTT<ݽy-[vssÇIII999666nnn]tӧOB&=~8888$$qqq陙R4??_,[YYժUA͛7ҥK^lmm^z5<<<66633Ʀvڍ5СC߾};w,ysjȨWޭuy֥s%=ϵ˅ƥnq`:N[&9} z⣇a{j!JC̙UxNn>Q̘蹳g޷sIZ<^QQQl2buOC,*,qqq'N xLg#GG֭A999۶m߃NxS/Xsez.ŞJ̌[;xmrB`˷dSGFk_7uhՇ;3˿^\mfY$0ѲdޛܿYوhzTdgg>~P!VBT*W}oC,LMM+D,_=z yyڵ)ߑd;w\tibb6J#G9rd̘1k׮SNW^++>m۶?CMѣs΍r~LL̗_~v%K̝;Ĥ߻*!-----MmuN }D2L1-Dy\s! ~ڔg?oP 4pޝo\pNsyVx؃;vŪOcyJ񸸘n4]B9*,,!Gy/}wAh۶msRSSyǏ\:\OAAA}>}?`aa>ϟ?5kxlٲ{߾}ej\XUvwnT_87e{>z9[i9gӿԅ=vjyV{WoJm |v7{[FyVeH0433itEq镛[+r0?08ʳrSjOLL>|  :uꔷΛcӦMׯ_?z˟={6t۷oCPPPΝ9R/Yxe 8TMyֹ˳4lX֍YVH"+۶QQBmg)Ok]RXXxpqo6mּ YX[s+ݽB7y!M4SЩܼO@AyPnڵkľ}J$=O9r7J7wݻٳgO.55uxFC=}׼UA@U|||QQb3u)Z_r߀69jpa]Ԯ$I"]՗g]v955E5ĩ01˖|?}YV"&XK< (EÆ ;wܡC//&MYZZdee={Ƿo߾zj@@KyyyF*672dHkժ}ԩ]vEDDu!{===ׯ_fMKKK333##$Dr̙ EDD1ܹs666ZXXX8i$ Y[=ztf͜bq|||ttcΝ;WPP:?==}׮]ӭWrys[휜^0--5έ6mKhU*^ԩ[6_p_syօsjZp VFۢLmSR 4@i4ܽ{~R;88;wnYJ{._|joꫯtV?ӝ;w7W_ }'NhÇKv.]Лi{155U}̙3;w*W-;k u ^8ʥ ܩb0,4$!!Mܾ8xx)+Gǚ̒>lZQ[[϶(w5oFiպYЀ,-YDSڵ+w&M57Xݼy300|R,YFΟ?߫W/~'n|<9s뒪=z۶mjg~g/=*U p+YY%-9sꅶv85j)67ڭL%{R58hHgت?W= 11^5Kv_͚_CyVޠaj* (*VPPj|ժUmڴr%K4iD5o>ӎ6lЃlȰ拕TN;يo#k˻Sךj?{DIHwvm?SRGOjMZ&ړkRTYeZmK1(]y{7pshبʡ< @aaaA///vQ[ۤ$vӍYjӻj]vvyFFFtt>;־OOS J"BBzy,T̫bhvn߬<}byUNoϢBԩ㜐}һ@} .WILM6((,LMI.,,sV[ђN11q-7Wnnnzâ< ( ')O>-JժU+ղx͛7'ST&(g pkW/$+>z ^Tuȁ -;sBiɐaIrx6R-Ϻs>{*5^*fee]>B{v5t! *H"""%QQ1Q2LS $QɩRFXZrQ2*Y@EIN.ή#ըQCm<))I<ٳgj_}r077777U+11Qmj99]⯏9dȰUֽ]FF<~ 6TvvҒ%hӶsJ\gꖷlHm3;ZhVҼx$B)DEH$C@jK{nnnn ==ԫ< (999pJ^^^%"gcccllZt_? `gggX\,, yЁAITDЭmy<¥5oZuXV ?v`bAKi~Z[>ϡuk_+cc^ԥ<>qj{:솊#63supux!--5--UV[{ZXZytFyPQT;UJ8EN$Y[[+>ْJǬ%(GߘT%¡{۶$酺ƕQʳkپ*7;ĤzSvviJ=ԭ<Aɯ,j5yMr׎?'L/իi;𥳐g/^8a25+Z7jKs=SQUij+IR6)fjͼZ ㇕׮|+hω~^jeLHiefn^R&M[n(*ar IDATj)ǏoyVXXxݺuU&&&qqqJ񐐐t]rv/dJ1wBV[aa!a!q1zK-PTƍ*CCC L Q :88ԪUK|///7oí[T֕|ϣ AT}ߚTE6{jh7~G sKU֨KjCϚ+/qWKkSl_{w}W[;1T& bGD<43+cY@jժjy֙3gVXatʁeKE tW)xU}rPm۶v)wVVVJN|#G%M3v6?Isy֔i˜\0g/>tF`۴k{6lj*NJ9}9|{:;-8^B&&&_X\<<< @]vߕ7oLJJSARG~~U]v-iI^k`ttKwCهR[<+++3*ը1olۺQƣǖra1/^^-CB}ͽA}tҠ7n5~Bp;ohkkuǁZ)AË -(P_Vu$$<2t H J$QiH-j< @}D2L1(6nܸ|rCe3gΤ[Ғ=zۧ)N?ԭ<jFnzV J$|y򬞽ծ>&Lj:mH$1*ήO3;zn}a2.!qBcCgeUVfMzl{,??,U(*sn.]̙cooot#V\SB+SSoܨ\ñk׮y5k֬L9ZJ5ޢE֭[ir L̫En\7hϘqWLr[bX >~=:oҢesuu/2ӾVT 5eՖ71Ss# PD1c'v˿STqSNU-JOO? no0ajyV~~ٳoT!̙3ߤ5jH;ӧOݻעE T3fgԪԧ ?ֶ:հroۆyeZkii3o@^[X[:Lb ׮]3gjxٚ׶h_?|R< `k֬2C}qiӦiIW^LLR|׮]_}AR*sV 9Nln^}&9M<>'/56THZNTڭ 6{vm!K\wĩ?fYϪڶp?%%Љѱf EPɭ5\XZm9:lӶ7) D٧OJM0(X133sۭ[Θ1CCfT-Gj7ԩ[l/^0aBr_[6m*uKRՇ-ZrJ###;9r7T͛L)& zsN{n۷=<<\g2L)\WN$5sBC.^GⲲ2,-j֪ݰQmuǻSR1VGϳnݐf^ʼnMmy[XV`UPŵRʶZwQQ 68ya3[6x6>@(+Lv ?g0W)H$83^u).DUߎ/Sn߾= @_gnSNݻg:=nܸ>ۻiiii_~/ߠ٭^Z<==.]b Շ֬Ys bO>]x;>ڿmj*ΨQT˳rrr t!//J$m۶TpϦ^Mޛ%KʺkކUV[R41!NVdģ =+S-zopY@D۶mԩӓ'OT ?Ν;wԩyR4+++%%%***44400088R_~g PTTw޽{֫WoС:uSNڵSRR/\pvXn]:u迉ΦOf͚Qӧz=ptt<|pvJZ|||G6//ݻw/xEVlHoՠa#kk[=Y@ܹ?3eʔK.鹕ʳWXxbݖתU… ={tΡM6swwy%ۯ_4_r,@AyP\\\n_OvvvRRR:utXkbb2|Cua˖-7lár۶m=zybcc˴j755'rײeϏ;VZPUEZmb'gV[ R=UY@eDoIvܹm۶+Wd2׶if&Mҭ6Kcc}ݿKOO/u9sƏoddۡJF9tЭ[nܸ188u֝:u9sj֬Y. ;//[n㏿˳gJ/xI&UBz@Rj$Qz'JU[m9;mڴ;*,_Ri5-oo-J-,,===6mչsgGGrIy֭^pոT###&Mter"X}vgeX%K.]`ٳgϟ?OD";;;;;;ggvy{{ٹS:woкM{xڲg9BG+++C/ʳsvv?~ ry>}cӋyyyyyy͙3ǀ9#X+5M,+E233 *L&۽{b$WwH$1jbe+Im(ջ{קvЎ?7ZEK>ʴϖ͏4ٹoٛ~e׎-i5mb'+vQluKAׯ_|+**2>6&##=77|mZʝ9VMX{W‚ dMߟ4g܄>[]ulغeClLgS3f&@ Lv ?g0WiH$83^u)խH${TRRuۺݳv*/M~Rjyֽs>x;!!^ޙ:nG ?h~*,,\8ǏhoVOTʳ88tpٶ<"uتҏ߯9o"6##])~"4 ?~Z=XIvIʒTvnz*3͛N26+Kj+((xrVgK*r?x{:hbjωUS,~/}w;sR6k3foT,ͽw79!ݭtdfggeRN&###$$}J^Y~89\%RuE$kD&Hs>b Vz3?ۿ dee]<? ׵tU O8ovH}ޝ1e&*E|bUT@+Z @U{j>lbdٛ׹KwSX3s:KTSSSieeŗɇ} {{nݶ}^}f9v:`G/-۴ZDᄅT%63wOVeś7#B& {m}{lD>ڭ瑓Mx6Kkk롯ʪL[ܾYuZ~&KK5aL5߯0\C+ ʞ ¾=ے]z}ǿ1ðm[77ۡBTQ <{ߤh*;wyKЂaݼ~m_V'+,,۳CH$Z .YqC}4wH$R;Ͽzs6m;hNyKkg~~~0!!.&&~}2<)&ڬ3?fe oٕ4Yln6}O}$\sy֎m[_DKW=g{32ҋ'\+SpnT,Ai,J  gN)-\˺[jJJX}]Nn|tzImmmKFmy+F$Ɨ4Y)zp_Pǵ[L,A"")wy~=qq1:*;AђH{ͼZ4v)a~~Gu;WA>PP eaJkٴE6=qaJwo+0uй%ݺR,xE R 08C'WN}צ..΢1ak9j{zn8MK]Rظxg DGG*&_b2_#[_{*]8?|h )[ná2BEK)H$jueeeFGEy A*(@esrjض CgQʳPXXX|<Ilemc8H/uIZ)z$);򡳳GÆzY/Q$QSvUS=[n+RiԉóPt_kծ]M2ƞʹYոIzn=z^YrC%'UY2qVjk23/{m=7G85 |9ed=O?rEw76YHN/~JR׹06VRR->PVtWKE"r"#RYyyyRX[vnߢᆵV^}M<= x訌 ss5kթӭ[hs7 }po҄~X[Ϧ]^p|ijٔu{.Oʳbggռ{E...ؑ#Fռ--.;K^$spt,g򡙙وݦ'wnsђW\pVwwo@}{v~2i^^ǟ٢s=\Z Qppf 79v6˧tʙ0-/޽a~aaavwdV*$ɖ.(,<ռջ?a&/_+QxIYm8t`7ͯdS/X\$).߳>;olټܜwc-dg+fڬ23{a,Ь}*cÛ?i6{ e AED2EElP)8زe 72doڲgݦ3IG~_Mi˫~θCUzg@;*VzY$g \z(BBB̟M=~5MWWsOP]leJJNNN$/o#]8oֺ5+4MXÏ+W0b*3|`N)ty IIIlR*BJrRVvVFFFj,BZZRVe'do_\oԤνVj9]VVV+T|冁U в%.A;2q^h3KOKnVQKOF!^4r`Mdۖ >emmkF߷K'NS_ Vj&6lY~C#0<+C$W?`ʳdArB.BzzB!OOOW*\!k*r5At\.+M\! IDAT \4P&L;ʕ++?5R||U jҢ}77mX_4*vղю3ilECQѰ9*N֬UTvV:t{nڰZ6kĨ #F4=< ܧ[7Ν=C]'\>!X2C %%j7LYUB?`ʳJJJJJlegeY6*UTxyuS yӆsgЎ|#{DφRSS4͉Sf꙰zZ.xPoMdgee?*VnDS[L+*hѼO,K+AXTζt WJiv|!.?;eh큽|8eڷϯh899k7cc]^-ݔ%&v׵ԯR9[jӬysgOQUTeʔHtuzvr|R5DIT(򔝝)+rAd r<]LRJEZZ I̬ԔTARR333333SRAHKKJNN`Z666N 8::YbX]d#V7lmD" {N"G[uz{;*.&vaJRy[3oYfdTD|"QRiDDsM^`C2MMy `ʳ)Z8,vx;l(BeeemJ*K+A(b)h;|hiG;9Qy_fr`+3}~EvWʳ\:?h'h? PZP֯0$,c>^}οU~թd2!ǎn6j,G& pժ/?CFФI3WAhGL\;T6kG:tbĢͧ@yUvvitssb)[J5+?rISw;"aeeee޾VV]''cΎ2UwzZXT(֮ń_ﵫ4˔)WJOwSǣ"#}[e(;o4mll7l-Wd- VIIU;qjܷ;,(b``= )Og)W|A\\]E"+[[[GA]N΂ 8::ml]AwrqqN$YtO@rűc~Z4qoo6RyZnn ]Ȕإ[Bi^[<33S\v;ZW$ɦ~9F;2`\ҥNJ?V*|zV;$*ۙS""k"MU0 d[7(rus%Bn>GϜ>I;2:- 1Jʳ}ڵCovF7KR%=/<ʕK'OIMMѧ\5뷕l qqwn4[`_&MKKK:M8e*?8tfFFƠ{aI=yh􈡏,hWano:瀁C׭[t\vݪOݴ'h;ÁC5eeei3fӳ:x/'55o$y{1pݢT@ ByF[6TPyy[3g-tuu5dΩT*Ms?F0bxHLt :ywO[#tϞ))?(zf[MKKsG_Db?kOֹN3uڬ so֪]I^>VVqW^r|VV}>lаq F#U7ӦLhܻ4ggo!׍?PPڬ*UOֶ}'猊|Q^!\Ea!FhanN"YzSٻπϟX&Rnݼv5=Z8}x^D|զtcbOu4vv+_BS?PXY:H`.o^mS!!>T|ekG0*P-qJ۴+ixxzX,1z_V}EWز}Zu/n={1|]*d,(lmmmm휝]|˖PF4lָ9*D"ѷ} B5LJʗ]ܺer9(o?x9{ӫC.kW/۱mTReG5lrwܲiݺ5+?㩻G>Hggg=?PP%XD1S7B6q7+ټEGu\.nҠa`+=3!>捫 "ӫz5jDqĨ#FaH鋴4k+kWwOZ-[ΐyxEbC v'ؘlUGՠu6.-O ge 8Yãu3R`@`+^f5Ϳ.VN^Og@< YP$("Ay QjjrrR\nckkoo),YbcN:qׯq MVuCTԐ..Ml}uK233,E>N%YheeeU*ՉO?ܷ_T}-^ۺ_~=kАO :\TܟU*JJ9ȡwREK.iiKg/tJ*=U`Ygbj˦uׯ]ٲmVh))_Nu[YY8*>>޺yu7llXG,&f7}+%%@*VRulZ7vͪPkS(,''KזȕS8= kwܜ2鋟1|HJ [Щs|{N7"έ¬ՠՠaS'Y:#yzzol,zTj6C,Dy(r"ZP լUr*=<<\\ݕJyBBBċ=s:|ԴEkXYkk54 nRZ+UtsswpHAHNJ|6o\у\1:t_'ۻ3GAkէiUTuvqLIMNIIy7oߺ~ P:n6wp=fҵJKR`6 6;X:\\Zh,@< (B={=J B,899-[600Fk׮Qo8;~ ݻnccԬYvڹ[:͢Rʷ>~>rtt.WB#GO:~+r7՛۴oFOoOF~٘,mVVV>O>޿oOguk]\\ آU;u[7ihN[tY=V];~իW,Y{nLVnݺ5jڴif_pɓs'L0k֬e,qq9wrr*T oذ_qo6mZUJMBRmٲE;":F$xsBie#bkgE=nQRGgݬR5aGXR刿fw5*55f2J"G8AR3HO$3Jz +˯E"JݻlذY .,]>Z;w""R6n4r|AP*{iݺu߾}̐a*̬R`y h4?Ч@ٿOli޹[ՠacjQ蓝={ÇjN=/z9dȐȂݱcG&M.]TG)>`oVƹ>w_ESLII~~^CszZ& @Eyw}wb={ֲe=T*ܹŋMٔԳW.Wc*E}hyZ^Cr;R1==ݔi((rڥKZ:Kl߾{ 9Orr;#JMٔվCg`\\l|\L$n0RWOKMLl(;;{Νӧjժ]tV J2===>>߿zjhhhPCPO3tCef|T yuͽnݪժS|6ߔ((r`=4ތ3iӦW\1ij3cƌ˗/M6fkkkMϯI&&M7oެYs PT[3bZA,1jswHH8sO/vs9 @@N|M|}}OP|8; vzV|\Cs4=˕PA(SiO%ZlۧzV")J..7^lٲ\/.%Y ,YD74rH'5jnllmtYY*=}f se) pW_<0O[vv'>t`_}9&<((^?t!!!}1>T*O1c zW߾}%:<~X*_+k+kcKA+Mz677wVm}zu=(^:sL&ӍؤĉIII9{.TӍ_x^)>zݴ?hqYյ;g4S\c}\`X ȩStg[ndlϟ ѣ1R}5vkս}gy V/߼qM||ݽKqb+׿h>=-ƍkqq1N|֭/wt"(y(^tniӦlf͚7nf|@Iw͙'b{נar+{;88)Ϟܸ~?ܹY4ʖ-?Y&N۹5޼qUOstowVzZ [Vl8v英@AQ 1&Yq9;; JnWrn|Ys~Թ#ëFڝ:wkledl%>7skV-ݷgGfff^wY?dK%6KkW/7mnJʳdffJRxݺu͟kUbbV|@I3Ok}!7ָIMMz_~jRkU+~z͜٤X:JIIt (y^;tҍ{zz?K%VjjjQ/aR};znPbo](Ws:ﭮεJ?gP8= Ozzzq77Slz P(̰J|odRBoݼ~t:{>>** 7o\7xe.0D2hȧ>ZFL;ݏ=CJy-у3O%Μ>Ezvdٝ76o\;d`޿;[׷ޱ+P9mۊD"J#m۶qƙ-#G|Ezu_JZriӊzuo@a[>}(11aU7R7m$vSJ]^ZP8~Ú ryzVV]&L^s۲eKEV\ծ][7w^3,)b?sbbJ~+ 9JP$%4KNN26KW.yom XL9#rPQb9s3&Mw}3>p@ݠL&=zVJFm7odhe4_}\@IGy},XuւΖa\ݺu fffN8Ѹ o߾e˖Սoٲe…fHo@!hnߡ3[1Za 9C&;ջIP܈-PH$ &?>G\R}3d/^|7o4.VZUX93f̢E@ӧO6lɓ'?ydE@)>Pҍ8-GZlmŝ:uj߾}͚5cbb]pXP(֭ѣ\Dzu)((U$ ѣ[n>}:;;;XCAxEHHHttt^\\\6mRV-777777R㰰߸qCR 0w1c./<ʳCg%YKy_%K̃nnZ.?WWwӫ~F(.]WhÏ /gX?hѢcǎ{D"޽'|ҺuMy#=}$/{x͝J'77f-ZY: -KLt C:t8rș3gž>}פIVZj H$#F1'OxÇ#""rH$NNNnnn>>>UTYf5 ,JϜ9sǏť*JuNNNAAAիWYfӦM===>R(&P.FEJw .2e\\LL,ʔ)3hРAY0` & ޽{mKT*bqff .H)w0gȁ, On} ]7jש'F$"###dDYbBbbB4")IfT*U\lL\lL==Lٲήf< ȟMJ*HOOoEGF-WT4*J %Dgee-mAHxe񖟿x=Pn2YBT.ފFFW%KLFJ_(J䬦ˣ"QR=lrNN.D< 0+WWwWWw}4qEGKk⊎TT q?77w Y@9+zͼ(x]IĨ(iTTݔՅ\q}|qQ7յ\eٙ'gCyP"{ .$Ybd4*J*Ӫ2OLĕxK .o_+++ ry\|\VVyqY....F< xiG)'$&K##"###dĸؘL3& zY[[[m) qqYYق 88K$8z#ܹ)* {{?25km;03\~S'ݺy-&:*--%bzjӮapY|EE'2Rzо3z~B|\fVw&ڶ$H$*u"#7رV˔)׺MUar>}3/?\jeeUjPppH4jeQvqEEGJs+R)HJ'g aU}3UӇoxS=D;|^{{_;tz SSSWXy,Q;$KJx?-YXF_hޢU5N!_Ri٫w[v񩓿w[C|G+@WWح[7۽Oc-ϺuქBo Q>5$heeeM{w۷nٰͭ+=|`r!o޸7,\֩sB YL]zd Ribbx+:J_,1!2R)}+:*J (>qdIGV!ߺu}ǎtر}Kz넩k,+(C7tvvNJJsr85{;g^rUW'W}%tP?H\ ӱcgwOOARRRNulܿbُ!Z4k^ ػmoڑV}uPRhnaʳrYӴ#?xڌ5Zu͚cuk|oʕ^W៏_ A߁DQ2e˔)WX`޷W=wf- .gEt Gϟef+J Z)nꃸQRuT,12R%%&⊊'g5R>KSýsfMSL`׎11fOkAC>w7TsRBB+GkNx|ʵՈbcl^ ;rTYzE9̝=ce_?៏} 4զVk^*,  ޻wWs+?|X10 lۺAs޴Y\y> ߰n|ُsbܢj*WW rɈ7S9yY^gܤ,JA\B<(uVttd4B}bbbBtTdddLY$+[Q|իupj)۶n|-&O3ݥ=n5fb^D}5~oHNc*5Ϟ=-_BôHrf d7URi\]]l+:lėGi";cty k4_DISfXYYs =)InطkƭfȻY7_-̊PLPD篿"==*:R[ Hip*WvΜF߬Y{Ynk T[owuŋgW\l$...f ^aj"ё)2+z331n_9/QM_ŋgƭ{էOi-[Qv^]\\zpՊfFF{h҆Yh/66F((@+_ŐS@`go_!R}Lu⊊FEIw)Dgee6ӳU]4m;N]@Jn\30W7o^>_QiѲq֍O'%'I$^^޾-Zmն!39ÚBo?T]4aY-Ƈh!<_ezdef޽vM>wG~1EPkZuon/۳;ʷ~@ |4I 9$нq]mwҶc9MUM.=]oϡXnxxڬ}4z #Pd((RfdT}oŲryQfSP$ Cs}BXjU${wwwJǁI^>F,pެukVhbW`TfyH [ؠaJmbYΝYYYni sःC>Elϟ?MLӧ9^>F%E"Y-ϝxMOL,(j׭f?$ӎRi9|ϋ\vdƷ Zn[y&*&ZǮúyz{jӮUvz>#,Q+>|L^gef~9=nDjԬӮsWΩ:tp߅~Y6Qq{qڙR̫z&1C؃̴H/?{ϟjj:Ǎ=,3#C$P"PJU+Y.UjTr>oϞ5I@V\~^y?&}, IDATݼE뀊..b٥rj]vzgѹ(J9]짅s#L@Q(Mkq>OwܵfTvV=7mX.MsĨ oG|*<}CRSS A_JNJ2d005'Sg`mmNwzycX?m.E<Ӧt z R%J}W>ݬ7mX=w '>y.h73J;;Ctq M+'>6v5֭ۏ=^w*++}TAޏS"k~]: ڈ nnظCFžr_1ѡwnj>ի4d=;fvALbv3]wY <{DφRSS4͉Sf꙰zZ.ӎl߶1;;;LrXYWOm1ѯt x"Ҳu۠/f{GͰ.ʳWJiv|wqMO! ABQP 2=Zֶvj]ZoֶڪZ@P 2ApMX!B~7=0HHu<'[Ó`Ln['O|d귶mW~affNs5A Ps_zy05gqUo ŨQ c;umq}.@eAgD (<+9d>gV׼YX,DEǽgOcF/+*g`@ѨRtdE}2fBs;^~YV27?Quqd"_] 0`–skkMfh;w~li}zu5}|>|M^IM"_5)] _:8:*RP_,~*232D~9ma?vRUA@CP*lꋏjH?]L8aM555I$N{ϝoȆo9c)ήGg??M:KyVU} =(ɗve2H$KS3efQ O^|0ehXT/ի.ΎZ0< @ u7 C"_^RMnա[EDmmMnN?'PۉP"nk(A@P_UYQQQ hߥ@P/6qd&3m|Y|Ռgş|KCCïFj)̪JNJhhheJ…˙Y %trp`*3Śeaa)okogaa`Tן#aشRȑ ' ,u{ >9G}knnv ܨ@WH$*_  ~e%_ ohm"k4x{|)7sGk=:eCD"*La}"}첱ao7MϞ:^xrLc?M>s.n}(]2JmhTCC=⪪_QOU_YD)U[gEm ]?"0]J鎏?5t骽?v=gnxf [|y쉵ݮ=_~mz~9h`t;X*nRyl?%oѳϽr> =d KvΝңq}'Z*oɃff6vv.BC_19lc}q(:ihmgч҇EԦ_Y5 [ɎG+5i{XP\|vI[Z]~Oe__0׷, [wS2fOݲ5~E=_~M]횏=v寴++mbWɃ~-b<9˛T9د~bD"Yӓ|,N(?uO`phdT ?ggHH}/ɗ.F9?nEqqkjkkO-B 1c [(d{Vuttajj '</33SYP"l̙3 O(%+e|~A\vԠU}}n][ZZ1VVVVVLQ\ZYZg~zfm!sÕ_lyu!*m5d#Wn1 {Y>_#RF^^fNJqn{𥳐ϫ(mlds^nRrߎvrC5 jkkk]S˯5aGmupppu1@y 6DRS]OUUEEwXZrpvw[>}O>9۷5_>ݺښ><ҫ/(?Kz*"~')USLQY+6k(㶽Srs{@],g:555 J~E_XxԅN3F3NVֲ,&IJVemmCt:‘4s}|} )qTh4|Ug_1izw铋o}уdOwp|'~E:CKCãG똓Ө踅yti~ܢѣ:z{}ii QkkG^,..7.%$_ɻ"?&R!s})W1 Ћ*mC[eeS Yښj_ *+`l+}}};{ţNNNʦN.۟ #c|8UlzM59g._J8'f{ԓ~"/˻SUUU_WCll&Mv +M9ikaabsP,nZZDVC:ڶMrUc@Vn,myEӭmlXVV֎L'GG3{(!2%"~Zv;A=Q:>j7? v"'Q"޼w=%KJWWUZ[BYY//olN@ 쬮U\UUUTeG *fݕJV}5l Cy۾JܢnJKcG#uҕ*-o?m~Jg Ltt,^ƽr1b~wO8MsFlkJy%^8/'44M$jnLpztS/ށٿ}~Ǔ'(9vsf |Sdⶶr>⪪\vԠR;,-dV5(+vY#K`򬦦Ƣ‚ɞSUZFdgZu53C1tDr%-ERϯ\015S~̙/?ꬆw~cSϫ:]q–n%Hj*~/;jR- ml(G ZXZ1F98SWh4bŋ~t8+SI,1b0+?f^QIG'&qM|ײ?r`HhlWU A}EuW-L'`92Pw(Y IMH_X3ʯ: 7b:"Aݽ)A#Q>~+ȿrY3,5}=^SS~1u`0uvv_]Yu`UV++)ٖWt##&s$;jʚtrptQY#NP\TI\e-^IJ š3*%oyo~V~燝Aᑱ_ 4ApM$c Acllbcckkgoggomckcckckkccgg`cckmckkkkmcLF(g8)|~h%ImqY+ԄUr F1giƍwukcccai-JΟE :G)qAO0qҘ1ncML  ݽvo:'41 taK oJ=5hai4сt~#0xM󱵵į,^R*=Ued[l&i)'CCìkw>djjbs^^z3[6)OPrGSS#8$llmlg?«v~'% eLIӧ^p{ewvc"􃑑{thQVV֎L'G  -"NۉP0 ?z%tyK==HH$ʜܘ%Ȝ2u29RsEC++ܹ]4~_Tɑt:}ƷFv]icY0tX,sɑ$teeetrrppB+IggUg\ɼ^l~1Nm=gDsC#˳RS.*9=9^SMM9ٙ3\$-5YTpHFS2 U=bŇN:75]IO9^J tʳF9si4SɃŮn}Np+%9AKsC#8Qq3ffPɗ4WEě[or{wƎ}A-bb#b"b O&fRUদƖfSS`033v T(llME''%~,x> YJō3,==9a}NKW*g޼M`Ų\g>CCSI֖s*/bb|NNfǒ, gWN@Fsg`k; *gP1ݔg]< er-&vvy⻔m\ jkkjml5/Q,ϪknGArxi\e^F@P L'c'̉Dzv("b>t;%v9SOOɉKѨQZ+)1,t1^15'ǹȬos4X5a$`sSvQ__{95_PiT/P3ށ<Ԅ1#?y3bK]=u(m3z,u-oI{[n<]0Y@oP0BMtrw9P ~-gfObTr2u]_Ig%_WD,6bqN`?|dd7ٳgYf@muí#S,r$nGOo  TJ$\vTh ʳFy(YA$ğ<|YA̙!anh;[7zؘ2_~#/ 50,aG }:--%%)M{Dlbs|g9h(OnݒEBabbbbb͛}||"##Y, rX[@acm"192ylw47RRTP߄,P0Gt˿]IRů$ğ!_~7~똒GE"b)\"wC IDAT"6nQm|M))HqK:݈`Xl˟3cLIVttttK tz꾁<_ҥKZT*R Fs3~ZGSȗ.LDm%32iH7PVRE1G|qByfffFʺʝO677^Iܙ~6˨?< gOR"f!!a~ 旟,aF*]fery +XYY9l6'08sʴl޿rٮ.`:9=q7!xn[jmڴ=444666,,hSQ])HG_d`^|}֗v#ϼ:nd|<-gq1Jy֥ɑ˘؅9ҔzWXx4_򃦠 w_1_jemcmmmbb&n [\}ԱVA#s3ҹ,^mM+̙cqO)ۑhA|ny]!xÃ)=)..޳gϞ={lll¢"""p8rpDI,X UDZ٘pmhW5]Y#]t6Q7F^T/oy$9q[e$ooi..n;>*<$  nؐqQLOIȻ&D,vak"IMe€TUUE";|N9I]]ݟZ'zLum%%mgf(,,F?{|p^AA첣#YɞS) zzzQ~'r򬳧+n974rBIM6YXX ~J'.5/#M/ȿѡt==^,6fs4'8TSrTzIDRd֬Y111 ,pu/9.,G,E(b, N;,/ʸRWWK~4&v! vBJy֩G7m&Y,)U6A c[ӵYjU&#GY_a@SٙcZJ%_{qV2@y&=D>/';ׯ.M:]q:sr/ Eog0;pz&USjgjMbېSZZѮ,v нy܌Ln:VUY 66CigPS%oi&wwٳg|ofɑ @{ xdc, =ǽJ~hYO,\<ѿ^^zE%KWJ_1 lmjj,}`(~]~]mgC˽HqӳxM dk"IЊrؓ;wܹ6,,,***<)X]Y^'E6QX"dei{ 2s_xާ3%W=JwJn-,{Y.157vpvvw:{4w>N=~ɆzKO]r\VJ|N~ŕ B ll,Ϝ27zy?" O]Ϊ/HaF]2m-Uڽ nJZaȐn`mk>cn _I*-(7kN/D`92*+~I`aYs­zy [9WB|˖?6C1X__7[v$;wq3jjjHiys-F0qbq?E-СC"-R{vIFm(hi8K-īU¼g8ZY..}؉YG_~Q)oXŽ J'#B;Z)D~WmubyCAYk^Qf͞h~ dEo#yyʞQZRI$两U/֤__۱:bKeY@d٣9X[S}'c{Mi:9SFzzzu{BS h4TJT,1s $.ݞ8Yh;rj/#M/ȿѡt==^A,6wwZS{"oebb2k֬ hnG B*~c;?U҂kŽ=jjn]zrXF©7?~W9_Z,KQ|΍[MP{Rp8o>M7Jȑ+Y9/}@Aġԓ^VfYE{ nfI>#V'߿>šݡ@W<|bp[Ւ#/}KVP"9=-\tqgNjH??rMo^>~66΢lm}fh;aڿe}jv^[̈ ڶ?]\s>1c;;#cQ0*N3 M67cƍmmm5H.x/>~kY2{5hyi͢ٯ+Y"!پ珷S~J\2lBŬ`'Q|~RinFZ&7K KK+ ehh<`Zh9'lx4¸MNӔaJQ?$ĨZ.ggO9*>pSݰit2]'n7?萜9帆9}||VDVa& 딡kSR6xUԻE= 05c :;i|4_odZx_AyRʳ(zy"zzz /z=־?)jKK%756n~}io PEYe"qcgO>|}ffϽ^s)Ԍ3;DYִggҹܜU`kk  cqOD0iBKFRKU=66666vFF8>y7,}}إaX@O;+Y,a)9g>>%#~T*mnn(/u+JXt< F H2yܖfUWpubsC]g9SJ#(hsNРJ.Z֭CK-U&IJp,v4q2db3ѵMֶ])䈱׿mǢtty%1K9"Jo> 7olrFYXZXxUrӋ>ljHo>Oh<~Rv4616c•}p|fXާ|Ư(OG3scyFY̝ۚ0w>ۥ϶~6 ƆhVjqxQsg|%'kyD__oݪg"W>g>KGǿmҪzߩwW[Qsپ~=~}}89ydiiiddرc ʳ_&طn_ e<=<= }m؀:'n|#KJ+>$߷g=KÈ!T*4wvv{kɆo|VŇ$GN8kײmmt|OKCC+%9 zGGJi44`8L' 0tvv~I˳r1 0<oڴ -T"J#吵ؙl1b{SmV'fMi1~=-I*LcYx]JC.'^kXXkn=酨U| f(-:<%o}Yt#GFG{Vt~fVj!t~گ`X?_70wmKvVW!}J|NJ|vPrzX'LKv%F]MϞ!YAmڴiaaaqqqVVV T@y upTTbzz5[%F_~2ENⶶR%ǎ\q[[Nv]]mGl|18`_h7]dmNg9^ٲn>N6]Jlge}_t~fVjqh.'6rfƞj07=mywvtE6gz񿧏4n~ѧS1gڵʕ+ϟsNt2ZP]L }#+V+Βe+#dO39 SSߚN{Ҳ_\E^߷d1٧V߷VMDd`Ύc.ީ<~:i;;#cm49 M67cƍb#ƞ/S"|J:򥕍3.Tf)sbs){j߅BINL%_Zۚy2CDrFٓ nOJ-~T/61U/ V89ۑ#WnU[rAh}=55MǬ[H[_ephnڵsmܸsҤI6l8{P(}:nTk]L ZZZ,P~{ OSKYea1T{9u.uyzz?Ke,n߃y#/M++`mm3;p2 NFRkΝ111 .tqqv^/]. Gƌw Сc7?ӌyVbk呼l5p #vfw!{Qr|69$VI7T5#>SǪ=7ݼZܯ~˧nXJN761ZTؗ$ONRis9IW::WAMG*h4ڏ?H߻wo׮]vb0AAAQQQ'NV F"✝]KKK };mk˳~~'mA=P\I9~ܙcJrgQq7q¤9a䙔k{Y[[xhɊeW#I bqNvU^zJȘ̂FUW5ʙ3;⌟࡯ߟ,HҒ(H(7!*5JdѣW^F Jdn OC.RUP"nk0+23sQ.W#{vqd,"J$j"BWnfXRS%ϡg]NFPCnrZ*k/JKv)OPm?+'1$вD7ovpp Ǥ ByPguߤPS?l@V3Rb|~kjjlnjZ&&&&&&VvvL&Ӊ4m[ zq˩/&ݸ[RrZ$jhL}}|fqNy022S.i7 -0c2bsnH`xCK-(qaH^6@*ȱ2!_vK]US {Bq` ilާnS"9uK7nfLRigW(ۊZyƃ6؈A~_>Tz#^9[bxmc#6K>hhhzSUUuȑ#GZjMYdmmxKWt:}nhШA1.5/#M/ȿj[8==^,6`P# Zjtzsyr 65 uLY/[4[1$ YO=[RYI%2U͙S7Re%K.h<|P5yɑ Zgj.N<|YDgO6KFܹO?sp[[[BBBBBkfooccc3ʳFҒ+i) e*䪫e-O3k,=== ]< `DJnfSy +XXXgf93Y}&}uCK-]˘[#Pf-hiRBT*iZ {BvɆFMSZc|inaG7nf-]4"C*1k]P^Zxߝ~p]A ]ЈX?I//; BJ^R.f@tyD:MHHHRR%Ivvvvvg}fjj:a„g}666Ii¿P0luvvMMNrӯfUUR6`vs IDAT2!Zvvvaaaaaa `@$.M #ɐ'rw# V: .͊4nOJ-::<5:ݰQgI:$ϩ;#5QVYGjpwO=ĉVzzzޡs /4`Xqyh&NbBBY,#-ã총lJ{*tuDuo.30ȚCк''/x8ZVf:E.0y?u3+5(_4::$>zjmjnȑ#G! 24O:Cy Va빵NDevvӦy3MU]]M䔤D7[jm }|X;QCyB@ QCR*޻]6aהH:KT7TF'y9w*'GRrcэr08%?y?'+ UP@~)JN"o>DF:!ᨴ*8̬+E¶'ЇZ,иVaW_|\WWPt1܆oBVInF^XF bsX3&*hootk; M F7>}ph#$/sH~νgM6&|9r5P3fĠ[*j''v9ِF=2[ɹ^ 3ɑ+E(98YI(׳T.!xakod3w+|v{RmgZ#oe``9}tTgQ(͹:tkR4z7#-WVU]ҊfsC'{zvA%']8uomg0h%KWg)=0cbbtd&g2Atr© WdM_IȥYKWȚZdbbDT*Sztr/`(ge;O cԓy#Z^Of>D+$ۺ|aҗ=Q wr5A]sq˗/o۶dFDD<3l6[y (knnv 8h{ggҹ̚*UWpBBXTg)C"G^;Z)Ÿ{)8?7E8XNrK:Q$tj;#-FFFNgh@yE"./g]嶴\bsnckk3SY UWf4䏛1kk똘@m~ֶ> m퟼5,æ\+R>=ޗkvDzgܓ./ cp0/X|xr??:xCY*#}Ѫ8'tK>Ov\r$vih48/q/95Zҹzm4fY֐SSSm3Ġ< ѣu4ECC}JE^F^ǏQ7˛X,#ICyల4@YheH+YA)~NO~8r Av 3kjxAOe._±n2O:%g^!GfMPi,koOovڰmz !%R]ۿ!+u Uܦ'KBRN`w[Eo_ʷV?enAmAXZ- [@DIq뉧3SQ"<<<444<ttth<4-5Mq޹%HTNӽ}fA!s}|T( y {{joa,0{{~n19?qU![sNOTn.nq?ܸUI8Ypsj-u$g ȑl(69E߿'_m?$lj58ڌR 5Y*WwHGRs>6Ϟf [DO.~+45c<| :\3]|tmD-JāohhhLL 60@ei;a`=/${njic_y*dm8(8wv;uNI[,aIOOر###_~Yd j`trCɯ{=n.tfgn .c=|{z 9?/٨! $K#_8X*__ [Dt?Bd#ζϞtrx,k㥳Yݿ$ufϱFSJʎLHbVzAûyit?Yahe;[SPqg"}6R (m9;ˁ3r_|~ؙt#jk7Z,,,,Yk׮۷ogddر# Yf7rit/ 69l)a&::&Mu{']#%cϴ<9n,j}F=YMƚuq.nSnyՇoJ;qX$qVfƦ (킇V1Azz+ ؓVBNu)+cmfMhw44ܿ].n bsS{"l%$GBcfc);ڲlvq9~'|`mk0w{;{K^/w*m4c9vɪ;eJ]/HUYۚaGvtH~g{JWiR) - j1h{{hԈb^$:(,eg~"=]1=;_ٯ]uڰ>+cahAޖ`[[j}w,w_v O*AnN??}g֯_W5[jڴӯgmuHH-n.!!O>#F2dH.]\a37}K[o|k`xVE V<𸭿|w-2mAvApp?1v૏'~m^2iL÷ژ;re>0kvKN7 BW+-7_d5 i) _~on[0gIZ+A33K:WWYVӷ^{{eQ#"ER):tn١s˳.Rñ_VEf\|ѷծ -M R#Hj*v!5k֑GYl2Xz_|ݿۯNiӦϾ] ҫw߮zo(>ة( ^|_l24əpH[|f9klQj I ^se1MiNAAA$$LV\Wd+aGIIM:x=[زFy <)癖tQ7Ϛȗuj*z}K>%AqVJپn5Β/O?nTʘ/tڕ͚5kС{jԨq2nٲeQ ,(gϛSSRS߫w^}vѫja4 D.延 fL;ed}-wc5k<󙙙n$VZv⊁֫W mi޲]zeGc[j{3רҵWہt;Aů٦m'߸vƔy|nS߰V~>^;I6P }>;no䔤t88QꙌ{My#GUdXUq1O~0'?4OJjrNw^m{ئmR(ٲMz]oO ًPȲ߰]p֨>kNǟ94kkѺ#|={ؿ8S ͝=iҏkVo#IIIm}ӷW~ի Iv?pܹsګI^dg\2??N'5< w4Q |_/>SOi99Y[NKRkOAA7!vÈH$rI>g`3wܡCΟ_;QV׮]GG_G"Ms2u7ݘJ54iVu&M+_cŲ."sպܔjiժ7kQmuըwQ/\19/߰6kjTQ^vگurJy39~}95ZkPyO%&VeMi9V-MJJLMOIZzz߰qJ<[69Kڸ.S5h\M ˤi fN]tʍjխzػJ՝Q &0vf~ܧ[JKhc;zt$ *.!7oСC͛ /LeuaGF|QlH$b,خH$RlVݺسw}uAPW,ZiieiS^އL:qIFFnnNzzFtߠ :,95kfgg}I?._t UT^FVtU5+(lW_\e lKWׯX|8ؓnO_{//m[>m3/2t\v|D۴jŊ_~7_=|Y'xgŦigqܨ*e+.;?0eħ{VZiV\yǕ4 .v0]N/vd6FYz/bf-8@[jY?}k87A ۢEÇ>|РAUVwkx~FׯkvÆѫw=z֭Wέ¡rtݷ{;n'+2DWJ,?D"ͷWLH=xݷbl޼YNݓN9,V233}螗^|f뭧VXQ"3gL?cׯ_Wd<--=))ikӦM*1Uhꔉ?_z#)9-FVVȳO3{Vwrkf?.ʯE˖)e9ݵmd8x`e 4'vǫ jթ~q8,r&Z7ug pٯk:yއ+:3rUmyowLIMo`}nۡmkժ]JO!|ٿ/\0/Z7ǞRJtZjq⏿ݷ  ijoؑSN;线-;֥_q<1=^{{ow}JQjv9y(}lGcelٶcS/ΨV)v%-P9ijv?Ϝq-}g\^^|y4dW^͙g}3EwTZzsO?q䃇vǽ؛nȣ朄Q\r1/=w*+UY;nӣiب~[j2-OF_׫_GfSRC'̝=U6eo?[$#Z9m>O ۆ7Gi׾>v<Ȁ^{-[,خ͛wx֎ǍlII6Aە4` :tܶ}A~M6mڔ~ ֭[9??7/'kcVk k3 ؘ1QKNNZZժUOHLLIIRj5k֊D"!H$Rf TPF TTz WRVXfuiNKOOeWW۩VFlnmf$&%ի`钌rUaҥ'-7nz*f%>B|żɧU~ on))){9ˊ=ƫ4ٳ(dIeKsUvs_fVJ<7\Aջ^P5=Gf5i2dȠAԩ#{,خ-[ApACQ+deefggedAAvVvvNvvt0';.,rsN*~5'';';{u6mĶXyyyEz\۵S}Xy6l[Μ94WUჅ9ŊxG'Apđa?csb׬^}2//}7 |E;n7_-:waтye-I&…7>t^/|]f;}8!!{UQÇ ֻwoe;q^掸Wa,4Xa,4Xa,b`Q  `]yv;;vD騯t%d~g&ǟgxbVN?ߕol޼Gf[mg~S&O\8ukխWa~:!իW/\tޏn5}SN8➿=ޮ}i-۵x#^-[[֫ߠ4WrڲeKw7+oQRme,`W!(X;O,;v0A f`Y?gcY5ky)wE G[UQNv>N XnJYy999O~я׮]`ݺuaR q{噧III-Zc`ɢEFt>ࠡݺjfZj''%eɜ:u;5_FGYY/|ZJ ! Y9SL?5_,9 92}YY2'hܸq[J]>u}ԨQt[쬌Kdl/AÆeMwpPYa._[ڻ^tY~6n$===)9nzt۝?5k֊N^`޳O?ՒߴF]e9i F]֣g{ݷ1ngۛ崜b)Mh˯MI+,=x#FUĎiwn==#W^}ñǟTEv(:t\bG^\t?]y٨z5Zu{_z/7VxV<&}RjlWxV^lYuEJW̛c4;vKy͈g칚5k~OWW\u}~ZjSFͤkwC;o6"JE<==vK: * &'';LjS"vXEˌӧN{roDQ_q57Fׯ]ɧ_x]쬋.<{Æ e}#ATEjڵ*3sMlYZyYy$.ÕENZ26{f9Mbˮ=zr{;r.`㎖[nMN-HC;rꔉ򧟾o"sVXqݷFJHH8ӻtqǭZpp钌ELo$ZĖ+V./U+Wlqc-)7^u钌;t4r,K8˗-6uR_AJsc_Ǝ~y^qM%5jĖY99%Ƴbw `y[y6W^sc1 v{=#…_P[+\|fZli-u-񪫯bh٭{;~$u^i'ֱӾiSпKVƼ ֩_dKb f[ƖkV*UE߳O 3%Ώu鞔\ xVY%ƽ#%^[Oef-y/LE<|{Il)۵/2!111-UP5?sTV(33A|ự(2ȁӧO Jaꖩ mof1z:Fƭ]v{ ^}ؑ!C-MijV*Znvܿ?%ɉ٫UkǾ%sG0Oo^~}*#>6Z<e׮=4iZdNzZl-?l%Kbhܵ[O+%&%} rя=ӧM=ݪMmΌ˷`"@uYNxwQ~uڭG!7;睷<Ёu͛7x|ժ|K/<}]V/-͛3Ca\=hcO6gfff^KcGN=c6gz׹99]5PPPpӍ,^0:[J;Դh?㭧o*vho .$)P1U^rU?ܼxRf))ݺLKF 9k3Ӿ?~ EAjּSϾҼEW[r)Wëhم7npޣWF9௬Zis#~2//ӎ;m99wHq6<oFnܸaI~AcLKK` 5e5my._ABbª+a6???O<[E,#+>_fYY>]9=*Tzz6Hܿ5 ,=QbBb٬۶|p\dQBvvփ9 I*UyN<2#cqtp)3OԴǟ|qYO]p-:RPP0yҏ'mBzÍo=^GO9eKD/_'ncrjC=Ӽym.o "! i9vJ=g^yʔ `Օگ5oѴiؑU]JYƼs;jf_|wۻ?yu+MIII.Ɀ?߰*#j޼ŘWu_7mب3/V{5gӫV^aMԱ>]u39%|kpTbYE"n瞒S8r.iߡcޥD͛x7c9s0 |YsAJZj)+/edl{ױƍzѧy^FN9QӦ^}1/>S/\8Wk׮s EիW/f[|Gl.˃ ݡPacƌu$97'{Ν^mO:xZh'ݣgn{ƷիVMòeVZDԭסCD"^sޜ3fN[hƍ 5jԬ]n}4mڬ+M>uڔI+/\v:mwonܦr[y걅 iyn`[|ΐD"'s`gOŻ `:rxw.O>-b>0D"v`ִiSY>]΀T-[iٺMEV(#СSv} 4Xh{FÆ x7յkxںt=gkժUn޽{-';۷oƍ駥Knڴ) vڵUVݖx֭[n:]n ,PgB< YHw7oʎwڼys[6,3gw.JpCxRZ[Dj@Qvw^{u4TPbbb@QYTFƻ 7x@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB!kIDAT ,PgB< In_wG{@(ijB! ,P$Ż*Y$IJJڴiSt$-%x;;gԩݟ DY;ƻ`#(xniӦ>BթSƍǻ %ŻBѭ[O>}ʕnM$&&֭[cǎM6w/ m5k֬Yf\7x@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,PgB< Yx@(ijB! ,ڹC0< ;54bpHʪޒsܟg<,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ,< !Hȳ???f{@Bg$Yc۶{޽p c˲z9ry w|:Zz~nm۫KceYu=4},>47{IENDB`distributed-1.20.2/docs/source/images/scheduler-detailed.svg000066400000000000000000001710531321233345200240700ustar00rootroot00000000000000 image/svg+xml 192.168.0.1 192.168.0.2 192.168.0.3 192.168.0.4 worker queues worker core worker core worker core worker core worker core worker core scheduler queue report queue Computationgraph whohaswhat center/deletion sockets Scheduler distributed-1.20.2/docs/source/images/task-state.dot000066400000000000000000000006661321233345200224110ustar00rootroot00000000000000digraph{ graph [ bgcolor="#FFFFFFF00", rankdir=LR, ]; released1 [label=released]; released2 [label=released]; new -> released1; released1 -> waiting; waiting -> processing; waiting -> "no-worker"; "no-worker" -> waiting; "no-worker" -> processing; processing -> memory; processing -> error; error -> forgotten; memory -> released2; released2 -> forgotten; } distributed-1.20.2/docs/source/images/task-state.svg000066400000000000000000000146231321233345200224200ustar00rootroot00000000000000 %3 released1 released waiting waiting released1->waiting released2 released forgotten forgotten released2->forgotten new new new->released1 processing processing waiting->processing no-worker no-worker waiting->no-worker memory memory processing->memory error error processing->error no-worker->waiting no-worker->processing memory->released2 error->forgotten distributed-1.20.2/docs/source/images/worker-dep-state.dot000066400000000000000000000002011321233345200235070ustar00rootroot00000000000000digraph{ graph [ bgcolor="#FFFFFFF00", rankdir=LR, ]; waiting -> flight; flight -> memory; } distributed-1.20.2/docs/source/images/worker-dep-state.svg000066400000000000000000000036401321233345200235320ustar00rootroot00000000000000 %3 waiting waiting flight flight waiting->flight memory memory flight->memory distributed-1.20.2/docs/source/images/worker-task-state.dot000066400000000000000000000004171321233345200237120ustar00rootroot00000000000000digraph{ graph [ bgcolor="#FFFFFFF00", rankdir=LR, ]; waiting -> ready; ready -> executing; executing -> "long-running"; executing -> memory; executing -> error; "long-running" -> memory; "long-running" -> error; } distributed-1.20.2/docs/source/images/worker-task-state.svg000066400000000000000000000107111321233345200237210ustar00rootroot00000000000000 %3 waiting waiting ready ready waiting->ready executing executing ready->executing long-running long-running executing->long-running memory memory executing->memory error error executing->error long-running->memory long-running->error distributed-1.20.2/docs/source/index.rst000066400000000000000000000103511321233345200202050ustar00rootroot00000000000000Dask.distributed ================ Dask.distributed is a lightweight library for distributed computing in Python. It extends both the ``concurrent.futures`` and ``dask`` APIs to moderate sized clusters. See :doc:`the quickstart ` to get started. Motivation ---------- ``Distributed`` serves to complement the existing PyData analysis stack. In particular it meets the following needs: * **Low latency:** Each task suffers about 1ms of overhead. A small computation and network roundtrip can complete in less than 10ms. * **Peer-to-peer data sharing:** Workers communicate with each other to share data. This removes central bottlenecks for data transfer. * **Complex Scheduling:** Supports complex workflows (not just map/filter/reduce) which are necessary for sophisticated algorithms used in nd-arrays, machine learning, image processing, and statistics. * **Pure Python:** Built in Python using well-known technologies. This eases installation, improves efficiency (for Python users), and simplifies debugging. * **Data Locality:** Scheduling algorithms cleverly execute computations where data lives. This minimizes network traffic and improves efficiency. * **Familiar APIs:** Compatible with the `concurrent.futures`_ API in the Python standard library. Compatible with `dask`_ API for parallel algorithms * **Easy Setup:** As a Pure Python package distributed is ``pip`` installable and easy to :doc:`set up ` on your own cluster. .. _`concurrent.futures`: https://www.python.org/dev/peps/pep-3148/ .. _`dask`: http://dask.pydata.org/en/latest/ Architecture ------------ Dask.distributed is a centrally managed, distributed, dynamic task scheduler. The central ``dask-scheduler`` process coordinates the actions of several ``dask-worker`` processes spread across multiple machines and the concurrent requests of several clients. The scheduler is asynchronous and event driven, simultaneously responding to requests for computation from multiple clients and tracking the progress of multiple workers. The event-driven and asynchronous nature makes it flexible to concurrently handle a variety of workloads coming from multiple users at the same time while also handling a fluid worker population with failures and additions. Workers communicate amongst each other for bulk data transfer over TCP. Internally the scheduler tracks all work as a constantly changing directed acyclic graph of tasks. A task is a Python function operating on Python objects, which can be the results of other tasks. This graph of tasks grows as users submit more computations, fills out as workers complete tasks, and shrinks as users leave or become disinterested in previous results. Users interact by connecting a local Python session to the scheduler and submitting work, either by individual calls to the simple interface ``client.submit(function, *args, **kwargs)`` or by using the large data collections and parallel algorithms of the parent ``dask`` library. The collections in the dask_ library like dask.array_ and dask.dataframe_ provide easy access to sophisticated algorithms and familiar APIs like NumPy and Pandas, while the simple ``client.submit`` interface provides users with custom control when they want to break out of canned "big data" abstractions and submit fully custom workloads. .. _dask.array: http://dask.pydata.org/en/latest/array.html .. _dask.dataframe: http://dask.pydata.org/en/latest/dataframe.html Contents -------- .. toctree:: :maxdepth: 1 :caption: Getting Started install quickstart setup client api faq .. toctree:: :maxdepth: 1 :caption: Build Understanding diagnosing-performance efficiency limitations locality manage-computation memory related-work resilience scheduling-policies scheduling-state worker work-stealing .. toctree:: :maxdepth: 1 :caption: Additional Features adaptive asynchronous configuration ec2 local-cluster ipython joblib publish queues resources submitting-applications task-launch tls web .. toctree:: :maxdepth: 1 :caption: Developer Documentation changelog communications develop foundations journey protocol serialization plugins distributed-1.20.2/docs/source/install.rst000066400000000000000000000023301321233345200205420ustar00rootroot00000000000000Install Dask.Distributed ======================== You can install dask.distributed with ``conda``, with ``pip``, or by installing from source. Conda ----- To install the latest version of dask.distributed from the `conda-forge `_ repository using `conda `_:: conda install dask distributed -c conda-forge Pip --- Or install distributed with ``pip``:: pip install dask distributed --upgrade Source ------ To install distributed from source, clone the repository from `github `_:: git clone https://github.com/dask/distributed.git cd distributed python setup.py install Notes ----- **Note for Macports users:** There `is a known issue `_. with python from macports that makes executables be placed in a location that is not available by default. A simple solution is to extend the ``PATH`` environment variable to the location where python from macports install the binaries:: $ export PATH=/opt/local/Library/Frameworks/Python.framework/Versions/3.5/bin:$PATH or $ export PATH=/opt/local/Library/Frameworks/Python.framework/Versions/2.7/bin:$PATH distributed-1.20.2/docs/source/ipython.rst000066400000000000000000000051001321233345200205640ustar00rootroot00000000000000IPython Integration =================== Dask.distributed integrates with IPython in three ways: 1. You can launch a Dask.distributed cluster from an IPyParallel_ cluster 2. You can launch IPython kernels from Dask Workers and Schedulers to assist with debugging 3. They both support the common concurrent.futures_ interface .. _IPyParallel: https://ipyparallel.readthedocs.io/en/latest/ .. _concurrent.futures: https://docs.python.org/3/library/concurrent.futures.html Launch Dask from IPyParallel ---------------------------- IPyParallel is IPython's distributed computing framework that allows you to easily manage many IPython engines on different computers. An IPyParallel ``Client`` can launch a ``dask.distributed`` Scheduler and Workers on those IPython engines, effectively launching a full dask.distributed system. This is possible with the Client.become_dask_ method:: $ ipcluster start .. code-block:: python >>> from ipyparallel import Client >>> c = Client() # connect to IPyParallel cluster >>> e = c.become_dask() # start dask on top of IPyParallel >>> e .. _Client.become_dask: https://ipyparallel.readthedocs.io/en/latest/api/ipyparallel.html#ipyparallel.Client.become_dask Launch IPython within Dask Workers ---------------------------------- It is sometimes convenient to inspect the ``Worker`` or ``Scheduler`` process interactively. Fortunately IPython gives us a way to launch interactive sessions within Python processes. This is available through the following methods: .. currentmodule:: distributed.client .. autosummary:: Client.start_ipython_workers Client.start_ipython_scheduler These methods start IPython kernels running in a separate thread within the specified Worker or Schedulers. These kernels are accessible either through IPython magics or a QT-Console. Example with IPython Magics ~~~~~~~~~~~~~~~~~~~~~~~~~~~ .. code-block:: python >>> e.start_ipython_scheduler() >>> %scheduler scheduler.processing {'127.0.0.1:3595': ['inc-1', 'inc-2'], '127.0.0.1:53589': ['inc-2', 'add-5']} >>> info = e.start_ipython_workers() >>> %remote info['127.0.0.1:3595'] worker.active {'inc-1', 'inc-2'} Example with qt-console ~~~~~~~~~~~~~~~~~~~~~~~ You can also open up a full interactive `IPython qt-console`_ on the scheduler or each of the workers: .. code-block:: python >>> e.start_ipython_scheduler(qtconsole=True) >>> e.start_ipython_workers(qtconsole=True) .. _`IPython qt-console`: https://ipython.org/ipython-doc/3/interactive/qtconsole.html distributed-1.20.2/docs/source/joblib.rst000066400000000000000000000051451321233345200203440ustar00rootroot00000000000000Joblib Integration ================== Dask.distributed integrates with Joblib_ by providing an alternative cluster-computing backend, alongside Joblib's builtin threading and multiprocessing backends. Joblib_ is a library for simple parallel programming primarily developed and used by the Scikit Learn community. As of version 0.10.0 it contains a plugin mechanism to allow Joblib code to use other parallel frameworks to execute computations. The ``dask.distributed`` scheduler implements such a plugin in the ``distributed.joblib`` module and registers it appropriately with Joblib. As a result, any joblib code (including many scikit-learn algorithms) will run on the distributed scheduler if you enclose it in a context manager as follows: .. code-block:: python import distributed.joblib from joblib import Parallel, parallel_backend with parallel_backend('dask.distributed', scheduler_host='HOST:PORT'): # normal Joblib code Note that scikit-learn bundles joblib internally, so if you want to specify the joblib backend you'll need to import ``parallel_backend`` from scikit-learn instead of ``joblib``. As an example you might distributed a randomized cross validated parameter search as follows. .. code-block:: python import distributed.joblib # Scikit-learn bundles joblib, so you need to import from # `sklearn.externals.joblib` instead of `joblib` directly from sklearn.externals.joblib import parallel_backend from sklearn.datasets import load_digits from sklearn.grid_search import RandomizedSearchCV from sklearn.svm import SVC import numpy as np digits = load_digits() param_space = { 'C': np.logspace(-6, 6, 13), 'gamma': np.logspace(-8, 8, 17), 'tol': np.logspace(-4, -1, 4), 'class_weight': [None, 'balanced'], } model = SVC(kernel='rbf') search = RandomizedSearchCV(model, param_space, cv=3, n_iter=50, verbose=10) with parallel_backend('dask.distributed', scheduler_host='localhost:8786'): search.fit(digits.data, digits.target) For large arguments that are used by multiple tasks, it may be more efficient to pre-scatter the data to every worker, rather than serializing it once for every task. This can be done using the ``scatter`` keyword argument, which takes an iterable of objects to send to each worker. .. code-block:: python # Serialize the training data only once to each worker with parallel_backend('dask.distributed', scheduler_host='localhost:8786', scatter=[digits.data, digits.target]): search.fit(digits.data, digits.target) .. _Joblib: https://pythonhosted.org/joblib/ distributed-1.20.2/docs/source/journey.rst000066400000000000000000000161341321233345200205760ustar00rootroot00000000000000Journey of a Task ================= We follow a single task through the user interface, scheduler, worker nodes, and back. Hopefully this helps to illustrate the inner workings of the system. User code --------- A user computes the addition of two variables already on the cluster, then pulls the result back to the local process. .. code-block:: python client = Client('host:port') x = e.submit(...) y = e.submit(...) z = client.submit(add, x, y) # we follow z print(z.result()) Step 1: Client ---------------- ``z`` begins its life when the ``Client.submit`` function sends the following message to the ``Scheduler``:: {'op': 'update-graph', 'tasks': {'z': (add, x, y)}, 'keys': ['z']} The client then creates a ``Future`` object with the key ``'z'`` and returns that object back to the user. This happens even before the message has been received by the scheduler. The status of the future says ``'pending'``. Step 2: Arrive in the Scheduler ------------------------------- A few milliseconds later, the scheduler receives this message on an open socket. The scheduler updates its state with this little graph that shows how to compute ``z``.:: scheduler.tasks.update(msg['tasks']) The scheduler also updates *a lot* of other state. Notably, it has to identify that ``x`` and ``y`` are themselves variables, and connect all of those dependencies. This is a long and detail oriented process that involves updating roughly 10 sets and dictionaries. Interested readers should investigate ``distributed/scheduler.py::update_state()``. While this is fairly complex and tedious to describe rest assured that it all happens in constant time and in about a millisecond. Step 3: Select a Worker ----------------------- Once the latter of ``x`` and ``y`` finishes, the scheduler notices that all of ``z``'s dependencies are in memory and that ``z`` itself may now run. Which worker should ``z`` select? We consider a sequence of criteria: 1. First, we quickly downselect to only those workers that have either ``x`` or ``y`` in local memory. 2. Then, we select the worker that would have to gather the least number of bytes in order to get both ``x`` and ``y`` locally. E.g. if two different workers have ``x`` and ``y`` and if ``y`` takes up more bytes than ``x`` then we select the machine that holds ``y`` so that we don't have to communicate as much. 3. If there are multiple workers that require the minimum number of communication bytes then we select the worker that is the least busy ``z`` considers the workers and chooses one based on the above criteria. In the common case the choice is pretty obvious after step 1. ``z`` waits on a stack associated with the chosen worker. The worker may still be busy though, so ``z`` may wait a while. *Note: This policy is under flux and this part of this document is quite possibly out of date.* Step 4: Transmit to the Worker ------------------------------ Eventually the worker finishes a task, has a spare core, and ``z`` finds itself at the top of the stack (note, that this may be some time after the last section if other tasks placed themselves on top of the worker's stack in the meantime.) We place ``z`` into a ``worker_queue`` associated with that worker and a ``worker_core`` coroutine pulls it out. ``z``'s function, the keys associated to its arguments, and the locations of workers that hold those keys are packed up into a message that looks like this:: {'op': 'compute', 'function': execute_task, 'args': ((add, 'x', 'y'),), 'who_has': {'x': {(worker_host, port)}, 'y': {(worker_host, port), (worker_host, port)}}, 'key': 'z'} This message is serialized and sent across a TCP socket to the worker. Step 5: Execute on the Worker ----------------------------- The worker unpacks the message, and notices that it needs to have both ``x`` and ``y``. If the worker does not already have both of these then it gathers them from the workers listed in the ``who_has`` dictionary also in the message. For each key that it doesn't have it selects a valid worker from ``who_has`` at random and gathers data from it. After this exchange, the worker has both the value for ``x`` and the value for ``y``. So it launches the computation ``add(x, y)`` in a local ``ThreadPoolExecutor`` and waits on the result. *In the mean time the worker repeats this process concurrently for other tasks. Nothing blocks.* Eventually the computation completes. The Worker stores this result in its local memory:: data['x'] = ... And transmits back a success, and the number of bytes of the result:: Worker: Hey Scheduler, 'z' worked great. I'm holding onto it. It takes up 64 bytes. The worker does not transmit back the actual value for ``z``. Step 6: Scheduler Aftermath ---------------------------- The scheduler receives this message and does a few things: 1. It notes that the worker has a free core, and sends up another task if available 2. If ``x`` or ``y`` are no longer needed then it sends a message out to relevant workers to delete them from local memory. 3. It sends a message to all of the clients that ``z`` is ready and so all client ``Future`` objects that are currently waiting should, wake up. In particular, this wakes up the ``z.result()`` command executed by the user originally. Step 7: Gather --------------- When the user calls ``z.result()`` they wait both on the completion of the computation and for the computation to be sent back over the wire to the local process. Usually this isn't necessary, usually you don't want to move data back to the local process but instead want to keep in on the cluster. But perhaps the user really wanted to actually know this value, so they called ``z.result()``. The scheduler checks who has ``z`` and sends them a message asking for the result. This message doesn't wait in a queue or for other jobs to complete, it starts instantly. The value gets serialized, sent over TCP, and then deserialized and returned to the user (passing through a queue or two on the way.) Step 8: Garbage Collection --------------------------- The user leaves this part of their code and the local variable ``z`` goes out of scope. The Python garbage collector cleans it up. This triggers a decremented reference on the client (we didn't mention this, but when we created the ``Future`` we also started a reference count.) If this is the only instance of a Future pointing to ``z`` then we send a message up to the scheduler that it is OK to release ``z``. The user no longer requires it to persist. The scheduler receives this message and, if there are no computations that might depend on ``z`` in the immediate future, it removes elements of this key from local scheduler state and adds the key to a list of keys to be deleted periodically. Every 500 ms a message goes out to relevant workers telling them which keys they can delete from their local memory. The graph/recipe to create the result of ``z`` persists in the scheduler for all time. Overhead -------- The user experiences this in about 10 milliseconds, depending on network latency. distributed-1.20.2/docs/source/limitations.rst000066400000000000000000000034461321233345200214410ustar00rootroot00000000000000Limitations =========== Dask.distributed has limitations. Understanding these can help you to reliably create efficient distributed computations. Performance ----------- - The central scheduler spends a few hundred microseconds on every task. For optimal performance, task durations should be greater than 10-100ms. - Dask can not parallelize within individual tasks. Individual tasks should be a comfortable size so as not to overwhelm any particular worker. - Dask assigns tasks to workers heuristically. It *usually* makes the right decision, but non-optimal situations do occur. - The workers are just Python processes, and inherit all capabilities and limitations of Python. They do not bound or limit themselves in any way. In production you may wish to run dask-workers within containers. Assumptions on Functions and Data --------------------------------- Dask assumes the following about your functions and your data: - All functions must be serializable either with pickle or `cloudpickle `_. This is *usually* the case except in fairly exotic situations. The following should work:: from cloudpickle import dumps, loads loads(dumps(my_object)) - All data must be serializable either with pickle, cloudpickle, or using Dask's custom serialization system. - Dask may run your functions multiple times, such as if a worker holding an intermediate result dies. Any side effects should be `idempotent `_. - Security -------- As a distributed computing framework, Dask enables the remote execution of arbitrary code. You should only host dask-workers within networks that you trust. This is standard among distributed computing frameworks, but is worth repeating. distributed-1.20.2/docs/source/local-cluster.rst000066400000000000000000000015541321233345200216540ustar00rootroot00000000000000Local Cluster ============= For convenience you can start a local cluster from your Python session. .. code-block:: python >>> from distributed import Client, LocalCluster >>> cluster = LocalCluster() LocalCluster("127.0.0.1:8786", workers=8, ncores=8) >>> client = Client(cluster) You can dynamically scale this cluster up and down: .. code-block:: python >>> worker = cluster.add_worker() >>> cluster.remove_worker(worker) Alternatively, a ``LocalCluster`` is made for you automatically if you create an ``Client`` with no arguments: .. code-block:: python >>> from distributed import Client >>> client = Client() >>> client API --- .. currentmodule:: distributed.deploy.local .. autoclass:: LocalCluster :members: distributed-1.20.2/docs/source/locality.rst000066400000000000000000000144721321233345200207260ustar00rootroot00000000000000Data Locality ============= *Data movement often needlessly limits performance.* This is especially true for analytic computations. Dask.distributed minimizes data movement when possible and enables the user to take control when necessary. This document describes current scheduling policies and user API around data locality. Current Policies ---------------- Task Submission ``````````````` In the common case distributed runs tasks on workers that already hold dependent data. If you have a task ``f(x)`` that requires some data ``x`` then that task will very likely be run on the worker that already holds ``x``. If a task requires data split among multiple workers, then the scheduler chooses to run the task on the worker that requires the least data transfer to it. The size of each data element is measured by the workers using the ``sys.getsizeof`` function, which depends on the ``__sizeof__`` protocol generally available on most relevant Python objects. Data Scatter ```````````` When a user scatters data from their local process to the distributed network this data is distributed in a round-robin fashion grouping by number of cores. So for example If we have two workers ``Alice`` and ``Bob``, each with two cores and we scatter out the list ``range(10)`` as follows: .. code-block:: python futures = client.scatter(range(10)) Then Alice and Bob receive the following data * Alice: ``[0, 1, 4, 5, 8, 9]`` * Bob: ``[2, 3, 6, 7]`` User Control ------------ Complex algorithms may require more user control. For example the existence of specialized hardware such as GPUs or database connections may restrict the set of valid workers for a particular task. In these cases use the ``workers=`` keyword argument to the ``submit``, ``map``, or ``scatter`` functions, providing a hostname, IP address, or alias as follows: .. code-block:: python future = client.submit(func, *args, workers=['Alice']) * Alice: ``[0, 1, 4, 5, 8, 9, new_result]`` * Bob: ``[2, 3, 6, 7]`` Required data will always be moved to these workers, even if the volume of that data is significant. If this restriction is only a preference and not a strict requirement, then add the ``allow_other_workers`` keyword argument to signal that in extreme cases such as when no valid worker is present, another may be used. .. code-block:: python future = client.submit(func, *args, workers=['Alice'], allow_other_workers=True) Additionally the ``scatter`` function supports a ``broadcast=`` keyword argument to enforce that the all data is sent to all workers rather than round-robined. If new workers arrive they will not automatically receive this data. .. code-block:: python futures = client.scatter([1, 2, 3], broadcast=True) # send data to all workers * Alice: ``[1, 2, 3]`` * Bob: ``[1, 2, 3]`` Valid arguments for ``workers=`` include the following: * A single IP addresses, IP/Port pair, or hostname like the following:: 192.168.1.100, 192.168.1.100:8989, alice, alice:8989 * A list or set of the above:: ['alice'], ['192.168.1.100', '192.168.1.101:9999'] If only a hostname or IP is given then any worker on that machine will be considered valid. Additionally, you can provide aliases to workers upon creation.:: $ dask-worker scheduler_address:8786 --name worker_1 And then use this name when specifying workers instead. .. code-block:: python client.map(func, sequence, workers='worker_1') Specify workers with Compute/Persist ------------------------------------ The ``workers=`` keyword in ``scatter``, ``submit``, and ``map`` is fairly straightforward, taking either a worker hostname, host:port pair or a sequence of those as valid inputs: .. code-block:: python client.submit(f, x, workers='127.0.0.1') client.submit(f, x, workers='127.0.0.1:55852') client.submit(f, x, workers=['192.168.1.101', '192.168.1.100']) For more complex computations, such as occur with dask collections like dask.dataframe or dask.delayed, we sometimes want to specify that certain parts of the computation run on certain workers while other parts run on other workers. .. code-block:: python x = delayed(f)(1) y = delayed(f)(2) z = delayed(g)(x, y) future = client.compute(z, workers={z: '127.0.0.1', x: '192.168.0.1'}) Here the values of the dictionary are of the same form as before, a host, a host:port pair, or a list of these. The keys in this case are either dask collections or tuples of dask collections. All of the *final* keys of these collections will run on the specified machines; dependencies can run anywhere unless they are also listed in ``workers=``. We explore this through a set of examples: The computation ``z = f(x, y)`` runs on the host ``127.0.0.1``. The other two computations for ``x`` and ``y`` can run anywhere. .. code-block:: python future = client.compute(z, workers={z: '127.0.0.1'}) The computations for both ``z`` and ``x`` must run on ``127.0.0.1`` .. code-block:: python future = client.compute(z, workers={z: '127.0.0.1', x: '127.0.0.1'}) Use a tuple to group collections. This is shorthand for the above. .. code-block:: python future = client.compute(z, workers={(x, y): '127.0.0.1'}) Recall that all options for ``workers=`` in ``scatter/submit/map`` hold here as well. .. code-block:: python future = client.compute(z, workers={(x, y): ['192.168.1.100', '192.168.1.101:9999']}) Set ``allow_other_workers=True`` to make these loose restrictions rather than hard requirements. .. code-block:: python future = client.compute(z, workers={(x, y): '127.0.0.1'}, allow_other_workers=True) Provide a collection to ``allow_other_workers=[...]`` to say that the keys for only some of the collections are loose. In the case below ``z`` *must* run on ``127.0.0.1`` while ``x`` *should* run on ``127.0.0.1`` but can run elsewhere if necessary: .. code-block:: python future = client.compute(z, workers={(x, y): '127.0.0.1'}, allow_other_workers=[x]) This works fine with ``persist`` and with any dask collection (any object with a ``.__dask_graph__()`` method): .. code-block:: python df = dd.read_csv('s3://...') df = client.persist(df, workers={df: ...}) See the :doc:`efficiency ` page to learn about best practices. distributed-1.20.2/docs/source/manage-computation.rst000066400000000000000000000146221321233345200226730ustar00rootroot00000000000000Managing Computation ==================== Data and Computation in Dask.distributed are always in one of three states 1. Concrete values in local memory. Example include the integer ``1`` or a numpy array in the local process. 2. Lazy computations in a dask graph, perhaps stored in a ``dask.delayed`` or ``dask.dataframe`` object. 3. Running computations or remote data, represented by ``Future`` objects pointing to computations currently in flight. All three of these forms are important and there are functions that convert between all three states. Dask Collections to Concrete Values ----------------------------------- You can turn any dask collection into a concrete value by calling the ``.compute()`` method or ``dask.compute(...)`` function. This function will block until the computation is finished, going straight from a lazy dask collection to a concrete value in local memory. This approach is the most familiar and straightforward, especially for people coming from the standard single-machine Dask experience or from just normal programming. It is great when you have data already in memory and want to get small fast results right to your local process. .. code-block:: python >>> df = dd.read_csv('s3://...') >>> df.value.sum().compute() 100000000 However, this approach often breaks down if you try to bring the entire dataset back to local RAM .. code-block:: python >>> df.compute() MemoryError(...) It also forces you to wait until the computation finishes before handing back control of the interpreter. Dask Collections to Futures --------------------------- You can asynchronously submit lazy dask graphs to run on the cluster with the ``client.compute`` and ``client.persist`` methods. These functions return Future objects immediately. These futures can then be queried to determine the state of the computation. client.compute ~~~~~~~~~~~~~~ The ``.compute`` method takes a collection and returns a single future. .. code-block:: python >>> df = dd.read_csv('s3://...') >>> total = client.compute(df.sum()) # Return a single future >>> total Future(..., status='pending') >>> total.result() # Block until finished 100000000 Because this is a single future the result must fit on a single worker machine. Like ``dask.compute`` above, the ``client.compute`` method is only appropriate when results are small and should fit in memory. The following would likely fail: .. code-block:: python >>> future = client.compute(df) # Blows up memory Instead, you should use ``client.persist`` client.persist ~~~~~~~~~~~~~~ The ``.persist`` method submits the task graph behind the Dask collection to the scheduler, obtaining Futures for all of the top-most tasks (for example one Future for each Pandas DataFrame in a Dask DataFrame). It then returns a copy of the collection pointing to these futures instead of the previous graph. This new collection is semantically equivalent but now points to actively running data rather than a lazy graph. If you look at the dask graph within the collection you will see the Future objects directly: .. code-block:: python >>> df = dd.read_csv('s3://...') >>> df.dask # Recipe to compute df in chunks {('read', 0): (load_s3_bytes, ...), ('parse', 0): (pd.read_csv, ('read', 0)), ('read', 1): (load_s3_bytes, ...), ('parse', 1): (pd.read_csv, ('read', 1)), ... } >>> df = client.persist(df) # Start computation >>> df.dask # Now points to running futures {('parse', 0): Future(..., status='finished'), ('parse', 1): Future(..., status='pending'), ... } The collection is returned immediately and the computation happens in the background on the cluster. Eventually all of the futures of this collection will be completed at which point further queries on this collection will likely be very fast. Typically the workflow is to define a computation with a tool like ``dask.dataframe`` or ``dask.delayed`` until a point where you have a nice dataset to work from, then persist that collection to the cluster and then perform many fast queries off of the resulting collection. Concrete Values to Futures -------------------------- We obtain futures through a few different ways. One is the mechanism above, by wrapping Futures within Dask collections. Another is by submitting data or tasks directly to the cluster with ``client.scatter``, ``client.submit`` or ``client.map``. .. code-block:: python futures = client.scatter(args) # Send data future = client.submit(function, *args, **kwrags) # Send single task futures = client.map(function, sequence, **kwargs) # Send many tasks In this case ``*args`` or ``**kwargs`` can be normal Python objects, like ``1`` or ``'hello'``, or they can be other ``Future`` objects if you want to link tasks together with dependencies. Unlike Dask collections like dask.delayed these task submissions happen immediately. The concurrent.futures interface is very similar to dask.delayed except that execution is immediate rather than lazy. Futures to Concrete Values -------------------------- You can turn an individual ``Future`` into a concrete value in the local process by calling the ``Future.result()`` method. You can convert a collection of futures into concrete values by calling the ``client.gather`` method. .. code-block:: python >>> future.result() 1 >>> client.gather(futures) [1, 2, 3, 4, ...] Futures to Dask Collections --------------------------- As seen in the Collection to futures section it is common to have currently computing ``Future`` objects within Dask graphs. This lets us build further computations on top of currently running computations. This is most often done with dask.delayed workflows on custom computations: .. code-block:: python >>> x = delayed(sum)(futures) >>> y = delayed(product)(futures) >>> future = client.compute(x + y) Mixing the two forms allow you to build and submit a computation in stages like ``sum(...) + product(...)``. This is often valuable if you want to wait to see the values of certain parts of the computation before determining how to proceed. Submitting many computations at once allows the scheduler to be slightly more intelligent when determining what gets run. *If this page interests you then you may also want to check out the doc page on* :doc:`Managing Memory` distributed-1.20.2/docs/source/memory.rst000066400000000000000000000162351321233345200204150ustar00rootroot00000000000000Managing Memory =============== Dask.distributed stores the results of tasks in the distributed memory of the worker nodes. The central scheduler tracks all data on the cluster and determines when data should be freed. Completed results are usually cleared from memory as quickly as possible in order to make room for more computation. The result of a task is kept in memory if either of the following conditions hold: 1. A client holds a future pointing to this task. The data should stay in RAM so that the client can gather the data on demand. 2. The task is necessary for ongoing computations that are working to produce the final results pointed to by futures. These tasks will be removed once no ongoing tasks require them. When users hold Future objects or persisted collections (which contain many such Futures inside their ``.dask`` attribute) they pin those results to active memory. When the user deletes futures or collections from their local Python process the scheduler removes the associated data from distributed RAM. Because of this relationship, distributed memory reflects the state of local memory. A user may free distributed memory on the cluster by deleting persisted collections in the local session. Creating Futures ---------------- The following functions produce Futures .. currentmodule:: distributed.client .. autosummary:: Client.submit Client.map Client.compute Client.persist Client.scatter The ``submit`` and ``map`` methods handle raw Python functions. The ``compute`` and ``persist`` methods handle Dask collections like arrays, bags, delayed values, and dataframes. The ``scatter`` method sends data directly from the local process. Persisting Collections ---------------------- Calls to ``Client.compute`` or ``Client.persist`` submit task graphs to the cluster and return ``Future`` objects that point to particular output tasks. Compute returns a single future per input, persist returns a copy of the collection with each block or partition replaced by a single future. In short, use ``persist`` to keep full collection on the cluster and use ``compute`` when you want a small result as a single future. Persist is more common and is often used as follows with collections: .. code-block:: python >>> # Construct dataframe, no work happens >>> df = dd.read_csv(...) >>> df = df[df.x > 0] >>> df = df.assign(z = df.x + df.y) >>> # Pin data in distributed ram, this triggers computation >>> df = client.persist(df) >>> # continue operating on df *Note for Spark users: this differs from what you're accustomed to. Persist is an immediate action. However, you'll get control back immediately as computation occurs in the background.* In this example we build a computation by parsing CSV data, filtering rows, and then adding a new column. Up until this point all work is lazy; we've just built up a recipe to perform the work as a graph in the ``df`` object. When we call ``df = client.persist(df)`` we cut this graph off of the ``df`` object, send it up to the scheduler, receive ``Future`` objects in return and create a new dataframe with a very shallow graph that points directly to these futures. This happens more or less immediately (as long as it takes to serialize and send the graph) and we can continue working on our new ``df`` object while the cluster works to evaluate the graph in the background. Difference with dask.compute ---------------------------- The operations ``client.persist(df)`` and ``client.compute(df)`` are asynchronous and so differ from the traditional ``df.compute()`` method or ``dask.compute`` function, which blocks until a result is available. The ``.compute()`` method does not persist any data on the cluster. The ``.compute()`` method also brings the entire result back to the local machine, so it is unwise to use it on large datasets. However, ``.compute()`` is very convenient for smaller results particularly because it does return concrete results in a way that most other tools expect. Typically we use asynchronous methods like ``client.persist`` to set up large collections and then use ``df.compute()`` for fast analyses. .. code-block:: python >>> # df.compute() # This is bad and would likely flood local memory >>> df = client.persist(df) # This is good and asynchronously pins df >>> df.x.sum().compute() # This is good because the result is small >>> future = client.compute(df.x.sum()) # This is also good but less intuitive Clearing data ------------- We remove data from distributed ram by removing the collection from our local process. Remote data is removed once all Futures pointing to that data are removed from all client machines. .. code-block:: python >>> del df # Deleting local data often deletes remote data If this is the only copy then this will likely trigger the cluster to delete the data as well. However if we have multiple copies or other collections based on this one then we'll have to delete them all. .. code-block:: python >>> df2 = df[df.x < 10] >>> del df # would not delete data, because df2 still tracks the futures Aggressively Clearing Data -------------------------- To definitely remove a computation and all computations that depend on it you can always ``cancel`` the futures/collection. .. code-block:: python >>> client.cancel(df) # kills df, df2, and every other dependent computation Alternatively, if you want a clean slate, you can restart the cluster. This clears all state and does a hard restart of all worker processes. It generally completes in around a second. .. code-block:: python >>> client.restart() Resilience ---------- Results are not intentionally copied unless necessary for computations on other worker nodes. Resilience is achieved through recomputation by maintaining the provenance of any result. If a worker node goes down the scheduler is able to recompute all of its results. The complete graph for any desired Future is maintained until no references to that future exist. For more information see :doc:`Resilience `. Advanced techniques ------------------- At first the result of a task is not intentionally copied, but only persists on the node where it was originally computed or scattered. However result may be copied to another worker node in the course of normal computation if that result is required by another task that is intended to by run by a different worker. This occurs if a task requires two pieces of data on different machines (at least one must move) or through work stealing. In these cases it is the policy for the second machine to maintain its redundant copy of the data. This helps to organically spread around data that is in high demand. However, advanced users may want to control the location, replication, and balancing of data more directly throughout the cluster. They may know ahead of time that certain data should be broadcast throughout the network or that their data has become particularly imbalanced, or that they want certain pieces of data to live on certain parts of their network. These considerations are not usually necessary. .. currentmodule:: distributed.client .. autosummary:: Client.rebalance Client.replicate Client.scatter distributed-1.20.2/docs/source/plugins.rst000066400000000000000000000001601321233345200205540ustar00rootroot00000000000000Scheduler Plugins ================= .. autoclass:: distributed.diagnostics.plugin.SchedulerPlugin :members: distributed-1.20.2/docs/source/protocol.rst000066400000000000000000000251411321233345200207420ustar00rootroot00000000000000.. _protocol: Protocol ======== The scheduler, workers, and clients pass messages between each other. Semantically these messages encode commands, status updates, and data, like the following: * Please compute the function ``sum`` on the data ``x`` and store in ``y`` * The computation ``y`` has been completed * Be advised that a new worker named ``alice`` is available for use * Here is the data for the keys ``'x'``, and ``'y'`` In practice we represent these messages with dictionaries/mappings:: {'op': 'compute', 'function': ... 'args': ['x']} {'op': 'task-complete', 'key': 'y', 'nbytes': 26} {'op': 'register-worker', 'address': '192.168.1.42', 'name': 'alice', 'ncores': 4} {'x': b'...', 'y': b'...'} When we communicate these messages between nodes we need to serialize these messages down to a string of bytes that can then be deserialized on the other end to their in-memory dictionary form. For simple cases several options exist like JSON, MsgPack, Protobuffers, and Thrift. The situation is made more complex by concerns like serializing Python functions and Python objects, optional compression, cross-language support, large messages, and efficiency. This document describes the protocol used by ``dask.distributed`` today. Be advised that this protocol changes rapidly as we continue to optimize for performance. Overview -------- We may split a single message into multiple message-part to suit different protocols. Generally small bits of data are encoded with MsgPack while large bytestrings and complex datatypes are handled by a custom format. Each message-part gets its own header, which is always encoded as msgpack. After serializing all message parts we have a sequence of bytestrings or *frames* which we send along the wire, prepended with length information. The application doesn't know any of this, it just sends us Python dictionaries with various datatypes and we produce a list of bytestrings that get written to a socket. This format is fast both for many frequent messages and for large messages. MsgPack for Messages -------------------- Most messages are encoded with MsgPack_, a self describing semi-structured serialization format that is very similar to JSON, but smaller, faster, not human-readable, and supporting of bytestrings and (soon) timestamps. We chose MsgPack as a base serialization format for the following reasons: * It does not require separate headers, and so is easy and flexible to use which is particularly important in an early stage project like ``dask.distributed`` * It is very fast, much faster than JSON, and there are nicely optimized implementations. With few exceptions (described later) MsgPack does not come anywhere near being a bottleneck, even under heavy use. * Unlike JSON it supports bytestrings * It covers the standard set of types necessary to encode most information * It is widely implemented in a number of languages (see cross language section below) However, MsgPack fails (correctly) in the following ways: * It does not provide any way for us to encode Python functions or user defined data types * It does not support bytestrings greater than 4GB and is generally inefficient for very large messages. Because of these failings we supplement it with a language-specific protocol and a special case for large bytestrings. CloudPickle for Functions and Some Data --------------------------------------- Pickle and CloudPickle are Python libraries to serialize almost any Python object, including functions. We use these libraries to transform the users' functions and data into bytes before we include them in the dictionary/map that we pass off to msgpack. In the introductory example you may have noticed that we skipped providing an example for the function argument:: {'op': 'compute', 'function': ... 'args': ['x']} That is because this value ``...`` will actually be the result of calling ``cloudpickle.dumps(myfunction)``. Those bytes will then be included in the dictionary that we send off to msgpack, which will only have to deal with bytes rather than obscure Python functions. *Note: we actually call some combination of pickle and cloudpickle, depending on the situation. This is for performance reasons.* Cross Language Specialization ----------------------------- The Client and Workers must agree on a language-specific serialization format. In the standard ``dask.distributed`` client and worker objects this ends up being the following:: bytes = cloudpickle.dumps(obj, protocol=pickle.HIGHEST_PROTOCOL) obj = cloudpickle.loads(bytes) This varies between Python 2 and 3 and so your client and workers must match their Python versions and software environments. However, the Scheduler never uses the language-specific serialization and instead only deals with MsgPack. If the client sends a pickled function up to the scheduler the scheduler will not unpack function but will instead keep it as bytes. Eventually those bytes will be sent to a worker, which will then unpack the bytes into a proper Python function. Because the Scheduler never unpacks language-specific serialized bytes it may be in a different language. **The client and workers must share the same language and software environment, the scheduler may differ.** This has a few advantages: 1. The Scheduler is protected from unpickling unsafe code 2. The Scheduler can be run under ``pypy`` for improved performance. This is only useful for larger clusters. 3. We could conceivably implement workers and clients for other languages (like R or Julia) and reuse the Python scheduler. The worker and client code is fairly simple and much easier to reimplement than the scheduler, which is complex. 4. The scheduler might some day be rewritten in more heavily optimized C or Go Compression ----------- Fast compression libraries like LZ4 or Snappy may increase effective bandwidth by compressing data before sending and decompressing it after reception. This is especially valuable on lower-bandwidth networks. If either of these libraries is available (we prefer LZ4 to Snappy) then for every message greater than 1kB we try to compress the message and, if the compression is at least a 10% improvement, we send the compressed bytes rather than the original payload. We record the compression used within the header as a string like ``'lz4'`` or ``'snappy'``. To avoid compressing large amounts of uncompressable data we first try to compress a sample. We take 10kB chunks from five locations in the dataset, arrange them together, and try compressing the result. If this doesn't result in significant compression then we don't try to compress the full result. Header ------ The header is a small dictionary encoded in msgpack that includes some metadata about the message, such as compression. Serializing Data ---------------- For administrative messages like updating status msgpack is sufficient. However for large results or Python specific data, like NumPy arrays or Pandas Dataframes, or for larger results we need to use something else to convert Python objects to bytestrings. Exactly how we do this is described more in the :doc:`Serialization documentation `. The application code marks Python specific results with the ``to_serialize`` function: .. code-block:: python >>> import numpy as np >>> x = np.ones(5) >>> from distributed.protocol import to_serialize >>> msg = {'status': 'OK', 'data': to_serialize(x)} >>> msg {'data': , 'status': 'OK'} We separate the message into two messages, one encoding all of the data to be serialized and, and one encoding everything else:: {'key': 'x', 'address': 'alice'} {'data': } The first message we pass normally with msgpack. The second we pass in multiple parts, one part for each serialized piece of data (see :doc:`serialization `) and one header including types, compression, etc. used for each value:: {'keys': ['data'], 'compression': ['lz4']} b'...' b'...' Frames ------ At the end of the pipeline we have a sequence of bytestrings or frames. We need to tell the receiving end how many frames there are and how long each these frames are. We order the frames and lengths of frames as follows: 1. The number of frames, stored as an 8 byte unsigned integer 2. The length of each frame, each stored as an 8 byte unsigned integer 3. Each of the frames In the following sections we describe how we create these frames. .. _MsgPack: http://msgpack.org/index.html Technical Version ----------------- A message is broken up into the following components: 1. 8 bytes encoding how many frames there are in the message (N) as a ``uint64`` 2. 8 * N frames encoding the length of each frame as ``uint64`` s 3. Header for the administrative message 4. The administrative message, msgpack encoded, possibly compressed 5. Header for all payload messages 6. Payload messages Header for Administrative Message ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ The administrative message is arbitrary msgpack-encoded data. Usually a dictionary. It may optionally be compressed. If so the compression type will be in the header. Payload frames and Header ~~~~~~~~~~~~~~~~~~~~~~~~~ These frames are optional. Payload frames are used to send large or language-specific data. These values will be inserted into the administrative message after they are decoded. The header is msgpack encoded and contains encoding and compression information for the all subsequent payload messages. A Payload may be spread across many frames. Each frame may be separately compressed. Simple Example ~~~~~~~~~~~~~~ This simple example shows a minimal message. There is only an empty header and a small msgpack message. There are no additional payload frames Message: ``{'status': 'OK'}`` Frames: * Header: ``{}`` * Administrative Message: ``{'status': 'OK'}`` Example with Custom Data ~~~~~~~~~~~~~~~~~~~~~~~~ This example contains a single payload message composed of a single frame. It uses a special serialization for NumPy arrays. Message: ``{'op': 'get-data', 'data': np.ones(5)}`` Frames: * Header: ``{}`` * Administrative Message: ``{'op': 'get-data'}`` * Payload header: :: {'headers': [{'type': 'numpy.ndarray', 'compression': 'lz4', 'count': 1, 'lengths': [40], 'dtype': '>> df2.head() name balance 0 Alice -100 1 Bob -200 2 Charlie -300 3 Dennis -400 4 Edith -500 **Publish** To share this collection with a colleague we publish it under the name ``'negative_accounts'`` .. code-block:: python client.publish_dataset(negative_accounts=df2) **Load published dataset from different client** Now any other client can connect to the scheduler and retrieve this published dataset. .. code-block:: python >>> from dask.distributed import Client >>> client = Client('scheduler-address:8786') >>> client.list_datasets() ['negative_accounts'] >>> df = client.get_dataset('negative_accounts') >>> df.head() name balance 0 Alice -100 1 Bob -200 2 Charlie -300 3 Dennis -400 4 Edith -500 This allows users to easily share results. It also allows for the persistence of important and commonly used datasets beyond a single session. Published datasets continue to reside in distributed memory even after all clients requesting them have disconnected. Dictionary interface -------------------- Alternatively you can use the ``.datasets`` mapping on the client to publish, list, get, and delete global datasets. .. code-block:: python >>> client.datasets['negative_accounts'] = df >>> list(client.datasets) ['negative_accounts'] >>> df = client.datasets['negative_accounts'] This mapping is globally shared among all clients connected to the same scheduler. Notes ----- Published collections are not automatically persisted. If you publish an un-persisted collection then others will still be able to get the collection from the scheduler, but operations on that collection will start from scratch. This allows you to publish views on data that do not permanently take up cluster memory but can be surprising if you expect "publishing" to automatically make a computed dataset rapidly available. Any client can publish or unpublish a dataset. Publishing too many large datasets can quickly consume a cluster's RAM. API --- .. currentmodule:: distributed.client .. autosummary:: Client.publish_dataset Client.list_datasets Client.get_dataset Client.unpublish_dataset distributed-1.20.2/docs/source/queues.rst000066400000000000000000000114321321233345200204060ustar00rootroot00000000000000Data 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. .. 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 distributed-1.20.2/docs/source/quickstart.rst000066400000000000000000000055071321233345200212770ustar00rootroot00000000000000Quickstart ========== Install ------- :: $ pip install dask distributed --upgrade See :doc:`installation ` document for more information. Setup Dask.distributed the Easy Way ----------------------------------- If you create an client without providing an address it will start up a local scheduler and worker for you. .. code-block:: python >>> from dask.distributed import Client >>> client = Client() # set up local cluster on your laptop >>> client Setup Dask.distributed the Hard Way ----------------------------------- This allows dask.distributed to use multiple machines as workers. Set up scheduler and worker processes on your local computer:: $ dask-scheduler Scheduler started at 127.0.0.1:8786 $ dask-worker 127.0.0.1:8786 $ dask-worker 127.0.0.1:8786 $ dask-worker 127.0.0.1:8786 .. note:: At least one ``dask-worker`` must be running after launching a scheduler. Launch an Client and point it to the IP/port of the scheduler. .. code-block:: python >>> from dask.distributed import Client >>> client = Client('127.0.0.1:8786') See :doc:`setup ` for advanced use. Map and Submit Functions ~~~~~~~~~~~~~~~~~~~~~~~~ Use the ``map`` and ``submit`` methods to launch computations on the cluster. The ``map/submit`` functions send the function and arguments to the remote workers for processing. They return ``Future`` objects that refer to remote data on the cluster. The ``Future`` returns immediately while the computations run remotely in the background. .. code-block:: python >>> def square(x): return x ** 2 >>> def neg(x): return -x >>> A = client.map(square, range(10)) >>> B = client.map(neg, A) >>> total = client.submit(sum, B) >>> total.result() -285 Gather ~~~~~~ The ``map/submit`` functions return ``Future`` objects, lightweight tokens that refer to results on the cluster. By default the results of computations *stay on the cluster*. .. code-block:: python >>> total # Function hasn't yet completed >>> total # Function completed, result ready on remote worker Gather results to your local machine either with the ``Future.result`` method for a single future, or with the ``Client.gather`` method for many futures at once. .. code-block:: python >>> total.result() # result for single future -285 >>> client.gather(A) # gather for many futures [0, 1, 4, 9, 16, 25, 36, 49, 64, 81] Restart ~~~~~~~ When things go wrong, or when you want to reset the cluster state, call the ``restart`` method. .. code-block:: python >>> client.restart() See :doc:`client ` for advanced use. distributed-1.20.2/docs/source/related-work.rst000066400000000000000000000210511321233345200214750ustar00rootroot00000000000000Related Work ============ Writing the "related work" for a project called "distributed", is a Sisyphean task. We'll list a few notable projects that you've probably already heard of down below. You may also find the `dask comparison with spark`_ of interest. .. _`dask comparison with spark`: http://dask.pydata.org/en/latest/spark.html Big Data World -------------- * The venerable Hadoop_ provides batch processing with the MapReduce programming paradigm. Python users typically use `Hadoop Streaming`_ or MRJob_. * Spark builds on top of HDFS systems with a nicer API and in-memory processing. Python users typically use PySpark_. * Storm_ provides streaming computation. Python users typically use streamparse_. This is a woefully inadequate representation of the excellent work blossoming in this space. A variety of projects have come into this space and rival or complement the projects above. Still, most "Big Data" processing hype probably centers around the three projects above, or their derivatives. .. _Hadoop: https://hadoop.apache.org/ .. _MRJob: https://pythonhosted.org/mrjob/ .. _`Hadoop Streaming`: https://hadoop.apache.org/docs/r1.2.1/streaming.html .. _Spark: http://spark.apache.org/ .. _PySpark: http://spark.apache.org/docs/latest/api/python/ .. _storm: http://storm.apache.org/ .. _streamparse: https://streamparse.readthedocs.io/en/latest/index.html .. _Disco: http://discoproject.org/ Python Projects --------------- There are dozens of Python projects for distributed computing. Here we list a few of the more prominent projects that we see in active use today. Task scheduling ~~~~~~~~~~~~~~~ * Celery_: An asynchronous task scheduler, focusing on real-time processing. * Luigi_: A bulk big-data/batch task scheduler, with hooks to a variety of interesting data sources. Ad hoc computation ~~~~~~~~~~~~~~~~~~ * `IPython Parallel`_: Allows for stateful remote control of several running ipython sessions. * Scoop_: Implements the `concurrent.futures`_ API on distributed workers. Notably allows tasks to spawn more tasks. Direct Communication ~~~~~~~~~~~~~~~~~~~~ * MPI4Py_: Wraps the Message Passing Interface popular in high performance computing. * PyZMQ_: Wraps ZeroMQ, the gentleman's socket. Venerable ~~~~~~~~~ There are a couple of older projects that often get mentioned * Dispy_: Embarrassingly parallel function evaluation * Pyro_: Remote objects / RPC .. _Luigi: https://luigi.readthedocs.io/en/latest/ .. _MPI4Py: http://mpi4py.readthedocs.io/en/stable/ .. _PyZMQ: https://github.com/zeromq/pyzmq .. _Celery: http://www.celeryproject.org/ .. _`IPython Parallel`: https://ipyparallel.readthedocs.io/en/latest/ .. _Scoop: https://github.com/soravux/scoop/ .. _`concurrent.futures`: https://docs.python.org/3/library/concurrent.futures.html .. _Dispy: http://dispy.sourceforge.net/ .. _Pyro: https://pythonhosted.org/Pyro4/ Relationship ------------ In relation to these projects ``distributed``... * Supports data-local computation like Hadoop and Spark * Uses a task graph with data dependencies abstraction like Luigi * In support of ad-hoc applications, like IPython Parallel and Scoop In depth comparison to particular projects ------------------------------------------ IPython Parallel ~~~~~~~~~~~~~~~~ **Short Description** `IPython Parallel`_ is a distributed computing framework from the IPython project. It uses a centralized hub to farm out jobs to several ``ipengine`` processes running on remote workers. It communicates over ZeroMQ sockets and centralizes communication through the central hub. IPython parallel has been around for a while and, while not particularly fancy, is quite stable and robust. IPython Parallel offers parallel ``map`` and remote ``apply`` functions that route computations to remote workers .. code-block:: python >>> view = Client(...)[:] >>> results = view.map(func, sequence) >>> result = view.apply(func, *args, **kwargs) >>> future = view.apply_async(func, *args, **kwargs) It also provides direct execution of code in the remote process and collection of data from the remote namespace. .. code-block:: python >>> view.execute('x = 1 + 2') >>> view['x'] [3, 3, 3, 3, 3, 3] **Brief Comparison** Distributed and IPython Parallel are similar in that they provide ``map`` and ``apply/submit`` abstractions over distributed worker processes running Python. Both manage the remote namespaces of those worker processes. They are dissimilar in terms of their maturity, how worker nodes communicate to each other, and in the complexity of algorithms that they enable. **Distributed Advantages** The primary advantages of ``distributed`` over IPython Parallel include 1. Peer-to-peer communication between workers 2. Dynamic task scheduling ``Distributed`` workers share data in a peer-to-peer fashion, without having to send intermediate results through a central bottleneck. This allows ``distributed`` to be more effective for more complex algorithms and to manage larger datasets in a more natural manner. IPython parallel does not provide a mechanism for workers to communicate with each other, except by using the central node as an intermediary for data transfer or by relying on some other medium, like a shared file system. Data transfer through the central node can easily become a bottleneck and so IPython parallel has been mostly helpful in embarrassingly parallel work (the bulk of applications) but has not been used extensively for more sophisticated algorithms that require non-trivial communication patterns. The distributed client includes a dynamic task scheduler capable of managing deep data dependencies between tasks. The IPython parallel docs include `a recipe`_ for executing task graphs with data dependencies. This same idea is core to all of ``distributed``, which uses a dynamic task scheduler for all operations. Notably, ``distributed.Future`` objects can be used within ``submit/map/get`` calls before they have completed. .. code-block:: python >>> x = client.submit(f, 1) # returns a future >>> y = client.submit(f, 2) # returns a future >>> z = client.submit(add, x, y) # consumes futures The ability to use futures cheaply within ``submit`` and ``map`` methods enables the construction of very sophisticated data pipelines with simple code. Additionally, distributed can serve as a full dask task scheduler, enabling support for distributed arrays, dataframes, machine learning pipelines, and any other application build on dask graphs. The dynamic task schedulers within ``distributed`` are adapted from the dask_ task schedulers and so are fairly sophisticated/efficient. **IPython Parallel Advantages** IPython Parallel has the following advantages over ``distributed`` 1. Maturity: IPython Parallel has been around for a while. 2. Explicit control over the worker processes: IPython parallel allows you to execute arbitrary statements on the workers, allowing it to serve in system administration tasks. 3. Deployment help: IPython Parallel has mechanisms built-in to aid deployment on SGE, MPI, etc.. Distributed does not have any such sugar, though is fairly simple to :doc:`set up ` by hand. 4. Various other advantages: Over the years IPython parallel has accrued a variety of helpful features like IPython interaction magics, ``@parallel`` decorators, etc.. .. _`a recipe`: https://ipython.org/ipython-doc/3/parallel/dag_dependencies.html#dag-dependencies .. _dask: http://dask.pydata.org/en/latest/ concurrent.futures ~~~~~~~~~~~~~~~~~~ The :class:`distributed.Client` API is modeled after :mod:`concurrent.futures` and :pep:`3184`. It has a few notable differences: * ``distributed`` accepts :class:`~distributed.client.Future` objects within calls to ``submit/map``. When chaining computations, it is preferable to submit Future objects directly rather than wait on them before submission. * The :meth:`~distributed.client.Client.map` method returns :class:`~distributed.client.Future` objects, not concrete results. The :meth:`~distributed.client.Client.map` method returns immediately. * Despite sharing a similar API, ``distributed`` :class:`~distributed.client.Future` objects cannot always be substituted for :class:`concurrent.futures.Future` objects, especially when using ``wait()`` or ``as_completed()``. * Distributed generally does not support callbacks. If you need full compatibility with the :class:`concurrent.futures.Executor` API, use the object returned by the :meth:`~distributed.client.Client.get_executor` method. .. _PEP-3184: https://www.python.org/dev/peps/pep-3148/ distributed-1.20.2/docs/source/resilience.rst000066400000000000000000000061611321233345200212240ustar00rootroot00000000000000Resilience ========== Software fails, Hardware fails, network connections fail, user code fails. This document describes how ``dask.distributed`` responds in the face of these failures and other known bugs. User code failures ------------------ When a function raises an error that error is kept and transmitted to the client on request. Any attempt to gather that result *or any dependent result* will raise that exception. .. code-block:: python >>> def div(a, b): ... return a / b >>> x = client.submit(div, 1, 0) >>> x.result() ZeroDivisionError: division by zero >>> y = client.submit(add, x, 10) >>> y.result() # same error as above ZeroDivisionError: division by zero This does not affect the smooth operation of the scheduler or worker in any way. Closed Network Connections -------------------------- If the connection to a remote worker unexpectedly closes and the local process appropriately raises an ``IOError`` then the scheduler will reroute all pending computations to other workers. If the lost worker was the only worker to hold vital results necessary for future computations then those results will be recomputed by surviving workers. The scheduler maintains a full history of how each result was produced and so is able to reproduce those same computations on other workers. This has some fail cases. 1. If results depend on impure functions then you may get a different (although still entirely accurate) result 2. If the worker failed due to a bad function, for example a function that causes a segmentation fault, then that bad function will repeatedly be called on other workers. This function will be marked as "bad" after it kills a fixed number of workers (defaults to three). 3. Data scattered out to the workers is not kept in the scheduler (it is often quite large) and so the loss of this data is irreparable. You may wish to call ``Client.replicate`` on the data with a suitable replication factor to ensure that it remains long-lived or else back the data off of some resilient store, like a file system. Hardware Failures ----------------- It is not clear under which circumstances the local process will know that the remote worker has closed the connection. If the socket does not close cleanly then the system will wait for a timeout, roughly three seconds, before marking the worker as failed and resuming smooth operation. Scheduler Failure ----------------- The process containing the scheduler might die. There is currently no persistence mechanism to record and recover the scheduler state. The workers and clients will all reconnect to the scheduler after it comes back online but records of ongoing computations will be lost. Restart and Nanny Processes --------------------------- The client provides a mechanism to restart all of the workers in the cluster. This is convenient if, during the course of experimentation, you find your workers in an inconvenient state that makes them unresponsive. The ``Client.restart`` method kills all workers, flushes all scheduler state, and then brings all workers back online, resulting in a clean cluster. distributed-1.20.2/docs/source/resources.rst000066400000000000000000000064321321233345200211150ustar00rootroot00000000000000Worker Resources ================ Access to scarce resources like memory, GPUs, or special hardware may constrain how many of certain tasks can run on particular machines. For example, we may have a cluster with ten computers, four of which have two GPUs each. We may have a thousand tasks, a hundred of which require a GPU and ten of which require two GPUs at once. In this case we want to balance tasks across the cluster with these resource constraints in mind, allocating GPU-constrained tasks to GPU-enabled workers. Additionally we need to be sure to constrain the number of GPU tasks that run concurrently on any given worker to ensure that we respect the provided limits. This situation arises not only for GPUs but for many resources like tasks that require a large amount of memory at runtime, special disk access, or access to special hardware. Dask allows you to specify abstract arbitrary resources to constrain how your tasks run on your workers. Dask does not model these resources in any particular way (Dask does not know what a GPU is) and it is up to the user to specify resource availability on workers and resource demands on tasks. Example ------- We consider a computation where we load data from many files, process each one with a function that requires a GPU, and then aggregate all of the intermediate results with a task that takes up 70GB of memory. We operate on a three-node cluster that has two machines with two GPUs each and one machine with 100GB of RAM. When we set up our cluster we define resources per worker:: dask-worker scheduler:8786 --resources "GPU=2" dask-worker scheduler:8786 --resources "GPU=2" dask-worker scheduler:8786 --resources "MEMORY=100e9" When we submit tasks to the cluster we specify constraints per task .. code-block:: python from distributed import Client client = Client('scheduler:8786') data = [client.submit(load, fn) for fn in filenames] processed = [client.submit(process, d, resources={'GPU': 1}) for d in data] final = client.submit(aggregate, processed, resources={'MEMORY': 70e9}) Resources are Abstract ---------------------- Resources listed in this way are just abstract quantities. We could equally well have used terms "mem", "memory", "bytes" etc. above because, from Dask's perspective, this is just an abstract term. You can choose any term as long as you are consistent across workers and clients. It's worth noting that Dask separately track number of cores and available memory as actual resources and uses these in normal scheduling operation. Resources with collections -------------------------- You can also use resources with Dask collections, like arrays, dataframes, and delayed objects. You can pass a dictionary mapping keys of the collection to resource requirements during compute or persist calls. .. code-block:: python x = dd.read_csv(...) y = x.map_partitions(func1) z = y.map_parititons(func2) z.compute(resources={tuple(y.__dask_keys__()): {'GPU': 1}) In some cases (such as the case above) the keys for ``y`` may be optimized away before execution. You can avoid that either by requiring them as an explicit output, or by passing the ``optimize_graph=False`` keyword. .. code-block:: python z.compute(resources={tuple(y.__dask_keys__()): {'GPU': 1}, optimize_graph=False) distributed-1.20.2/docs/source/scheduling-policies.rst000066400000000000000000000147411321233345200230370ustar00rootroot00000000000000Scheduling Policies =================== This document describes the policies used to select the preference of tasks and to select the preference of workers used by Dask's distributed scheduler. For more information on how this these policies are enacted efficiently see :doc:`Scheduling State`. .. _decide-worker: Choosing Workers ---------------- When a task transitions from waiting to a processing state we decide a suitable worker for that task. If the task has significant data dependencies or if the workers are under heavy load then this choice of worker can strongly impact global performance. Currently workers for tasks are determined as follows: 1. If the task has no major dependencies and no restrictions then we find the least occupied worker. 2. Otherwise, if a task has user-provided restrictions (for example it must run on a machine with a GPU) then we restrict the available pool of workers to just that set, otherwise we consider all workers 3. From among this pool of workers we determine the workers to whom the least amount of data would need to be transferred. 4. We break ties by choosing the worker that currently has the fewest tasks, counting both those tasks in memory and those tasks processing currently. This process is easy to change (and indeed this document may be outdated). We encourage readers to inspect the ``decide_worker`` function in scheduler.py .. currentmodule:: distributed.scheduler .. autosummary:: decide_worker Choosing Tasks -------------- We often have a choice between running many valid tasks. There are a few competing interests that might motivate our choice: 1. Run tasks on a first-come-first-served basis for fairness between multiple clients 2. Run tasks that are part of the critical path in an effort to reduce total running time and minimize straggler workloads 3. Run tasks that allow us to release many dependencies in an effort to keep the memory footprint small 4. Run tasks that are related so that large chunks of work can be completely eliminated before running new chunks of work Accomplishing all of these objectives simultaneously is impossible. Optimizing for any of these objectives perfectly can result in costly overhead. The heuristics with the scheduler do a decent but imperfect job of optimizing for all of these (they all come up in important workloads) quickly. Last in, first out ~~~~~~~~~~~~~~~~~~ When a worker finishes a task the immediate dependencies of that task get top priority. This encourages a behavior of finishing ongoing work immediately before starting new work. This often conflicts with the first-come-first-served objective but often results in shorter total runtimes and significantly reduced memory footprints. Break ties with children and depth ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ Often a task has multiple dependencies and we need to break ties between them with some other objective. Breaking these ties has a surprisingly strong impact on performance and memory footprint. When a client submits a graph we perform a few linear scans over the graph to determine something like the number of descendants of each node (not quite, because it's a DAG rather than a tree, but this is a close proxy). This number can be used to break ties and helps us to prioritize nodes with longer critical paths and nodes with many children. The actual algorithms used are somewhat more complex and are described in detail in `dask/order.py`_ .. _`dask/order.py`: https://github.com/dask/dask/blob/master/dask/order.py Initial Task Placement ~~~~~~~~~~~~~~~~~~~~~~ When a new large batch of tasks come in and there are many idle workers then we want to give each worker a set of tasks that are close together/related and unrelated from the tasks given to other workers. This usually avoids inter-worker communication down the line. The same depth-first-with-child-weights priority given to workers described above can usually be used to properly segment the leaves of a graph into decently well separated sub-graphs with relatively low inter-sub-graph connectedness. First-Come-First-Served, Coarsely ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ The last-in-first-out behavior used by the workers to minimize memory footprint can distort the task order provided by the clients. Tasks submitted recently may run sooner than tasks submitted long ago because they happen to be more convenient given the current data in memory. This behavior can be *unfair* but improves global runtimes and system efficiency, sometimes quite significantly. However, workers inevitably run out of tasks that were related to tasks they were just working on and the last-in-first-out policy eventually exhausts itself. In these cases workers often pull tasks from the common task pool. The tasks in this pool *are* ordered in a first-come-first-served basis and so workers do behave in a fair scheduling manner at a *coarse* level if not a fine grained one. Dask's scheduling policies are short-term-efficient and long-term-fair. Where these decisions are made ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ The objectives above are mostly followed by small decisions made by the client, scheduler, and workers at various points in the computation. 1. As we submit a graph from the client to the scheduler we assign a numeric priority to each task of that graph. This priority focuses on computing deeply before broadly, preferring critical paths, preferring nodes with many dependencies, etc.. This is the same logic used by the single-machine scheduler and lives in `dask/order.py `_. 2. When the graph reaches the scheduler the scheduler changes each of these numeric priorities into a tuple of two numbers, the first of which is an increasing counter, the second of which is the client-generated priority described above. This per-graph counter encourages a first-in-first-out policy between computations. All tasks from a previous call to compute have a higher priority than all tasks from a subsequent call to compute (or submit, persist, map, or any operation that generates futures). 3. Whenever a task is ready to run the scheduler assigns it to a worker. The scheduler does not wait based on priority. 4. However when the worker receives these tasks it considers their priorities when determining which tasks to prioritize for communication or for computation. The worker maintains a heap of all ready-to-run tasks ordered by this priority. distributed-1.20.2/docs/source/scheduling-state.rst000066400000000000000000000524041321233345200223460ustar00rootroot00000000000000Scheduling State ================ Overview -------- The life of a computation with Dask can be described in the following stages: 1. The user authors a graph using some library, perhaps Dask.delayed or dask.dataframe or the ``submit/map`` functions on the client. They submit these tasks to the scheduler. 2. The schedulers assimilates these tasks into its graph of all tasks to track and as their dependencies become available it asks workers to run each of these tasks. 3. The worker receives information about how to run the task, communicates with its peer workers to collect dependencies, and then runs the relevant function on the appropriate data. It reports back to the scheduler that it has finished. 4. The scheduler reports back to the user that the task has completed. If the user desires, it then fetches the data from the worker through the scheduler. Most relevant logic is in tracking tasks as they evolve from newly submitted, to waiting for dependencies, to actively running on some worker, to finished in memory, to garbage collected. Tracking this process, and tracking all effects that this task has on other tasks that might depend on it, is the majority of the complexity of the dynamic task scheduler. This section describes the system used to perform this tracking. For more abstract information about the policies used by the scheduler, see :doc:`Scheduling Policies`. State Variables --------------- We start with a description of the state that the scheduler keeps on each task. Each of the following is a dictionary keyed by task name (described below). Task description variables '''''''''''''''''''''''''' These containers keep task information accross a task's whole lifetime. * **tasks:** ``{key: task}``: Dictionary mapping key to a serialized task. A key is the name of a task, generally formed from the name of the function, followed by a hash of the function and arguments, like ``'inc-ab31c010444977004d656610d2d421ec'``. The value of this dictionary is the task, which is an unevaluated function and arguments. This is stored in one of two forms: * ``{'function': inc, 'args': (1,), 'kwargs': {}}``; a dictionary with the function, arguments, and keyword arguments (kwargs). However in the scheduler these are stored serialized, as they were sent from the client, so it looks more like ``{'function': b'\x80\x04\x95\xcb\...', 'args': b'...', }`` * ``(inc, 1)``: a tuple satisfying the `dask graph protocol `_. This again is stored serialized. These are the values that will eventually be sent to a worker when the task is ready to run. * **dependencies and dependents:** ``{key: {keys}}``: These are dictionaries which show which tasks depend on which others. They contain redundant information. If ``dependencies[a] == {b, c}`` then the task with the name of ``a`` depends on the results of the two tasks with the names of ``b`` and ``c``. There will be complimentary entries in dependents such that ``a in dependents[b]`` and ``a in dependents[c]`` such as ``dependents[b] == {a, d}``. Keeping the information around twice allows for constant-time access for either direction of query, so we can both look up a task's out-edges or in-edges efficiently. * **priority:** ``{key: tuple}``: The ``priority`` dictionary provides each key with a relative ranking which is used to break ties when many keys are being considered for execution. This ranking is generally a tuple of two parts. The first (and dominant) part corresponds to when it was submitted. Generally earlier tasks take precedence. The second part is determined by the client, and is a way to prioritize tasks within a large graph that may be important, such as if they are on the critical path, or good to run in order to release many dependencies. This is explained further in :doc:`Scheduling Policy `. * **host_restrictions:** ``{key: {hostnames}}``: A set of hostnames per key of where that key can be run. Usually this is empty unless a key has been specifically restricted to only run on certain hosts. A hostname may correspond to one or several connected workers. * **worker_restrictions:** ``{key: {worker addresses}}``: A set of complete worker addresses per key of where that key can be run. Usually this is empty unless a key has been specifically restricted to only run on certain workers. * **resource_restrictions:** ``{key: {resource: quantity}}``: Resources required by a task, such as ``{'gpu': 1}`` or ``{'memory': 1e9}``. These are user-defined names and are matched against the contents of the ``worker_resources`` dictionary. * **loose_restrictions:** ``{key}``: Set of keys for which we are allowed to violate restrictions (see above) if no valid workers are present and the task would otherwise go into the ``unrunnable`` set. In other words, if a key is in ``loose_restrictions``, then its restrictions become mere preferences, otherwise they are mandatory. * **who_wants:** ``{key: {client}}``: When a client submits a graph to the scheduler it also specifies which output keys it desires. Those keys are tracked here where each desired key knows which clients want it. These keys will not be released from memory and, when they complete, messages will be sent to all of these clients that the task is ready. * **wants_what:** ``{client: {key}}``: The transpose of ``who_wants``. Task state flow ''''''''''''''' These state variables reflect the current status of a task and may get updated at each state transition. * **task_state:** ``{key: string}``: The ``task_state`` dictionary holds the current state of every key. Current valid states include ``released``, ``waiting``, ``no-worker``, ``processing``, ``memory``, and ``erred``. These states are explained :ref:`further below `. * **waiting and waiting_data:** ``{key: {keys}}``: These dictionaries are a subset of ``dependencies`` and ``dependents`` respectively, as they only track keys that are still in play. For example ``waiting`` looks like ``dependencies``, tracking all of the tasks that a certain task requires before it can run. However, as tasks are completed and arrive in memory they are removed from their dependents sets in ``waiting``, so that when a set becomes empty we know that a key is ready to run and ready to be allocated to a worker. Similarly, the ``waiting_data`` dictionary holds all of the dependents of a key that have yet to run and still require that this task stay in memory in services of tasks that may depend on it (its ``dependents``). When a value set in this dictionary becomes empty its task may be garbage-collected (unless some client actively desires that this task stay in memory, as tracked in ``who_wants``). * **processing:** ``{worker: {key: cost}}``: Keys that are currently allocated to a worker. This is keyed by worker address and contains the expected cost in seconds of running each task, summing both the task's expected computation time and the expected communication time of its result. Multiple tasks may be submitted to a worker in advance and the worker will run them eventually, depending on its execution resources (but see :doc:`work-stealing`). * **rprocessing:** ``{key: worker}``: The reverse of the ``processing`` dictionary. This tracks the worker processing each task that is currently running. This is redundant with ``processing`` and just here for faster indexed querying. * **who_has:** ``{key: {worker}}``: For keys that are in memory this shows on which workers they currently reside. * **has_what:** ``{worker: {key}}``: This is the transpose of ``who_has``, showing all keys that currently reside on each worker. * **released:** ``{keys}`` The set of keys that are neither waiting to be processed, nor in memory. These typically are just-initialized tasks, or tasks that have already been computed but which it is not necessary to keep in memory. * **unrunnable:** ``{key}`` The set of keys that are not currently able to run, either because they have a user-defined restriction (described in ``host_restrictions``, ``worker_restrictions`` and ``resource_restrictions``) that is not met by any connected worker, or because no worker is connected at all. These keys already have all their ``dependencies`` satisfied (their ``waiting`` set is empty), and are waiting for an appropriate worker to join the network before computing. * **exceptions and tracebacks:** ``{key: Exception/Traceback}``: Dictionaries mapping keys to remote exceptions and tracebacks. When tasks fail we store their exceptions and tracebacks (serialized from the worker) here so that users may gather the exceptions to see the error. * **exceptions_blame:** ``{key: key}``: If a task fails then we mark all of its dependent tasks as failed as well. This dictionary lets any failed task see which task was the origin of its failure. * **suspicious_tasks:** ``{key: int}`` Number of times a task has been involved in a worker failure. Some tasks may cause workers to fail (such as ``sys.exit(0)``). When a worker fails all of the tasks on that worker are reassigned to others. This combination of behaviors can cause a bad task to catastrophically destroy all workers on the cluster, one after another. Whenever a worker fails we mark each task currently running on that worker as suspicious. If a task is involved in three failures (or some other fixed constant) then we mark the task as ``erred``. * **retries:** ``{key: int}`` Number of times a task is automatically retried in case of failure. If present, this is decremented each time a task's execution failed, until it reaches 0. * **nbytes:** ``{key: int}``: The number of bytes, as determined by ``sizeof``, of the result of each finished task. This number is used for diagnostics and to help prioritize work. Worker state variables '''''''''''''''''''''' These state variables track the current state of each worker, and are involved in deciding :ref:`which worker to run a task on `. * **ncores:** ``{worker: int}`` The number of CPU cores made available on each worker. * **worker_resources:** ``{worker: {str: Number}}``: The available resources on each worker like ``{'gpu': 2, 'mem': 1e9}``. These are abstract quantities that constrain certain tasks from running at the same time on a given worker. * **used_resources:** ``{worker: {str: Number}}``: The sum of each resource used by all tasks allocated to a particular worker. The numbers in this dictionary can only be less or equal than those in ``worker_resources``. * **worker_bytes:** ``{worker: int}``: The total memory size, in bytes, used by the keys currently held in memory on each given worker. * **occupancy:** ``{worker: duration}``: The total expected runtime, in seconds, of all tasks currently processing on a worker. * **idle and saturated:** ``{worker}``: Two sets of workers indicating their ability to start computing a new task in a relatively short timespan. "Idle" workers will be preferred when :ref:`deciding a suitable worker ` to run a new task on. Conversely, "saturated" workers may see their workload lightened through :doc:`work-stealing`. These two sets are computed based on each worker's number of cores (``ncores``), task queue (``processing``) and ``occupancy`` numbers. These two sets are disjoint. Also, some workers may be *neither* "idle" nor "saturated". .. XXX list invariants somewhere? For every worker ``w``, * ``worker_bytes[w] == sum(nbytes[k] for k in has_what[w])`` * ``occupancy[w] == sum(processing[w].values())`` Example Event and Response -------------------------- Whenever an event happens, like when a client sends up more tasks, or when a worker finishes a task, the scheduler changes the state above. For example when a worker reports that a task has finished we perform actions like the following: **Task `key` finished by `worker`**: .. code-block:: python # Update task state task_state[key] = 'memory' processing[worker].remove(key) del rprocessing[key] who_has[key].add(worker) has_what[worker].add(key) # Update memory / resource / occupancy counters nbytes[key] = nbytes worker_bytes[worker] += nbytes for key, value in resource_restrictions[key]: used_resources[key] -= value update_worker_occupancy_and_idleness(worker) # Notify clients interested in this task's result if key in who_wants: send_done_message_to_clients(who_wants[key]) # Transitively update dependent tasks for dep in dependencies[key]: waiting_data[dep].remove(key) for dep in dependents[key]: waiting[dep].remove(key) for task in ready_tasks(): worker = best_worker(task): send_task_to_worker(task, worker) .. _task-states: State Transitions ----------------- The code presented in the section above is just for demonstration. In practice writing this code for every possible event is highly error prone, resulting in hard-to-track-down bugs. Instead the scheduler moves tasks between a fixed set of states, notably ``released``, ``waiting``, ``no-worker``, ``processing``, ``memory``, ``error``. Tasks fall into the following states with the following allowed transitions .. image:: images/task-state.svg :alt: Dask scheduler task states * *Released*: Known but not actively computing or in memory * *Waiting*: On track to be computed, waiting on dependencies to arrive in memory * *No-worker* (ready, rare): Ready to be computed, but no appropriate worker exists * *Processing*: Actively being computed by one or more workers * *Memory*: In memory on one or more workers * *Erred*: Task computation, or one of its dependencies, has encountered an error * Forgotten (not actually a state): Task is no longer needed by any client and so is removed from state Tasks and task states ''''''''''''''''''''' The table below shows which state variable a task is in, depending on the task's state. Cells with a check mark (`✓`) indicate the task key *must* be present in the given state variable; cells with an question mark (`?`) indicate the task key *may* be present in the given state variable. ======================= ======== ======= ========= ========== ====== ===== State variable Released Waiting No-worker Processing Memory Erred ======================= ======== ======= ========= ========== ====== ===== tasks ✓ ✓ ✓ ✓ ✓ ✓ priority ✓ ✓ ✓ ✓ ✓ ✓ dependencies ✓ ✓ ✓ ✓ ✓ ✓ dependents ✓ ✓ ✓ ✓ ✓ ✓ ----------------------- -------- ------- --------- ---------- ------ ----- host_restrictions ? ? ? ? ? ? worker_restrictions ? ? ? ? ? ? resource_restrictions ? ? ? ? ? ? loose_restrictions ? ? ? ? ? ? ----------------------- -------- ------- --------- ---------- ------ ----- released ✓ waiting ✓ waiting_data ✓ unrunnable ✓ processing ✓ rprocessing ✓ who_has ✓ has_what ✓ nbytes *(1)* ? ? ? ? ✓ ? exceptions ✓ tracebacks ✓ exceptions_blame ✓ retries ? ? ? ? ? ? suspicious_tasks ? ? ? ? ? ? ======================= ======== ======= ========= ========== ====== ===== Notes: 1. **nbytes**: a task can be in this collection as long as it was already computed, even if not currently held in a worker's memory. Transitions and worker state '''''''''''''''''''''''''''' The table below shows which worker state variables are updated on each task state transition. ==================================== ========================================================== Transition Affected worker state ==================================== ========================================================== released → waiting occupancy, idle, saturated waiting → processing occupancy, idle, saturated, used_resources waiting → memory idle, saturated, worker_bytes processing → memory occupancy, idle, saturated, used_resources, worker_bytes processing → erred occupancy, idle, saturated, used_resources processing → released occupancy, idle, saturated, used_resources memory → released worker_bytes memory → forgotten worker_bytes ==================================== ========================================================== .. note:: Another way of understanding this table is to observe that entering or exiting a specific task state updates a well-defined set of worker state variables. For example, entering and exiting the Memory state updates ``worker_bytes``. Implementation -------------- Every transition between states is a separate method in the scheduler. These task transition functions are prefixed with ``transition`` and then have the name of the start and finish task state like the following. .. code-block:: python def transition_released_waiting(self, key): def transition_processing_memory(self, key): def transition_processing_erred(self, key): These functions each have three effects. 1. They perform the necessary transformations on the scheduler state (the 20 dicts/lists/sets) to move one key between states. 2. They return a dictionary of recommended ``{key: state}`` transitions to enact directly afterwards on other keys. For example after we transition a key into memory we may find that many waiting keys are now ready to transition from waiting to a ready state. 3. Optionally they include a set of validation checks that can be turned on for testing. Rather than call these functions directly we call the central function ``transition``: .. code-block:: python def transition(self, key, final_state): """ Transition key to the suggested state """ This transition function finds the appropriate path from the current to the final state. It also serves as a central point for logging and diagnostics. Often we want to enact several transitions at once or want to continually respond to new transitions recommended by initial transitions until we reach a steady state. For that we use the ``transitions`` function (note the plural ``s``). .. code-block:: python def transitions(self, recommendations): recommendations = recommendations.copy() while recommendations: key, finish = recommendations.popitem() new = self.transition(key, finish) recommendations.update(new) This function runs ``transition``, takes the recommendations and runs them as well, repeating until no further task-transitions are recommended. Stimuli ------- Transitions occur from stimuli, which are state-changing messages to the scheduler from workers or clients. The scheduler responds to the following stimuli: * **Workers** * Task finished: A task has completed on a worker and is now in memory * Task erred: A task ran and erred on a worker * Task missing data: A task tried to run but was unable to find necessary data on other workers * Worker added: A new worker was added to the network * Worker removed: An existing worker left the network * **Clients** * Update graph: The client sends more tasks to the scheduler * Release keys: The client no longer desires the result of certain keys Stimuli functions are prepended with the text ``stimulus``, and take a variety of keyword arguments from the message as in the following examples: .. code-block:: python def stimulus_task_finished(self, key=None, worker=None, nbytes=None, type=None, compute_start=None, compute_stop=None, transfer_start=None, transfer_stop=None): def stimulus_task_erred(self, key=None, worker=None, exception=None, traceback=None) These functions change some non-essential administrative state and then call transition functions. Note that there are several other non-state-changing messages that we receive from the workers and clients, such as messages requesting information about the current state of the scheduler. These are not considered stimuli. API --- .. currentmodule:: distributed.scheduler .. autoclass:: Scheduler :members: .. autofunction:: decide_worker distributed-1.20.2/docs/source/serialization.rst000066400000000000000000000030141321233345200217510ustar00rootroot00000000000000Custom Serialization ==================== When we communicate data between computers we first convert that data into a sequence of bytes that can be communicated across the network. Dask can convert data to bytes using the standard solutions of Pickle and Cloudpickle. However, sometimes pickle and cloudpickle are suboptimal so Dask also supports custom serialization formats for special types. This helps Dask to be faster on common formats like NumPy and Pandas and gives power-users more control about how their objects get moved around on the network if they want to extend the system. We include a small example and then follow with the full API documentation describing the ``serialize`` and ``deserialize`` functions, which convert objects into a msgpack header and a list of bytestrings and back. Example ------- Here is how we special case handling raw Python bytes objects. In this case there is no need to call ``pickle.dumps`` on the object. The object is already a sequnce of bytes. .. code-block:: python def serialize_bytes(obj): header = {} # no special metadata frames = [obj] return header, frames def deserialize_bytes(header, frames): return frames[0] register_serialization(bytes, serialize_bytes, deserialize_bytes) API --- .. currentmodule:: distributed.protocol.serialize .. autosummary:: register_serialization serialize deserialize .. autofunction:: register_serialization .. autofunction:: serialize .. autofunction:: deserialize distributed-1.20.2/docs/source/setup.rst000066400000000000000000000234721321233345200202460ustar00rootroot00000000000000Setup Network ============= A ``dask.distributed`` network consists of one ``Scheduler`` node and several ``Worker`` nodes. One can set these up in a variety of ways Using the Command Line ---------------------- We launch the ``dask-scheduler`` executable in one process and the ``dask-worker`` executable in several processes, possibly on different machines. Launch ``dask-scheduler`` on one node:: $ dask-scheduler Start scheduler at 192.168.0.1:8786 Then launch ``dask-worker`` on the rest of the nodes, providing the address to the node that hosts ``dask-scheduler``:: $ dask-worker 192.168.0.1:8786 Start worker at: 192.168.0.2:12345 Registered with center at: 192.168.0.1:8786 $ dask-worker 192.168.0.1:8786 Start worker at: 192.168.0.3:12346 Registered with center at: 192.168.0.1:8786 $ dask-worker 192.168.0.1:8786 Start worker at: 192.168.0.4:12347 Registered with center at: 192.168.0.1:8786 There are various mechanisms to deploy these executables on a cluster, ranging from manualy SSH-ing into all of the nodes to more automated systems like SGE/SLURM/Torque or Yarn/Mesos. Additionally, cluster SSH tools exist to send the same commands to many machines. One example is `tmux-cssh`__. .. note:: - The scheduler and worker both need to accept TCP connections. By default the scheduler uses port 8786 and the worker binds to a random open port. If you are behind a firewall then you may have to open particular ports or tell Dask to use particular ports with the ``--port`` and ``-worker-port`` keywords. Other ports like 8787, 8788, and 8789 are also useful to keep open for the diagnostic web interfaces. - More information about relevant ports is available by looking at the help pages with ``dask-scheduler --help`` and ``dask-worker --help`` __ https://github.com/dennishafemann/tmux-cssh Using SSH --------- The convenience script ``dask-ssh`` opens several SSH connections to your target computers and initializes the network accordingly. You can give it a list of hostnames or IP addresses:: $ dask-ssh 192.168.0.1 192.168.0.2 192.168.0.3 192.168.0.4 Or you can use normal UNIX grouping:: $ dask-ssh 192.168.0.{1,2,3,4} Or you can specify a hostfile that includes a list of hosts:: $ cat hostfile.txt 192.168.0.1 192.168.0.2 192.168.0.3 192.168.0.4 $ dask-ssh --hostfile hostfile.txt The ``dask-ssh`` utility depends on the ``paramiko``:: pip install paramiko Using a Shared Network File System and a Job Scheduler ------------------------------------------------------ Some clusters benefit from a shared network file system (NFS) and can use this to communicate the scheduler location to the workers:: dask-scheduler --scheduler-file /path/to/scheduler.json dask-worker --scheduler-file /path/to/scheduler.json dask-worker --scheduler-file /path/to/scheduler.json .. code-block:: python >>> client = Client(scheduler_file='/path/to/scheduler.json') This can be particularly useful when deploying ``dask-scheduler`` and ``dask-worker`` processes using a job scheduler like ``SGE/SLURM/Torque/etc..`` Here is an example using SGE's ``qsub`` command:: # Start a dask-scheduler somewhere and write connection information to file qsub -b y /path/to/dask-scheduler --scheduler-file /path/to/scheduler.json # Start 100 dask-worker processes in an array job pointing to the same file qsub -b y -t 1-100 /path/to/dask-worker --scheduler-file /path/to/scheduler.json Note, the ``--scheduler-file`` option is *only* valuable if your scheduler and workers share a standard POSIX file system. Using MPI --------- You can launch a Dask network using ``mpirun`` or ``mpiexec`` and the ``dask-mpi`` command line executable. .. code-block:: bash mpirun --np 4 dask-mpi --scheduler-file /path/to/scheduler.json .. code-block:: python from dask.distributed import Client client = Client(scheduler_file='/path/to/scheduler.json') This depends on the `mpi4py `_ library. It only uses MPI to start the Dask cluster, and not for inter-node communication. You may want to specify a high-bandwidth network interface like infiniband using the ``--interface`` keyword .. code-block:: bash mpirun --np 4 dask-mpi --nthreads 1 \ --interface ib0 \ --scheduler-file /path/to/scheduler.json Using the Python API -------------------- Alternatively you can start up the ``distributed.scheduler.Scheduler`` and ``distributed.worker.Worker`` objects within a Python session manually. Start the Scheduler, provide the listening port (defaults to 8786) and Tornado IOLoop (defaults to ``IOLoop.current()``) .. code-block:: python from distributed import Scheduler from tornado.ioloop import IOLoop from threading import Thread loop = IOLoop.current() t = Thread(target=loop.start, daemon=True) t.start() s = Scheduler(loop=loop) s.start('tcp://:8786') # Listen on TCP port 8786 On other nodes start worker processes that point to the URL of the scheduler. .. code-block:: python from distributed import Worker from tornado.ioloop import IOLoop from threading import Thread loop = IOLoop.current() t = Thread(target=loop.start, daemon=True) t.start() w = Worker('tcp://127.0.0.1:8786', loop=loop) w.start() # choose randomly assigned port Alternatively, replace ``Worker`` with ``Nanny`` if you want your workers to be managed in a separate process by a local nanny process. This allows workers to restart themselves in case of failure, provides some additional monitoring, and is useful when coordinating many workers that should live in different processes to avoid the GIL_. .. _GIL: https://docs.python.org/3/glossary.html#term-gil Using LocalCluster ------------------ You can do the work above easily using :doc:`LocalCluster`. .. code-block:: python from distributed import LocalCluster c = LocalCluster(processes=False) A scheduler will be available under ``c.scheduler`` and a list of workers under ``c.workers``. There is an IOLoop running in a background thread. Using Amazon EC2 ---------------- See the :doc:`EC2 quickstart ` for information on the ``dask-ec2`` easy setup script to launch a canned cluster on EC2. Using Google Cloud ------------------ See the dask-kubernetes_ project to easily launch clusters on `Google Kubernetes Engine`_. .. _dask-kubernetes: https://github.com/dask/dask-kubernetes .. _`Google Kubernetes Engine`: https://cloud.google.com/kubernetes-engine/ Cluster Resource Managers ------------------------- Dask.distributed has been deployed on dozens of different cluster resource managers. This section contains links to some external projects, scripts, and instructions that may serve as useful starting points. Kubernetes ~~~~~~~~~~ * https://github.com/martindurant/dask-kubernetes * https://github.com/ogrisel/docker-distributed * https://github.com/hammerlab/dask-distributed-on-kubernetes/ Marathon ~~~~~~~~ * https://github.com/mrocklin/dask-marathon DRMAA (SGE, SLURM, Torque, etc..) ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ * https://github.com/dask/dask-drmaa * https://github.com/mfouesneau/dasksge YARN ~~~~ * https://github.com/dask/dask-yarn * https://knit.readthedocs.io/en/latest/ Software Environment -------------------- The workers and clients should all share the same software environment. That means that they should all have access to the same libraries and that those libraries should be the same version. Dask generally assumes that it can call a function on any worker with the same outcome (unless explicitly told otherwise.) This is typically enforced through external means, such as by having a network file system (NFS) mount for libraries, by starting the ``dask-worker`` processes in equivalent Docker_ containers, using Conda_ environments, or through any of the other means typically employed by cluster administrators. .. _Docker: https://www.docker.com/ .. _Conda: http://conda.pydata.org/docs/ Windows ~~~~~~~ .. note:: - Running a ``dask-scheduler`` on Windows architectures is supported for only a limited number of workers (roughly 100). This is a detail of the underlying tcp server implementation and is discussed `here`__. - Running ``dask-worker`` processes on Windows is well supported, performant, and without limit. If you wish to run in a primarily Windows environment, it is recommneded to run a ``dask-scheduler`` on a linux or MacOSX environment, with ``dask-worker`` workers on the Windows boxes. This works because the scheduler environment is de-coupled from that of the workers. __ https://github.com/jfisteus/ztreamy/issues/26 Customizing initialization -------------------------- Both ``dask-scheduler`` and ``dask-worker`` support a ``--preload`` option that allows custom initialization of each scheduler/worker respectively. A module or python file passed as a ``--preload`` value is guaranteed to be imported before establishing any connection. A ``dask_setup(service)`` function is called if found, with a ``Scheduler`` or ``Worker`` instance as the argument. As the service stops, ``dask_teardown(service)`` is called if present. As an example, consider the following file that creates a :doc:`scheduler plugin ` and registers it with the scheduler .. code-block:: python # scheduler-setup.py from distributed.diagnostics.plugin import SchedulerPlugin class MyPlugin(SchedulerPlugin): def add_worker(self, scheduler=None, worker=None, **kwargs): print("Added a new worker at", worker) def dask_setup(scheduler): plugin = MyPlugin() scheduler.add_plugin(plugin) We can then run this preload script by referring to its filename (or module name if it is on the path) when we start the scheduler:: dask-scheduler --preload scheduler-setup.py distributed-1.20.2/docs/source/submitting-applications.rst000066400000000000000000000030621321233345200237500ustar00rootroot00000000000000Submitting Applications ======================= The ``dask-submit`` cli can be used to submit an application to the dask cluster running remotely. If your code depends on resources that can only be access from cluster running dask, ``dask-submit`` provides a mechanism to send the script to the cluster for execution from a different machine. For example, S3 buckets could not be visible from your local machine and hence any attempt to create a dask graph from local machine may not work. Submitting dask Applications with `dask-submit` ----------------------------------------------- In order to remotely submit scripts to the cluster from a local machine or a CI/CD environment, we need to run a remote client on the same machine as the scheduler:: #scheduler machine dask-remote --port 8788 After making sure the `dask-remote` is running, you can submit a script by:: #local machine dask-submit : Some of the commonly used arguments are: - ``REMOTE_CLIENT_ADDRESS``: host name where dask-remote client is running - ``FILEPATH``: Local path to file containing dask application For example, given the following dask application saved in a file called ``script.py``: .. code-block:: python from distributed import Client def inc(x): return x + 1 if __name__=='__main__': client = Client('127.0.0.1:8786') x = client.submit(inc, 10) print(x.result()) We can submit this application from a local machine by running:: dask-submit : script.py distributed-1.20.2/docs/source/task-launch.rst000066400000000000000000000141641321233345200213160ustar00rootroot00000000000000Launch Tasks from Tasks ======================= Sometimes it is convenient to launch tasks from other tasks. For example you may not know what computations to run until you have the results of some initial computations. Motivating example ------------------ We want to download one piece of data and turn it into a list. Then we want to submit one task for every element of that list. We don't know how long the list will be until we have the data. So we send off our original ``download_and_convert_to_list`` function, which downloads the data and converts it to a list on one of our worker machines: .. code-block:: python future = e.submit(download_and_convert_to_list, uri) But now we need to submit new tasks for individual parts of this data. We have three options. 1. Gather the data back to the local process and then submit new jobs from the local process 2. Gather only enough information about the data back to the local process and submit jobs from the local process 3. Submit a task to the cluster that will submit other tasks directly from that worker Gather the data locally ----------------------- If the data is not large then we can bring it back to the client to perform the necessary logic on our local machine: .. code-block:: python >>> data = future.result() # gather data to local process >>> data # data is a list [...] >>> futures = e.map(process_element, data) # submit new tasks on data >>> analysis = e.submit(aggregate, futures) # submit final aggregation task This is straightforward and, if ``data`` is small then it is probably the simplest, and therefore correct choice. However, if ``data`` is large then we have to choose another option. Submit tasks from client ------------------------ We can run small functions on our remote data to determine enough to submit the right kinds of tasks. In the following example we compute the ``len`` function on ``data`` remotely and then break up data into its various elements. .. code-block:: python >>> n = e.submit(len, data) # compute number of elements >>> n = n.result() # gather n (small) locally >>> from operator import getitem >>> elements = [e.submit(getitem, data, i) for i in range(n)] # split data >>> futures = e.map(process_element, elements) >>> analysis = e.submit(aggregate, futures) We compute the length remotely, gather back this very small result, and then use it to submit more tasks to break up the data and process on the cluster. This is more complex because we had to go back and forth a couple of times between the cluster and the local process, but the data moved was very small, and so this only added a few milliseconds to our total processing time. Submit tasks from worker ------------------------ *Note: this interface is new and experimental. It may be changed without warning in future versions.* We can submit tasks from other tasks. This allows us to make decisions while on worker nodes. To submit new tasks from a worker that worker must first create a new client object that connects to the scheduler. There is a convenience function to do this for you so that you don't have to pass around connection information. However you must use this function ``worker_client`` as a context manager to ensure proper cleanup on the worker. .. code-block:: python from distributed import worker_client def process_all(data): with worker_client() as e: elements = e.scatter(data) futures = e.map(process_element, elements) analysis = e.submit(aggregate, futures) result = analysis.result() return result analysis = e.submit(process_all, data) # spawns many tasks This approach is somewhat complex but very powerful. It allows you to spawn tasks that themselves act as potentially long-running clients, managing their own independent workloads. Extended Example ~~~~~~~~~~~~~~~~ This example computing the Fibonacci numbers creates tasks that submit tasks that submit tasks that submit other tasks, etc.. .. code-block:: python In [1]: from distributed import Client, worker_client In [2]: client = Client() In [3]: def fib(n): ...: if n < 2: ...: return n ...: else: ...: with worker_client() as c ...: a = c.submit(fib, n - 1) ...: b = c.submit(fib, n - 2) ...: a, b = c.gather([a, b]) ...: return a + b ...: In [4]: future = e.submit(fib, 100) In [5]: future Out[5]: In [6]: future.result() Out[6]: 354224848179261915075 This example is a bit extreme and spends most of its time establishing client connections from the worker rather than doing actual work, but does demonstrate that even pathological cases function robustly. Technical details ~~~~~~~~~~~~~~~~~ Tasks that invoke ``worker_client`` are conservatively assumed to be *long running*. They can take a long time blocking, waiting for other tasks to finish, gathering results, etc.. In order to avoid having them take up processing slots the following actions occur whenever a task invokes ``worker_client``. 1. The thread on the worker running this function *secedes* from the thread pool and goes off on its own. This allows the thread pool to populate that slot with a new thread and continue processing additional tasks without counting this long running task against its normal quota. 2. The Worker sends a message back to the scheduler temporarily increasing its allowed number of tasks by one. This likewise lets the scheduler allocate more tasks to this worker, not counting this long running task against it. Because of this behavior you can happily launch long running control tasks that manage worker-side clients happily, without fear of deadlocking the cluster. Establishing a connection to the scheduler takes on the order of 10-20 ms and so it is wise for computations that use this feature to be at least a few times longer in duration than this. distributed-1.20.2/docs/source/tls.rst000066400000000000000000000076211321233345200177060ustar00rootroot00000000000000.. _tls: TLS/SSL ======= Currently dask distributed has experimental support for TLS/SSL communication, providing mutual authentication and encryption of communications between cluster endpoints (Clients, Schedulers and Workers). TLS is enabled by using a ``tls`` address such as ``tls://`` (the default being ``tcp``, which sends data unauthenticated and unencrypted). In TLS mode, all cluster endpoints must present a valid TLS certificate signed by a given Certificate Authority (CA). It is generally recommended to use a custom CA for your organization, as it will allow signing certificates for internal hostnames or IP addresses. Parameters ---------- When using TLS, one has to provide additional parameters: * a *CA certificate(s) file*, which allows TLS to decide whether an endpoint's certificate has been signed by the correct authority; * a *certificate file* for each endpoint, which is presented to other endpoints so as to achieve mutual authentication; * a *private key file*, which is the cryptographic means to prove to other endpoints that you are the authorized user of a given certificate. .. note:: As per OpenSSL's requirements, all those files should be in PEM format. Also, it is allowed to concatenate the certificate and private key into a single file (you can then just specify the *certificate* parameter and leave the *private key* parameter absent). It is up to you whether each endpoint uses a different certificate and private key, or whether all endpoints share the same, or whether each endpoint kind (Client, Scheduler, Worker) gets its own certificate / key pair. Unless you have extraordinary requirements, however, the CA certificate should probably be the same for all endpoints. One can also pass additional parameters: * a set of allowed *ciphers*, if you have strong requirements as to which algorithms are considered secure; this setting's value should be an `OpenSSL cipher string `_; * whether to *require encryption*, to avoid using plain TCP communications by mistake. All those parameters can be passed in several ways: * through the Dask :ref:`configuration file `; * if using the command line, through options to ``dask-scheduler`` and ``dask-worker``; * if using the API, through a ``Security`` object. For example, here is how you might configure a ``Security`` object for client use: .. code-block:: python from distributed import Client from distributed.security import Security sec = Security(tls_ca_file='cluster_ca.pem', tls_client_cert='cli_cert.pem', tls_client_key='cli_key.pem', require_encryption=True) client = Client(..., security=sec) Security policy --------------- Dask always verifies the certificate presented by a remote endpoint against the configured CA certificate(s). Certificates are verified for both "client" and "server" endpoints (in the TCP sense), ensuring the endpoints are mutually authenticated. The hostname or IP address for which a certificate has been issued is not checked; this should not be an issue if you are using your own internal Certificate Authority. It is not possible to disable certificate verification, as it would render the communications vulnerable to Man-in-the-Middle attacks. Performance implications ------------------------ Encryption is fast on recent CPUs, most of which have hardware acceleration for AES-based encryption. AES is normally selected by the TLS layer unless you have forced the *ciphers* parameter to something else. However, encryption may still have a non-negligible overhead if you are transferring very large data over very high speed network links. .. seealso:: `A study of AES-NI acceleration `_ shows recent x86 CPUs can AES-encrypt more than 1 GB per second on each CPU core. distributed-1.20.2/docs/source/web.rst000066400000000000000000000171551321233345200176640ustar00rootroot00000000000000Web Interface ============= .. image:: https://raw.githubusercontent.com/dask/dask-org/master/images/daskboard.gif :alt: Dask Bokeh Dashboard :width: 80% Information about the current state of the network helps to track progress, identify performance issues, and debug failures. Dask.distributed includes a web interface to help deliver this information over a normal web page in real time. This web interface is launched by default wherever the scheduler is launched if the scheduler machine has Bokeh_ installed (``conda install bokeh -c bokeh``). List of Servers --------------- There are a few sets of diagnostic pages served at different ports: * Main Scheduler pages at ``http://scheduler-address:8787``. These pages, particularly the ``/status`` page are the main page that most people associate with Dask. These pages are served from a separate standalone Bokeh server application running in a separate process. * Debug Scheduler pages at ``http://scheduler-address:8788``. These pages have more detailed diagnostic information about the scheduler. They are more often used by developers than by users, but may still be of interest to the performance-conscious. These pages run from inside the scheduler process, and so compete for resources with the main scheduler. * Debug Worker pages for each worker at ``http://worker-address:8789``. These pages have detailed diagnostic information about the worker. Like the diagnostic scheduler pages they are of more utility to developers or to people looking to understand the performance of their underlying cluster. If port 8789 is unavailable (for example it is in use by another worker) then a random port is chosen. A list of all ports can be obtained from looking at the service ports for each worker in the result of calling ``client.scheduler_info()`` The rest of this document will be about the main pages at ``http://scheduler-address:8787``. The available pages are ``http://scheduler-address:8787//`` where ```` is one of - ``status``: a stream of recently run tasks, progress bars, resource use - ``tasks``: a larger stream of the last 100k tasks - ``workers``: basic information about workers and their current load .. _Bokeh: http://bokeh.pydata.org/en/latest/ Plots ----- Example Computation ~~~~~~~~~~~~~~~~~~~ The following plots show a trace of the following computation: .. code-block:: python from distributed import Client from time import sleep import random def inc(x): sleep(random.random() / 10) return x + 1 def dec(x): sleep(random.random() / 10) return x - 1 def add(x, y): sleep(random.random() / 10) return x + y client = Client('127.0.0.1:8786') incs = client.map(inc, range(100)) decs = client.map(dec, range(100)) adds = client.map(add, incs, decs) total = client.submit(sum, adds) del incs, decs, adds total.result() Progress ~~~~~~~~ The interface shows the progress of the various computations as well as the exact number completed. .. image:: https://raw.githubusercontent.com/dask/dask-org/master/images/bokeh-progress.gif :alt: Resources view of Dask web interface Each bar is assigned a color according to the function being run. Each bar has a few components. On the left the lighter shade is the number of tasks that have both completed and have been released from memory. The darker shade to the right corresponds to the tasks that are completed and whose data still reside in memory. If errors occur then they appear as a black colored block to the right. Typical computations may involve dozens of kinds of functions. We handle this visually with the following approaches: 1. Functions are ordered by the number of total tasks 2. The colors are assigned in a round-robin fashion from a standard palette 3. The progress bars shrink horizontally to make space for more functions 4. Only the largest functions (in terms of number of tasks) are displayed .. image:: https://raw.githubusercontent.com/dask/dask-org/master/images/bokeh-progress-large.gif :alt: Progress bar plot of Dask web interface Counts of tasks processing, waiting for dependencies, processing, etc.. are displayed in the title bar. Memory Use ~~~~~~~~~~ The interface shows the relative memory use of each function with a horizontal bar sorted by function name. .. image:: https://raw.githubusercontent.com/dask/dask-org/master/images/bokeh-memory-use.gif :alt: Memory use plot of Dask web interface The title shows the number of total bytes in use. Hovering over any bar tells you the specific function and how many bytes its results are actively taking up in memory. This does not count data that has been released. Task Stream ~~~~~~~~~~~ The task stream plot shows when tasks complete on which workers. Worker cores are on the y-axis and time is on the x-axis. As a worker completes a task its start and end times are recorded and a rectangle is added to this plot accordingly. .. image:: https://raw.githubusercontent.com/dask/dask-org/master/images/bokeh-task-stream.gif :alt: Task stream plot of Dask web interface If data transfer occurs between workers a *red* bar appears preceding the task bar showing the duration of the transfer. If an error occurs than a *black* bar replaces the normal color. This plot show the last 1000 tasks. It resets if there is a delay greater than 10 seconds. For a full history of the last 100,000 tasks see the ``tasks/`` page. Resources ~~~~~~~~~ The resources plot show the average CPU and Memory use over time as well as average network traffic. More detailed information on a per-worker basis is available in the ``workers/`` page. .. image:: https://raw.githubusercontent.com/dask/dask-org/master/images/bokeh-resources.gif :alt: Resources view of Dask web interface Connecting to Web Interface --------------------------- Default ~~~~~~~ By default, ``dask-scheduler`` prints out the address of the web interface:: INFO - Bokeh UI at: http://10.129.39.91:8787/status ... INFO - Starting Bokeh server on port 8787 with applications at paths ['/status', '/tasks'] The machine hosting the scheduler runs an HTTP server serving at that address. Troubleshooting --------------- Some clusters restrict the ports that are visible to the outside world. These ports may include the default port for the web interface, ``8787``. There are a few ways to handle this: 1. Open port ``8787`` to the outside world. Often this involves asking your cluster administrator. 2. Use a different port that is publicly accessible using the ``--bokeh-port PORT`` option on the ``dask-scheduler`` command. 3. Use fancier techniques, like `Port Forwarding`_ Running distributed on a remote machine can cause issues with viewing the web UI -- this depends on the remote machines network configuration. .. _`Port Forwarding`: https://en.wikipedia.org/wiki/Port_forwarding Port Forwarding ~~~~~~~~~~~~~~~ If you have SSH access then one way to gain access to a blocked port is through SSH port forwarding. A typical use case looks like the following: .. code:: bash local$ ssh -L 8000:localhost:8787 user@remote remote$ dask-scheduler # now, the web UI is visible at localhost:8000 remote$ # continue to set up dask if needed -- add workers, etc It is then possible to go to ``localhost:8000`` and see Dask Web UI. This same approach is not specific to dask.distributed, but can be used by any service that operates over a network, such as Jupyter notebooks. For example, if we chose to do this we could forward port 8888 (the default Jupyter port) to port 8001 with ``ssh -L 8001:localhost:8888 user@remote``. distributed-1.20.2/docs/source/work-stealing.rst000066400000000000000000000120501321233345200216620ustar00rootroot00000000000000Work Stealing ============= Some tasks prefer to run on certain workers. This may be because that worker holds data dependencies of the task or because the user has expressed a loose desire that the task run in a particular place. Occasionally this results in a few very busy workers and several idle workers. In this situation the idle workers may choose to steal work from the busy workers, even if stealing work requires the costly movement of data. This is a performance optimization and not required for correctness. Work stealing provides robustness in many ad-hoc cases, but can also backfire when we steal the wrong tasks and reduce performance. Criteria for stealing --------------------- Computation to Communication Ratio ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ Stealing is profitable when the computation time for a task is much longer than the communication time of the task's dependencies. **Bad example** We do not want to steal tasks that require moving a large dependent piece of data across a wire from the victim to the thief if the computation is fast. We end up spending far more time in communication than just waiting a bit longer and giving up on parallelism. .. code-block:: python [data] = client.scatter([np.arange(1000000000)]) x = client.submit(np.sum, data) **Good example** We do want to steal task tasks that only need to move dependent pieces of data, especially when the computation time is expensive (here 100 seconds.) .. code-block:: python [data] = client.scatter([100]) x = client.submit(sleep, data) Fortunately we often know both the number of bytes of dependencies (as reported by calling ``sys.getsizeof`` on the workers) and the runtime cost of previously seen functions, which is maintained as an exponentially weighted moving average. Saturated Worker Burden ~~~~~~~~~~~~~~~~~~~~~~~ Stealing may be profitable even when the computation-time to communication-time ratio is poor. This occurs when the saturated workers have a very long backlog of tasks and there are a large number of idle workers. We determine if it acceptable to steal a task if the last task to be run by the saturated workers would finish more quickly if stolen or if it remains on the original/victim worker. The longer the backlog of stealable tasks, and the smaller the number of active workers we have both increase our willingness to steal. This is balanced against the compute-to-communicate cost ratio. Replicate Popular Data ~~~~~~~~~~~~~~~~~~~~~~ It is also good long term if stealing causes highly-sought-after data to be replicated on more workers. Steal from the Rich ~~~~~~~~~~~~~~~~~~~ We would like to steal tasks from particularly over-burdened workers rather than workers with just a few excess tasks. Restrictions ~~~~~~~~~~~~ If a task has been specifically restricted to run on particular workers (such as is the case when special hardware is required) then we do not steal. Choosing tasks to steal ----------------------- We maintain a list of sets of stealable tasks, ordered into bins by computation-to-communication time ratio. The first bin contains all tasks with a compute-to-communicate ratio greater than or equal to 8 (considered high enough to always steal), the next bin with a ratio of 4, the next bin with a ratio of 2, etc.., all the way down to a ratio of 1/256, which we will never steal. This data structure provides a somewhat-ordered view of all stealable tasks that we can add to and remove from in constant time, rather than ``log(n)`` as with more traditional data structures, like a heap. During any stage when we submit tasks to workers we check if there are both idle and saturated workers and if so we quickly run through this list of sets, selecting tasks from the best buckets first, working our way down to the buckets of less desirable stealable tasks. We stop either when there are no more stealable tasks, no more idle workers, or when the quality of the task-to-be-stolen is not high enough given the current backlog. This approach is fast, optimizes to steal the tasks with the best computation-to-communication cost ratio (up to a factor of two) and tends to steal from the workers that have the largest backlogs, just by nature that random selection tends to draw from the largest population. Transactional Work Stealing --------------------------- To avoid running the same task twice, Dask implements transactional work stealing. When the scheduler identifies a task that should be moved it first sends a request to the busy worker. The worker inspects its current state of the task and sends a response to the scheduler: 1. If the task is not yet running, then the worker cancels the task and informs the scheduler that it can reroute the ask elsewhere. 2. If the task is already running or complete then the worker tells the scheduler that it should not replicate the task elsewhere. This avoids redundant work, and also the duplication of side effects for more exotic tasks. However, concurrent or repeated execution of the same task *is still possible* in the event of worker death or a disrupted network connection. distributed-1.20.2/docs/source/worker.rst000066400000000000000000000213421321233345200204110ustar00rootroot00000000000000Worker ====== Overview -------- Workers provide two functions: 1. Compute tasks as directed by the scheduler 2. Store and serve computed results to other workers or clients Each worker contains a ThreadPool that it uses to evaluate tasks as requested by the scheduler. It stores the results of these tasks locally and serves them to other workers or clients on demand. If the worker is asked to evaluate a task for which it does not have all of the necessary data then it will reach out to its peer workers to gather the necessary dependencies. A typical conversation between a scheduler and two workers Alice and Bob may look like the following:: Scheduler -> Alice: Compute ``x <- add(1, 2)``! Alice -> Scheduler: I've computed x and am holding on to it! Scheduler -> Bob: Compute ``y <- add(x, 10)``! You will need x. Alice has x. Bob -> Alice: Please send me x. Alice -> Bob: Sure. x is 3! Bob -> Scheduler: I've computed y and am holding on to it! Storing Data ------------ Data is stored locally in a dictionary in the ``.data`` attribute that maps keys to the results of function calls. .. code-block:: python >>> worker.data {'x': 3, 'y': 13, ... '(df, 0)': pd.DataFrame(...), ... } This ``.data`` attribute is a ``MutableMapping`` that is typically a combination of in-memory and on-disk storage with an LRU policy to move data between them. Thread Pool ----------- Each worker sends computations to a thread in a `concurrent.futures.ThreadPoolExecutor `_ for computation. These computations occur in the same process as the Worker communication server so that they can access and share data efficiently between each other. For the purposes of data locality all threads within a worker are considered the same worker. If your computations are mostly numeric in nature (for example NumPy and Pandas computations) and release the GIL entirely then it is advisable to run ``dask-worker`` processes with many threads and one process. This reduces communication costs and generally simplifies deployment. If your computations are mostly Python code and don't release the GIL then it is advisable to run ``dask-worker`` processes with many processes and one thread per core:: $ dask-worker scheduler:8786 --nprocs 8 If your computations are external to Python and long-running and don't release the GIL then beware that while the computation is running the worker process will not be able to communicate to other workers or to the scheduler. This situation should be avoided. If you don't link in your own custom C/Fortran code then this topic probably doesn't apply to you. Command Line tool ----------------- Use the ``dask-worker`` command line tool to start an individual worker. Here are the available options:: $ dask-worker --help Usage: dask-worker [OPTIONS] SCHEDULER Options: --worker-port INTEGER Serving worker port, defaults to randomly assigned --http-port INTEGER Serving http port, defaults to randomly assigned --nanny-port INTEGER Serving nanny port, defaults to randomly assigned --port INTEGER Deprecated, see --nanny-port --host TEXT Serving host. Defaults to an ip address that can hopefully be visible from the scheduler network. --nthreads INTEGER Number of threads per process. Defaults to number of cores --nprocs INTEGER Number of worker processes. Defaults to one. --name TEXT Alias --memory-limit TEXT Number of bytes before spilling data to disk --no-nanny --help Show this message and exit. Internal Scheduling ------------------- Internally tasks that come to the scheduler proceed through the following pipeline: .. image:: images/worker-task-state.svg :alt: Dask worker task states The worker also tracks data dependencies that are required to run the tasks above. These follow through a simpler pipeline: .. image:: images/worker-dep-state.svg :alt: Dask worker dependency states As tasks arrive they are prioritized and put into a heap. They are then taken from this heap in turn to have any remote dependencies collected. For each dependency we select a worker at random that has that data and collect the dependency from that worker. To improve bandwidth we opportunistically gather other dependencies of other tasks that are known to be on that worker, up to a maximum of 200MB of data (too little data and bandwidth suffers, too much data and responsiveness suffers). We use a fixed number of connections (around 10-50) so as to avoid overly-fragmenting our network bandwidth. After all dependencies for a task are in memory we transition the task to the ready state and put the task again into a heap of tasks that are ready to run. We collect from this heap and put the task into a thread from a local thread pool to execute. Optionally, this task may identify itself as a long-running task (see :doc:`Tasks launching tasks `), at which point it secedes from the thread pool. A task either errs or its result is put into memory. In either case a response is sent back to the scheduler. Memory Management ----------------- Workers are given a target memory limit to stay under with the command line ``--memory-limit`` keyword or the ``memory_limit=`` Python keyword argument.:: $ dask-worker tcp://scheduler:port --memory-limit=auto # total available RAM $ dask-worker tcp://scheduler:port --memory-limit=4e9 # four gigabytes Workers use a few different policies to keep memory use beneath this limit: 1. At 60% of memory load (as estimated by ``sizeof``), spill least recently used data to disk 2. At 70% of memory load, spill least recently used data to disk regardless of what is reported by ``sizeof`` 3. At 80% of memory load, stop accepting new work on local thread pool 4. At 95% of memory load, terminate and restart the worker These values can be configured by modifying the ``~/.dask/config.yaml`` file .. code-block:: yaml # Fractions of worker memory at which we take action to avoid memory blowup # Set any of the lower three values to False to turn off the behavior entirely worker-memory-target: 0.60 # target fraction to stay below worker-memory-spill: 0.70 # fraction at which we spill to disk worker-memory-pause: 0.80 # fraction at which we pause worker threads worker-memory-terminate: 0.95 # fraction at which we terminate the worker Spill data to Disk ~~~~~~~~~~~~~~~~~~ Every time the worker finishes a task it estimates the size in bytes that the result costs to keep in memory using the ``sizeof`` function. This function defaults to ``sys.getsizeof`` for arbitrary objects which uses the standard Python `__sizeof__ protocol `_, but also has special-cased implementations for common data types like NumPy arrays and Pandas dataframes. When the sum of the number of bytes of the data in memory exceeds 60% of the available threshold the worker will begin to dump the least recently used data to disk. You can control this location with the ``--local-directory`` keyword.:: $ dask-worker tcp://scheduler:port --memory-limit 4e9 --local-directory /scratch That data is still available and will be read back from disk when necessary. On the diagnostic dashboard status page disk I/O will show up in the task stream plot as orange blocks. Additionally the memory plot in the upper left will become orange and then red. Monitor process memory load ~~~~~~~~~~~~~~~~~~~~~~~~~~~ The approach above can fail for a few reasons 1. Custom objects may not report their memory size accurately 2. User functions may take up more RAM than expected 3. Significant amounts of data may accumulate in network I/O buffers To address this we periodically monitor the memory of the worker process every 200 ms. If the system reported memory use is above 70% of the target memory usage then the worker will start dumping unused data to disk, even if internal ``sizeof`` recording hasn't yet reached the normal 60% limit. Halt worker threads ~~~~~~~~~~~~~~~~~~~ At 80% load the worker's thread pool will stop accepting new tasks. This gives time for the write-to-disk functionality to take effect even in the face of rapidly accumulating data. Kill Worker ~~~~~~~~~~~ At 95% memory load a worker's nanny process will terminate it. This is to avoid having our worker job being terminated by an external job scheduler (like YARN, Mesos, SGE, etc..). After termination the nanny will restart the worker in a fresh state. API Documentation ----------------- .. autoclass:: distributed.worker.Worker distributed-1.20.2/requirements.txt000066400000000000000000000003601321233345200173770ustar00rootroot00000000000000click >= 6.6 cloudpickle >= 0.2.2 dask >= 0.16.0 msgpack-python psutil six sortedcontainers tblib toolz >= 0.7.4 tornado >= 4.5.1 zict >= 0.1.3 # Compatibility packages futures; python_version < '3.0' singledispatch; python_version < '3.4' distributed-1.20.2/setup.cfg000066400000000000000000000025701321233345200157410ustar00rootroot00000000000000[flake8] # References: # https://flake8.readthedocs.io/en/latest/user/configuration.html # https://flake8.readthedocs.io/en/latest/user/error-codes.html # Note: there cannot be spaces after comma's here exclude = __init__.py,distributed/_concurrent_futures_thread.py ignore = # Extra space in brackets E20, # Multiple spaces around "," E231,E241, # Comments E26, # Import formatting E4, # Comparing types instead of isinstance E721, # Assigning lambda expression E731, # continuation line under-indented for hanging indent E121, # continuation line over-indented for hanging indent E126, # continuation line over-indented for visual indent E127, # E128 continuation line under-indented for visual indent E128, # multiple statements on one line (semicolon) E702, # line break before binary operator W503, # visually indented line with same indent as next logical line E129, # unexpected indentation E116, # redefinition of unused 'loop' from line 10 F811, # local variable is assigned to but never used F841, # Ambiguous variable names E741 max-line-length = 120 [versioneer] VCS = git style = pep440 versionfile_source = distributed/_version.py versionfile_build = distributed/_version.py tag_prefix = parentdir_prefix = distributed- [bdist_wheel] universal=1 distributed-1.20.2/setup.py000077500000000000000000000045571321233345200156440ustar00rootroot00000000000000#!/usr/bin/env python import os from setuptools import setup import sys import versioneer requires = open('requirements.txt').read().strip().split('\n') install_requires = [] extras_require = {} for r in requires: 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 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 computing', 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.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", "License :: OSI Approved :: BSD License", "Operating System :: OS Independent", "Programming Language :: Python", "Programming Language :: Python :: 2.7", "Programming Language :: Python :: 3.5", "Programming Language :: Python :: 3.6", "Topic :: Scientific/Engineering", "Topic :: System :: Distributed Computing", ], entry_points=''' [console_scripts] dask-ssh=distributed.cli.dask_ssh:go dask-submit=distributed.cli.dask_submit:go dask-remote=distributed.cli.dask_remote:go dask-scheduler=distributed.cli.dask_scheduler:go dask-worker=distributed.cli.dask_worker:go dask-mpi=distributed.cli.dask_mpi:go ''', zip_safe=False) distributed-1.20.2/versioneer.py000066400000000000000000002057531321233345200166630ustar00rootroot00000000000000 # Version: 0.17 """The Versioneer - like a rocketeer, but for versions. The Versioneer ============== * like a rocketeer, but for versions! * https://github.com/warner/python-versioneer * Brian Warner * License: Public Domain * Compatible With: python2.6, 2.7, 3.2, 3.3, 3.4, 3.5, and pypy * [![Latest Version] (https://pypip.in/version/versioneer/badge.svg?style=flat) ](https://pypi.python.org/pypi/versioneer/) * [![Build Status] (https://travis-ci.org/warner/python-versioneer.png?branch=master) ](https://travis-ci.org/warner/python-versioneer) This is a tool for managing a recorded version number in distutils-based python projects. The goal is to remove the tedious and error-prone "update the embedded version string" step from your release process. Making a new release should be as easy as recording a new tag in your version-control system, and maybe making new tarballs. ## Quick Install * `pip install versioneer` to somewhere to your $PATH * add a `[versioneer]` section to your setup.cfg (see below) * run `versioneer install` in your source tree, commit the results ## Version Identifiers Source trees come from a variety of places: * a version-control system checkout (mostly used by developers) * a nightly tarball, produced by build automation * a snapshot tarball, produced by a web-based VCS browser, like github's "tarball from tag" feature * a release tarball, produced by "setup.py sdist", distributed through PyPI Within each source tree, the version identifier (either a string or a number, this tool is format-agnostic) can come from a variety of places: * ask the VCS tool itself, e.g. "git describe" (for checkouts), which knows about recent "tags" and an absolute revision-id * the name of the directory into which the tarball was unpacked * an expanded VCS keyword ($Id$, etc) * a `_version.py` created by some earlier build step For released software, the version identifier is closely related to a VCS tag. Some projects use tag names that include more than just the version string (e.g. "myproject-1.2" instead of just "1.2"), in which case the tool needs to strip the tag prefix to extract the version identifier. For unreleased software (between tags), the version identifier should provide enough information to help developers recreate the same tree, while also giving them an idea of roughly how old the tree is (after version 1.2, before version 1.3). Many VCS systems can report a description that captures this, for example `git describe --tags --dirty --always` reports things like "0.7-1-g574ab98-dirty" to indicate that the checkout is one revision past the 0.7 tag, has a unique revision id of "574ab98", and is "dirty" (it has uncommitted changes. The version identifier is used for multiple purposes: * to allow the module to self-identify its version: `myproject.__version__` * to choose a name and prefix for a 'setup.py sdist' tarball ## Theory of Operation Versioneer works by adding a special `_version.py` file into your source tree, where your `__init__.py` can import it. This `_version.py` knows how to dynamically ask the VCS tool for version information at import time. `_version.py` also contains `$Revision$` markers, and the installation process marks `_version.py` to have this marker rewritten with a tag name during the `git archive` command. As a result, generated tarballs will contain enough information to get the proper version. To allow `setup.py` to compute a version too, a `versioneer.py` is added to the top level of your source tree, next to `setup.py` and the `setup.cfg` that configures it. This overrides several distutils/setuptools commands to compute the version when invoked, and changes `setup.py build` and `setup.py sdist` to replace `_version.py` with a small static file that contains just the generated version data. ## Installation See [INSTALL.md](./INSTALL.md) for detailed installation instructions. ## Version-String Flavors Code which uses Versioneer can learn about its version string at runtime by importing `_version` from your main `__init__.py` file and running the `get_versions()` function. From the "outside" (e.g. in `setup.py`), you can import the top-level `versioneer.py` and run `get_versions()`. Both functions return a dictionary with different flavors of version information: * `['version']`: A condensed version string, rendered using the selected style. This is the most commonly used value for the project's version string. The default "pep440" style yields strings like `0.11`, `0.11+2.g1076c97`, or `0.11+2.g1076c97.dirty`. See the "Styles" section below for alternative styles. * `['full-revisionid']`: detailed revision identifier. For Git, this is the full SHA1 commit id, e.g. "1076c978a8d3cfc70f408fe5974aa6c092c949ac". * `['date']`: Date and time of the latest `HEAD` commit. For Git, it is the commit date in ISO 8601 format. This will be None if the date is not available. * `['dirty']`: a boolean, True if the tree has uncommitted changes. Note that this is only accurate if run in a VCS checkout, otherwise it is likely to be False or None * `['error']`: if the version string could not be computed, this will be set to a string describing the problem, otherwise it will be None. It may be useful to throw an exception in setup.py if this is set, to avoid e.g. creating tarballs with a version string of "unknown". Some variants are more useful than others. Including `full-revisionid` in a bug report should allow developers to reconstruct the exact code being tested (or indicate the presence of local changes that should be shared with the developers). `version` is suitable for display in an "about" box or a CLI `--version` output: it can be easily compared against release notes and lists of bugs fixed in various releases. The installer adds the following text to your `__init__.py` to place a basic version in `YOURPROJECT.__version__`: from ._version import get_versions __version__ = get_versions()['version'] del get_versions ## Styles The setup.cfg `style=` configuration controls how the VCS information is rendered into a version string. The default style, "pep440", produces a PEP440-compliant string, equal to the un-prefixed tag name for actual releases, and containing an additional "local version" section with more detail for in-between builds. For Git, this is TAG[+DISTANCE.gHEX[.dirty]] , using information from `git describe --tags --dirty --always`. For example "0.11+2.g1076c97.dirty" indicates that the tree is like the "1076c97" commit but has uncommitted changes (".dirty"), and that this commit is two revisions ("+2") beyond the "0.11" tag. For released software (exactly equal to a known tag), the identifier will only contain the stripped tag, e.g. "0.11". Other styles are available. See details.md in the Versioneer source tree for descriptions. ## Debugging Versioneer tries to avoid fatal errors: if something goes wrong, it will tend to return a version of "0+unknown". To investigate the problem, run `setup.py version`, which will run the version-lookup code in a verbose mode, and will display the full contents of `get_versions()` (including the `error` string, which may help identify what went wrong). ## Known Limitations Some situations are known to cause problems for Versioneer. This details the most significant ones. More can be found on Github [issues page](https://github.com/warner/python-versioneer/issues). ### Subprojects Versioneer has limited support for source trees in which `setup.py` is not in the root directory (e.g. `setup.py` and `.git/` are *not* siblings). The are two common reasons why `setup.py` might not be in the root: * Source trees which contain multiple subprojects, such as [Buildbot](https://github.com/buildbot/buildbot), which contains both "master" and "slave" subprojects, each with their own `setup.py`, `setup.cfg`, and `tox.ini`. Projects like these produce multiple PyPI distributions (and upload multiple independently-installable tarballs). * Source trees whose main purpose is to contain a C library, but which also provide bindings to Python (and perhaps other langauges) in subdirectories. Versioneer will look for `.git` in parent directories, and most operations should get the right version string. However `pip` and `setuptools` have bugs and implementation details which frequently cause `pip install .` from a subproject directory to fail to find a correct version string (so it usually defaults to `0+unknown`). `pip install --editable .` should work correctly. `setup.py install` might work too. Pip-8.1.1 is known to have this problem, but hopefully it will get fixed in some later version. [Bug #38](https://github.com/warner/python-versioneer/issues/38) is tracking this issue. The discussion in [PR #61](https://github.com/warner/python-versioneer/pull/61) describes the issue from the Versioneer side in more detail. [pip PR#3176](https://github.com/pypa/pip/pull/3176) and [pip PR#3615](https://github.com/pypa/pip/pull/3615) contain work to improve pip to let Versioneer work correctly. Versioneer-0.16 and earlier only looked for a `.git` directory next to the `setup.cfg`, so subprojects were completely unsupported with those releases. ### Editable installs with setuptools <= 18.5 `setup.py develop` and `pip install --editable .` allow you to install a project into a virtualenv once, then continue editing the source code (and test) without re-installing after every change. "Entry-point scripts" (`setup(entry_points={"console_scripts": ..})`) are a convenient way to specify executable scripts that should be installed along with the python package. These both work as expected when using modern setuptools. When using setuptools-18.5 or earlier, however, certain operations will cause `pkg_resources.DistributionNotFound` errors when running the entrypoint script, which must be resolved by re-installing the package. This happens when the install happens with one version, then the egg_info data is regenerated while a different version is checked out. Many setup.py commands cause egg_info to be rebuilt (including `sdist`, `wheel`, and installing into a different virtualenv), so this can be surprising. [Bug #83](https://github.com/warner/python-versioneer/issues/83) describes this one, but upgrading to a newer version of setuptools should probably resolve it. ### Unicode version strings While Versioneer works (and is continually tested) with both Python 2 and Python 3, it is not entirely consistent with bytes-vs-unicode distinctions. Newer releases probably generate unicode version strings on py2. It's not clear that this is wrong, but it may be surprising for applications when then write these strings to a network connection or include them in bytes-oriented APIs like cryptographic checksums. [Bug #71](https://github.com/warner/python-versioneer/issues/71) investigates this question. ## Updating Versioneer To upgrade your project to a new release of Versioneer, do the following: * install the new Versioneer (`pip install -U versioneer` or equivalent) * edit `setup.cfg`, if necessary, to include any new configuration settings indicated by the release notes. See [UPGRADING](./UPGRADING.md) for details. * re-run `versioneer install` in your source tree, to replace `SRC/_version.py` * commit any changed files ## Future Directions This tool is designed to make it easily extended to other version-control systems: all VCS-specific components are in separate directories like src/git/ . The top-level `versioneer.py` script is assembled from these components by running make-versioneer.py . In the future, make-versioneer.py will take a VCS name as an argument, and will construct a version of `versioneer.py` that is specific to the given VCS. It might also take the configuration arguments that are currently provided manually during installation by editing setup.py . Alternatively, it might go the other direction and include code from all supported VCS systems, reducing the number of intermediate scripts. ## License To make Versioneer easier to embed, all its code is dedicated to the public domain. The `_version.py` that it creates is also in the public domain. Specifically, both are released under the Creative Commons "Public Domain Dedication" license (CC0-1.0), as described in https://creativecommons.org/publicdomain/zero/1.0/ . """ from __future__ import print_function try: import configparser except ImportError: import ConfigParser as configparser import errno import json import os import re import subprocess import sys class VersioneerConfig: """Container for Versioneer configuration parameters.""" def get_root(): """Get the project root directory. We require that all commands are run from the project root, i.e. the directory that contains setup.py, setup.cfg, and versioneer.py . """ root = os.path.realpath(os.path.abspath(os.getcwd())) setup_py = os.path.join(root, "setup.py") versioneer_py = os.path.join(root, "versioneer.py") if not (os.path.exists(setup_py) or os.path.exists(versioneer_py)): # allow 'python path/to/setup.py COMMAND' root = os.path.dirname(os.path.realpath(os.path.abspath(sys.argv[0]))) setup_py = os.path.join(root, "setup.py") versioneer_py = os.path.join(root, "versioneer.py") if not (os.path.exists(setup_py) or os.path.exists(versioneer_py)): err = ("Versioneer was unable to run the project root directory. " "Versioneer requires setup.py to be executed from " "its immediate directory (like 'python setup.py COMMAND'), " "or in a way that lets it use sys.argv[0] to find the root " "(like 'python path/to/setup.py COMMAND').") raise VersioneerBadRootError(err) try: # Certain runtime workflows (setup.py install/develop in a setuptools # tree) execute all dependencies in a single python process, so # "versioneer" may be imported multiple times, and python's shared # module-import table will cache the first one. So we can't use # os.path.dirname(__file__), as that will find whichever # versioneer.py was first imported, even in later projects. me = os.path.realpath(os.path.abspath(__file__)) me_dir = os.path.normcase(os.path.splitext(me)[0]) vsr_dir = os.path.normcase(os.path.splitext(versioneer_py)[0]) if me_dir != vsr_dir: print("Warning: build in %s is using versioneer.py from %s" % (os.path.dirname(me), versioneer_py)) except NameError: pass return root def get_config_from_root(root): """Read the project setup.cfg file to determine Versioneer config.""" # This might raise EnvironmentError (if setup.cfg is missing), or # configparser.NoSectionError (if it lacks a [versioneer] section), or # configparser.NoOptionError (if it lacks "VCS="). See the docstring at # the top of versioneer.py for instructions on writing your setup.cfg . setup_cfg = os.path.join(root, "setup.cfg") parser = configparser.SafeConfigParser() with open(setup_cfg, "r") as f: parser.readfp(f) VCS = parser.get("versioneer", "VCS") # mandatory def get(parser, name): if parser.has_option("versioneer", name): return parser.get("versioneer", name) return None cfg = VersioneerConfig() cfg.VCS = VCS cfg.style = get(parser, "style") or "" cfg.versionfile_source = get(parser, "versionfile_source") cfg.versionfile_build = get(parser, "versionfile_build") cfg.tag_prefix = get(parser, "tag_prefix") if cfg.tag_prefix in ("''", '""'): cfg.tag_prefix = "" cfg.parentdir_prefix = get(parser, "parentdir_prefix") cfg.verbose = get(parser, "verbose") return cfg class NotThisMethod(Exception): """Exception raised if a method is not valid for the current scenario.""" # these dictionaries contain VCS-specific tools LONG_VERSION_PY = {} HANDLERS = {} def register_vcs_handler(vcs, method): # decorator """Decorator to mark a method as the handler for a particular VCS.""" def decorate(f): """Store f in HANDLERS[vcs][method].""" if vcs not in HANDLERS: HANDLERS[vcs] = {} HANDLERS[vcs][method] = f return f return decorate def run_command(commands, args, cwd=None, verbose=False, hide_stderr=False, env=None): """Call the given command(s).""" assert isinstance(commands, list) p = None for c in commands: try: dispcmd = str([c] + args) # remember shell=False, so use git.cmd on windows, not just git p = subprocess.Popen([c] + args, cwd=cwd, env=env, stdout=subprocess.PIPE, stderr=(subprocess.PIPE if hide_stderr else None)) break except EnvironmentError: e = sys.exc_info()[1] if e.errno == errno.ENOENT: continue if verbose: print("unable to run %s" % dispcmd) print(e) return None, None else: if verbose: print("unable to find command, tried %s" % (commands,)) return None, None stdout = p.communicate()[0].strip() if sys.version_info[0] >= 3: stdout = stdout.decode() if p.returncode != 0: if verbose: print("unable to run %s (error)" % dispcmd) print("stdout was %s" % stdout) return None, p.returncode return stdout, p.returncode LONG_VERSION_PY['git'] = ''' # This file helps to compute a version number in source trees obtained from # git-archive tarball (such as those provided by githubs download-from-tag # feature). Distribution tarballs (built by setup.py sdist) and build # directories (produced by setup.py build) will contain a much shorter file # that just contains the computed version number. # This file is released into the public domain. Generated by # versioneer-0.17 (https://github.com/warner/python-versioneer) """Git implementation of _version.py.""" import errno import os import re import subprocess import sys def get_keywords(): """Get the keywords needed to look up the version information.""" # these strings will be replaced by git during git-archive. # setup.py/versioneer.py will grep for the variable names, so they must # each be defined on a line of their own. _version.py will just call # get_keywords(). git_refnames = "%(DOLLAR)sFormat:%%d%(DOLLAR)s" git_full = "%(DOLLAR)sFormat:%%H%(DOLLAR)s" git_date = "%(DOLLAR)sFormat:%%ci%(DOLLAR)s" keywords = {"refnames": git_refnames, "full": git_full, "date": git_date} return keywords class VersioneerConfig: """Container for Versioneer configuration parameters.""" def get_config(): """Create, populate and return the VersioneerConfig() object.""" # these strings are filled in when 'setup.py versioneer' creates # _version.py cfg = VersioneerConfig() cfg.VCS = "git" cfg.style = "%(STYLE)s" cfg.tag_prefix = "%(TAG_PREFIX)s" cfg.parentdir_prefix = "%(PARENTDIR_PREFIX)s" cfg.versionfile_source = "%(VERSIONFILE_SOURCE)s" cfg.verbose = False return cfg class NotThisMethod(Exception): """Exception raised if a method is not valid for the current scenario.""" LONG_VERSION_PY = {} HANDLERS = {} def register_vcs_handler(vcs, method): # decorator """Decorator to mark a method as the handler for a particular VCS.""" def decorate(f): """Store f in HANDLERS[vcs][method].""" if vcs not in HANDLERS: HANDLERS[vcs] = {} HANDLERS[vcs][method] = f return f return decorate def run_command(commands, args, cwd=None, verbose=False, hide_stderr=False, env=None): """Call the given command(s).""" assert isinstance(commands, list) p = None for c in commands: try: dispcmd = str([c] + args) # remember shell=False, so use git.cmd on windows, not just git p = subprocess.Popen([c] + args, cwd=cwd, env=env, stdout=subprocess.PIPE, stderr=(subprocess.PIPE if hide_stderr else None)) break except EnvironmentError: e = sys.exc_info()[1] if e.errno == errno.ENOENT: continue if verbose: print("unable to run %%s" %% dispcmd) print(e) return None, None else: if verbose: print("unable to find command, tried %%s" %% (commands,)) return None, None stdout = p.communicate()[0].strip() if sys.version_info[0] >= 3: stdout = stdout.decode() if p.returncode != 0: if verbose: print("unable to run %%s (error)" %% dispcmd) print("stdout was %%s" %% stdout) return None, p.returncode return stdout, p.returncode def versions_from_parentdir(parentdir_prefix, root, verbose): """Try to determine the version from the parent directory name. Source tarballs conventionally unpack into a directory that includes both the project name and a version string. We will also support searching up two directory levels for an appropriately named parent directory """ rootdirs = [] for i in range(3): dirname = os.path.basename(root) if dirname.startswith(parentdir_prefix): return {"version": dirname[len(parentdir_prefix):], "full-revisionid": None, "dirty": False, "error": None, "date": None} else: rootdirs.append(root) root = os.path.dirname(root) # up a level if verbose: print("Tried directories %%s but none started with prefix %%s" %% (str(rootdirs), parentdir_prefix)) raise NotThisMethod("rootdir doesn't start with parentdir_prefix") @register_vcs_handler("git", "get_keywords") def git_get_keywords(versionfile_abs): """Extract version information from the given file.""" # the code embedded in _version.py can just fetch the value of these # keywords. When used from setup.py, we don't want to import _version.py, # so we do it with a regexp instead. This function is not used from # _version.py. keywords = {} try: f = open(versionfile_abs, "r") for line in f.readlines(): if line.strip().startswith("git_refnames ="): mo = re.search(r'=\s*"(.*)"', line) if mo: keywords["refnames"] = mo.group(1) if line.strip().startswith("git_full ="): mo = re.search(r'=\s*"(.*)"', line) if mo: keywords["full"] = mo.group(1) if line.strip().startswith("git_date ="): mo = re.search(r'=\s*"(.*)"', line) if mo: keywords["date"] = mo.group(1) f.close() except EnvironmentError: pass return keywords @register_vcs_handler("git", "keywords") def git_versions_from_keywords(keywords, tag_prefix, verbose): """Get version information from git keywords.""" if not keywords: raise NotThisMethod("no keywords at all, weird") date = keywords.get("date") if date is not None: # git-2.2.0 added "%%cI", which expands to an ISO-8601 -compliant # datestamp. However we prefer "%%ci" (which expands to an "ISO-8601 # -like" string, which we must then edit to make compliant), because # it's been around since git-1.5.3, and it's too difficult to # discover which version we're using, or to work around using an # older one. date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) refnames = keywords["refnames"].strip() if refnames.startswith("$Format"): if verbose: print("keywords are unexpanded, not using") raise NotThisMethod("unexpanded keywords, not a git-archive tarball") refs = set([r.strip() for r in refnames.strip("()").split(",")]) # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of # just "foo-1.0". If we see a "tag: " prefix, prefer those. TAG = "tag: " tags = set([r[len(TAG):] for r in refs if r.startswith(TAG)]) if not tags: # Either we're using git < 1.8.3, or there really are no tags. We use # a heuristic: assume all version tags have a digit. The old git %%d # expansion behaves like git log --decorate=short and strips out the # refs/heads/ and refs/tags/ prefixes that would let us distinguish # between branches and tags. By ignoring refnames without digits, we # filter out many common branch names like "release" and # "stabilization", as well as "HEAD" and "master". tags = set([r for r in refs if re.search(r'\d', r)]) if verbose: print("discarding '%%s', no digits" %% ",".join(refs - tags)) if verbose: print("likely tags: %%s" %% ",".join(sorted(tags))) for ref in sorted(tags): # sorting will prefer e.g. "2.0" over "2.0rc1" if ref.startswith(tag_prefix): r = ref[len(tag_prefix):] if verbose: print("picking %%s" %% r) return {"version": r, "full-revisionid": keywords["full"].strip(), "dirty": False, "error": None, "date": date} # no suitable tags, so version is "0+unknown", but full hex is still there if verbose: print("no suitable tags, using unknown + full revision id") return {"version": "0+unknown", "full-revisionid": keywords["full"].strip(), "dirty": False, "error": "no suitable tags", "date": None} @register_vcs_handler("git", "pieces_from_vcs") def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): """Get version from 'git describe' in the root of the source tree. This only gets called if the git-archive 'subst' keywords were *not* expanded, and _version.py hasn't already been rewritten with a short version string, meaning we're inside a checked out source tree. """ GITS = ["git"] if sys.platform == "win32": GITS = ["git.cmd", "git.exe"] out, rc = run_command(GITS, ["rev-parse", "--git-dir"], cwd=root, hide_stderr=True) if rc != 0: if verbose: print("Directory %%s not under git control" %% root) raise NotThisMethod("'git rev-parse --git-dir' returned error") # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] # if there isn't one, this yields HEX[-dirty] (no NUM) describe_out, rc = run_command(GITS, ["describe", "--tags", "--dirty", "--always", "--long", "--match", "%%s*" %% tag_prefix], cwd=root) # --long was added in git-1.5.5 if describe_out is None: raise NotThisMethod("'git describe' failed") describe_out = describe_out.strip() full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) if full_out is None: raise NotThisMethod("'git rev-parse' failed") full_out = full_out.strip() pieces = {} pieces["long"] = full_out pieces["short"] = full_out[:7] # maybe improved later pieces["error"] = None # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] # TAG might have hyphens. git_describe = describe_out # look for -dirty suffix dirty = git_describe.endswith("-dirty") pieces["dirty"] = dirty if dirty: git_describe = git_describe[:git_describe.rindex("-dirty")] # now we have TAG-NUM-gHEX or HEX if "-" in git_describe: # TAG-NUM-gHEX mo = re.search(r'^(.+)-(\d+)-g([0-9a-f]+)$', git_describe) if not mo: # unparseable. Maybe git-describe is misbehaving? pieces["error"] = ("unable to parse git-describe output: '%%s'" %% describe_out) return pieces # tag full_tag = mo.group(1) if not full_tag.startswith(tag_prefix): if verbose: fmt = "tag '%%s' doesn't start with prefix '%%s'" print(fmt %% (full_tag, tag_prefix)) pieces["error"] = ("tag '%%s' doesn't start with prefix '%%s'" %% (full_tag, tag_prefix)) return pieces pieces["closest-tag"] = full_tag[len(tag_prefix):] # distance: number of commits since tag pieces["distance"] = int(mo.group(2)) # commit: short hex revision ID pieces["short"] = mo.group(3) else: # HEX: no tags pieces["closest-tag"] = None count_out, rc = run_command(GITS, ["rev-list", "HEAD", "--count"], cwd=root) pieces["distance"] = int(count_out) # total number of commits # commit date: see ISO-8601 comment in git_versions_from_keywords() date = run_command(GITS, ["show", "-s", "--format=%%ci", "HEAD"], cwd=root)[0].strip() pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) return pieces def plus_or_dot(pieces): """Return a + if we don't already have one, else return a .""" if "+" in pieces.get("closest-tag", ""): return "." return "+" def render_pep440(pieces): """Build up version string, with post-release "local version identifier". Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty Exceptions: 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"] or pieces["dirty"]: rendered += plus_or_dot(pieces) rendered += "%%d.g%%s" %% (pieces["distance"], pieces["short"]) if pieces["dirty"]: rendered += ".dirty" else: # exception #1 rendered = "0+untagged.%%d.g%%s" %% (pieces["distance"], pieces["short"]) if pieces["dirty"]: rendered += ".dirty" return rendered def render_pep440_pre(pieces): """TAG[.post.devDISTANCE] -- No -dirty. Exceptions: 1: no tags. 0.post.devDISTANCE """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"]: rendered += ".post.dev%%d" %% pieces["distance"] else: # exception #1 rendered = "0.post.dev%%d" %% pieces["distance"] return rendered def render_pep440_post(pieces): """TAG[.postDISTANCE[.dev0]+gHEX] . The ".dev0" means dirty. Note that .dev0 sorts backwards (a dirty tree will appear "older" than the corresponding clean one), but you shouldn't be releasing software with -dirty anyways. Exceptions: 1: no tags. 0.postDISTANCE[.dev0] """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"] or pieces["dirty"]: rendered += ".post%%d" %% pieces["distance"] if pieces["dirty"]: rendered += ".dev0" rendered += plus_or_dot(pieces) rendered += "g%%s" %% pieces["short"] else: # exception #1 rendered = "0.post%%d" %% pieces["distance"] if pieces["dirty"]: rendered += ".dev0" rendered += "+g%%s" %% pieces["short"] return rendered def render_pep440_old(pieces): """TAG[.postDISTANCE[.dev0]] . The ".dev0" means dirty. Eexceptions: 1: no tags. 0.postDISTANCE[.dev0] """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"] or pieces["dirty"]: rendered += ".post%%d" %% pieces["distance"] if pieces["dirty"]: rendered += ".dev0" else: # exception #1 rendered = "0.post%%d" %% pieces["distance"] if pieces["dirty"]: rendered += ".dev0" return rendered def render_git_describe(pieces): """TAG[-DISTANCE-gHEX][-dirty]. Like 'git describe --tags --dirty --always'. Exceptions: 1: no tags. HEX[-dirty] (note: no 'g' prefix) """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"]: rendered += "-%%d-g%%s" %% (pieces["distance"], pieces["short"]) else: # exception #1 rendered = pieces["short"] if pieces["dirty"]: rendered += "-dirty" return rendered def render_git_describe_long(pieces): """TAG-DISTANCE-gHEX[-dirty]. Like 'git describe --tags --dirty --always -long'. The distance/hash is unconditional. Exceptions: 1: no tags. HEX[-dirty] (note: no 'g' prefix) """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] rendered += "-%%d-g%%s" %% (pieces["distance"], pieces["short"]) else: # exception #1 rendered = pieces["short"] if pieces["dirty"]: rendered += "-dirty" return rendered def render(pieces, style): """Render the given version pieces into the requested style.""" if pieces["error"]: return {"version": "unknown", "full-revisionid": pieces.get("long"), "dirty": None, "error": pieces["error"], "date": None} if not style or style == "default": style = "pep440" # the default if style == "pep440": rendered = render_pep440(pieces) elif style == "pep440-pre": rendered = render_pep440_pre(pieces) elif style == "pep440-post": rendered = render_pep440_post(pieces) elif style == "pep440-old": rendered = render_pep440_old(pieces) elif style == "git-describe": rendered = render_git_describe(pieces) elif style == "git-describe-long": rendered = render_git_describe_long(pieces) else: raise ValueError("unknown style '%%s'" %% style) return {"version": rendered, "full-revisionid": pieces["long"], "dirty": pieces["dirty"], "error": None, "date": pieces.get("date")} def get_versions(): """Get version information or return default if unable to do so.""" # I am in _version.py, which lives at ROOT/VERSIONFILE_SOURCE. If we have # __file__, we can work backwards from there to the root. Some # py2exe/bbfreeze/non-CPython implementations don't do __file__, in which # case we can only use expanded keywords. cfg = get_config() verbose = cfg.verbose try: return git_versions_from_keywords(get_keywords(), cfg.tag_prefix, verbose) except NotThisMethod: pass try: root = os.path.realpath(__file__) # versionfile_source is the relative path from the top of the source # tree (where the .git directory might live) to this file. Invert # this to find the root from __file__. for i in cfg.versionfile_source.split('/'): root = os.path.dirname(root) except NameError: return {"version": "0+unknown", "full-revisionid": None, "dirty": None, "error": "unable to find root of source tree", "date": None} try: pieces = git_pieces_from_vcs(cfg.tag_prefix, root, verbose) return render(pieces, cfg.style) except NotThisMethod: pass try: if cfg.parentdir_prefix: return versions_from_parentdir(cfg.parentdir_prefix, root, verbose) except NotThisMethod: pass return {"version": "0+unknown", "full-revisionid": None, "dirty": None, "error": "unable to compute version", "date": None} ''' @register_vcs_handler("git", "get_keywords") def git_get_keywords(versionfile_abs): """Extract version information from the given file.""" # the code embedded in _version.py can just fetch the value of these # keywords. When used from setup.py, we don't want to import _version.py, # so we do it with a regexp instead. This function is not used from # _version.py. keywords = {} try: f = open(versionfile_abs, "r") for line in f.readlines(): if line.strip().startswith("git_refnames ="): mo = re.search(r'=\s*"(.*)"', line) if mo: keywords["refnames"] = mo.group(1) if line.strip().startswith("git_full ="): mo = re.search(r'=\s*"(.*)"', line) if mo: keywords["full"] = mo.group(1) if line.strip().startswith("git_date ="): mo = re.search(r'=\s*"(.*)"', line) if mo: keywords["date"] = mo.group(1) f.close() except EnvironmentError: pass return keywords @register_vcs_handler("git", "keywords") def git_versions_from_keywords(keywords, tag_prefix, verbose): """Get version information from git keywords.""" if not keywords: raise NotThisMethod("no keywords at all, weird") date = keywords.get("date") if date is not None: # git-2.2.0 added "%cI", which expands to an ISO-8601 -compliant # datestamp. However we prefer "%ci" (which expands to an "ISO-8601 # -like" string, which we must then edit to make compliant), because # it's been around since git-1.5.3, and it's too difficult to # discover which version we're using, or to work around using an # older one. date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) refnames = keywords["refnames"].strip() if refnames.startswith("$Format"): if verbose: print("keywords are unexpanded, not using") raise NotThisMethod("unexpanded keywords, not a git-archive tarball") refs = set([r.strip() for r in refnames.strip("()").split(",")]) # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of # just "foo-1.0". If we see a "tag: " prefix, prefer those. TAG = "tag: " tags = set([r[len(TAG):] for r in refs if r.startswith(TAG)]) if not tags: # Either we're using git < 1.8.3, or there really are no tags. We use # a heuristic: assume all version tags have a digit. The old git %d # expansion behaves like git log --decorate=short and strips out the # refs/heads/ and refs/tags/ prefixes that would let us distinguish # between branches and tags. By ignoring refnames without digits, we # filter out many common branch names like "release" and # "stabilization", as well as "HEAD" and "master". tags = set([r for r in refs if re.search(r'\d', r)]) if verbose: print("discarding '%s', no digits" % ",".join(refs - tags)) if verbose: print("likely tags: %s" % ",".join(sorted(tags))) for ref in sorted(tags): # sorting will prefer e.g. "2.0" over "2.0rc1" if ref.startswith(tag_prefix): r = ref[len(tag_prefix):] if verbose: print("picking %s" % r) return {"version": r, "full-revisionid": keywords["full"].strip(), "dirty": False, "error": None, "date": date} # no suitable tags, so version is "0+unknown", but full hex is still there if verbose: print("no suitable tags, using unknown + full revision id") return {"version": "0+unknown", "full-revisionid": keywords["full"].strip(), "dirty": False, "error": "no suitable tags", "date": None} @register_vcs_handler("git", "pieces_from_vcs") def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): """Get version from 'git describe' in the root of the source tree. This only gets called if the git-archive 'subst' keywords were *not* expanded, and _version.py hasn't already been rewritten with a short version string, meaning we're inside a checked out source tree. """ GITS = ["git"] if sys.platform == "win32": GITS = ["git.cmd", "git.exe"] out, rc = run_command(GITS, ["rev-parse", "--git-dir"], cwd=root, hide_stderr=True) if rc != 0: if verbose: print("Directory %s not under git control" % root) raise NotThisMethod("'git rev-parse --git-dir' returned error") # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] # if there isn't one, this yields HEX[-dirty] (no NUM) describe_out, rc = run_command(GITS, ["describe", "--tags", "--dirty", "--always", "--long", "--match", "%s*" % tag_prefix], cwd=root) # --long was added in git-1.5.5 if describe_out is None: raise NotThisMethod("'git describe' failed") describe_out = describe_out.strip() full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) if full_out is None: raise NotThisMethod("'git rev-parse' failed") full_out = full_out.strip() pieces = {} pieces["long"] = full_out pieces["short"] = full_out[:7] # maybe improved later pieces["error"] = None # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] # TAG might have hyphens. git_describe = describe_out # look for -dirty suffix dirty = git_describe.endswith("-dirty") pieces["dirty"] = dirty if dirty: git_describe = git_describe[:git_describe.rindex("-dirty")] # now we have TAG-NUM-gHEX or HEX if "-" in git_describe: # TAG-NUM-gHEX mo = re.search(r'^(.+)-(\d+)-g([0-9a-f]+)$', git_describe) if not mo: # unparseable. Maybe git-describe is misbehaving? pieces["error"] = ("unable to parse git-describe output: '%s'" % describe_out) return pieces # tag full_tag = mo.group(1) if not full_tag.startswith(tag_prefix): if verbose: fmt = "tag '%s' doesn't start with prefix '%s'" print(fmt % (full_tag, tag_prefix)) pieces["error"] = ("tag '%s' doesn't start with prefix '%s'" % (full_tag, tag_prefix)) return pieces pieces["closest-tag"] = full_tag[len(tag_prefix):] # distance: number of commits since tag pieces["distance"] = int(mo.group(2)) # commit: short hex revision ID pieces["short"] = mo.group(3) else: # HEX: no tags pieces["closest-tag"] = None count_out, rc = run_command(GITS, ["rev-list", "HEAD", "--count"], cwd=root) pieces["distance"] = int(count_out) # total number of commits # commit date: see ISO-8601 comment in git_versions_from_keywords() date = run_command(GITS, ["show", "-s", "--format=%ci", "HEAD"], cwd=root)[0].strip() pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) return pieces def do_vcs_install(manifest_in, versionfile_source, ipy): """Git-specific installation logic for Versioneer. For Git, this means creating/changing .gitattributes to mark _version.py for export-subst keyword substitution. """ GITS = ["git"] if sys.platform == "win32": GITS = ["git.cmd", "git.exe"] files = [manifest_in, versionfile_source] if ipy: files.append(ipy) try: me = __file__ if me.endswith(".pyc") or me.endswith(".pyo"): me = os.path.splitext(me)[0] + ".py" versioneer_file = os.path.relpath(me) except NameError: versioneer_file = "versioneer.py" files.append(versioneer_file) present = False try: f = open(".gitattributes", "r") for line in f.readlines(): if line.strip().startswith(versionfile_source): if "export-subst" in line.strip().split()[1:]: present = True f.close() except EnvironmentError: pass if not present: f = open(".gitattributes", "a+") f.write("%s export-subst\n" % versionfile_source) f.close() files.append(".gitattributes") run_command(GITS, ["add", "--"] + files) def versions_from_parentdir(parentdir_prefix, root, verbose): """Try to determine the version from the parent directory name. Source tarballs conventionally unpack into a directory that includes both the project name and a version string. We will also support searching up two directory levels for an appropriately named parent directory """ rootdirs = [] for i in range(3): dirname = os.path.basename(root) if dirname.startswith(parentdir_prefix): return {"version": dirname[len(parentdir_prefix):], "full-revisionid": None, "dirty": False, "error": None, "date": None} else: rootdirs.append(root) root = os.path.dirname(root) # up a level if verbose: print("Tried directories %s but none started with prefix %s" % (str(rootdirs), parentdir_prefix)) raise NotThisMethod("rootdir doesn't start with parentdir_prefix") SHORT_VERSION_PY = """ # This file was generated by 'versioneer.py' (0.17) from # revision-control system data, or from the parent directory name of an # unpacked source archive. Distribution tarballs contain a pre-generated copy # of this file. import json version_json = ''' %s ''' # END VERSION_JSON def get_versions(): return json.loads(version_json) """ def versions_from_file(filename): """Try to determine the version from _version.py if present.""" try: with open(filename) as f: contents = f.read() except EnvironmentError: raise NotThisMethod("unable to read _version.py") mo = re.search(r"version_json = '''\n(.*)''' # END VERSION_JSON", contents, re.M | re.S) if not mo: mo = re.search(r"version_json = '''\r\n(.*)''' # END VERSION_JSON", contents, re.M | re.S) if not mo: raise NotThisMethod("no version_json in _version.py") return json.loads(mo.group(1)) def write_to_version_file(filename, versions): """Write the given version number to the given _version.py file.""" os.unlink(filename) contents = json.dumps(versions, sort_keys=True, indent=1, separators=(",", ": ")) with open(filename, "w") as f: f.write(SHORT_VERSION_PY % contents) print("set %s to '%s'" % (filename, versions["version"])) def plus_or_dot(pieces): """Return a + if we don't already have one, else return a .""" if "+" in pieces.get("closest-tag", ""): return "." return "+" def render_pep440(pieces): """Build up version string, with post-release "local version identifier". Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty Exceptions: 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"] or pieces["dirty"]: rendered += plus_or_dot(pieces) rendered += "%d.g%s" % (pieces["distance"], pieces["short"]) if pieces["dirty"]: rendered += ".dirty" else: # exception #1 rendered = "0+untagged.%d.g%s" % (pieces["distance"], pieces["short"]) if pieces["dirty"]: rendered += ".dirty" return rendered def render_pep440_pre(pieces): """TAG[.post.devDISTANCE] -- No -dirty. Exceptions: 1: no tags. 0.post.devDISTANCE """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"]: rendered += ".post.dev%d" % pieces["distance"] else: # exception #1 rendered = "0.post.dev%d" % pieces["distance"] return rendered def render_pep440_post(pieces): """TAG[.postDISTANCE[.dev0]+gHEX] . The ".dev0" means dirty. Note that .dev0 sorts backwards (a dirty tree will appear "older" than the corresponding clean one), but you shouldn't be releasing software with -dirty anyways. Exceptions: 1: no tags. 0.postDISTANCE[.dev0] """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"] or pieces["dirty"]: rendered += ".post%d" % pieces["distance"] if pieces["dirty"]: rendered += ".dev0" rendered += plus_or_dot(pieces) rendered += "g%s" % pieces["short"] else: # exception #1 rendered = "0.post%d" % pieces["distance"] if pieces["dirty"]: rendered += ".dev0" rendered += "+g%s" % pieces["short"] return rendered def render_pep440_old(pieces): """TAG[.postDISTANCE[.dev0]] . The ".dev0" means dirty. Eexceptions: 1: no tags. 0.postDISTANCE[.dev0] """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"] or pieces["dirty"]: rendered += ".post%d" % pieces["distance"] if pieces["dirty"]: rendered += ".dev0" else: # exception #1 rendered = "0.post%d" % pieces["distance"] if pieces["dirty"]: rendered += ".dev0" return rendered def render_git_describe(pieces): """TAG[-DISTANCE-gHEX][-dirty]. Like 'git describe --tags --dirty --always'. Exceptions: 1: no tags. HEX[-dirty] (note: no 'g' prefix) """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] if pieces["distance"]: rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) else: # exception #1 rendered = pieces["short"] if pieces["dirty"]: rendered += "-dirty" return rendered def render_git_describe_long(pieces): """TAG-DISTANCE-gHEX[-dirty]. Like 'git describe --tags --dirty --always -long'. The distance/hash is unconditional. Exceptions: 1: no tags. HEX[-dirty] (note: no 'g' prefix) """ if pieces["closest-tag"]: rendered = pieces["closest-tag"] rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) else: # exception #1 rendered = pieces["short"] if pieces["dirty"]: rendered += "-dirty" return rendered def render(pieces, style): """Render the given version pieces into the requested style.""" if pieces["error"]: return {"version": "unknown", "full-revisionid": pieces.get("long"), "dirty": None, "error": pieces["error"], "date": None} if not style or style == "default": style = "pep440" # the default if style == "pep440": rendered = render_pep440(pieces) elif style == "pep440-pre": rendered = render_pep440_pre(pieces) elif style == "pep440-post": rendered = render_pep440_post(pieces) elif style == "pep440-old": rendered = render_pep440_old(pieces) elif style == "git-describe": rendered = render_git_describe(pieces) elif style == "git-describe-long": rendered = render_git_describe_long(pieces) else: raise ValueError("unknown style '%s'" % style) return {"version": rendered, "full-revisionid": pieces["long"], "dirty": pieces["dirty"], "error": None, "date": pieces.get("date")} class VersioneerBadRootError(Exception): """The project root directory is unknown or missing key files.""" def get_versions(verbose=False): """Get the project version from whatever source is available. Returns dict with two keys: 'version' and 'full'. """ if "versioneer" in sys.modules: # see the discussion in cmdclass.py:get_cmdclass() del sys.modules["versioneer"] root = get_root() cfg = get_config_from_root(root) assert cfg.VCS is not None, "please set [versioneer]VCS= in setup.cfg" handlers = HANDLERS.get(cfg.VCS) assert handlers, "unrecognized VCS '%s'" % cfg.VCS verbose = verbose or cfg.verbose assert cfg.versionfile_source is not None, \ "please set versioneer.versionfile_source" assert cfg.tag_prefix is not None, "please set versioneer.tag_prefix" versionfile_abs = os.path.join(root, cfg.versionfile_source) # extract version from first of: _version.py, VCS command (e.g. 'git # describe'), parentdir. This is meant to work for developers using a # source checkout, for users of a tarball created by 'setup.py sdist', # and for users of a tarball/zipball created by 'git archive' or github's # download-from-tag feature or the equivalent in other VCSes. get_keywords_f = handlers.get("get_keywords") from_keywords_f = handlers.get("keywords") if get_keywords_f and from_keywords_f: try: keywords = get_keywords_f(versionfile_abs) ver = from_keywords_f(keywords, cfg.tag_prefix, verbose) if verbose: print("got version from expanded keyword %s" % ver) return ver except NotThisMethod: pass try: ver = versions_from_file(versionfile_abs) if verbose: print("got version from file %s %s" % (versionfile_abs, ver)) return ver except NotThisMethod: pass from_vcs_f = handlers.get("pieces_from_vcs") if from_vcs_f: try: pieces = from_vcs_f(cfg.tag_prefix, root, verbose) ver = render(pieces, cfg.style) if verbose: print("got version from VCS %s" % ver) return ver except NotThisMethod: pass try: if cfg.parentdir_prefix: ver = versions_from_parentdir(cfg.parentdir_prefix, root, verbose) if verbose: print("got version from parentdir %s" % ver) return ver except NotThisMethod: pass if verbose: print("unable to compute version") return {"version": "0+unknown", "full-revisionid": None, "dirty": None, "error": "unable to compute version", "date": None} def get_version(): """Get the short version string for this project.""" return get_versions()["version"] def get_cmdclass(): """Get the custom setuptools/distutils subclasses used by Versioneer.""" if "versioneer" in sys.modules: del sys.modules["versioneer"] # this fixes the "python setup.py develop" case (also 'install' and # 'easy_install .'), in which subdependencies of the main project are # built (using setup.py bdist_egg) in the same python process. Assume # a main project A and a dependency B, which use different versions # of Versioneer. A's setup.py imports A's Versioneer, leaving it in # sys.modules by the time B's setup.py is executed, causing B to run # with the wrong versioneer. Setuptools wraps the sub-dep builds in a # sandbox that restores sys.modules to it's pre-build state, so the # parent is protected against the child's "import versioneer". By # removing ourselves from sys.modules here, before the child build # happens, we protect the child from the parent's versioneer too. # Also see https://github.com/warner/python-versioneer/issues/52 cmds = {} # we add "version" to both distutils and setuptools from distutils.core import Command class cmd_version(Command): description = "report generated version string" user_options = [] boolean_options = [] def initialize_options(self): pass def finalize_options(self): pass def run(self): vers = get_versions(verbose=True) print("Version: %s" % vers["version"]) print(" full-revisionid: %s" % vers.get("full-revisionid")) print(" dirty: %s" % vers.get("dirty")) print(" date: %s" % vers.get("date")) if vers["error"]: print(" error: %s" % vers["error"]) cmds["version"] = cmd_version # we override "build_py" in both distutils and setuptools # # most invocation pathways end up running build_py: # distutils/build -> build_py # distutils/install -> distutils/build ->.. # setuptools/bdist_wheel -> distutils/install ->.. # setuptools/bdist_egg -> distutils/install_lib -> build_py # setuptools/install -> bdist_egg ->.. # setuptools/develop -> ? # pip install: # copies source tree to a tempdir before running egg_info/etc # if .git isn't copied too, 'git describe' will fail # then does setup.py bdist_wheel, or sometimes setup.py install # setup.py egg_info -> ? # we override different "build_py" commands for both environments if "setuptools" in sys.modules: from setuptools.command.build_py import build_py as _build_py else: from distutils.command.build_py import build_py as _build_py class cmd_build_py(_build_py): def run(self): root = get_root() cfg = get_config_from_root(root) versions = get_versions() _build_py.run(self) # now locate _version.py in the new build/ directory and replace # it with an updated value if cfg.versionfile_build: target_versionfile = os.path.join(self.build_lib, cfg.versionfile_build) print("UPDATING %s" % target_versionfile) write_to_version_file(target_versionfile, versions) cmds["build_py"] = cmd_build_py if "cx_Freeze" in sys.modules: # cx_freeze enabled? from cx_Freeze.dist import build_exe as _build_exe # nczeczulin reports that py2exe won't like the pep440-style string # as FILEVERSION, but it can be used for PRODUCTVERSION, e.g. # setup(console=[{ # "version": versioneer.get_version().split("+", 1)[0], # FILEVERSION # "product_version": versioneer.get_version(), # ... class cmd_build_exe(_build_exe): def run(self): root = get_root() cfg = get_config_from_root(root) versions = get_versions() target_versionfile = cfg.versionfile_source print("UPDATING %s" % target_versionfile) write_to_version_file(target_versionfile, versions) _build_exe.run(self) os.unlink(target_versionfile) with open(cfg.versionfile_source, "w") as f: LONG = LONG_VERSION_PY[cfg.VCS] f.write(LONG % {"DOLLAR": "$", "STYLE": cfg.style, "TAG_PREFIX": cfg.tag_prefix, "PARENTDIR_PREFIX": cfg.parentdir_prefix, "VERSIONFILE_SOURCE": cfg.versionfile_source, }) cmds["build_exe"] = cmd_build_exe del cmds["build_py"] if 'py2exe' in sys.modules: # py2exe enabled? try: from py2exe.distutils_buildexe import py2exe as _py2exe # py3 except ImportError: from py2exe.build_exe import py2exe as _py2exe # py2 class cmd_py2exe(_py2exe): def run(self): root = get_root() cfg = get_config_from_root(root) versions = get_versions() target_versionfile = cfg.versionfile_source print("UPDATING %s" % target_versionfile) write_to_version_file(target_versionfile, versions) _py2exe.run(self) os.unlink(target_versionfile) with open(cfg.versionfile_source, "w") as f: LONG = LONG_VERSION_PY[cfg.VCS] f.write(LONG % {"DOLLAR": "$", "STYLE": cfg.style, "TAG_PREFIX": cfg.tag_prefix, "PARENTDIR_PREFIX": cfg.parentdir_prefix, "VERSIONFILE_SOURCE": cfg.versionfile_source, }) cmds["py2exe"] = cmd_py2exe # we override different "sdist" commands for both environments if "setuptools" in sys.modules: from setuptools.command.sdist import sdist as _sdist else: from distutils.command.sdist import sdist as _sdist class cmd_sdist(_sdist): def run(self): versions = get_versions() self._versioneer_generated_versions = versions # unless we update this, the command will keep using the old # version self.distribution.metadata.version = versions["version"] return _sdist.run(self) def make_release_tree(self, base_dir, files): root = get_root() cfg = get_config_from_root(root) _sdist.make_release_tree(self, base_dir, files) # now locate _version.py in the new base_dir directory # (remembering that it may be a hardlink) and replace it with an # updated value target_versionfile = os.path.join(base_dir, cfg.versionfile_source) print("UPDATING %s" % target_versionfile) write_to_version_file(target_versionfile, self._versioneer_generated_versions) cmds["sdist"] = cmd_sdist return cmds CONFIG_ERROR = """ setup.cfg is missing the necessary Versioneer configuration. You need a section like: [versioneer] VCS = git style = pep440 versionfile_source = src/myproject/_version.py versionfile_build = myproject/_version.py tag_prefix = parentdir_prefix = myproject- You will also need to edit your setup.py to use the results: import versioneer setup(version=versioneer.get_version(), cmdclass=versioneer.get_cmdclass(), ...) Please read the docstring in ./versioneer.py for configuration instructions, edit setup.cfg, and re-run the installer or 'python versioneer.py setup'. """ SAMPLE_CONFIG = """ # See the docstring in versioneer.py for instructions. Note that you must # re-run 'versioneer.py setup' after changing this section, and commit the # resulting files. [versioneer] #VCS = git #style = pep440 #versionfile_source = #versionfile_build = #tag_prefix = #parentdir_prefix = """ INIT_PY_SNIPPET = """ from ._version import get_versions __version__ = get_versions()['version'] del get_versions """ def do_setup(): """Main VCS-independent setup function for installing Versioneer.""" root = get_root() try: cfg = get_config_from_root(root) except (EnvironmentError, configparser.NoSectionError, configparser.NoOptionError) as e: if isinstance(e, (EnvironmentError, configparser.NoSectionError)): print("Adding sample versioneer config to setup.cfg", file=sys.stderr) with open(os.path.join(root, "setup.cfg"), "a") as f: f.write(SAMPLE_CONFIG) print(CONFIG_ERROR, file=sys.stderr) return 1 print(" creating %s" % cfg.versionfile_source) with open(cfg.versionfile_source, "w") as f: LONG = LONG_VERSION_PY[cfg.VCS] f.write(LONG % {"DOLLAR": "$", "STYLE": cfg.style, "TAG_PREFIX": cfg.tag_prefix, "PARENTDIR_PREFIX": cfg.parentdir_prefix, "VERSIONFILE_SOURCE": cfg.versionfile_source, }) ipy = os.path.join(os.path.dirname(cfg.versionfile_source), "__init__.py") if os.path.exists(ipy): try: with open(ipy, "r") as f: old = f.read() except EnvironmentError: old = "" if INIT_PY_SNIPPET not in old: print(" appending to %s" % ipy) with open(ipy, "a") as f: f.write(INIT_PY_SNIPPET) else: print(" %s unmodified" % ipy) else: print(" %s doesn't exist, ok" % ipy) ipy = None # Make sure both the top-level "versioneer.py" and versionfile_source # (PKG/_version.py, used by runtime code) are in MANIFEST.in, so # they'll be copied into source distributions. Pip won't be able to # install the package without this. manifest_in = os.path.join(root, "MANIFEST.in") simple_includes = set() try: with open(manifest_in, "r") as f: for line in f: if line.startswith("include "): for include in line.split()[1:]: simple_includes.add(include) except EnvironmentError: pass # That doesn't cover everything MANIFEST.in can do # (http://docs.python.org/2/distutils/sourcedist.html#commands), so # it might give some false negatives. Appending redundant 'include' # lines is safe, though. if "versioneer.py" not in simple_includes: print(" appending 'versioneer.py' to MANIFEST.in") with open(manifest_in, "a") as f: f.write("include versioneer.py\n") else: print(" 'versioneer.py' already in MANIFEST.in") if cfg.versionfile_source not in simple_includes: print(" appending versionfile_source ('%s') to MANIFEST.in" % cfg.versionfile_source) with open(manifest_in, "a") as f: f.write("include %s\n" % cfg.versionfile_source) else: print(" versionfile_source already in MANIFEST.in") # Make VCS-specific changes. For git, this means creating/changing # .gitattributes to mark _version.py for export-subst keyword # substitution. do_vcs_install(manifest_in, cfg.versionfile_source, ipy) return 0 def scan_setup_py(): """Validate the contents of setup.py against Versioneer's expectations.""" found = set() setters = False errors = 0 with open("setup.py", "r") as f: for line in f.readlines(): if "import versioneer" in line: found.add("import") if "versioneer.get_cmdclass()" in line: found.add("cmdclass") if "versioneer.get_version()" in line: found.add("get_version") if "versioneer.VCS" in line: setters = True if "versioneer.versionfile_source" in line: setters = True if len(found) != 3: print("") print("Your setup.py appears to be missing some important items") print("(but I might be wrong). Please make sure it has something") print("roughly like the following:") print("") print(" import versioneer") print(" setup( version=versioneer.get_version(),") print(" cmdclass=versioneer.get_cmdclass(), ...)") print("") errors += 1 if setters: print("You should remove lines like 'versioneer.VCS = ' and") print("'versioneer.versionfile_source = ' . This configuration") print("now lives in setup.cfg, and should be removed from setup.py") print("") errors += 1 return errors if __name__ == "__main__": cmd = sys.argv[1] if cmd == "setup": errors = do_setup() errors += scan_setup_py() if errors: sys.exit(1)