def test_parse_timedelta(): for text, value in [ ("1s", 1), ("100ms", 0.1), ("5S", 5), ("5.5s", 5.5), ("5.5 s", 5.5), ("1 second", 1), ("3.3 seconds", 3.3), ("3.3 milliseconds", 0.0033), ("3500 us", 0.0035), ("1 ns", 1e-9), ("2m", 120), ("2 minutes", 120), (None, None), (3, 3), (datetime.timedelta(seconds=2), 2), (datetime.timedelta(milliseconds=100), 0.1), ]: result = parse_timedelta(text) assert result == value or abs(result - value) < 1e-14 assert parse_timedelta("1ms", default="seconds") == 0.001 assert parse_timedelta("1", default="seconds") == 1 assert parse_timedelta("1", default="ms") == 0.001 assert parse_timedelta(1, default="ms") == 0.001
def __init__(self, scheduler): self.scheduler = scheduler # { level: { task states } } self.stealable_all = [set() for i in range(15)] # { worker: { level: { task states } } } self.stealable = dict() # { task state: (worker, level) } self.key_stealable = dict() 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) self._callback_time = parse_timedelta( dask.config.get("distributed.scheduler.work-stealing-interval"), default="ms", ) # `callback_time` is in milliseconds self.scheduler.add_plugin(self) self.scheduler.extensions["stealing"] = self self.scheduler.events["stealing"] = deque(maxlen=100000) self.count = 0 # { task state: <stealing info dict> } self.in_flight = dict() # { worker state: occupancy } self.in_flight_occupancy = defaultdict(lambda: 0) self._in_flight_event = asyncio.Event() self.scheduler.stream_handlers[ "steal-response"] = self.move_task_confirm
async def on_connection(self, comm: Comm, handshake_overrides=None): local_info = {**comm.handshake_info(), **(handshake_overrides or {})} timeout = dask.config.get("distributed.comm.timeouts.connect") timeout = parse_timedelta(timeout, default="seconds") try: # Timeout is to ensure that we'll terminate connections eventually. # Connector side will employ smaller timeouts and we should only # reach this if the comm is dead anyhow. await asyncio.wait_for(comm.write(local_info), timeout=timeout) handshake = await asyncio.wait_for(comm.read(), timeout=timeout) # This would be better, but connections leak if worker is closed quickly # write, handshake = await asyncio.gather(comm.write(local_info), comm.read()) except Exception as e: with suppress(Exception): await comm.close() raise CommClosedError(f"Comm {comm!r} closed.") from e comm.remote_info = handshake comm.remote_info["address"] = comm.peer_address comm.local_info = local_info comm.local_info["address"] = comm.local_address comm.handshake_options = comm.handshake_configuration( comm.local_info, comm.remote_info)
def __init__( self, asynchronous=False, loop=None, quiet=False, name=None, scheduler_sync_interval=1, ): self._loop_runner = LoopRunner(loop=loop, asynchronous=asynchronous) self.loop = self._loop_runner.loop self.scheduler_info = {"workers": {}} self.periodic_callbacks = {} self._watch_worker_status_comm = None self._watch_worker_status_task = None self._cluster_manager_logs = [] self.quiet = quiet self.scheduler_comm = None self._adaptive = None self._sync_interval = parse_timedelta(scheduler_sync_interval, default="seconds") self._sync_cluster_info_task = None if name is None: name = str(uuid.uuid4())[:8] # Mask class attribute with instance attribute self._cluster_info = { "name": name, "type": typename(type(self)), **type(self)._cluster_info, } self.status = Status.created
def wait(self, timeout=None): """Wait until the event is set. Parameters ---------- timeout : number or string or timedelta, optional Seconds to wait on the event in the scheduler. This does not include local coroutine time, network transfer time, etc.. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. "200ms". Examples -------- >>> event = Event('a') # doctest: +SKIP >>> event.wait(timeout="1s") # doctest: +SKIP Returns ------- True if the event was set of false, if a timeout happend """ timeout = parse_timedelta(timeout) result = self.client.sync(self.client.scheduler.event_wait, name=self.name, timeout=timeout) return result
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 ------ concurrent.futures.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: timeout = parse_timedelta(timeout) 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) # Below iterator relies on fs being an iterator itself, and not just an iterable # (such as a list), in order to cancel remaining futures fs = iter(fs) # 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 TimeoutError: raise cf.TimeoutError else: yield future.result() finally: remaining = list(fs) self._futures.update(remaining) self._client.cancel(remaining) return result_iterator()
def __init__( self, cluster=None, interval=None, minimum=None, maximum=None, wait_count=None, target_duration=None, worker_key=None, **kwargs, ): self.cluster = cluster self.worker_key = worker_key self._workers_to_close_kwargs = kwargs if interval is None: interval = dask.config.get("distributed.adaptive.interval") if minimum is None: minimum = dask.config.get("distributed.adaptive.minimum") if maximum is None: maximum = dask.config.get("distributed.adaptive.maximum") if wait_count is None: wait_count = dask.config.get("distributed.adaptive.wait-count") if target_duration is None: target_duration = dask.config.get("distributed.adaptive.target-duration") self.target_duration = parse_timedelta(target_duration) logger.info("Adaptive scaling started: minimum=%s maximum=%s", minimum, maximum) super().__init__( minimum=minimum, maximum=maximum, wait_count=wait_count, interval=interval )
def set_tcp_timeout(comm): """ Set kernel-level TCP timeout on the stream. """ if comm.closed(): return timeout = dask.config.get("distributed.comm.timeouts.tcp") timeout = int(parse_timedelta(timeout, default="seconds")) sock = comm.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 OSError as e: logger.warning("Could not set timeout on TCP stream: %s", e)
async def retry_operation(coro, *args, operation=None, **kwargs): """ Retry an operation using the configuration values for the retry parameters """ retry_count = dask.config.get("distributed.comm.retry.count") retry_delay_min = parse_timedelta( dask.config.get("distributed.comm.retry.delay.min"), default="s") retry_delay_max = parse_timedelta( dask.config.get("distributed.comm.retry.delay.max"), default="s") return await retry( partial(coro, *args, **kwargs), count=retry_count, delay_min=retry_delay_min, delay_max=retry_delay_max, operation=operation, )
def sync(loop, func, *args, callback_timeout=None, **kwargs): """ Run coroutine in loop running in separate thread. """ callback_timeout = parse_timedelta(callback_timeout, "s") # Tornado's PollIOLoop doesn't raise when using closed, do it ourselves if PollIOLoop and ( (isinstance(loop, PollIOLoop) and getattr(loop, "_closing", False)) or (hasattr(loop, "asyncio_loop") and loop.asyncio_loop._closed) ): raise RuntimeError("IOLoop is closed") try: if loop.asyncio_loop.is_closed(): # tornado 6 raise RuntimeError("IOLoop is closed") except AttributeError: pass e = threading.Event() main_tid = threading.get_ident() result = [None] error = [False] @gen.coroutine def f(): # We flag the thread state asynchronous, which will make sync() call # within `func` use async semantic. In order to support concurrent # calls to sync(), `asynchronous` is used as a ref counter. thread_state.asynchronous = getattr(thread_state, "asynchronous", 0) thread_state.asynchronous += 1 try: if main_tid == threading.get_ident(): raise RuntimeError("sync() called from thread of running loop") yield gen.moment future = func(*args, **kwargs) if callback_timeout is not None: future = asyncio.wait_for(future, callback_timeout) result[0] = yield future except Exception as exc: error[0] = sys.exc_info() finally: assert thread_state.asynchronous > 0 thread_state.asynchronous -= 1 e.set() loop.add_callback(f) if callback_timeout is not None: if not e.wait(callback_timeout): raise TimeoutError("timed out after %s s." % (callback_timeout,)) else: while not e.is_set(): e.wait(10) if error[0]: typ, exc, tb = error[0] raise exc.with_traceback(tb) else: return result[0]
def collect(self, start=None, stop=None, count=None): def bisect(target, left, right): if left == right: return left mid = (left + right) // 2 value = max( startstop["stop"] for startstop in self.buffer[mid]["startstops"] ) if value < target: return bisect(target, mid + 1, right) else: return bisect(target, left, mid) if isinstance(start, str): start = time() - parse_timedelta(start) if start is not None: start = bisect(start, 0, len(self.buffer)) if isinstance(stop, str): stop = time() - parse_timedelta(stop) if stop is not None: stop = bisect(stop, 0, len(self.buffer)) if count is not None: if start is None and stop is None: stop = len(self.buffer) start = stop - count elif start is None and stop is not None: start = stop - count elif start is not None and stop is None: stop = start + count if stop is None: stop = len(self.buffer) if start is None: start = 0 start = max(0, start) stop = min(stop, len(self.buffer)) return [self.buffer[i] for i in range(start, stop)]
def __init__(self, scheduler): self.scheduler = scheduler # {semaphore_name: asyncio.Event} self.events = defaultdict(asyncio.Event) # {semaphore_name: max_leases} self.max_leases = dict() # {semaphore_name: {lease_id: lease_last_seen_timestamp}} self.leases = defaultdict(dict) self.scheduler.handlers.update( { "semaphore_register": self.create, "semaphore_acquire": self.acquire, "semaphore_release": self.release, "semaphore_close": self.close, "semaphore_refresh_leases": self.refresh_leases, "semaphore_value": self.get_value, } ) self.scheduler.extensions["semaphores"] = self # {metric_name: {semaphore_name: metric}} self.metrics = { "acquire_total": defaultdict(int), # counter "release_total": defaultdict(int), # counter "average_pending_lease_time": defaultdict(float), # gauge "pending": defaultdict(int), # gauge } validation_callback_time = parse_timedelta( dask.config.get("distributed.scheduler.locks.lease-validation-interval"), default="s", ) self._pc_lease_timeout = PeriodicCallback( self._check_lease_timeout, validation_callback_time * 1000 ) self._pc_lease_timeout.start() self.lease_timeout = parse_timedelta( dask.config.get("distributed.scheduler.locks.lease-timeout"), default="s" )
def get(self, timeout=None, **kwargs): """Get the value of this variable Parameters ---------- timeout : number or string or timedelta, optional Time in seconds to wait before timing out. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. "200ms". """ timeout = parse_timedelta(timeout) return self.client.sync(self._get, timeout=timeout, **kwargs)
def put(self, value, timeout=None, **kwargs): """Put data into the queue Parameters ---------- timeout : number or string or timedelta, optional Time in seconds to wait before timing out. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. "200ms". """ timeout = parse_timedelta(timeout) return self.client.sync(self._put, value, timeout=timeout, **kwargs)
def _watch(thread_id, log, interval="20ms", cycle="2s", omit=None, stop=lambda: False): interval = parse_timedelta(interval) cycle = parse_timedelta(cycle) recent = create() last = time() while not stop(): if time() > last + cycle: log.append((time(), recent)) recent = create() last = time() try: frame = sys._current_frames()[thread_id] except KeyError: return process(frame, None, recent, omit=omit) sleep(interval)
def sync(self, func, *args, asynchronous=None, callback_timeout=None, **kwargs): callback_timeout = parse_timedelta(callback_timeout) if ( asynchronous or self.asynchronous or getattr(thread_state, "asynchronous", False) ): future = func(*args, **kwargs) if callback_timeout is not None: future = asyncio.wait_for(future, callback_timeout) return future else: return sync( self.loop, func, *args, callback_timeout=callback_timeout, **kwargs )
def test_parse_timedelta(): for text, value in [ ("1s", 1), ("100ms", 0.1), ("5S", 5), ("5.5s", 5.5), ("5.5 s", 5.5), ("1 second", 1), ("3.3 seconds", 3.3), ("3.3 milliseconds", 0.0033), ("3500 us", 0.0035), ("1 ns", 1e-9), ("2m", 120), ("2 minutes", 120), (datetime.timedelta(seconds=2), 2), (datetime.timedelta(milliseconds=100), 0.1), ]: result = parse_timedelta(text) assert abs(result - value) < 1e-14 assert parse_timedelta("1ms", default="seconds") == 0.001 assert parse_timedelta("1", default="seconds") == 1 assert parse_timedelta("1", default="ms") == 0.001 assert parse_timedelta(1, default="ms") == 0.001
def __init__( self, minimum: int = 0, maximum: int = math.inf, wait_count: int = 3, interval: str = "1s", ): self.minimum = minimum self.maximum = maximum self.wait_count = wait_count self.interval = parse_timedelta(interval, "seconds") if interval else interval self.periodic_callback = None def f(): try: self.periodic_callback.start() except AttributeError: pass if self.interval: import weakref self_ref = weakref.ref(self) async def _adapt(): core = self_ref() if core: await core.adapt() self.periodic_callback = PeriodicCallback(_adapt, self.interval * 1000) try: self.loop.add_callback(f) except AttributeError: IOLoop.current().add_callback(f) try: self.plan = set() self.requested = set() self.observed = set() except Exception: pass # internal state self.close_counts = collections.defaultdict(int) self._adapting = False self.log = collections.deque(maxlen=10000)
def get(self, timeout=None, batch=False, **kwargs): """Get data from the queue Parameters ---------- timeout : number or string or timedelta, optional Time in seconds to wait before timing out. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. "200ms". 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 """ timeout = parse_timedelta(timeout) return self.client.sync(self._get, timeout=timeout, batch=batch, **kwargs)
def __init__( self, max_leases=1, name=None, register=True, scheduler_rpc=None, loop=None, ): try: worker = get_worker() self.scheduler = scheduler_rpc or worker.scheduler self.loop = loop or worker.loop except ValueError: client = get_client() self.scheduler = scheduler_rpc or client.scheduler self.loop = loop or client.io_loop self.name = name or "semaphore-" + uuid.uuid4().hex self.max_leases = max_leases self.id = uuid.uuid4().hex self._leases = deque() self.refresh_leases = True self._registered = None if register: self._registered = self.register() # this should give ample time to refresh without introducing another # config parameter since this *must* be smaller than the timeout anyhow refresh_leases_interval = ( parse_timedelta( dask.config.get("distributed.scheduler.locks.lease-timeout"), default="s", ) / 5 ) pc = PeriodicCallback( self._refresh_leases, callback_time=refresh_leases_interval * 1000 ) self.refresh_callback = pc # Need to start the callback using IOLoop.add_callback to ensure that the # PC uses the correct event loop. self.loop.add_callback(pc.start)
def __init__( self, minimum: int = 0, maximum: int | float = math.inf, wait_count: int = 3, interval: str | int | float | timedelta | None = "1s", ): if not isinstance(maximum, int) and not math.isinf(maximum): raise TypeError(f"maximum must be int or inf; got {maximum}") self.minimum = minimum self.maximum = maximum self.wait_count = wait_count self.interval = parse_timedelta(interval, "seconds") self.periodic_callback = None def f(): try: self.periodic_callback.start() except AttributeError: pass if self.interval: import weakref self_ref = weakref.ref(self) async def _adapt(): core = self_ref() if core: await core.adapt() self.periodic_callback = PeriodicCallback(_adapt, self.interval * 1000) self.loop.add_callback(f) try: self.plan = set() self.requested = set() self.observed = set() except Exception: pass # internal state self.close_counts = defaultdict(int) self._adapting = False self.log = deque(maxlen=10000)
def __init__( self, cluster=None, interval="1s", minimum=0, maximum=math.inf, wait_count=3, target_duration="5s", **kwargs ): self.cluster = cluster self.target_duration = parse_timedelta(target_duration) self._workers_to_close_kwargs = kwargs super().__init__( minimum=minimum, maximum=maximum, wait_count=wait_count, interval=interval )
def __init__(self, interval, loop=None, serializers=None): # XXX is the loop arg useful? self.loop = loop or IOLoop.current() self.interval = parse_timedelta(interval, default="ms") 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=dask.config.get( "distributed.comm.recent-messages-log-length")) self.serializers = serializers self._consecutive_failures = 0
def acquire(self, timeout=None): """ Acquire a semaphore. If the internal counter is greater than zero, decrement it by one and return True immediately. If it is zero, wait until a release() is called and return True. Parameters ---------- timeout : number or string or timedelta, optional Seconds to wait on acquiring the semaphore. This does not include local coroutine time, network transfer time, etc.. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. "200ms". """ timeout = parse_timedelta(timeout) return self.sync(self._acquire, timeout=timeout)
def _update_worker_status(self, op, msg): if op == "remove": name = self.scheduler_info["workers"][msg]["name"] def f(): if (name in self.workers and msg not in self.scheduler_info["workers"] and not any( d["name"] == name for d in self.scheduler_info["workers"].values())): self._futures.add( asyncio.ensure_future(self.workers[name].close())) del self.workers[name] delay = parse_timedelta( dask.config.get("distributed.deploy.lost-worker-timeout")) asyncio.get_event_loop().call_later(delay, f) super()._update_worker_status(op, msg)
def __init__( self, scheduler: SchedulerState, # The following parameters are exposed so that one may create, run, and throw # away on the fly a specialized manager, separate from the main one. policies: set[ActiveMemoryManagerPolicy] | None = None, register: bool = True, start: bool | None = None, interval: float | None = None, ): self.scheduler = scheduler self.policies = set() if policies is None: # Initialize policies from config policies = set() for kwargs in dask.config.get( "distributed.scheduler.active-memory-manager.policies"): kwargs = kwargs.copy() cls = import_term(kwargs.pop("class")) policies.add(cls(**kwargs)) for policy in policies: self.add_policy(policy) if register: scheduler.extensions["amm"] = self scheduler.handlers.update({ "amm_run_once": self.run_once, "amm_start": self.start, "amm_stop": self.stop, }) if interval is None: interval = parse_timedelta( dask.config.get( "distributed.scheduler.active-memory-manager.interval")) self.interval = interval if start is None: start = dask.config.get( "distributed.scheduler.active-memory-manager.start") if start: self.start()
def get(self, timeout=None): """Get a single message Parameters ---------- timeout : number or string or timedelta, optional Time in seconds to wait before timing out. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. "200ms". """ timeout = parse_timedelta(timeout) if self.client: return self.client.sync(self._get, timeout=timeout) elif self.worker.thread_id == threading.get_ident(): return self._get() else: if self.buffer: # fastpath return self.buffer.popleft() return sync(self.loop, self._get, timeout=timeout)
def acquire(self, blocking=True, timeout=None, num_locks=None): """Acquire the lock Parameters ---------- blocking : bool, optional If false, don't wait on the lock in the scheduler at all. timeout : string or number or timedelta, optional Seconds to wait on the lock in the scheduler. This does not include local coroutine time, network transfer time, etc.. It is forbidden to specify a timeout when blocking is false. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. "200ms". num_locks : int, optional Number of locks needed. If None, all locks are needed Examples -------- >>> lock = MultiLock(['x', 'y']) # doctest: +SKIP >>> lock.acquire(timeout="1s") # doctest: +SKIP Returns ------- True or False whether or not it successfully acquired the lock """ timeout = parse_timedelta(timeout) if not blocking: if timeout is not None: raise ValueError( "can't specify a timeout for a non-blocking call") timeout = 0 result = self.client.sync( self.client.scheduler.multi_lock_acquire, locks=self.names, id=self.id, timeout=timeout, num_locks=num_locks or len(self.names), ) self._locked = True return result
def __init__( self, nanny: Nanny, *, memory_limit: str | float = "auto", ): self.memory_limit = parse_memory_limit(memory_limit, nanny.nthreads) self.memory_terminate_fraction = dask.config.get( "distributed.worker.memory.terminate" ) self.memory_monitor_interval = parse_timedelta( dask.config.get("distributed.worker.memory.monitor-interval"), default=None, ) assert isinstance(self.memory_monitor_interval, (int, float)) if self.memory_limit and self.memory_terminate_fraction is not False: pc = PeriodicCallback( partial(self.memory_monitor, nanny), self.memory_monitor_interval * 1000, ) nanny.periodic_callbacks["memory_monitor"] = pc
def __init__(self, asynchronous, quiet=False, name=None, scheduler_sync_interval=1): self.scheduler_info = {"workers": {}} self.periodic_callbacks = {} self._asynchronous = asynchronous self._watch_worker_status_comm = None self._watch_worker_status_task = None self._cluster_manager_logs = [] self.quiet = quiet self.scheduler_comm = None self._adaptive = None self._sync_interval = parse_timedelta(scheduler_sync_interval, default="seconds") if name is None: name = str(uuid.uuid4())[:8] self._cluster_info = {"name": name, "type": typename(type(self))} self.status = Status.created
def __init__( self, handlers, blocked_handlers=None, stream_handlers=None, connection_limit=512, deserialize=True, serializers=None, deserializers=None, connection_args=None, timeout=None, io_loop=None, ): self.handlers = { "identity": self.identity, "echo": self.echo, "connection_stream": self.handle_stream, "dump_state": self._to_dict, } self.handlers.update(handlers) if blocked_handlers is None: blocked_handlers = dask.config.get( "distributed.%s.blocked-handlers" % type(self).__name__.lower(), [] ) self.blocked_handlers = blocked_handlers self.stream_handlers = {} self.stream_handlers.update(stream_handlers or {}) 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._ongoing_coroutines = weakref.WeakSet() self._event_finished = asyncio.Event() self.listeners = [] self.io_loop = io_loop or IOLoop.current() self.loop = self.io_loop if not hasattr(self.io_loop, "profile"): ref = weakref.ref(self.io_loop) def stop(): loop = ref() return loop is None or loop.asyncio_loop.is_closed() self.io_loop.profile = profile.watch( omit=("profile.py", "selectors.py"), interval=dask.config.get("distributed.worker.profile.interval"), cycle=dask.config.get("distributed.worker.profile.cycle"), stop=stop, ) # Statistics counters for various events with suppress(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.periodic_callbacks = dict() pc = PeriodicCallback( self.monitor.update, parse_timedelta( dask.config.get("distributed.admin.system-monitor.interval") ) * 1000, ) self.periodic_callbacks["monitor"] = pc self._last_tick = time() measure_tick_interval = parse_timedelta( dask.config.get("distributed.admin.tick.interval"), default="ms" ) pc = PeriodicCallback(self._measure_tick, measure_tick_interval * 1000) self.periodic_callbacks["tick"] = pc self.thread_id = 0 def set_thread_ident(): self.thread_id = threading.get_ident() self.io_loop.add_callback(set_thread_ident) self._startup_lock = asyncio.Lock() self.status = Status.undefined self.rpc = ConnectionPool( limit=connection_limit, deserialize=deserialize, serializers=serializers, deserializers=deserializers, connection_args=connection_args, timeout=timeout, server=self, ) self.__stopped = False