From a7560228331cd389294b31f617ea5abd74ae0fe8 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Mon, 26 Apr 2021 19:43:33 +0100 Subject: [PATCH 01/31] partial prototype --- distributed/distributed-schema.yaml | 34 ++++++++++++ distributed/distributed.yaml | 15 +++++ distributed/scheduler.py | 85 +++++++++++++++++++++++++++++ 3 files changed, 134 insertions(+) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index 0e8faf2f19d..a10d917d08e 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -404,6 +404,40 @@ properties: non-time-sensitive heuristics. This should be set to be longer than the duration of most dask tasks. + rebalance: + type: object + description: >- + Settings for data rebalance operations + properties: + sender_min: + type: number + minimum: 0 + maximum: 1 + description: >- + Fraction of worker process memory at which we stop potentially + receiving data from other workers. Ignored when max_memory is not + set. + + receiver_max: + type: number + minimum: 0 + maximum: 1 + description: >- + Fraction of worker process memory at which we start potentially + transferring data to other workers. + + sender_recipient_gap: 0.1 + type: number + minimum: 0 + maximum: 1 + description: >- + Fraction of worker process memory, around the cluster mean, where + a worker is neither a sender nor a recipient of data during a + rebalance operation. E.g. if the mean cluster occupation is 50%, + no_rebalance_gap=0.1 means that only nodes above 55% will donate + data and only nodes below 45% will receive them. This helps avoid + data from bouncing around the cluster repeatedly. + target: oneOf: - {type: number, minimum: 0, maximum: 1} diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index c71845e3769..7e8aae284e3 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -90,6 +90,21 @@ distributed: # This should be set to be longer than the duration of most dask tasks. recent_to_old_time: 30s + rebalance: + # Fraction of worker process memory at which we start potentially transferring + # data to other workers. Ignored when max_memory is not set. + sender_min: 0.40 + # Fraction of worker process memory at which we stop potentially receiving + # data from other workers. Ignored when max_memory is not set. + receiver_max: 0.60 + # Fraction of worker process memory, around the cluster mean, where a worker is + # neither a sender nor a recipient of data during a rebalance operation. E.g. + # if the mean cluster occupation is 50%, no_rebalance_gap=0.10 means that only + # nodes above 55% will donate data and only nodes below 45% will receive them. + # This helps avoid data from bouncing around the cluster repeatedly.# + # Ignored when max_memory is not set. + sender_recipient_gap: 0.10 + # Fractions of worker process memory at which we take action to avoid memory # blowup. Set any of the values to False to turn off the behavior entirely. target: 0.60 # target fraction to stay below diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 58d1ccc4965..9ceaadd0fa6 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1,4 +1,5 @@ import asyncio +import heapq import inspect import itertools import json @@ -168,6 +169,19 @@ def nogil(func): double, parse_timedelta(dask.config.get("distributed.worker.memory.recent_to_old_time")), ) +MEMORY_REBALANCE_SENDER_MIN = declare( + double, + dask.config.get("distributed.worker.memory.rebalance.sender_min"), +) +MEMORY_REBALANCE_RECEIVER_MAX = declare( + double, + dask.config.get("distributed.worker.memory.rebalance.receiver_max"), +) +MEMORY_REBALANCE_HALF_GAP = declare( + double, + dask.config.get("distributed.worker.memory.rebalance.sender_recipient_gap") / 2.0, +) + DEFAULT_EXTENSIONS = [ LockExtension, @@ -5475,6 +5489,8 @@ async def rebalance(self, comm=None, keys=None, workers=None): **Policy** + TODO TODO TODO + 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 @@ -5483,8 +5499,77 @@ async def rebalance(self, comm=None, keys=None, workers=None): """ parent: SchedulerState = cast(SchedulerState, self) ts: TaskState + ws: WorkerState + with log_errors(): async with self._lock: + + # Identify workers that need to lose keys and those that can receive + # them, together with how many bytes each needs to lose/receive. + # This operation is O(n) to the number of workers in the cluster. + if workers: + workers = [parent._workers_dv[w] for w in workers] + else: + workers = parent._workers_dv.values() + + # optimistic memory = RSS - unmanaged memory that appeared over the last + # 30 seconds (distributed.worker.memory.recent_to_old_time). + # This lets us ignore temporary spikes caused by task heap usage. + memory_by_worker = [(ws, ws.memory.optimistic) for ws in workers] + mean_memory = sum(m for _, m in memory_by_worker) / len( + memory_by_worker + ) + + # Smallest-first heaps (managed by the heapq modules) of workers that + # need to send/receive data, with how many bytes each needs to + # send/receive. Bytes are negative, so that the workers farthest from + # the cluster mean are at the top of the heaps. + # Second element of each tuple is an arbitrary unique number, there just + # to to make sure that WorkerState objects are never used for sorting in + # the unlikely event that two processes have exactly the same number of + # bytes allocated. + senders: "list[tuple[int, int, WorkerState]]" = [] + recipients: "list[tuple[int, int, WorkerState]]" = [] + + for ws, ws_memory in memory_by_worker: + if ws.memory_limit: + half_gap = MEMORY_REBALANCE_HALF_GAP * ws.memory_limit + if ( + ws_memory >= MEMORY_REBALANCE_SENDER_MIN * ws.memory_limit + and ws_memory >= mean_memory + half_gap + ): + senders.append((mean_memory - ws_memory, id(ws), ws)) + elif ( + ws_memory < MEMORY_REBALANCE_RECEIVER_MAX * ws.memory_limit + and ws_memory < mean_memory - half_gap + ): + recipients.append((ws_memory - mean_memory, id(ws), ws)) + elif ws_memory >= mean_memory: + senders.append((mean_memory - ws_memory, id(ws), ws)) + else: + recipients.append((ws_memory - mean_memory, id(ws), ws)) + + # Fast exit in case no transfers are necessary or possible + if not senders or not recipients: + self.log_event( + "all", + { + "action": "rebalance", + "senders": len(senders), + "recipients": len(recipients), + "moved_keys": 0, + }, + ) + return {"status": "OK"} + + heapq.heapify(senders) + heapq.heapify(recipients) + + if keys is not None and not isinstance(keys, Set): + keys = set(keys) # unless already a set-like + + # TODO TODO TODO + if keys: tasks = {parent._tasks[k] for k in keys} missing_data = [ts._key for ts in tasks if not ts._who_has] From 36589b100dbc49fd184f2037382a48d102ffcce3 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Tue, 27 Apr 2021 17:14:29 +0100 Subject: [PATCH 02/31] incomplete poc --- distributed/scheduler.py | 152 ++++++++++++++++++++++++++++----------- 1 file changed, 111 insertions(+), 41 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 9ceaadd0fa6..fa1087b09ee 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -422,7 +422,7 @@ class WorkerState: .. attribute:: has_what: {TaskState} - The set of tasks which currently reside on this worker. + An insertion-sorted set-like of tasks which currently reside on this worker. All the tasks here are in the "memory" state. This is the reverse mapping of :class:`TaskState.who_has`. @@ -482,7 +482,7 @@ class WorkerState: _bandwidth: double _executing: dict _extra: dict - _has_what: set + _has_what: dict _hash: Py_hash_t _last_seen: double _local_directory: str @@ -570,7 +570,7 @@ def __init__( ) self._actors = set() - self._has_what = set() + self._has_what = {} self._processing = {} self._executing = {} self._resources = {} @@ -611,8 +611,8 @@ def extra(self): return self._extra @property - def has_what(self): - return self._has_what + def has_what(self) -> "Set[TaskState]": + return self._has_what.keys() @property def host(self): @@ -2590,7 +2590,7 @@ def transition_memory_released(self, key, safe: bint = False): "report": False, } for ws in ts._who_has: - ws._has_what.remove(ts) + del ws._has_what[ts] ws._nbytes -= ts_nbytes ts._group._nbytes_in_memory -= ts_nbytes worker_msgs[ws._address] = [worker_msg] @@ -3046,7 +3046,7 @@ def get_comm_cost(self, ts: TaskState, ws: WorkerState) -> double: on the given worker. """ dts: TaskState - deps: set = ts._dependencies - ws._has_what + deps: set = ts._dependencies.difference(ws._has_what) nbytes: Py_ssize_t = 0 for dts in deps: nbytes += dts._nbytes @@ -4468,7 +4468,7 @@ def stimulus_missing_data( ws: WorkerState cts_nbytes: Py_ssize_t = cts.get_nbytes() for ws in cts._who_has: # TODO: this behavior is extreme - ws._has_what.remove(cts) + del ws._has_what[ts] ws._nbytes -= cts_nbytes cts._who_has.clear() recommendations[cause] = "released" @@ -5054,7 +5054,7 @@ def handle_missing_data(self, key=None, errant_worker=None, **kwargs): ws: WorkerState = parent._workers_dv.get(errant_worker) if ws is not None and ws in ts._who_has: ts._who_has.remove(ws) - ws._has_what.remove(ts) + del ws._has_what[ts] ws._nbytes -= ts.get_nbytes() if not ts._who_has: if ts._run_spec: @@ -5066,12 +5066,12 @@ def release_worker_data(self, comm=None, keys=None, worker=None): parent: SchedulerState = cast(SchedulerState, self) ws: WorkerState = parent._workers_dv[worker] tasks: set = {parent._tasks[k] for k in keys} - removed_tasks: set = tasks & ws._has_what - ws._has_what -= removed_tasks + removed_tasks: set = tasks.intersection(ws._has_what) ts: TaskState recommendations: dict = {} for ts in removed_tasks: + del ws._has_what[ts] ws._nbytes -= ts.get_nbytes() wh: set = ts._who_has wh.remove(ws) @@ -5312,7 +5312,7 @@ async def gather(self, comm=None, keys=None, serializers=None): for worker in workers: ws = parent._workers_dv.get(worker) if ws is not None and ts in ws._has_what: - ws._has_what.remove(ts) + del ws._has_what[ts] ts._who_has.remove(ws) ws._nbytes -= ts_nbytes parent._transitions( @@ -5478,8 +5478,8 @@ async def _delete_worker_data(self, worker_address, keys): ws: WorkerState = parent._workers_dv[worker_address] ts: TaskState tasks: set = {parent._tasks[key] for key in keys} - ws._has_what -= tasks for ts in tasks: + del ws._has_what[ts] ts._who_has.remove(ws) ws._nbytes -= ts.get_nbytes() self.log_event(ws._address, {"action": "remove-worker-data", "keys": keys}) @@ -5516,7 +5516,7 @@ async def rebalance(self, comm=None, keys=None, workers=None): # 30 seconds (distributed.worker.memory.recent_to_old_time). # This lets us ignore temporary spikes caused by task heap usage. memory_by_worker = [(ws, ws.memory.optimistic) for ws in workers] - mean_memory = sum(m for _, m in memory_by_worker) / len( + mean_memory = sum(m for _, m in memory_by_worker) // len( memory_by_worker ) @@ -5524,30 +5524,46 @@ async def rebalance(self, comm=None, keys=None, workers=None): # need to send/receive data, with how many bytes each needs to # send/receive. Bytes are negative, so that the workers farthest from # the cluster mean are at the top of the heaps. - # Second element of each tuple is an arbitrary unique number, there just - # to to make sure that WorkerState objects are never used for sorting in - # the unlikely event that two processes have exactly the same number of - # bytes allocated. - senders: "list[tuple[int, int, WorkerState]]" = [] - recipients: "list[tuple[int, int, WorkerState]]" = [] + # Heap elements: + # - maximum number of bytes to send or receive + # - number of bytes after which the worker should not be considered + # anymore + # - arbitrary unique number, there just to to make sure that WorkerState + # objects are never used for sorting in the unlikely event that two + # processes have exactly the same number of bytes allocated. + # - WorkerState + # - iterator of all tasks in memory on the worker (senders only) + senders: "list[tuple[int, int, int, WorkerState, Iterator[TaskState]]]" = ( + [] + ) + recipients: "list[tuple[int, int, int, WorkerState]" = [] for ws, ws_memory in memory_by_worker: if ws.memory_limit: - half_gap = MEMORY_REBALANCE_HALF_GAP * ws.memory_limit - if ( - ws_memory >= MEMORY_REBALANCE_SENDER_MIN * ws.memory_limit - and ws_memory >= mean_memory + half_gap - ): - senders.append((mean_memory - ws_memory, id(ws), ws)) - elif ( - ws_memory < MEMORY_REBALANCE_RECEIVER_MAX * ws.memory_limit - and ws_memory < mean_memory - half_gap - ): - recipients.append((ws_memory - mean_memory, id(ws), ws)) - elif ws_memory >= mean_memory: - senders.append((mean_memory - ws_memory, id(ws), ws)) + half_gap = int(MEMORY_REBALANCE_HALF_GAP * ws.memory_limit) + sender_min = MEMORY_REBALANCE_SENDER_MIN * ws.memory_limit + receiver_max = MEMORY_REBALANCE_RECEIVER_MAX * ws.memory_limit else: - recipients.append((ws_memory - mean_memory, id(ws), ws)) + half_gap = 0 + sender_min = 0.0 + receiver_max = math.inf + + if ( + ws._has_what + and ws_memory >= mean_memory + half_gap + and ws_memory >= sender_min + ): + # This may send the worker below sender_min (by design) + nbytes = mean_memory - ws_memory # negative + senders.append( + (nbytes, nbytes + half_gap, id(ws), ws, iter(ws._has_what)) + ) + elif ( + ws_memory < mean_memory - half_gap and ws_memory < receiver_max + ): + # This may send the worker above receiver_max (by design) + nbytes = ws_memory - mean_memory # negative + recipients.append((nbytes, nbytes + half_gap, id(ws), ws)) # Fast exit in case no transfers are necessary or possible if not senders or not recipients: @@ -5568,6 +5584,60 @@ async def rebalance(self, comm=None, keys=None, workers=None): if keys is not None and not isinstance(keys, Set): keys = set(keys) # unless already a set-like + # A task from a sender should be omitted from rebalance if: + # - it is not in memory (nbytes=-1) + # - there are no recipients that have (1) enough available RAM and + # and (2) don't hold a copy already + # - TODO it is needed as the input of a running or queued task on the + # same worker + # These tasks are pushed to the bottom of the insertion-sorted dict + # WorkerState._has_what. + + while senders and recipients: + snd_nbytes_max, snd_nbytes_min, _, snd_ws, ts_iter = senders[0] + + # Iterate through tasks in memory, least recently inserted first + reinsert_sender = True + for ts in ts_iter: + if keys is not None and ts.key not in keys: + continue + if ts.nbytes + snd_nbytes_max > 0: + # Moving this task would cause the sender to go below mean + # and potentially risk becoming a receiver, which would + # cause tasks to bounce around. + continue + # Find a receiver for this task + # TODO TODO TODO + rec_nbytes_max, rec_nbytes_min, _, rec_ws = recipients[0] + + snd_nbytes_max += ts.nbytes + snd_nbytes_min += ts.nbytes + rec_nbytes_max += ts.nbytes + rec_nbytes_min += ts.nbytes + + break + else: + # worker has no more tasks to donate + reinsert_sender = False + + if snd_nbytes_min >= 0: + reinsert_sender = False + + # Move skipped tasks to the end of the insertion-sorted dict + # ws._has_what + if reinsert_sender: + heapq.heapreplace( + senders, + ( + snd_nbytes_max, + snd_nbytes_min, + id(snd_ws), + snd_ws, + ts_iter, + ), + ) + else: + heapq.heappop(senders) # TODO TODO TODO if keys: @@ -5670,7 +5740,7 @@ async def rebalance(self, comm=None, keys=None, workers=None): for sender, recipient, ts in msgs: assert ts._state == "memory" ts._who_has.add(recipient) - recipient.has_what.add(ts) + recipient._has_what[ts] = None recipient.nbytes += ts.get_nbytes() self.log.append( ( @@ -6017,7 +6087,7 @@ async def retire_workers( logger.info("Retire workers %s", workers) # Keys orphaned by retiring those workers - keys = set.union(*[w.has_what for w in workers]) + keys = set.union(*[w._has_what for w in workers]) keys = {ts._key for ts in keys if ts._who_has.issubset(workers)} if keys: @@ -6071,7 +6141,7 @@ def add_keys(self, comm=None, worker=None, keys=()): if ts is not None and ts._state == "memory": if ts not in ws._has_what: ws._nbytes += ts.get_nbytes() - ws._has_what.add(ts) + ws._has_what[ts] = None ts._who_has.add(ws) else: self.worker_send( @@ -6116,7 +6186,7 @@ def update_data( ws: WorkerState = parent._workers_dv[w] if ts not in ws._has_what: ws._nbytes += ts_nbytes - ws._has_what.add(ts) + ws._has_what[ts] = None ts._who_has.add(ws) self.report( {"op": "key-in-memory", "key": key, "workers": list(workers)} @@ -6232,7 +6302,7 @@ def get_has_what(self, comm=None, workers=None): if workers is not None: workers = map(self.coerce_address, workers) return { - w: [ts._key for ts in parent._workers_dv[w].has_what] + w: [ts._key for ts in parent._workers_dv[w]._has_what] if w in parent._workers_dv else [] for w in workers @@ -6991,7 +7061,7 @@ def _add_to_memory( assert ts not in ws._has_what ts._who_has.add(ws) - ws._has_what.add(ts) + ws._has_what[ts] = None ws._nbytes += ts.get_nbytes() deps: list = list(ts._dependents) @@ -7075,7 +7145,7 @@ def _propagate_forgotten( ws: WorkerState for ws in ts._who_has: - ws._has_what.remove(ts) + del ws._has_what[ts] ws._nbytes -= ts_nbytes w: str = ws._address if w in state._workers_dv: # in case worker has died From 9ac044ac9a7b2be4261d6d2d3a00c827b203e296 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Tue, 27 Apr 2021 17:20:57 +0100 Subject: [PATCH 03/31] poc (incomplete) --- distributed/scheduler.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index fa1087b09ee..9c81bd4f5cd 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -5584,15 +5584,12 @@ async def rebalance(self, comm=None, keys=None, workers=None): if keys is not None and not isinstance(keys, Set): keys = set(keys) # unless already a set-like + msgs = [] # (sender, recipient, key) + # A task from a sender should be omitted from rebalance if: # - it is not in memory (nbytes=-1) # - there are no recipients that have (1) enough available RAM and - # and (2) don't hold a copy already - # - TODO it is needed as the input of a running or queued task on the - # same worker - # These tasks are pushed to the bottom of the insertion-sorted dict - # WorkerState._has_what. - + # (2) don't hold a copy already while senders and recipients: snd_nbytes_max, snd_nbytes_min, _, snd_ws, ts_iter = senders[0] @@ -5607,8 +5604,14 @@ async def rebalance(self, comm=None, keys=None, workers=None): # cause tasks to bounce around. continue # Find a receiver for this task - # TODO TODO TODO rec_nbytes_max, rec_nbytes_min, _, rec_ws = recipients[0] + if ts.nbytes + rec_nbytes_max > 0: + # There are no receivers with enough available RAM to accept + # this task without going above mean + continue + + # TODO skip receiver if it already holds the task + msgs.append((snd_ws, rec_ws, ts.key)) snd_nbytes_max += ts.nbytes snd_nbytes_min += ts.nbytes From 2a5b5cd56666775fe8187306af09643ce9eac851 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 29 Apr 2021 11:37:56 +0100 Subject: [PATCH 04/31] complete POC --- distributed/scheduler.py | 534 +++++++++++++++++++++------------------ 1 file changed, 284 insertions(+), 250 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 61fe6b7b06f..4a355049b59 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -12,11 +12,12 @@ import warnings import weakref from collections import defaultdict, deque -from collections.abc import Mapping, Set +from collections.abc import Hashable, Iterable, Iterator, Mapping, Set from contextlib import suppress from datetime import timedelta from functools import partial from numbers import Number +from typing import Optional import psutil import sortedcontainers @@ -5491,282 +5492,315 @@ async def _delete_worker_data(self, worker_address, keys): ws._nbytes -= ts.get_nbytes() self.log_event(ws._address, {"action": "remove-worker-data", "keys": keys}) - async def rebalance(self, comm=None, keys=None, workers=None): + async def rebalance( + self, + comm=None, + keys: "Iterable[Hashable]" = None, + workers: "Iterable[str]" = None, + ) -> dict: """Rebalance keys so that each worker stores roughly equal bytes - **Policy** + **Algorithm** + + #. Find the mean occupancy of the cluster, defined as data managed by dask + + unmanaged process memory that has been there for at least 30 seconds + (``distributed.worker.memory.recent_to_old_time``). + This lets us ignore temporary spikes caused by task heap usage. + #. Discard workers whose occupancy is within 5% of the mean cluster occupancy + (``distributed.worker.memory.rebalance.sender_recipient_gap`` / 2). + This helps avoid data from bouncing around the cluster repeatedly. + #. Workers above the mean are senders; those below are receivers. + #. Discard senders whose absolute occupancy is below 40% + (``distributed.worker.memory.rebalance.sender_min``). In other words, no data + is moved regardless of imbalancing as long as all workers are below 40%. + #. Discard receivers whose absolute occupancy is above 60% + (``distributed.worker.memory.rebalance.receiver_max``). + Note that this threshold by default is the same as + ``distributed.worker.memory.target`` to prevent workers from accepting data + and immediately spilling it out to disk. + #. Iteratively pick the sender and receiver that are farthest from the mean and + move the *least recently inserted* key between the two, until either all + senders or all receivers fall within 5% of the mean. + + A receiver will be skipped if it already has a copy of the data. In other + words, this method does not degrade replication. + A key will be skipped if there are no receivers that have both enough memory + to accept and don't already hold a copy. + + The least recently insertd (LRI) policy is a greedy choice with the advantage of + being O(1), trivial to implement (it relies on python dict insertion-sorting) + and hopefully good enough in most cases. Discarded alternative policies were: + + - Largest first. O(n*log(n)) save for non-trivial additional data structures and + risks causing the largest chunks of data to repeatedly move around the + cluster like pinballs. + - Least recently utilized. This information is currently available on the + workers only and not trivial to replicate on the scheduler; transmitting it + over the network would be very expensive. Also, note that dask will go out of + its way to minimise the amount of time intermediate keys are held in memory, + so in such a case LRI is a close approximation of LRU. - TODO TODO TODO - - 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. + Parameters + ---------- + keys: optional + whitelist of dask keys that should be considered for moving. All other keys + will be ignored. Note that this offers no guarantee that a key will actually + be moved (e.g. because it is unnecessary or because there are no viable + receiver workers for it). + workers: optional + whitelist of workers addresses to be considered as senders or receivers. All + other workers will be ignored. The mean cluster occupancy will be calculated + only using the whitelisted workers. """ - parent: SchedulerState = cast(SchedulerState, self) - ts: TaskState - ws: WorkerState - with log_errors(): - async with self._lock: - - # Identify workers that need to lose keys and those that can receive - # them, together with how many bytes each needs to lose/receive. - # This operation is O(n) to the number of workers in the cluster. - if workers: - workers = [parent._workers_dv[w] for w in workers] - else: - workers = parent._workers_dv.values() - - # optimistic memory = RSS - unmanaged memory that appeared over the last - # 30 seconds (distributed.worker.memory.recent_to_old_time). - # This lets us ignore temporary spikes caused by task heap usage. - memory_by_worker = [(ws, ws.memory.optimistic) for ws in workers] - mean_memory = sum(m for _, m in memory_by_worker) // len( - memory_by_worker - ) - - # Smallest-first heaps (managed by the heapq modules) of workers that - # need to send/receive data, with how many bytes each needs to - # send/receive. Bytes are negative, so that the workers farthest from - # the cluster mean are at the top of the heaps. - # Heap elements: - # - maximum number of bytes to send or receive - # - number of bytes after which the worker should not be considered - # anymore - # - arbitrary unique number, there just to to make sure that WorkerState - # objects are never used for sorting in the unlikely event that two - # processes have exactly the same number of bytes allocated. - # - WorkerState - # - iterator of all tasks in memory on the worker (senders only) - senders: "list[tuple[int, int, int, WorkerState, Iterator[TaskState]]]" = ( - [] - ) - recipients: "list[tuple[int, int, int, WorkerState]" = [] - - for ws, ws_memory in memory_by_worker: - if ws.memory_limit: - half_gap = int(MEMORY_REBALANCE_HALF_GAP * ws.memory_limit) - sender_min = MEMORY_REBALANCE_SENDER_MIN * ws.memory_limit - receiver_max = MEMORY_REBALANCE_RECEIVER_MAX * ws.memory_limit - else: - half_gap = 0 - sender_min = 0.0 - receiver_max = math.inf - - if ( - ws._has_what - and ws_memory >= mean_memory + half_gap - and ws_memory >= sender_min - ): - # This may send the worker below sender_min (by design) - nbytes = mean_memory - ws_memory # negative - senders.append( - (nbytes, nbytes + half_gap, id(ws), ws, iter(ws._has_what)) - ) - elif ( - ws_memory < mean_memory - half_gap and ws_memory < receiver_max - ): - # This may send the worker above receiver_max (by design) - nbytes = ws_memory - mean_memory # negative - recipients.append((nbytes, nbytes + half_gap, id(ws), ws)) - - # Fast exit in case no transfers are necessary or possible - if not senders or not recipients: - self.log_event( - "all", - { - "action": "rebalance", - "senders": len(senders), - "recipients": len(recipients), - "moved_keys": 0, - }, - ) - return {"status": "OK"} - - heapq.heapify(senders) - heapq.heapify(recipients) + if workers is not None: + workers = [self._workers_dv[w] for w in workers] + else: + workers = self._workers_dv.values() - if keys is not None and not isinstance(keys, Set): + if keys is not None: + if not isinstance(keys, Set): keys = set(keys) # unless already a set-like + tasks = [self._tasks[k] for k in keys] + missing_data = [ts._key for ts in tasks if not ts._who_has] + if missing_data: + return {"status": "missing-data", "keys": missing_data} - msgs = [] # (sender, recipient, key) - - # A task from a sender should be omitted from rebalance if: - # - it is not in memory (nbytes=-1) - # - there are no recipients that have (1) enough available RAM and - # (2) don't hold a copy already - while senders and recipients: - snd_nbytes_max, snd_nbytes_min, _, snd_ws, ts_iter = senders[0] - - # Iterate through tasks in memory, least recently inserted first - reinsert_sender = True - for ts in ts_iter: - if keys is not None and ts.key not in keys: - continue - if ts.nbytes + snd_nbytes_max > 0: - # Moving this task would cause the sender to go below mean - # and potentially risk becoming a receiver, which would - # cause tasks to bounce around. - continue - # Find a receiver for this task - rec_nbytes_max, rec_nbytes_min, _, rec_ws = recipients[0] - if ts.nbytes + rec_nbytes_max > 0: - # There are no receivers with enough available RAM to accept - # this task without going above mean - continue - - # TODO skip receiver if it already holds the task - msgs.append((snd_ws, rec_ws, ts.key)) - - snd_nbytes_max += ts.nbytes - snd_nbytes_min += ts.nbytes - rec_nbytes_max += ts.nbytes - rec_nbytes_min += ts.nbytes - - break - else: - # worker has no more tasks to donate - reinsert_sender = False - - if snd_nbytes_min >= 0: - reinsert_sender = False - - # Move skipped tasks to the end of the insertion-sorted dict - # ws._has_what - if reinsert_sender: - heapq.heapreplace( - senders, - ( - snd_nbytes_max, - snd_nbytes_min, - id(snd_ws), - snd_ws, - ts_iter, - ), - ) - else: - heapq.heappop(senders) - # TODO TODO TODO + msgs = self._rebalance_find_msgs(keys, workers) + if not msgs: + return {"status": "OK"} - if keys: - tasks = {parent._tasks[k] for k in keys} - missing_data = [ts._key for ts in tasks if not ts._who_has] - if missing_data: - return {"status": "missing-data", "keys": missing_data} - else: - tasks = set(parent._tasks.values()) + async with self._lock: + return await self._rebalance_move_data(msgs) + + def _rebalance_find_msgs( + self: SchedulerState, + keys: "Optional[Set[Hashable]]", + workers: "Iterable[WorkerState]", + ) -> "list[tuple[WorkerState, WorkerState, TaskState]]": + """Identify workers that need to lose keys and those that can receive them, + together with how many bytes each needs to lose/receive. Then, pair a sender + worker with a receiver worker for each key, until the cluster is rebalanced. + + This method only defines the work to be performed; it does not start any network + transfers itself. + + The big-O complexity is O(wt + ke*log(we)), where + + - wt is the total number of workers on the cluster (or the number of whitelisted + workers, if explicitly stated by the user) + - we is the number of workers that are eligible to be senders or receivers + - kt is the total number of keys on the cluster (or on the whitelisted workers) + - ke is the number of keys that need to be moved in order to achieve a balanced + cluster + + There is a degenerate edge case O(wt + kt*log(we)) when kt >> the number of + whitelisted keys, or when most keys are replicated or cannot be moved for some + other reason. + + Returns list of tuples to feed into _rebalance_move_data: + + - sender worker + - receiver worker + - task to be transferred + """ + ts: TaskState + ws: WorkerState - if workers: - workers = {parent._workers_dv[w] for w in workers} - workers_by_task = {ts: ts._who_has & workers for ts in tasks} - else: - workers = set(parent._workers_dv.values()) - workers_by_task = {ts: ts._who_has for ts in tasks} + # Smallest-first heapsof workers that need to send/receive data, with how many + # bytes each needs to send/receive. Bytes are negative, so that the workers + # farthest from the cluster mean are at the top of the heaps. + # Heap elements: + # - maximum number of bytes to send or receive + # - number of bytes after which the worker should not be considered anymore + # - arbitrary unique number, there just to to make sure that WorkerState objects + # are never used for sorting in the unlikely event that two processes have + # exactly the same number of bytes allocated. + # - WorkerState + # - iterator of all tasks in memory on the worker (senders only) + senders: "list[tuple[int, int, int, WorkerState, Iterator[TaskState]]]" = [] + recipients: "list[tuple[int, int, int, WorkerState]]" = [] + + # Output: [(sender, recipient, task), ...] + msgs: "list[tuple[WorkerState, WorkerState, TaskState]]" = [] + + # optimistic memory = RSS - unmanaged memory that appeared over the last 30 + # seconds (distributed.worker.memory.recent_to_old_time). + # This lets us ignore temporary spikes caused by task heap usage. + memory_by_worker = [(ws, ws.memory.optimistic) for ws in workers] + mean_memory = sum(m for _, m in memory_by_worker) // len(memory_by_worker) + + for ws, ws_memory in memory_by_worker: + if ws.memory_limit: + half_gap = int(MEMORY_REBALANCE_HALF_GAP * ws.memory_limit) + sender_min = MEMORY_REBALANCE_SENDER_MIN * ws.memory_limit + receiver_max = MEMORY_REBALANCE_RECEIVER_MAX * ws.memory_limit + else: + half_gap = 0 + sender_min = 0.0 + receiver_max = math.inf + + if ( + ws._has_what + and ws_memory >= mean_memory + half_gap + and ws_memory >= sender_min + ): + # This may send the worker below sender_min (by design) + nbytes = mean_memory - ws_memory # negative + senders.append( + (nbytes, nbytes + half_gap, id(ws), ws, iter(ws._has_what)) + ) + elif ws_memory < mean_memory - half_gap and ws_memory < receiver_max: + # This may send the worker above receiver_max (by design) + nbytes = ws_memory - mean_memory # negative + recipients.append((nbytes, nbytes + half_gap, id(ws), ws)) - ws: WorkerState - tasks_by_worker = {ws: set() for ws in workers} + # Fast exit in case no transfers are necessary or possible + if not senders or not recipients: + self.log_event( + "all", + { + "action": "rebalance", + "senders": len(senders), + "recipients": len(recipients), + "moved_keys": 0, + }, + ) + return [] - for k, v in workers_by_task.items(): - for vv in v: - tasks_by_worker[vv].add(k) + heapq.heapify(senders) + heapq.heapify(recipients) - worker_bytes = { - ws: sum(ts.get_nbytes() for ts in v) - for ws, v in tasks_by_worker.items() - } + while senders and recipients: + snd_nbytes_max, snd_nbytes_min, _, snd_ws, ts_iter = senders[0] - avg = sum(worker_bytes.values()) / len(worker_bytes) + # Iterate through tasks in memory, least recently inserted first + for ts in ts_iter: + if keys is not None and ts.key not in keys: + continue + nbytes = ts.nbytes + if nbytes + snd_nbytes_max > 0: + # Moving this task would cause the sender to go below mean + # and potentially risk becoming a receiver, which would + # cause tasks to bounce around. Move on to the next task + # of the same sender. + continue - sorted_workers = list( - map(first, sorted(worker_bytes.items(), key=second, reverse=True)) - ) + # Find the recipient, farthest from the mean, which + # 1. has enough available RAM for this task + # 2. don't hold a copy already + # There may not be any that satisfies these conditions; in this + # case this task won't be moved. + skipped_recipients = [] + use_recipient = False + while recipients and not use_recipient: + rec_nbytes_max, rec_nbytes_min, _, rec_ws = recipients[0] + if nbytes + rec_nbytes_max > 0: + break # recipients are sorted by rec_nbytes_max + use_recipient = ts.key not in rec_ws._tasks + if not use_recipient: + skipped_recipients.append(heapq.heappop(recipients)) + + for recipient in skipped_recipients: + heapq.heappush(recipients, recipient) + + if not use_recipient: + # This task has no receivers available. Leave it on the + # sender and move on to the next task of the same sender + continue - recipients = reversed(sorted_workers) - recipient = next(recipients) - msgs = [] # (sender, recipient, key) - for sender in sorted_workers[: len(workers) // 2]: - sender_keys = { - ts: ts.get_nbytes() for ts in tasks_by_worker[sender] - } - sender_keys = iter( - sorted(sender_keys.items(), key=second, reverse=True) + # Stop iterating on the tasks of this sender for now and, if it + # still has bytes to lose, push it back into the senders heap. + # It may or may not come back on top again. + msgs.append((snd_ws, rec_ws, ts)) + # nbytes_max/nbytes_min are all negative for heap sorting + snd_nbytes_max += nbytes + snd_nbytes_min += nbytes + rec_nbytes_max += nbytes + rec_nbytes_min += nbytes + if snd_nbytes_min < 0: + heapq.heapreplace( + senders, + (snd_nbytes_max, snd_nbytes_min, id(snd_ws), snd_ws, ts_iter), ) + if rec_nbytes_min < 0: + heapq.heapreplace( + recipients, + (rec_nbytes_max, rec_nbytes_min, id(rec_ws), rec_ws), + ) + break + else: + # Exhausted tasks on this sender + heapq.heappop(senders) + return msgs - try: - while avg < worker_bytes[sender]: - while worker_bytes[recipient] < avg < worker_bytes[sender]: - ts, nb = next(sender_keys) - if ts not in tasks_by_worker[recipient]: - tasks_by_worker[recipient].add(ts) - # tasks_by_worker[sender].remove(ts) - msgs.append((sender, recipient, ts)) - worker_bytes[sender] -= nb - worker_bytes[recipient] += nb - if avg < worker_bytes[sender]: - recipient = next(recipients) - except StopIteration: - break + async def _rebalance_move_data( + self, msgs: "list[tuple[WorkerState, WorkerState, TaskState]]" + ) -> dict: + """Perform the actual transfer of data across the network in rebalance(). + Takes in input the output of _rebalance_find_msgs(). - to_recipients = defaultdict(lambda: defaultdict(list)) - to_senders = defaultdict(list) - for sender, recipient, ts in msgs: - to_recipients[recipient.address][ts._key].append(sender.address) - to_senders[sender.address].append(ts._key) + FIXME this method is not robust when the cluster is not idle + """ + ts: TaskState + ws: WorkerState - result = await asyncio.gather( - *( - retry_operation(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}) + to_recipients = defaultdict(lambda: defaultdict(list)) + to_senders = defaultdict(list) + for sender, recipient, ts in msgs: + to_recipients[recipient.address][ts._key].append(sender.address) + to_senders[sender.address].append(ts._key) - self.log_event( - "all", - { - "action": "rebalance", - "total-keys": len(tasks), - "senders": valmap(len, to_senders), - "recipients": valmap(len, to_recipients), - "moved_keys": len(msgs), - }, - ) + result = await asyncio.gather( + *( + retry_operation(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}) - if any(r["status"] != "OK" for r in result): - return { - "status": "missing-data", - "keys": tuple( - concat( - r["keys"].keys() - for r in result - if r["status"] == "missing-data" - ) - ), - } + self.log_event( + "all", + { + "action": "rebalance", + "senders": valmap(len, to_senders), + "recipients": valmap(len, to_recipients), + "moved_keys": len(msgs), + }, + ) - for sender, recipient, ts in msgs: - assert ts._state == "memory" - ts._who_has.add(recipient) - recipient._has_what[ts] = None - recipient.nbytes += ts.get_nbytes() - self.log.append( - ( - "rebalance", - ts._key, - time(), - sender.address, - recipient.address, - ) + if any(r["status"] != "OK" for r in result): + return { + "status": "missing-data", + "keys": list( + concat( + r["keys"].keys() + for r in result + if r["status"] == "missing-data" ) + ), + } - await asyncio.gather( - *(self._delete_worker_data(r, v) for r, v in to_senders.items()) + for sender, recipient, ts in msgs: + assert ts._state == "memory" + ts._who_has.add(recipient) + recipient._has_what[ts] = None + recipient.nbytes += ts.get_nbytes() + self.log.append( + ( + "rebalance", + ts._key, + time(), + sender.address, + recipient.address, ) + ) - return {"status": "OK"} + await asyncio.gather( + *(self._delete_worker_data(r, v) for r, v in to_senders.items()) + ) + return {"status": "OK"} async def replicate( self, From 60fe5a40e73e3cf409876ab66cac86bcdc81eec1 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 29 Apr 2021 11:39:41 +0100 Subject: [PATCH 05/31] polish --- distributed/scheduler.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 4a355049b59..ce8d862f08a 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -5498,7 +5498,10 @@ async def rebalance( keys: "Iterable[Hashable]" = None, workers: "Iterable[str]" = None, ) -> dict: - """Rebalance keys so that each worker stores roughly equal bytes + """Rebalance keys so that each worker ends up with roughly the same process + memory (managed+unmanaged). + + FIXME this method is not robust when the cluster is not idle. **Algorithm** @@ -5740,7 +5743,7 @@ async def _rebalance_move_data( """Perform the actual transfer of data across the network in rebalance(). Takes in input the output of _rebalance_find_msgs(). - FIXME this method is not robust when the cluster is not idle + FIXME this method is not robust when the cluster is not idle. """ ts: TaskState ws: WorkerState From a7e46b381cfad3c1eca66c5ef61cd4ff44f055bf Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 29 Apr 2021 13:16:59 +0100 Subject: [PATCH 06/31] polish --- distributed/scheduler.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index ce8d862f08a..b6faecc3853 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -5597,9 +5597,9 @@ def _rebalance_find_msgs( - ke is the number of keys that need to be moved in order to achieve a balanced cluster - There is a degenerate edge case O(wt + kt*log(we)) when kt >> the number of - whitelisted keys, or when most keys are replicated or cannot be moved for some - other reason. + There is a degenerate edge case O(wt + kt*log(we)) when kt is much greater than + the number of whitelisted keys, or when most keys are replicated or cannot be + moved for some other reason. Returns list of tuples to feed into _rebalance_move_data: From 5d267d317d54c565e50a2df411d93818cd4b2f50 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 29 Apr 2021 14:30:49 +0100 Subject: [PATCH 07/31] bugfix --- distributed/distributed-schema.yaml | 2 +- distributed/scheduler.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index a10d917d08e..4e219e75069 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -426,7 +426,7 @@ properties: Fraction of worker process memory at which we start potentially transferring data to other workers. - sender_recipient_gap: 0.1 + sender_recipient_gap: type: number minimum: 0 maximum: 1 diff --git a/distributed/scheduler.py b/distributed/scheduler.py index b6faecc3853..60b5d3fe229 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -6134,7 +6134,7 @@ async def retire_workers( logger.info("Retire workers %s", workers) # Keys orphaned by retiring those workers - keys = set.union(*[w._has_what for w in workers]) + keys = {k for w in workers for k in w._has_what} keys = {ts._key for ts in keys if ts._who_has.issubset(workers)} if keys: From b275d880ee229cc2f572c41e2ba1780a85b6d8c3 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 29 Apr 2021 18:29:30 +0100 Subject: [PATCH 08/31] fixes --- distributed/client.py | 7 +++++-- distributed/scheduler.py | 6 +++++- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/distributed/client.py b/distributed/client.py index 695c9dd39f2..2b1f2d18428 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -3051,8 +3051,11 @@ def upload_file(self, filename, **kwargs): ) async def _rebalance(self, futures=None, workers=None): - await _wait(futures) - keys = list({stringify(f.key) for f in self.futures_of(futures)}) + if futures is not None: + await _wait(futures) + keys = list({stringify(f.key) for f in self.futures_of(futures)}) + else: + keys = None result = await self.scheduler.rebalance(keys=keys, workers=workers) if result["status"] == "missing-data": raise ValueError( diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 60b5d3fe229..c9d1bdd96af 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -5560,10 +5560,14 @@ async def rebalance( workers = [self._workers_dv[w] for w in workers] else: workers = self._workers_dv.values() + if not workers: + return {"status": "OK"} if keys is not None: if not isinstance(keys, Set): keys = set(keys) # unless already a set-like + if not keys: + return {"status": "OK"} tasks = [self._tasks[k] for k in keys] missing_data = [ts._key for ts in tasks if not ts._who_has] if missing_data: @@ -5700,7 +5704,7 @@ def _rebalance_find_msgs( rec_nbytes_max, rec_nbytes_min, _, rec_ws = recipients[0] if nbytes + rec_nbytes_max > 0: break # recipients are sorted by rec_nbytes_max - use_recipient = ts.key not in rec_ws._tasks + use_recipient = ts.key not in rec_ws._has_what if not use_recipient: skipped_recipients.append(heapq.heappop(recipients)) From c3fd176e9a768fef6b7b631a80760473a60a5647 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 29 Apr 2021 20:58:24 +0100 Subject: [PATCH 09/31] fix --- distributed/distributed-schema.yaml | 2 +- distributed/distributed.yaml | 6 ++-- distributed/scheduler.py | 56 +++++++++++++++++------------ 3 files changed, 37 insertions(+), 27 deletions(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index 4e219e75069..fb814fa874f 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -418,7 +418,7 @@ properties: receiving data from other workers. Ignored when max_memory is not set. - receiver_max: + recipient_max: type: number minimum: 0 maximum: 1 diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index 7e8aae284e3..b40246f8118 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -91,12 +91,12 @@ distributed: recent_to_old_time: 30s rebalance: - # Fraction of worker process memory at which we start potentially transferring + # Fraction of worker process memory at which we start potentially sending # data to other workers. Ignored when max_memory is not set. sender_min: 0.40 - # Fraction of worker process memory at which we stop potentially receiving + # Fraction of worker process memory at which we stop potentially accepting # data from other workers. Ignored when max_memory is not set. - receiver_max: 0.60 + recipient_max: 0.60 # Fraction of worker process memory, around the cluster mean, where a worker is # neither a sender nor a recipient of data during a rebalance operation. E.g. # if the mean cluster occupation is 50%, no_rebalance_gap=0.10 means that only diff --git a/distributed/scheduler.py b/distributed/scheduler.py index c9d1bdd96af..bce316d1167 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -174,9 +174,9 @@ def nogil(func): double, dask.config.get("distributed.worker.memory.rebalance.sender_min"), ) -MEMORY_REBALANCE_RECEIVER_MAX = declare( +MEMORY_REBALANCE_RECIPIENT_MAX = declare( double, - dask.config.get("distributed.worker.memory.rebalance.receiver_max"), + dask.config.get("distributed.worker.memory.rebalance.recipient_max"), ) MEMORY_REBALANCE_HALF_GAP = declare( double, @@ -5512,22 +5512,22 @@ async def rebalance( #. Discard workers whose occupancy is within 5% of the mean cluster occupancy (``distributed.worker.memory.rebalance.sender_recipient_gap`` / 2). This helps avoid data from bouncing around the cluster repeatedly. - #. Workers above the mean are senders; those below are receivers. + #. Workers above the mean are senders; those below are recipients. #. Discard senders whose absolute occupancy is below 40% (``distributed.worker.memory.rebalance.sender_min``). In other words, no data is moved regardless of imbalancing as long as all workers are below 40%. - #. Discard receivers whose absolute occupancy is above 60% - (``distributed.worker.memory.rebalance.receiver_max``). + #. Discard recipients whose absolute occupancy is above 60% + (``distributed.worker.memory.rebalance.recipient_max``). Note that this threshold by default is the same as ``distributed.worker.memory.target`` to prevent workers from accepting data and immediately spilling it out to disk. - #. Iteratively pick the sender and receiver that are farthest from the mean and + #. Iteratively pick the sender and recipient that are farthest from the mean and move the *least recently inserted* key between the two, until either all - senders or all receivers fall within 5% of the mean. + senders or all recipients fall within 5% of the mean. - A receiver will be skipped if it already has a copy of the data. In other + A recipient will be skipped if it already has a copy of the data. In other words, this method does not degrade replication. - A key will be skipped if there are no receivers that have both enough memory + A key will be skipped if there are no recipients that have both enough memory to accept and don't already hold a copy. The least recently insertd (LRI) policy is a greedy choice with the advantage of @@ -5549,11 +5549,11 @@ async def rebalance( whitelist of dask keys that should be considered for moving. All other keys will be ignored. Note that this offers no guarantee that a key will actually be moved (e.g. because it is unnecessary or because there are no viable - receiver workers for it). + recipient workers for it). workers: optional - whitelist of workers addresses to be considered as senders or receivers. All - other workers will be ignored. The mean cluster occupancy will be calculated - only using the whitelisted workers. + whitelist of workers addresses to be considered as senders or recipients. + All other workers will be ignored. The mean cluster occupancy will be + calculated only using the whitelisted workers. """ with log_errors(): if workers is not None: @@ -5587,7 +5587,7 @@ def _rebalance_find_msgs( ) -> "list[tuple[WorkerState, WorkerState, TaskState]]": """Identify workers that need to lose keys and those that can receive them, together with how many bytes each needs to lose/receive. Then, pair a sender - worker with a receiver worker for each key, until the cluster is rebalanced. + worker with a recipient worker for each key, until the cluster is rebalanced. This method only defines the work to be performed; it does not start any network transfers itself. @@ -5596,7 +5596,7 @@ def _rebalance_find_msgs( - wt is the total number of workers on the cluster (or the number of whitelisted workers, if explicitly stated by the user) - - we is the number of workers that are eligible to be senders or receivers + - we is the number of workers that are eligible to be senders or recipients - kt is the total number of keys on the cluster (or on the whitelisted workers) - ke is the number of keys that need to be moved in order to achieve a balanced cluster @@ -5608,7 +5608,7 @@ def _rebalance_find_msgs( Returns list of tuples to feed into _rebalance_move_data: - sender worker - - receiver worker + - recipient worker - task to be transferred """ ts: TaskState @@ -5641,11 +5641,11 @@ def _rebalance_find_msgs( if ws.memory_limit: half_gap = int(MEMORY_REBALANCE_HALF_GAP * ws.memory_limit) sender_min = MEMORY_REBALANCE_SENDER_MIN * ws.memory_limit - receiver_max = MEMORY_REBALANCE_RECEIVER_MAX * ws.memory_limit + recipient_max = MEMORY_REBALANCE_RECIPIENT_MAX * ws.memory_limit else: half_gap = 0 sender_min = 0.0 - receiver_max = math.inf + recipient_max = math.inf if ( ws._has_what @@ -5657,8 +5657,8 @@ def _rebalance_find_msgs( senders.append( (nbytes, nbytes + half_gap, id(ws), ws, iter(ws._has_what)) ) - elif ws_memory < mean_memory - half_gap and ws_memory < receiver_max: - # This may send the worker above receiver_max (by design) + elif ws_memory < mean_memory - half_gap and ws_memory < recipient_max: + # This may send the worker above recipient_max (by design) nbytes = ws_memory - mean_memory # negative recipients.append((nbytes, nbytes + half_gap, id(ws), ws)) @@ -5688,7 +5688,7 @@ def _rebalance_find_msgs( nbytes = ts.nbytes if nbytes + snd_nbytes_max > 0: # Moving this task would cause the sender to go below mean - # and potentially risk becoming a receiver, which would + # and potentially risk becoming a recipient, which would # cause tasks to bounce around. Move on to the next task # of the same sender. continue @@ -5703,7 +5703,9 @@ def _rebalance_find_msgs( while recipients and not use_recipient: rec_nbytes_max, rec_nbytes_min, _, rec_ws = recipients[0] if nbytes + rec_nbytes_max > 0: - break # recipients are sorted by rec_nbytes_max + # recipients are sorted by rec_nbytes_max. + # The next ones will be worse; no reason to continue iterating + break use_recipient = ts.key not in rec_ws._has_what if not use_recipient: skipped_recipients.append(heapq.heappop(recipients)) @@ -5712,7 +5714,7 @@ def _rebalance_find_msgs( heapq.heappush(recipients, recipient) if not use_recipient: - # This task has no receivers available. Leave it on the + # This task has no recipients available. Leave it on the # sender and move on to the next task of the same sender continue @@ -5730,11 +5732,19 @@ def _rebalance_find_msgs( senders, (snd_nbytes_max, snd_nbytes_min, id(snd_ws), snd_ws, ts_iter), ) + else: + heapq.heappop(senders) + if rec_nbytes_min < 0: heapq.heapreplace( recipients, (rec_nbytes_max, rec_nbytes_min, id(rec_ws), rec_ws), ) + else: + heapq.heappop(recipients) + + # Move to next sender with the most data to lose. + # It may or may not be the same sender again. break else: # Exhausted tasks on this sender From c7e8ed631a9fbb432e81f52373abda15bbd053d1 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Fri, 30 Apr 2021 10:11:38 +0100 Subject: [PATCH 10/31] Use arbitrary measure in rebalance --- distributed/distributed-schema.yaml | 9 +++++++++ distributed/distributed.yaml | 17 ++++++++++++++++- distributed/scheduler.py | 7 ++++++- 3 files changed, 31 insertions(+), 2 deletions(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index fb814fa874f..20050b522ce 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -409,6 +409,15 @@ properties: description: >- Settings for data rebalance operations properties: + measure: + enum: + - process + - optimistic + - managed + - managed_in_memory + description: >- + Which of the properties of distributed.scheduler.MemoryState + should be used for measuring worker memory usage sender_min: type: number minimum: 0 diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index b40246f8118..a732cdb1155 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -91,9 +91,24 @@ distributed: recent_to_old_time: 30s rebalance: + # Memory measure to rebalance upon. Possible choices are: + # process + # Total process memory, as measured by the OS. + # optimistic + # Managed by dask (instantaneous) + unmanaged (without any increases + # happened in the last ). + # Recommended for use on CPython with large (2MiB+) numpy-based data chunks. + # managed_in_memory + # Only consider the data allocated by dask in RAM. Recommended if RAM is not + # released in a timely fashion back to the OS after the Python objects are + # dereferenced, but remains available for reuse by PyMalloc. + # managed + # Only consider data allocated by dask, including that spilled to disk. + # Recommended if disk occupation of the spill file is an issue. + measure: optimistic # Fraction of worker process memory at which we start potentially sending # data to other workers. Ignored when max_memory is not set. - sender_min: 0.40 + sender_min: 0.30 # Fraction of worker process memory at which we stop potentially accepting # data from other workers. Ignored when max_memory is not set. recipient_max: 0.60 diff --git a/distributed/scheduler.py b/distributed/scheduler.py index a161fd9cc66..98360edd128 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -170,6 +170,9 @@ def nogil(func): double, parse_timedelta(dask.config.get("distributed.worker.memory.recent_to_old_time")), ) +MEMORY_REBALANCE_MEASURE = declare( + str, dask.config.get("distributed.worker.memory.rebalance.measure") +) MEMORY_REBALANCE_SENDER_MIN = declare( double, dask.config.get("distributed.worker.memory.rebalance.sender_min"), @@ -5634,7 +5637,9 @@ def _rebalance_find_msgs( # optimistic memory = RSS - unmanaged memory that appeared over the last 30 # seconds (distributed.worker.memory.recent_to_old_time). # This lets us ignore temporary spikes caused by task heap usage. - memory_by_worker = [(ws, ws.memory.optimistic) for ws in workers] + memory_by_worker = [ + (ws, getattr(ws.memory, MEMORY_REBALANCE_MEASURE)) for ws in workers + ] mean_memory = sum(m for _, m in memory_by_worker) // len(memory_by_worker) for ws, ws_memory in memory_by_worker: From 428fd8fb3ee0b9d57d175dd65265242bc42b514e Mon Sep 17 00:00:00 2001 From: crusaderky Date: Fri, 7 May 2021 14:22:02 +0100 Subject: [PATCH 11/31] Code review --- distributed/distributed-schema.yaml | 14 ++++++------- distributed/distributed.yaml | 16 +++++++-------- distributed/scheduler.py | 32 ++++++++++++++++++----------- distributed/tests/test_scheduler.py | 4 ++-- 4 files changed, 37 insertions(+), 29 deletions(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index 20050b522ce..27fb561813f 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -395,7 +395,7 @@ properties: description: >- Settings for memory management properties: - recent_to_old_time: + recent-to-old-time: type: string description: >- When there is an increase in process memory (as observed by the @@ -418,7 +418,7 @@ properties: description: >- Which of the properties of distributed.scheduler.MemoryState should be used for measuring worker memory usage - sender_min: + sender-min: type: number minimum: 0 maximum: 1 @@ -427,7 +427,7 @@ properties: receiving data from other workers. Ignored when max_memory is not set. - recipient_max: + recipient-max: type: number minimum: 0 maximum: 1 @@ -435,7 +435,7 @@ properties: Fraction of worker process memory at which we start potentially transferring data to other workers. - sender_recipient_gap: + sender-recipient-gap: type: number minimum: 0 maximum: 1 @@ -443,9 +443,9 @@ properties: Fraction of worker process memory, around the cluster mean, where a worker is neither a sender nor a recipient of data during a rebalance operation. E.g. if the mean cluster occupation is 50%, - no_rebalance_gap=0.1 means that only nodes above 55% will donate - data and only nodes below 45% will receive them. This helps avoid - data from bouncing around the cluster repeatedly. + sender-recipient-gap=0.1 means that only nodes above 55% will + donate data and only nodes below 45% will receive them. This helps + avoid data from bouncing around the cluster repeatedly. target: oneOf: diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index a732cdb1155..b0bc6bd68d4 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -88,7 +88,7 @@ distributed: # system) that is not accounted for by the dask keys stored on the worker, ignore # it for this long before considering it in non-critical memory measures. # This should be set to be longer than the duration of most dask tasks. - recent_to_old_time: 30s + recent-to-old-time: 30s rebalance: # Memory measure to rebalance upon. Possible choices are: @@ -96,7 +96,7 @@ distributed: # Total process memory, as measured by the OS. # optimistic # Managed by dask (instantaneous) + unmanaged (without any increases - # happened in the last ). + # happened in the last ). # Recommended for use on CPython with large (2MiB+) numpy-based data chunks. # managed_in_memory # Only consider the data allocated by dask in RAM. Recommended if RAM is not @@ -108,17 +108,17 @@ distributed: measure: optimistic # Fraction of worker process memory at which we start potentially sending # data to other workers. Ignored when max_memory is not set. - sender_min: 0.30 + sender-min: 0.30 # Fraction of worker process memory at which we stop potentially accepting # data from other workers. Ignored when max_memory is not set. - recipient_max: 0.60 + recipient-max: 0.60 # Fraction of worker process memory, around the cluster mean, where a worker is - # neither a sender nor a recipient of data during a rebalance operation. E.g. - # if the mean cluster occupation is 50%, no_rebalance_gap=0.10 means that only + # neither a sender nor a recipient of data during a rebalance operation. E.g. if + # the mean cluster occupation is 50%, sender-recipient-gap=0.10 means that only # nodes above 55% will donate data and only nodes below 45% will receive them. - # This helps avoid data from bouncing around the cluster repeatedly.# + # This helps avoid data from bouncing around the cluster repeatedly. # Ignored when max_memory is not set. - sender_recipient_gap: 0.10 + sender-recipient-gap: 0.10 # Fractions of worker process memory at which we take action to avoid memory # blowup. Set any of the values to False to turn off the behavior entirely. diff --git a/distributed/scheduler.py b/distributed/scheduler.py index a81f46bc44b..d3cfc19a2fe 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -168,22 +168,22 @@ def nogil(func): ) MEMORY_RECENT_TO_OLD_TIME = declare( double, - parse_timedelta(dask.config.get("distributed.worker.memory.recent_to_old_time")), + parse_timedelta(dask.config.get("distributed.worker.memory.recent-to-old-time")), ) MEMORY_REBALANCE_MEASURE = declare( str, dask.config.get("distributed.worker.memory.rebalance.measure") ) MEMORY_REBALANCE_SENDER_MIN = declare( double, - dask.config.get("distributed.worker.memory.rebalance.sender_min"), + dask.config.get("distributed.worker.memory.rebalance.sender-min"), ) MEMORY_REBALANCE_RECIPIENT_MAX = declare( double, - dask.config.get("distributed.worker.memory.rebalance.recipient_max"), + dask.config.get("distributed.worker.memory.rebalance.recipient-max"), ) MEMORY_REBALANCE_HALF_GAP = declare( double, - dask.config.get("distributed.worker.memory.rebalance.sender_recipient_gap") / 2.0, + dask.config.get("distributed.worker.memory.rebalance.sender-recipient-gap") / 2.0, ) @@ -309,7 +309,7 @@ class MemoryState: unmanaged_old Minimum of the 'unmanaged' measures over the last - ``distributed.memory.recent_to_old_time`` seconds + ``distributed.memory.recent-to-old-time`` seconds unmanaged_recent unmanaged - unmanaged_old; in other words process memory that has been recently allocated but is not accounted for by dask; hopefully it's mostly a temporary @@ -496,6 +496,8 @@ class WorkerState: _bandwidth: double _executing: dict _extra: dict + # _has_what is a dict with all values set to None as we rely on the property of + # Python >=3.7 dicts to be insertion-sorted. _has_what: dict _hash: Py_hash_t _last_seen: double @@ -5511,17 +5513,17 @@ async def rebalance( #. Find the mean occupancy of the cluster, defined as data managed by dask + unmanaged process memory that has been there for at least 30 seconds - (``distributed.worker.memory.recent_to_old_time``). + (``distributed.worker.memory.recent-to-old-time``). This lets us ignore temporary spikes caused by task heap usage. #. Discard workers whose occupancy is within 5% of the mean cluster occupancy - (``distributed.worker.memory.rebalance.sender_recipient_gap`` / 2). + (``distributed.worker.memory.rebalance.sender-recipient-gap`` / 2). This helps avoid data from bouncing around the cluster repeatedly. #. Workers above the mean are senders; those below are recipients. #. Discard senders whose absolute occupancy is below 40% - (``distributed.worker.memory.rebalance.sender_min``). In other words, no data + (``distributed.worker.memory.rebalance.sender-min``). In other words, no data is moved regardless of imbalancing as long as all workers are below 40%. #. Discard recipients whose absolute occupancy is above 60% - (``distributed.worker.memory.rebalance.recipient_max``). + (``distributed.worker.memory.rebalance.recipient-max``). Note that this threshold by default is the same as ``distributed.worker.memory.target`` to prevent workers from accepting data and immediately spilling it out to disk. @@ -5572,8 +5574,7 @@ async def rebalance( keys = set(keys) # unless already a set-like if not keys: return {"status": "OK"} - tasks = [self._tasks[k] for k in keys] - missing_data = [ts._key for ts in tasks if not ts._who_has] + missing_data = [k for k in keys if not self._tasks[k]._who_has] if missing_data: return {"status": "missing-data", "keys": missing_data} @@ -5636,7 +5637,7 @@ def _rebalance_find_msgs( msgs: "list[tuple[WorkerState, WorkerState, TaskState]]" = [] # optimistic memory = RSS - unmanaged memory that appeared over the last 30 - # seconds (distributed.worker.memory.recent_to_old_time). + # seconds (distributed.worker.memory.recent-to-old-time). # This lets us ignore temporary spikes caused by task heap usage. memory_by_worker = [ (ws, getattr(ws.memory, MEMORY_REBALANCE_MEASURE)) for ws in workers @@ -5660,12 +5661,17 @@ def _rebalance_find_msgs( ): # This may send the worker below sender_min (by design) nbytes = mean_memory - ws_memory # negative + # See definition of senders above + # Note that the iterator of keys will typically *not* be exhausted. + # It will only be exhausted if moving away from the worker all keys that + # can be moved is insufficient to drop nbytes + half_gap above 0. senders.append( (nbytes, nbytes + half_gap, id(ws), ws, iter(ws._has_what)) ) elif ws_memory < mean_memory - half_gap and ws_memory < recipient_max: # This may send the worker above recipient_max (by design) nbytes = ws_memory - mean_memory # negative + # See definition of recipients above recipients.append((nbytes, nbytes + half_gap, id(ws), ws)) # Fast exit in case no transfers are necessary or possible @@ -5734,6 +5740,7 @@ def _rebalance_find_msgs( rec_nbytes_max += nbytes rec_nbytes_min += nbytes if snd_nbytes_min < 0: + # See definition of senders above heapq.heapreplace( senders, (snd_nbytes_max, snd_nbytes_min, id(snd_ws), snd_ws, ts_iter), @@ -5742,6 +5749,7 @@ def _rebalance_find_msgs( heapq.heappop(senders) if rec_nbytes_min < 0: + # See definition of recipients above heapq.heapreplace( recipients, (rec_nbytes_max, rec_nbytes_min, id(rec_ws), rec_ws), diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 18e9dc5a259..2cf12005cef 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2333,7 +2333,7 @@ def assert_memory(scheduler_or_workerstate, attr: str, min_, max_, timeout=10): # This test is heavily influenced by hard-to-control factors such as memory management # by the Python interpreter and the OS, so it occasionally glitches @pytest.mark.flaky(reruns=3, reruns_delay=5) -# ~33s runtime, or distributed.memory.recent_to_old_time + 3s +# ~33s runtime, or distributed.memory.recent-to-old-time + 3s @pytest.mark.slow def test_memory(): pytest.importorskip("zict") @@ -2384,7 +2384,7 @@ def test_memory(): sleep(2) assert_memory(s, "managed_spilled", 1, 999) # Wait for the spilling to finish. Note that this does not make the test take - # longer as we're waiting for recent_to_old_time anyway. + # longer as we're waiting for recent-to-old-time anyway. sleep(10) # Delete spilled keys From f73ace8d82ed3002d0f3dc66f96765ee31087a30 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Fri, 7 May 2021 14:48:33 +0100 Subject: [PATCH 12/31] renames --- distributed/scheduler.py | 67 +++++++++++++++++++++++----------------- 1 file changed, 39 insertions(+), 28 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index d3cfc19a2fe..ac0151dbd89 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -5619,17 +5619,23 @@ def _rebalance_find_msgs( ts: TaskState ws: WorkerState - # Smallest-first heapsof workers that need to send/receive data, with how many - # bytes each needs to send/receive. Bytes are negative, so that the workers - # farthest from the cluster mean are at the top of the heaps. - # Heap elements: - # - maximum number of bytes to send or receive - # - number of bytes after which the worker should not be considered anymore + # Heaps of workers that need to send/receive data, with how many bytes each + # needs to send/receive. + # + # Each element of the heap is a tuple constructed as follows: + # - snd_bytes_max/rec_bytes_max: maximum number of bytes to send or receive. + # This number is negative, so that the workers farthest from the cluster mean + # are at the top of the smallest-first heaps. + # - snd_bytes_min/rec_bytes_min: minimum number of bytes after sending/receiving + # which the worker should not be considered anymore. This is also negative. # - arbitrary unique number, there just to to make sure that WorkerState objects # are never used for sorting in the unlikely event that two processes have # exactly the same number of bytes allocated. # - WorkerState - # - iterator of all tasks in memory on the worker (senders only) + # - iterator of all tasks in memory on the worker (senders only). + # Note that this iterator of keys will typically *not* be exhausted. It will + # only be exhausted if, after moving away from the worker all keys that can be + # moved, is insufficient to drop snd_bytes_min above 0. senders: "list[tuple[int, int, int, WorkerState, Iterator[TaskState]]]" = [] recipients: "list[tuple[int, int, int, WorkerState]]" = [] @@ -5660,19 +5666,18 @@ def _rebalance_find_msgs( and ws_memory >= sender_min ): # This may send the worker below sender_min (by design) - nbytes = mean_memory - ws_memory # negative + snd_bytes_max = mean_memory - ws_memory # negative + snd_bytes_min = snd_bytes_max + half_gap # negative # See definition of senders above - # Note that the iterator of keys will typically *not* be exhausted. - # It will only be exhausted if moving away from the worker all keys that - # can be moved is insufficient to drop nbytes + half_gap above 0. senders.append( - (nbytes, nbytes + half_gap, id(ws), ws, iter(ws._has_what)) + (snd_bytes_max, snd_bytes_min, id(ws), ws, iter(ws._has_what)) ) elif ws_memory < mean_memory - half_gap and ws_memory < recipient_max: # This may send the worker above recipient_max (by design) - nbytes = ws_memory - mean_memory # negative + rec_bytes_max = ws_memory - mean_memory # negative + rec_bytes_min = rec_bytes_max + half_gap # negative # See definition of recipients above - recipients.append((nbytes, nbytes + half_gap, id(ws), ws)) + recipients.append((rec_bytes_max, rec_bytes_min, id(ws), ws)) # Fast exit in case no transfers are necessary or possible if not senders or not recipients: @@ -5691,14 +5696,14 @@ def _rebalance_find_msgs( heapq.heapify(recipients) while senders and recipients: - snd_nbytes_max, snd_nbytes_min, _, snd_ws, ts_iter = senders[0] + snd_bytes_max, snd_bytes_min, _, snd_ws, ts_iter = senders[0] # Iterate through tasks in memory, least recently inserted first for ts in ts_iter: if keys is not None and ts.key not in keys: continue nbytes = ts.nbytes - if nbytes + snd_nbytes_max > 0: + if nbytes + snd_bytes_max > 0: # Moving this task would cause the sender to go below mean # and potentially risk becoming a recipient, which would # cause tasks to bounce around. Move on to the next task @@ -5713,9 +5718,9 @@ def _rebalance_find_msgs( skipped_recipients = [] use_recipient = False while recipients and not use_recipient: - rec_nbytes_max, rec_nbytes_min, _, rec_ws = recipients[0] - if nbytes + rec_nbytes_max > 0: - # recipients are sorted by rec_nbytes_max. + rec_bytes_max, rec_bytes_min, _, rec_ws = recipients[0] + if nbytes + rec_bytes_max > 0: + # recipients are sorted by rec_bytes_max. # The next ones will be worse; no reason to continue iterating break use_recipient = ts.key not in rec_ws._has_what @@ -5734,25 +5739,31 @@ def _rebalance_find_msgs( # still has bytes to lose, push it back into the senders heap. # It may or may not come back on top again. msgs.append((snd_ws, rec_ws, ts)) - # nbytes_max/nbytes_min are all negative for heap sorting - snd_nbytes_max += nbytes - snd_nbytes_min += nbytes - rec_nbytes_max += nbytes - rec_nbytes_min += nbytes - if snd_nbytes_min < 0: + # *_bytes_max/min are all negative for heap sorting + snd_bytes_max += nbytes + snd_bytes_min += nbytes + rec_bytes_max += nbytes + rec_bytes_min += nbytes + if snd_bytes_min < 0: # See definition of senders above heapq.heapreplace( senders, - (snd_nbytes_max, snd_nbytes_min, id(snd_ws), snd_ws, ts_iter), + ( + snd_bytes_max, + snd_bytes_min, + id(snd_ws), + snd_ws, + ts_iter, + ), ) else: heapq.heappop(senders) - if rec_nbytes_min < 0: + if rec_bytes_min < 0: # See definition of recipients above heapq.heapreplace( recipients, - (rec_nbytes_max, rec_nbytes_min, id(rec_ws), rec_ws), + (rec_bytes_max, rec_bytes_min, id(rec_ws), rec_ws), ) else: heapq.heappop(recipients) From 1ad35ea7a77ca2ef3603207f37bcc10696567c1c Mon Sep 17 00:00:00 2001 From: crusaderky Date: Mon, 10 May 2021 13:49:09 +0100 Subject: [PATCH 13/31] suggest tweaking malloc_trim --- distributed/distributed.yaml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index b0bc6bd68d4..333a0188040 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -102,6 +102,12 @@ distributed: # Only consider the data allocated by dask in RAM. Recommended if RAM is not # released in a timely fashion back to the OS after the Python objects are # dereferenced, but remains available for reuse by PyMalloc. + # + # If this is your problem on Linux, you should alternatively consider + # setting the MALLOC_TRIM_THRESHOLD_ environment variable (note the final + # underscore) to a low value; refer to the mallopt man page and to the + # comments about M_TRIM_THRESHOLD on + # https://sourceware.org/git/?p=glibc.git;a=blob;f=malloc/malloc.c # managed # Only consider data allocated by dask, including that spilled to disk. # Recommended if disk occupation of the spill file is an issue. From 1ad9d51b670091b321ae1d4051ab9e7942bb4ef0 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Tue, 11 May 2021 15:29:59 +0100 Subject: [PATCH 14/31] self-review --- distributed/distributed-schema.yaml | 2 +- distributed/scheduler.py | 70 +++++++++++++++-------------- 2 files changed, 37 insertions(+), 35 deletions(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index 27fb561813f..2ea3866d4e4 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -407,7 +407,7 @@ properties: rebalance: type: object description: >- - Settings for data rebalance operations + Settings for memory rebalance operations properties: measure: enum: diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 884666308b8..57021abc297 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -497,8 +497,8 @@ class WorkerState: _bandwidth: double _executing: dict _extra: dict - # _has_what is a dict with all values set to None as we rely on the property of - # Python >=3.7 dicts to be insertion-sorted. + # _has_what is a dict with all values set to None as rebalance() relies on the + # property of Python >=3.7 dicts to be insertion-sorted. _has_what: dict _hash: Py_hash_t _last_seen: double @@ -5645,15 +5645,15 @@ def _rebalance_find_msgs( Returns list of tuples to feed into _rebalance_move_data: - - sender worker - - recipient worker - - task to be transferred + - sender worker + - recipient worker + - task to be transferred """ ts: TaskState ws: WorkerState - # Heaps of workers that need to send/receive data, with how many bytes each - # needs to send/receive. + # Heaps of workers, managed by the heapq module, that need to send/receive data, + # with how many bytes each needs to send/receive. # # Each element of the heap is a tuple constructed as follows: # - snd_bytes_max/rec_bytes_max: maximum number of bytes to send or receive. @@ -5665,18 +5665,20 @@ def _rebalance_find_msgs( # are never used for sorting in the unlikely event that two processes have # exactly the same number of bytes allocated. # - WorkerState - # - iterator of all tasks in memory on the worker (senders only). - # Note that this iterator of keys will typically *not* be exhausted. It will - # only be exhausted if, after moving away from the worker all keys that can be - # moved, is insufficient to drop snd_bytes_min above 0. + # - iterator of all tasks in memory on the worker (senders only), insertion + # sorted (least recently inserted first). + # Note that this iterator will typically *not* be exhausted. It will only be + # exhausted if, after moving away from the worker all keys that can be moved, + # is insufficient to drop snd_bytes_min above 0. senders: "list[tuple[int, int, int, WorkerState, Iterator[TaskState]]]" = [] recipients: "list[tuple[int, int, int, WorkerState]]" = [] # Output: [(sender, recipient, task), ...] msgs: "list[tuple[WorkerState, WorkerState, TaskState]]" = [] - # optimistic memory = RSS - unmanaged memory that appeared over the last 30 - # seconds (distributed.worker.memory.recent-to-old-time). + # By default, this is the optimistic memory, meaning total process memory minus + # unmanaged memory that appeared over the last 30 seconds + # (distributed.worker.memory.recent-to-old-time). # This lets us ignore temporary spikes caused by task heap usage. memory_by_worker = [ (ws, getattr(ws.memory, MEMORY_REBALANCE_MEASURE)) for ws in workers @@ -5737,17 +5739,16 @@ def _rebalance_find_msgs( continue nbytes = ts.nbytes if nbytes + snd_bytes_max > 0: - # Moving this task would cause the sender to go below mean - # and potentially risk becoming a recipient, which would - # cause tasks to bounce around. Move on to the next task - # of the same sender. + # Moving this task would cause the sender to go below mean and + # potentially risk becoming a recipient, which would cause tasks to + # bounce around. Move on to the next task of the same sender. continue # Find the recipient, farthest from the mean, which - # 1. has enough available RAM for this task - # 2. don't hold a copy already - # There may not be any that satisfies these conditions; in this - # case this task won't be moved. + # 1. has enough available RAM for this task, and + # 2. doesn't hold a copy of this task already + # There may not be any that satisfies these conditions; in this case + # this task won't be moved. skipped_recipients = [] use_recipient = False while recipients and not use_recipient: @@ -5764,34 +5765,33 @@ def _rebalance_find_msgs( heapq.heappush(recipients, recipient) if not use_recipient: - # This task has no recipients available. Leave it on the - # sender and move on to the next task of the same sender + # This task has no recipients available. Leave it on the sender and + # move on to the next task of the same sender. continue - # Stop iterating on the tasks of this sender for now and, if it - # still has bytes to lose, push it back into the senders heap. - # It may or may not come back on top again. + # Schedule task for transfer from sender to receiver msgs.append((snd_ws, rec_ws, ts)) + # *_bytes_max/min are all negative for heap sorting snd_bytes_max += nbytes snd_bytes_min += nbytes rec_bytes_max += nbytes rec_bytes_min += nbytes + + # Stop iterating on the tasks of this sender for now and, if it still + # has bytes to lose, push it back into the senders heap; it may or may + # not come back on top again. if snd_bytes_min < 0: # See definition of senders above heapq.heapreplace( senders, - ( - snd_bytes_max, - snd_bytes_min, - id(snd_ws), - snd_ws, - ts_iter, - ), + (snd_bytes_max, snd_bytes_min, id(snd_ws), snd_ws, ts_iter), ) else: heapq.heappop(senders) + # If receiver still has bytes to gain, push it back into the receivers + # heap; it may or may not come back on top again. if rec_bytes_min < 0: # See definition of recipients above heapq.heapreplace( @@ -5804,9 +5804,11 @@ def _rebalance_find_msgs( # Move to next sender with the most data to lose. # It may or may not be the same sender again. break - else: + + else: # for ts in ts_iter # Exhausted tasks on this sender heapq.heappop(senders) + return msgs async def _rebalance_move_data( From 32a1f322aefae1001f6e5eecfbf2215028172f00 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Tue, 11 May 2021 18:26:20 +0100 Subject: [PATCH 15/31] test_tls_functional --- distributed/tests/test_tls_functional.py | 36 +++++++++++++++++------- 1 file changed, 26 insertions(+), 10 deletions(-) diff --git a/distributed/tests/test_tls_functional.py b/distributed/tests/test_tls_functional.py index 59e0bbb429d..a0b374f502d 100644 --- a/distributed/tests/test_tls_functional.py +++ b/distributed/tests/test_tls_functional.py @@ -6,11 +6,9 @@ import pytest -from distributed import Client, Nanny, Queue, Scheduler, Worker, worker_client -from distributed.client import wait +from distributed import Client, Nanny, Queue, Scheduler, Worker, wait, worker_client from distributed.core import Status from distributed.metrics import time -from distributed.nanny import Nanny from distributed.utils_test import ( # noqa: F401 cleanup, double, @@ -101,16 +99,34 @@ async def test_nanny(c, s, a, b): assert result == 11 -@gen_tls_cluster(client=True) +@gen_tls_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "2GiB"}) async def test_rebalance(c, s, a, b): - x, y = await c._scatter([1, 2], workers=[a.address]) - assert len(a.data) == 2 - assert len(b.data) == 0 + # We used nannies to have separate processes for each worker + a_addr, _ = s.workers + assert a_addr.startswith("tls://") + + # Generate 10 buffers worth 1 GiB total on worker a. This sends its memory + # utilisation slightly above 50% (after counting unmanaged) which is above the + # distributed.worker.memory.rebalance.sender-min threshold. + futures = [ + c.submit(lambda: "x" * (2 ** 30 // 10), workers=[a_addr], pure=False) + for _ in range(10) + ] + await wait(futures) + + # Wait for heartbeats + while s.memory.process < 2 ** 30: + await asyncio.sleep(0.1) + + ndata = await c.run(lambda dask_worker: len(dask_worker.data)) + assert set(ndata.values()) == {10, 0} - await c._rebalance() + await c.rebalance() - assert len(a.data) == 1 - assert len(b.data) == 1 + ndata = await c.run(lambda dask_worker: len(dask_worker.data)) + # Allow for some uncertainty as the unmanaged memory is not stable + assert sum(ndata.values()) == 10 + assert 3 < next(iter(ndata.values())) < 7 @gen_tls_cluster(client=True, nthreads=[("tls://127.0.0.1", 2)] * 2) From 6557c52b3afa25e69edec24e5bb50f0c7cbc2c8b Mon Sep 17 00:00:00 2001 From: crusaderky Date: Wed, 12 May 2021 17:48:38 +0100 Subject: [PATCH 16/31] test_memory to use gen_cluster --- distributed/tests/test_scheduler.py | 174 ++++++++++++++-------------- 1 file changed, 88 insertions(+), 86 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 367e367c797..10d731fffe1 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2328,7 +2328,7 @@ def clear_leak(): gc.collect() -def assert_memory(scheduler_or_workerstate, attr: str, min_, max_, timeout=10): +async def assert_memory(scheduler_or_workerstate, attr: str, min_, max_, timeout=10): t0 = time() while True: minfo = scheduler_or_workerstate.memory @@ -2339,7 +2339,7 @@ def assert_memory(scheduler_or_workerstate, attr: str, min_, max_, timeout=10): raise TimeoutError( f"Expected {min_} MiB <= {attr} <= {max_} MiB; got:\n{minfo!r}" ) - sleep(0.1) + await asyncio.sleep(0.1) # This test is heavily influenced by hard-to-control factors such as memory management @@ -2347,93 +2347,95 @@ def assert_memory(scheduler_or_workerstate, attr: str, min_, max_, timeout=10): @pytest.mark.flaky(reruns=3, reruns_delay=5) # ~33s runtime, or distributed.memory.recent_to_old_time + 3s @pytest.mark.slow -def test_memory(): +@gen_cluster( + client=True, Worker=Nanny, worker_kwargs={"memory_limit": "500 MiB"}, timeout=60 +) +async def test_memory(c, s, *_): pytest.importorskip("zict") - with Client(n_workers=2, threads_per_worker=1, memory_limit=500 * 2 ** 20) as c: - c.wait_for_workers(2) - s = c.cluster.scheduler - a, b = s.workers.values() - - s_m0 = s.memory - assert s_m0.process == a.memory.process + b.memory.process - assert s_m0.managed == 0 - assert a.memory.managed == 0 - assert b.memory.managed == 0 - # When a worker first goes online, its RAM is immediately counted as - # unmanaged_old - assert_memory(s, "unmanaged_recent", 0, 40, timeout=0) - assert_memory(a, "unmanaged_recent", 0, 20, timeout=0) - assert_memory(b, "unmanaged_recent", 0, 20, timeout=0) - - f1 = c.submit(leaking, 100, 50, 5, pure=False, workers=[a.name]) - f2 = c.submit(leaking, 100, 50, 5, pure=False, workers=[b.name]) - assert_memory(s, "unmanaged_recent", 300, 380) - assert_memory(a, "unmanaged_recent", 150, 190) - assert_memory(b, "unmanaged_recent", 150, 190) - c.gather([f1, f2]) - - # On each worker, we now have 100 MiB managed + 50 MiB fresh leak - assert_memory(s, "managed_in_memory", 200, 201) - assert_memory(a, "managed_in_memory", 100, 101) - assert_memory(b, "managed_in_memory", 100, 101) - assert_memory(s, "unmanaged_recent", 100, 180) - assert_memory(a, "unmanaged_recent", 50, 90) - assert_memory(b, "unmanaged_recent", 50, 90) - - # Force the output of f1 and f2 to spill to disk. - # With target=0.6 and memory_limit=500 MiB, we'll start spilling at 300 MiB - # process memory per worker, or roughly after 3~7 rounds of the below depending - # on how much RAM the interpreter is using. - more_futs = [] - for _ in range(8): - if s.memory.managed_spilled > 0: - break - more_futs += [ - c.submit(leaking, 20, 0, 0, pure=False, workers=[a.name]), - c.submit(leaking, 20, 0, 0, pure=False, workers=[b.name]), - ] - sleep(2) - assert_memory(s, "managed_spilled", 1, 999) - # Wait for the spilling to finish. Note that this does not make the test take - # longer as we're waiting for recent_to_old_time anyway. - sleep(10) - - # Delete spilled keys - prev = s.memory - del f1 - del f2 - assert_memory(s, "managed_spilled", 0, prev.managed_spilled / 2 ** 20 - 19) - - # Empty the cluster, with the exception of leaked memory - del more_futs - assert_memory(s, "managed", 0, 0) - - orig_unmanaged = s_m0.unmanaged / 2 ** 20 - orig_old = s_m0.unmanaged_old / 2 ** 20 - - # Wait until 30s have passed since the spill to observe unmanaged_recent - # transition into unmanaged_old - c.run(gc.collect) - assert_memory(s, "unmanaged_recent", 0, 90, timeout=40) - assert_memory( - s, - "unmanaged_old", - orig_old + 90, - # On MacOS, the process memory of the Python interpreter does not shrink as - # fast as on Linux/Windows - 9999 if MACOS else orig_old + 190, - timeout=40, - ) + # WorkerState objects, as opposed to the Nanny objects passed by gen_cluster + a, b = s.workers.values() + + s_m0 = s.memory + assert s_m0.process == a.memory.process + b.memory.process + assert s_m0.managed == 0 + assert a.memory.managed == 0 + assert b.memory.managed == 0 + # When a worker first goes online, its RAM is immediately counted as + # unmanaged_old + await assert_memory(s, "unmanaged_recent", 0, 40, timeout=0) + await assert_memory(a, "unmanaged_recent", 0, 20, timeout=0) + await assert_memory(b, "unmanaged_recent", 0, 20, timeout=0) + + f1 = c.submit(leaking, 100, 50, 5, pure=False, workers=[a.name]) + f2 = c.submit(leaking, 100, 50, 5, pure=False, workers=[b.name]) + await assert_memory(s, "unmanaged_recent", 300, 380) + await assert_memory(a, "unmanaged_recent", 150, 190) + await assert_memory(b, "unmanaged_recent", 150, 190) + await wait([f1, f2]) + + # On each worker, we now have 100 MiB managed + 50 MiB fresh leak + await assert_memory(s, "managed_in_memory", 200, 201) + await assert_memory(a, "managed_in_memory", 100, 101) + await assert_memory(b, "managed_in_memory", 100, 101) + await assert_memory(s, "unmanaged_recent", 100, 180) + await assert_memory(a, "unmanaged_recent", 50, 90) + await assert_memory(b, "unmanaged_recent", 50, 90) + + # Force the output of f1 and f2 to spill to disk. + # With target=0.6 and memory_limit=500 MiB, we'll start spilling at 300 MiB + # process memory per worker, or roughly after 3~7 rounds of the below depending + # on how much RAM the interpreter is using. + more_futs = [] + for _ in range(8): + if s.memory.managed_spilled > 0: + break + more_futs += [ + c.submit(leaking, 20, 0, 0, pure=False, workers=[a.name]), + c.submit(leaking, 20, 0, 0, pure=False, workers=[b.name]), + ] + await asyncio.sleep(2) + await assert_memory(s, "managed_spilled", 1, 999) + + # Wait for the spilling to finish. Note that this does not make the test take + # longer as we're waiting for recent_to_old_time anyway. + await asyncio.sleep(10) + + # Delete spilled keys + prev = s.memory + del f1 + del f2 + await assert_memory(s, "managed_spilled", 0, prev.managed_spilled / 2 ** 20 - 19) + + # Empty the cluster, with the exception of leaked memory + del more_futs + await assert_memory(s, "managed", 0, 0) + + orig_unmanaged = s_m0.unmanaged / 2 ** 20 + orig_old = s_m0.unmanaged_old / 2 ** 20 + + # Wait until 30s have passed since the spill to observe unmanaged_recent + # transition into unmanaged_old + await c.run(gc.collect) + await assert_memory(s, "unmanaged_recent", 0, 90, timeout=40) + await assert_memory( + s, + "unmanaged_old", + orig_old + 90, + # On MacOS, the process memory of the Python interpreter does not shrink as + # fast as on Linux/Windows + 9999 if MACOS else orig_old + 190, + timeout=40, + ) - # When the leaked memory is cleared, unmanaged and unmanaged_old drop - # On MacOS, the process memory of the Python interpreter does not shrink as fast - # as on Linux/Windows - if not MACOS: - c.run(clear_leak) - assert_memory(s, "unmanaged", 0, orig_unmanaged + 95) - assert_memory(s, "unmanaged_old", 0, orig_old + 95) - assert_memory(s, "unmanaged_recent", 0, 90) + # When the leaked memory is cleared, unmanaged and unmanaged_old drop + # On MacOS, the process memory of the Python interpreter does not shrink as fast + # as on Linux/Windows + if not MACOS: + await c.run(clear_leak) + await assert_memory(s, "unmanaged", 0, orig_unmanaged + 95) + await assert_memory(s, "unmanaged_old", 0, orig_old + 95) + await assert_memory(s, "unmanaged_recent", 0, 90) @gen_cluster(client=True, worker_kwargs={"memory_limit": 0}) From 5a5a775761413a115c01e4c773cac9234d7884ae Mon Sep 17 00:00:00 2001 From: crusaderky Date: Wed, 12 May 2021 17:51:55 +0100 Subject: [PATCH 17/31] test_memory to use gen_cluster --- distributed/tests/test_scheduler.py | 174 ++++++++++++++-------------- 1 file changed, 88 insertions(+), 86 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 4e1b0a47b36..2aaf154a7dd 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2328,7 +2328,7 @@ def clear_leak(): gc.collect() -def assert_memory(scheduler_or_workerstate, attr: str, min_, max_, timeout=10): +async def assert_memory(scheduler_or_workerstate, attr: str, min_, max_, timeout=10): t0 = time() while True: minfo = scheduler_or_workerstate.memory @@ -2339,7 +2339,7 @@ def assert_memory(scheduler_or_workerstate, attr: str, min_, max_, timeout=10): raise TimeoutError( f"Expected {min_} MiB <= {attr} <= {max_} MiB; got:\n{minfo!r}" ) - sleep(0.1) + await asyncio.sleep(0.1) # This test is heavily influenced by hard-to-control factors such as memory management @@ -2347,93 +2347,95 @@ def assert_memory(scheduler_or_workerstate, attr: str, min_, max_, timeout=10): @pytest.mark.flaky(reruns=3, reruns_delay=5) # ~33s runtime, or distributed.memory.recent-to-old-time + 3s @pytest.mark.slow -def test_memory(): +@gen_cluster( + client=True, Worker=Nanny, worker_kwargs={"memory_limit": "500 MiB"}, timeout=60 +) +async def test_memory(c, s, *_): pytest.importorskip("zict") - with Client(n_workers=2, threads_per_worker=1, memory_limit=500 * 2 ** 20) as c: - c.wait_for_workers(2) - s = c.cluster.scheduler - a, b = s.workers.values() - - s_m0 = s.memory - assert s_m0.process == a.memory.process + b.memory.process - assert s_m0.managed == 0 - assert a.memory.managed == 0 - assert b.memory.managed == 0 - # When a worker first goes online, its RAM is immediately counted as - # unmanaged_old - assert_memory(s, "unmanaged_recent", 0, 40, timeout=0) - assert_memory(a, "unmanaged_recent", 0, 20, timeout=0) - assert_memory(b, "unmanaged_recent", 0, 20, timeout=0) - - f1 = c.submit(leaking, 100, 50, 5, pure=False, workers=[a.name]) - f2 = c.submit(leaking, 100, 50, 5, pure=False, workers=[b.name]) - assert_memory(s, "unmanaged_recent", 300, 380) - assert_memory(a, "unmanaged_recent", 150, 190) - assert_memory(b, "unmanaged_recent", 150, 190) - c.gather([f1, f2]) - - # On each worker, we now have 100 MiB managed + 50 MiB fresh leak - assert_memory(s, "managed_in_memory", 200, 201) - assert_memory(a, "managed_in_memory", 100, 101) - assert_memory(b, "managed_in_memory", 100, 101) - assert_memory(s, "unmanaged_recent", 100, 180) - assert_memory(a, "unmanaged_recent", 50, 90) - assert_memory(b, "unmanaged_recent", 50, 90) - - # Force the output of f1 and f2 to spill to disk. - # With target=0.6 and memory_limit=500 MiB, we'll start spilling at 300 MiB - # process memory per worker, or roughly after 3~7 rounds of the below depending - # on how much RAM the interpreter is using. - more_futs = [] - for _ in range(8): - if s.memory.managed_spilled > 0: - break - more_futs += [ - c.submit(leaking, 20, 0, 0, pure=False, workers=[a.name]), - c.submit(leaking, 20, 0, 0, pure=False, workers=[b.name]), - ] - sleep(2) - assert_memory(s, "managed_spilled", 1, 999) - # Wait for the spilling to finish. Note that this does not make the test take - # longer as we're waiting for recent-to-old-time anyway. - sleep(10) - - # Delete spilled keys - prev = s.memory - del f1 - del f2 - assert_memory(s, "managed_spilled", 0, prev.managed_spilled / 2 ** 20 - 19) - - # Empty the cluster, with the exception of leaked memory - del more_futs - assert_memory(s, "managed", 0, 0) - - orig_unmanaged = s_m0.unmanaged / 2 ** 20 - orig_old = s_m0.unmanaged_old / 2 ** 20 - - # Wait until 30s have passed since the spill to observe unmanaged_recent - # transition into unmanaged_old - c.run(gc.collect) - assert_memory(s, "unmanaged_recent", 0, 90, timeout=40) - assert_memory( - s, - "unmanaged_old", - orig_old + 90, - # On MacOS, the process memory of the Python interpreter does not shrink as - # fast as on Linux/Windows - 9999 if MACOS else orig_old + 190, - timeout=40, - ) + # WorkerState objects, as opposed to the Nanny objects passed by gen_cluster + a, b = s.workers.values() + + s_m0 = s.memory + assert s_m0.process == a.memory.process + b.memory.process + assert s_m0.managed == 0 + assert a.memory.managed == 0 + assert b.memory.managed == 0 + # When a worker first goes online, its RAM is immediately counted as + # unmanaged_old + await assert_memory(s, "unmanaged_recent", 0, 40, timeout=0) + await assert_memory(a, "unmanaged_recent", 0, 20, timeout=0) + await assert_memory(b, "unmanaged_recent", 0, 20, timeout=0) + + f1 = c.submit(leaking, 100, 50, 5, pure=False, workers=[a.name]) + f2 = c.submit(leaking, 100, 50, 5, pure=False, workers=[b.name]) + await assert_memory(s, "unmanaged_recent", 300, 380) + await assert_memory(a, "unmanaged_recent", 150, 190) + await assert_memory(b, "unmanaged_recent", 150, 190) + await wait([f1, f2]) + + # On each worker, we now have 100 MiB managed + 50 MiB fresh leak + await assert_memory(s, "managed_in_memory", 200, 201) + await assert_memory(a, "managed_in_memory", 100, 101) + await assert_memory(b, "managed_in_memory", 100, 101) + await assert_memory(s, "unmanaged_recent", 100, 180) + await assert_memory(a, "unmanaged_recent", 50, 90) + await assert_memory(b, "unmanaged_recent", 50, 90) + + # Force the output of f1 and f2 to spill to disk. + # With target=0.6 and memory_limit=500 MiB, we'll start spilling at 300 MiB + # process memory per worker, or roughly after 3~7 rounds of the below depending + # on how much RAM the interpreter is using. + more_futs = [] + for _ in range(8): + if s.memory.managed_spilled > 0: + break + more_futs += [ + c.submit(leaking, 20, 0, 0, pure=False, workers=[a.name]), + c.submit(leaking, 20, 0, 0, pure=False, workers=[b.name]), + ] + await asyncio.sleep(2) + await assert_memory(s, "managed_spilled", 1, 999) + + # Wait for the spilling to finish. Note that this does not make the test take + # longer as we're waiting for recent-to-old-time anyway. + await asyncio.sleep(10) + + # Delete spilled keys + prev = s.memory + del f1 + del f2 + await assert_memory(s, "managed_spilled", 0, prev.managed_spilled / 2 ** 20 - 19) + + # Empty the cluster, with the exception of leaked memory + del more_futs + await assert_memory(s, "managed", 0, 0) + + orig_unmanaged = s_m0.unmanaged / 2 ** 20 + orig_old = s_m0.unmanaged_old / 2 ** 20 + + # Wait until 30s have passed since the spill to observe unmanaged_recent + # transition into unmanaged_old + await c.run(gc.collect) + await assert_memory(s, "unmanaged_recent", 0, 90, timeout=40) + await assert_memory( + s, + "unmanaged_old", + orig_old + 90, + # On MacOS, the process memory of the Python interpreter does not shrink as + # fast as on Linux/Windows + 9999 if MACOS else orig_old + 190, + timeout=40, + ) - # When the leaked memory is cleared, unmanaged and unmanaged_old drop - # On MacOS, the process memory of the Python interpreter does not shrink as fast - # as on Linux/Windows - if not MACOS: - c.run(clear_leak) - assert_memory(s, "unmanaged", 0, orig_unmanaged + 95) - assert_memory(s, "unmanaged_old", 0, orig_old + 95) - assert_memory(s, "unmanaged_recent", 0, 90) + # When the leaked memory is cleared, unmanaged and unmanaged_old drop + # On MacOS, the process memory of the Python interpreter does not shrink as fast + # as on Linux/Windows + if not MACOS: + await c.run(clear_leak) + await assert_memory(s, "unmanaged", 0, orig_unmanaged + 95) + await assert_memory(s, "unmanaged_old", 0, orig_old + 95) + await assert_memory(s, "unmanaged_recent", 0, 90) @gen_cluster(client=True, worker_kwargs={"memory_limit": 0}) From fbeda366a23c9833ab7bc5ca8d7d61f9132132bf Mon Sep 17 00:00:00 2001 From: crusaderky Date: Wed, 12 May 2021 17:56:39 +0100 Subject: [PATCH 18/31] half memory --- distributed/tests/test_tls_functional.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/distributed/tests/test_tls_functional.py b/distributed/tests/test_tls_functional.py index a0b374f502d..ef6aa5d6727 100644 --- a/distributed/tests/test_tls_functional.py +++ b/distributed/tests/test_tls_functional.py @@ -99,23 +99,23 @@ async def test_nanny(c, s, a, b): assert result == 11 -@gen_tls_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "2GiB"}) +@gen_tls_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) async def test_rebalance(c, s, a, b): # We used nannies to have separate processes for each worker a_addr, _ = s.workers assert a_addr.startswith("tls://") - # Generate 10 buffers worth 1 GiB total on worker a. This sends its memory + # Generate 10 buffers worth 512 MiB total on worker a. This sends its memory # utilisation slightly above 50% (after counting unmanaged) which is above the # distributed.worker.memory.rebalance.sender-min threshold. futures = [ - c.submit(lambda: "x" * (2 ** 30 // 10), workers=[a_addr], pure=False) + c.submit(lambda: "x" * (2 ** 29 // 10), workers=[a_addr], pure=False) for _ in range(10) ] await wait(futures) # Wait for heartbeats - while s.memory.process < 2 ** 30: + while s.memory.process < 2 ** 29: await asyncio.sleep(0.1) ndata = await c.run(lambda dask_worker: len(dask_worker.data)) From d5708d4e0e7919b867dddd7e78b47cbde8f55cc9 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 13 May 2021 12:39:49 +0100 Subject: [PATCH 19/31] tests --- distributed/client.py | 2 +- distributed/scheduler.py | 6 +- distributed/tests/test_client.py | 132 ++++++++++++++--------- distributed/tests/test_scheduler.py | 101 +++++++++++++++++ distributed/tests/test_tls_functional.py | 14 +-- 5 files changed, 194 insertions(+), 61 deletions(-) diff --git a/distributed/client.py b/distributed/client.py index bef13b81094..17f5f3bd427 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -3066,7 +3066,7 @@ async def _rebalance(self, futures=None, workers=None): keys = None result = await self.scheduler.rebalance(keys=keys, workers=workers) if result["status"] == "missing-data": - raise ValueError( + raise KeyError( f"During rebalance {len(result['keys'])} keys were found to be missing" ) assert result["status"] == "OK" diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 57021abc297..57cd54274b0 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -5607,7 +5607,11 @@ async def rebalance( keys = set(keys) # unless already a set-like if not keys: return {"status": "OK"} - missing_data = [k for k in keys if not self._tasks[k]._who_has] + missing_data = [ + k + for k in keys + if k not in self._tasks or not self._tasks[k]._who_has + ] if missing_data: return {"status": "missing-data", "keys": missing_data} diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index 57bdc67b17e..a77dbfd43a1 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -2900,70 +2900,101 @@ def __reduce__(self): assert False -@gen_cluster(client=True) -async def test_rebalance(c, s, a, b): - aws = s.workers[a.address] - bws = s.workers[b.address] +@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) +async def test_rebalance(c, s, *_): + """Test Client.rebalance(). These are just to test the Client wrapper around + Scheduler.rebalance(); for more thorough tests on the latter see test_scheduler.py. + """ + # We used nannies to have separate processes for each worker + a, b = s.workers + + # Generate 10 buffers worth 512 MiB total on worker a. This sends its memory + # utilisation slightly above 50% (after counting unmanaged) which is above the + # distributed.worker.memory.rebalance.sender-min threshold. + futures = [ + c.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) + for _ in range(10) + ] + await wait(futures) + # Wait for heartbeats + while s.memory.process < 2 ** 29: + await asyncio.sleep(0.1) - x, y = await c.scatter([1, 2], workers=[a.address]) - assert len(a.data) == 2 - assert len(b.data) == 0 + assert await c.run(lambda dask_worker: len(dask_worker.data)) == {a: 10, b: 0} - s.validate_state() await c.rebalance() - s.validate_state() - assert len(b.data) == 1 - assert {ts.key for ts in bws.has_what} == set(b.data) - assert bws in s.tasks[x.key].who_has or bws in s.tasks[y.key].who_has + ndata = await c.run(lambda dask_worker: len(dask_worker.data)) + # Allow for some uncertainty as the unmanaged memory is not stable + assert sum(ndata.values()) == 10 + assert 3 <= ndata[a] <= 7 + assert 3 <= ndata[b] <= 7 - assert len(a.data) == 1 - assert {ts.key for ts in aws.has_what} == set(a.data) - assert aws not in s.tasks[x.key].who_has or aws not in s.tasks[y.key].who_has +@gen_cluster( + nthreads=[("127.0.0.1", 1)] * 3, + client=True, + Worker=Nanny, + worker_kwargs={"memory_limit": "1 GiB"}, +) +async def test_rebalance_workers_and_keys(client, s, *_): + """Test Client.rebalance(). These are just to test the Client wrapper around + Scheduler.rebalance(); for more thorough tests on the latter see test_scheduler.py. + """ + a, b, c = s.workers + futures = [ + client.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) + for _ in range(10) + ] + await wait(futures) + # Wait for heartbeats + while s.memory.process < 2 ** 29: + await asyncio.sleep(0.1) -@gen_cluster(nthreads=[("127.0.0.1", 1)] * 4, client=True) -async def test_rebalance_workers(e, s, a, b, c, d): - w, x, y, z = await 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 - - await 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} - - await 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() + # Passing empty iterables is not the same as omitting the arguments + await client.rebalance([]) + await client.rebalance(workers=[]) + assert await client.run(lambda dask_worker: len(dask_worker.data)) == { + a: 10, + b: 0, + c: 0, + } + # Limit rebalancing to two arbitrary keys and two arbitrary workers. + await client.rebalance([futures[3], futures[7]], [a, b]) + assert await client.run(lambda dask_worker: len(dask_worker.data)) == { + a: 8, + b: 2, + c: 0, + } -@gen_cluster(client=True) -async def test_rebalance_execution(c, s, a, b): - futures = c.map(inc, range(10), workers=a.address) - await c.rebalance(futures) - assert len(a.data) == len(b.data) == 5 - s.validate_state() + with pytest.raises(KeyError): + await client.rebalance(workers=["notexist"]) +@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) def test_rebalance_sync(c, s, a, b): - futures = c.map(inc, range(10), workers=[a["address"]]) - c.rebalance(futures) + a, b = s.workers + futures = [ + c.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) + for _ in range(10) + ] + # Wait for heartbeat + while s.memory.process < 2 ** 29: + sleep(0.1) - has_what = c.has_what() - assert len(has_what) == 2 - assert list(valmap(len, has_what).values()) == [5, 5] + assert c.run(lambda dask_worker: len(dask_worker.data)) == {a: 10, b: 0} + c.rebalance() + ndata = c.run(lambda dask_worker: len(dask_worker.data)) + # Allow for some uncertainty as the unmanaged memory is not stable + assert sum(ndata.values()) == 10 + assert 5 <= ndata[a] <= 6 + assert 4 <= ndata[b] <= 5 @gen_cluster(client=True) async def test_rebalance_unprepared(c, s, a, b): + """Client.rebalance() internally waits for unfinished futures""" futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) await asyncio.sleep(0.1) await c.rebalance(futures) @@ -2972,11 +3003,8 @@ async def test_rebalance_unprepared(c, s, a, b): @gen_cluster(client=True) async def test_rebalance_raises_missing_data(c, s, a, b): - with pytest.raises(ValueError, match="keys were found to be missing"): - futures = await c.scatter(range(100)) - keys = [f.key for f in futures] - del futures - await c.rebalance(keys) + with pytest.raises(KeyError, match="keys were found to be missing"): + await c.rebalance(["notexist"]) @gen_cluster(client=True) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 2aaf154a7dd..52db2f6ca1a 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2496,3 +2496,104 @@ async def test_close_scheduler__close_workers_Nanny(s, a, b): await asyncio.sleep(0.05) log = log.getvalue() assert "retry" not in log + + +async def assert_ndata(client, by_addr, total=None): + """Test that the number of elements in Worker.data is as expected. + To be used when the worker is wrapped by a nanny. + + by_addr: dict of either exact numbers or (min, max) tuples + total: optional exact match on the total number of keys (with duplicates) across all + workers + """ + out = await client.run(lambda dask_worker: len(dask_worker.data)) + try: + for k, v in by_addr.items(): + if isinstance(v, tuple): + assert v[0] <= out[k] <= v[1] + else: + assert out[k] == v + if total is not None: + assert sum(out.values()) == total + except AssertionError: + raise AssertionError(f"Expected {by_addr}, total={total}; got {out}") + + +@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) +async def test_rebalance(c, s, *_): + # We used nannies to have separate processes for each worker + a, b = s.workers + + # Generate 10 buffers worth 512 MiB total on worker a. This sends its memory + # utilisation slightly above 50% (after counting unmanaged) which is above the + # distributed.worker.memory.rebalance.sender-min threshold. + futures = [ + c.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) + for _ in range(10) + ] + await wait(futures) + # Wait for heartbeats + await assert_memory(s, "process", 512, 1024) + await assert_ndata(c, {a: 10, b: 0}) + await s.rebalance() + # Allow for some uncertainty as the unmanaged memory is not stable + await assert_ndata(c, {a: (3, 7), b: (3, 7)}, total=10) + + # rebalance() when there is nothing to do + await s.rebalance() + await assert_ndata(c, {a: (3, 7), b: (3, 7)}, total=10) + s.validate_state() + + +@gen_cluster( + nthreads=[("127.0.0.1", 1)] * 3, + client=True, + Worker=Nanny, + worker_kwargs={"memory_limit": "1 GiB"}, +) +async def test_rebalance_workers_and_keys(client, s, *_): + a, b, c = s.workers + futures = [ + client.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) + for _ in range(10) + ] + await wait(futures) + # Wait for heartbeats + await assert_memory(s, "process", 512, 1024) + + # Passing empty iterables is not the same as omitting the arguments + await s.rebalance(keys=[]) + await assert_ndata(client, {a: 10, b: 0, c: 0}) + await s.rebalance(workers=[]) + await assert_ndata(client, {a: 10, b: 0, c: 0}) + # Limit operation to workers that have nothing to do + await s.rebalance(workers=[b, c]) + await assert_ndata(client, {a: 10, b: 0, c: 0}) + + # Limit rebalancing to two arbitrary keys and two arbitrary workers + await s.rebalance(keys=[futures[3].key, futures[7].key], workers=[a, b]) + await assert_ndata(client, {a: 8, b: 2, c: 0}, total=10) + + with pytest.raises(KeyError): + await s.rebalance(workers=["notexist"]) + + s.validate_state() + + +@gen_cluster(client=True) +async def test_rebalance_missing_data(c, s, a, b): + out = await s.rebalance(keys=["notexist"]) + assert out == {"status": "missing-data", "keys": ["notexist"]} + + futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) + await asyncio.sleep(0.1) + out = await s.rebalance(keys=[f.key for f in futures]) + assert out["status"] == "missing-data" + assert 8 <= len(out["keys"]) <= 10 + s.validate_state() + + +@gen_cluster(nthreads=[]) +async def test_rebalance_no_workers(s): + await s.rebalance() + s.validate_state() diff --git a/distributed/tests/test_tls_functional.py b/distributed/tests/test_tls_functional.py index ef6aa5d6727..c6962f496c2 100644 --- a/distributed/tests/test_tls_functional.py +++ b/distributed/tests/test_tls_functional.py @@ -100,16 +100,16 @@ async def test_nanny(c, s, a, b): @gen_tls_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) -async def test_rebalance(c, s, a, b): +async def test_rebalance(c, s, *_): # We used nannies to have separate processes for each worker - a_addr, _ = s.workers - assert a_addr.startswith("tls://") + a, b = s.workers + assert a.startswith("tls://") # Generate 10 buffers worth 512 MiB total on worker a. This sends its memory # utilisation slightly above 50% (after counting unmanaged) which is above the # distributed.worker.memory.rebalance.sender-min threshold. futures = [ - c.submit(lambda: "x" * (2 ** 29 // 10), workers=[a_addr], pure=False) + c.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) for _ in range(10) ] await wait(futures) @@ -118,15 +118,15 @@ async def test_rebalance(c, s, a, b): while s.memory.process < 2 ** 29: await asyncio.sleep(0.1) - ndata = await c.run(lambda dask_worker: len(dask_worker.data)) - assert set(ndata.values()) == {10, 0} + assert await c.run(lambda dask_worker: len(dask_worker.data)) == {a: 10, b: 0} await c.rebalance() ndata = await c.run(lambda dask_worker: len(dask_worker.data)) # Allow for some uncertainty as the unmanaged memory is not stable assert sum(ndata.values()) == 10 - assert 3 < next(iter(ndata.values())) < 7 + assert 3 <= ndata[a] <= 7 + assert 3 <= ndata[b] <= 7 @gen_tls_cluster(client=True, nthreads=[("tls://127.0.0.1", 2)] * 2) From 535342fabc77926639017a2efdbcf67da42d3a95 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 13 May 2021 12:53:58 +0100 Subject: [PATCH 20/31] tests --- distributed/tests/test_client.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index a77dbfd43a1..60e299acbc5 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -2972,15 +2972,17 @@ async def test_rebalance_workers_and_keys(client, s, *_): await client.rebalance(workers=["notexist"]) -@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) -def test_rebalance_sync(c, s, a, b): - a, b = s.workers +def test_rebalance_sync(c): + a, b = c.run(lambda dask_worker: dask_worker.address).values() futures = [ c.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) for _ in range(10) ] # Wait for heartbeat - while s.memory.process < 2 ** 29: + while ( + c.run_on_scheduler(lambda dask_scheduler: dask_scheduler.memory.process) + < 2 ** 29 + ): sleep(0.1) assert c.run(lambda dask_worker: len(dask_worker.data)) == {a: 10, b: 0} @@ -2988,8 +2990,8 @@ def test_rebalance_sync(c, s, a, b): ndata = c.run(lambda dask_worker: len(dask_worker.data)) # Allow for some uncertainty as the unmanaged memory is not stable assert sum(ndata.values()) == 10 - assert 5 <= ndata[a] <= 6 - assert 4 <= ndata[b] <= 5 + assert 3 <= ndata[a] <= 7 + assert 3 <= ndata[b] <= 7 @gen_cluster(client=True) From efc96f6bb3ee246bcc91ab10cbfd6acfde6ed0b8 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 13 May 2021 14:24:27 +0100 Subject: [PATCH 21/31] tests --- distributed/tests/test_client.py | 32 ++++++++++---------- distributed/tests/test_scheduler.py | 37 +++++++++++++++++------- distributed/tests/test_tls_functional.py | 5 +--- 3 files changed, 45 insertions(+), 29 deletions(-) diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index 60e299acbc5..2d968ebc780 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -2911,10 +2911,7 @@ async def test_rebalance(c, s, *_): # Generate 10 buffers worth 512 MiB total on worker a. This sends its memory # utilisation slightly above 50% (after counting unmanaged) which is above the # distributed.worker.memory.rebalance.sender-min threshold. - futures = [ - c.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) - for _ in range(10) - ] + futures = c.map(lambda _: "x" * (2 ** 29 // 10), range(10), workers=[a]) await wait(futures) # Wait for heartbeats while s.memory.process < 2 ** 29: @@ -2942,10 +2939,7 @@ async def test_rebalance_workers_and_keys(client, s, *_): Scheduler.rebalance(); for more thorough tests on the latter see test_scheduler.py. """ a, b, c = s.workers - futures = [ - client.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) - for _ in range(10) - ] + futures = client.map(lambda _: "x" * (2 ** 29 // 10), range(10), workers=[a]) await wait(futures) # Wait for heartbeats while s.memory.process < 2 ** 29: @@ -2974,10 +2968,7 @@ async def test_rebalance_workers_and_keys(client, s, *_): def test_rebalance_sync(c): a, b = c.run(lambda dask_worker: dask_worker.address).values() - futures = [ - c.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) - for _ in range(10) - ] + futures = c.map(lambda _: "x" * (2 ** 29 // 10), range(10), workers=[a]) # Wait for heartbeat while ( c.run_on_scheduler(lambda dask_scheduler: dask_scheduler.memory.process) @@ -3003,10 +2994,21 @@ async def test_rebalance_unprepared(c, s, a, b): s.validate_state() -@gen_cluster(client=True) -async def test_rebalance_raises_missing_data(c, s, a, b): +@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) +async def test_rebalance_raises_missing_data(c, s, *_): + a, b = s.workers + futures = c.map(lambda _: "x" * (2 ** 29 // 10), range(10), workers=[a]) + await wait(futures) + # Wait for heartbeats + while s.memory.process < 2 ** 29: + await asyncio.sleep(0.1) + + # Descoping the futures enqueues a coroutine to release the data on the server + del futures with pytest.raises(KeyError, match="keys were found to be missing"): - await c.rebalance(["notexist"]) + # During the synchronous part of rebalance, the futures still exist, but they + # will be (partially) gone by the time the actual transferring happens. + await c.rebalance() @gen_cluster(client=True) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 52db2f6ca1a..80207b84744 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2527,10 +2527,7 @@ async def test_rebalance(c, s, *_): # Generate 10 buffers worth 512 MiB total on worker a. This sends its memory # utilisation slightly above 50% (after counting unmanaged) which is above the # distributed.worker.memory.rebalance.sender-min threshold. - futures = [ - c.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) - for _ in range(10) - ] + futures = c.map(lambda _: "x" * (2 ** 29 // 10), range(10), workers=[a]) await wait(futures) # Wait for heartbeats await assert_memory(s, "process", 512, 1024) @@ -2553,10 +2550,7 @@ async def test_rebalance(c, s, *_): ) async def test_rebalance_workers_and_keys(client, s, *_): a, b, c = s.workers - futures = [ - client.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) - for _ in range(10) - ] + futures = client.map(lambda _: "x" * (2 ** 29 // 10), range(10), workers=[a]) await wait(futures) # Wait for heartbeats await assert_memory(s, "process", 512, 1024) @@ -2580,11 +2574,17 @@ async def test_rebalance_workers_and_keys(client, s, *_): s.validate_state() -@gen_cluster(client=True) -async def test_rebalance_missing_data(c, s, a, b): +@gen_cluster() +async def test_rebalance_missing_data1(s, a, b): + """key never existed""" out = await s.rebalance(keys=["notexist"]) assert out == {"status": "missing-data", "keys": ["notexist"]} + s.validate_state() + +@gen_cluster(client=True) +async def test_rebalance_missing_data2(c, s, a, b): + """keys exist but belong to unfinished futures""" futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) await asyncio.sleep(0.1) out = await s.rebalance(keys=[f.key for f in futures]) @@ -2593,6 +2593,23 @@ async def test_rebalance_missing_data(c, s, a, b): s.validate_state() +@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) +async def test_rebalance_raises_missing_data3(c, s, *_): + """keys exist when the sync part of rebalance runs, but are gone by the time the + actual data movement runs + """ + a, _ = s.workers + futures = c.map(lambda _: "x" * (2 ** 29 // 10), range(10), workers=[a]) + await wait(futures) + # Wait for heartbeats + await assert_memory(s, "process", 512, 1024) + del futures + out = await s.rebalance() + assert out["status"] == "missing-data" + assert 1 <= len(out["keys"]) <= 10 + s.validate_state() + + @gen_cluster(nthreads=[]) async def test_rebalance_no_workers(s): await s.rebalance() diff --git a/distributed/tests/test_tls_functional.py b/distributed/tests/test_tls_functional.py index c6962f496c2..ae685f2c222 100644 --- a/distributed/tests/test_tls_functional.py +++ b/distributed/tests/test_tls_functional.py @@ -108,10 +108,7 @@ async def test_rebalance(c, s, *_): # Generate 10 buffers worth 512 MiB total on worker a. This sends its memory # utilisation slightly above 50% (after counting unmanaged) which is above the # distributed.worker.memory.rebalance.sender-min threshold. - futures = [ - c.submit(lambda: "x" * (2 ** 29 // 10), workers=[a], pure=False) - for _ in range(10) - ] + futures = c.map(lambda _: "x" * (2 ** 29 // 10), range(10), workers=[a]) await wait(futures) # Wait for heartbeats From 8818f99078533820c0c422c9c74c4a2ab92369b0 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 13 May 2021 14:34:54 +0100 Subject: [PATCH 22/31] tests --- distributed/tests/test_client.py | 32 ++++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index 2d968ebc780..1c00e355915 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -2966,23 +2966,23 @@ async def test_rebalance_workers_and_keys(client, s, *_): await client.rebalance(workers=["notexist"]) -def test_rebalance_sync(c): - a, b = c.run(lambda dask_worker: dask_worker.address).values() - futures = c.map(lambda _: "x" * (2 ** 29 // 10), range(10), workers=[a]) - # Wait for heartbeat - while ( - c.run_on_scheduler(lambda dask_scheduler: dask_scheduler.memory.process) - < 2 ** 29 - ): - sleep(0.1) +def test_rebalance_sync(): + # can't use the 'c' fixture because we need workers to run in a separate process + with Client(n_workers=2, memory_limit="1 GiB") as c: + s = c.cluster.scheduler + a, b = [ws.address for ws in s.workers.values()] + futures = c.map(lambda _: "x" * (2 ** 29 // 10), range(10), workers=[a]) + # Wait for heartbeat + while s.memory.process < 2 ** 29: + sleep(0.1) - assert c.run(lambda dask_worker: len(dask_worker.data)) == {a: 10, b: 0} - c.rebalance() - ndata = c.run(lambda dask_worker: len(dask_worker.data)) - # Allow for some uncertainty as the unmanaged memory is not stable - assert sum(ndata.values()) == 10 - assert 3 <= ndata[a] <= 7 - assert 3 <= ndata[b] <= 7 + assert c.run(lambda dask_worker: len(dask_worker.data)) == {a: 10, b: 0} + c.rebalance() + ndata = c.run(lambda dask_worker: len(dask_worker.data)) + # Allow for some uncertainty as the unmanaged memory is not stable + assert sum(ndata.values()) == 10 + assert 3 <= ndata[a] <= 7 + assert 3 <= ndata[b] <= 7 @gen_cluster(client=True) From b99e2209bae18e5a13d0ecb78a81fe04f643ffec Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 13 May 2021 15:48:14 +0100 Subject: [PATCH 23/31] make Cython happy --- distributed/scheduler.py | 36 ++++++++++++++++---------------- distributed/tests/test_client.py | 9 ++------ 2 files changed, 20 insertions(+), 25 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 57cd54274b0..e53f8a434b1 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -5594,11 +5594,13 @@ async def rebalance( All other workers will be ignored. The mean cluster occupancy will be calculated only using the whitelisted workers. """ + parent: SchedulerState = self + with log_errors(): if workers is not None: - workers = [self._workers_dv[w] for w in workers] + workers = [parent._workers_dv[w] for w in workers] else: - workers = self._workers_dv.values() + workers = parent._workers_dv.values() if not workers: return {"status": "OK"} @@ -5610,7 +5612,7 @@ async def rebalance( missing_data = [ k for k in keys - if k not in self._tasks or not self._tasks[k]._who_has + if k not in parent._tasks or not parent._tasks[k].who_has ] if missing_data: return {"status": "missing-data", "keys": missing_data} @@ -5734,6 +5736,9 @@ def _rebalance_find_msgs( heapq.heapify(senders) heapq.heapify(recipients) + snd_ws: WorkerState + rec_ws: WorkerState + while senders and recipients: snd_bytes_max, snd_bytes_min, _, snd_ws, ts_iter = senders[0] @@ -5824,7 +5829,8 @@ async def _rebalance_move_data( FIXME this method is not robust when the cluster is not idle. """ ts: TaskState - ws: WorkerState + snd_ws: WorkerState + rec_ws: WorkerState to_recipients = defaultdict(lambda: defaultdict(list)) to_senders = defaultdict(list) @@ -5863,19 +5869,13 @@ async def _rebalance_move_data( ), } - for sender, recipient, ts in msgs: + for snd_ws, rec_ws, ts in msgs: assert ts._state == "memory" - ts._who_has.add(recipient) - recipient._has_what[ts] = None - recipient.nbytes += ts.get_nbytes() + ts._who_has.add(rec_ws) + rec_ws._has_what[ts] = None + rec_ws.nbytes += ts.get_nbytes() self.log.append( - ( - "rebalance", - ts._key, - time(), - sender.address, - recipient.address, - ) + ("rebalance", ts._key, time(), snd_ws.address, rec_ws.address) ) await asyncio.gather( @@ -6212,7 +6212,7 @@ async def retire_workers( logger.info("Retire workers %s", workers) # Keys orphaned by retiring those workers - keys = {k for w in workers for k in w._has_what} + keys = {k for w in workers for k in w.has_what} keys = {ts._key for ts in keys if ts._who_has.issubset(workers)} if keys: @@ -6431,14 +6431,14 @@ def get_has_what(self, comm=None, workers=None): if workers is not None: workers = map(self.coerce_address, workers) return { - w: [ts._key for ts in parent._workers_dv[w]._has_what] + w: [ts._key for ts in parent._workers_dv[w].has_what] if w in parent._workers_dv else [] for w in workers } else: return { - w: [ts._key for ts in ws._has_what] + w: [ts._key for ts in ws.has_what] for w, ws in parent._workers_dv.items() } diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index 1c00e355915..071afdf566a 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -2891,13 +2891,8 @@ def __reduce__(self): raise BadlySerializedException("hello world") x = c.submit(f) - - try: - result = await x - except Exception as e: - assert "hello world" in str(e) - else: - assert False + with pytest.raises(Exception, match="hello world"): + await x @gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) From 4b1b16e403e253129e2ab8e69fb71269a32439fa Mon Sep 17 00:00:00 2001 From: crusaderky Date: Sun, 16 May 2021 22:19:35 +0100 Subject: [PATCH 24/31] test_rebalance_managed_memory --- distributed/tests/test_scheduler.py | 21 +++++++++++++++++++ distributed/utils_test.py | 31 +++++++++++++++++++++++++++++ 2 files changed, 52 insertions(+) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 80207b84744..fb09637532b 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -39,6 +39,7 @@ inc, loop, nodebug, + set_config_and_reload, slowadd, slowdec, slowinc, @@ -2542,6 +2543,26 @@ async def test_rebalance(c, s, *_): s.validate_state() +@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "250 MiB"}) +async def test_rebalance_managed_memory(c, s, a, b): + # Generate 100 buffers worth 100 MiB total on worker a. This sends its memory + # utilisation to exactly 40%, ignoring unmanaged, which is above the + # distributed.worker.memory.rebalance.sender-min threshold. + futures = c.map(lambda _: "x" * (2 ** 20), range(100), workers=[a.address]) + await wait(futures) + # Even if we're just using managed memory, which is instantaneously accounted for as + # soon as the tasks finish, MemoryState.managed is still capped by the process + # memory, so we need to wait for the heartbeat. + await assert_memory(s, "managed", 100, 101) + await assert_ndata(c, {a.address: 100, b.address: 0}) + await s.rebalance() + # We can expect an exact, stable result because we are completely bypassing the + # unpredictability of unmanaged memory. + await assert_ndata(c, {a.address: 62, b.address: 38}) + s.validate_state() + + @gen_cluster( nthreads=[("127.0.0.1", 1)] * 3, client=True, diff --git a/distributed/utils_test.py b/distributed/utils_test.py index c05f142b4f1..d897f4072b4 100644 --- a/distributed/utils_test.py +++ b/distributed/utils_test.py @@ -3,6 +3,7 @@ import copy import functools import gc +import importlib import io import itertools import logging @@ -21,6 +22,7 @@ import warnings import weakref from contextlib import contextmanager, nullcontext, suppress +from functools import wraps from glob import glob from time import sleep @@ -1567,3 +1569,32 @@ def transition(self, key, start, finish, **kwargs): ts.metadata["start_time"] = time() elif start == "executing" and finish == "memory": ts.metadata["stop_time"] = time() + + +def set_config_and_reload(cfg): + """Test function decorator that temporarily sets dask.config variables and updates + the cached copy in the global variables of the distributed.scheduler module. + + Tests decorated with this function are skipped when Cython is enabled. + + Note: this decorator must be placed *before* @gen_cluster. + """ + from . import scheduler + + def decorator(f): + @wraps(f) + def wrapper(*args, **kwargs): + try: + with dask.config.set(cfg): + importlib.reload(scheduler) + return f(*args, **kwargs) + finally: + importlib.reload(scheduler) + + if scheduler.COMPILED: + return pytest.mark.skip( + "Monkey-patching module variables doesn't work with Cython" + )(wrapper) + return wrapper + + return decorator From bc9294a280bf8a67029fad9272f93b7fd5c38305 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Mon, 17 May 2021 00:29:11 +0100 Subject: [PATCH 25/31] tests --- distributed/scheduler.py | 2 +- distributed/tests/test_scheduler.py | 145 ++++++++++++++++++++++++---- 2 files changed, 126 insertions(+), 21 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 5530ceac877..3e6bf91a902 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -5768,7 +5768,7 @@ def _rebalance_find_msgs( # recipients are sorted by rec_bytes_max. # The next ones will be worse; no reason to continue iterating break - use_recipient = ts.key not in rec_ws._has_what + use_recipient = ts not in rec_ws._has_what if not use_recipient: skipped_recipients.append(heapq.heappop(recipients)) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index fb09637532b..fbd5065d16d 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2543,26 +2543,6 @@ async def test_rebalance(c, s, *_): s.validate_state() -@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "250 MiB"}) -async def test_rebalance_managed_memory(c, s, a, b): - # Generate 100 buffers worth 100 MiB total on worker a. This sends its memory - # utilisation to exactly 40%, ignoring unmanaged, which is above the - # distributed.worker.memory.rebalance.sender-min threshold. - futures = c.map(lambda _: "x" * (2 ** 20), range(100), workers=[a.address]) - await wait(futures) - # Even if we're just using managed memory, which is instantaneously accounted for as - # soon as the tasks finish, MemoryState.managed is still capped by the process - # memory, so we need to wait for the heartbeat. - await assert_memory(s, "managed", 100, 101) - await assert_ndata(c, {a.address: 100, b.address: 0}) - await s.rebalance() - # We can expect an exact, stable result because we are completely bypassing the - # unpredictability of unmanaged memory. - await assert_ndata(c, {a.address: 62, b.address: 38}) - s.validate_state() - - @gen_cluster( nthreads=[("127.0.0.1", 1)] * 3, client=True, @@ -2635,3 +2615,128 @@ async def test_rebalance_raises_missing_data3(c, s, *_): async def test_rebalance_no_workers(s): await s.rebalance() s.validate_state() + + +@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "250 MiB"}) +async def test_rebalance_managed_memory(c, s, a, b): + # Generate 100 buffers worth 100 MiB total on worker a. This sends its memory + # utilisation to exactly 40%, ignoring unmanaged, which is above the + # distributed.worker.memory.rebalance.sender-min threshold. + futures = c.map(lambda _: "x" * (2 ** 20), range(100), workers=[a.address]) + await wait(futures) + # Even if we're just using managed memory, which is instantaneously accounted for as + # soon as the tasks finish, MemoryState.managed is still capped by the process + # memory, so we need to wait for the heartbeat. + await assert_memory(s, "managed", 100, 101) + await assert_ndata(c, {a.address: 100, b.address: 0}) + await s.rebalance() + # We can expect an exact, stable result because we are completely bypassing the + # unpredictability of unmanaged memory. + await assert_ndata(c, {a.address: 62, b.address: 38}) + s.validate_state() + + +@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": 0}) +async def test_rebalance_no_limit(c, s, a, b): + # See notes in test_rebalance_managed_memory + futures = c.map(lambda _: "x", range(100), workers=[a.address]) + await wait(futures) + # No reason to wait for memory here as we're allocating hundreds of bytes, so + # there's plenty of unmanaged process memory to pad it out + await assert_ndata(c, {a.address: 100, b.address: 0}) + await s.rebalance() + # Disabling memory_limit made us ignore all % thresholds set in the config + await assert_ndata(c, {a.address: 50, b.address: 50}) + s.validate_state() + + +@set_config_and_reload( + { + "distributed.worker.memory.rebalance.measure": "managed", + "distributed.worker.memory.rebalance.recipient-max": 0.4, + } +) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "250 MiB"}) +async def test_rebalance_no_recipients(c, s, a, b): + """There are sender workers, but no recipient workers""" + futures = [ + c.submit(lambda: "x" * (100 * 2 ** 20), pure=False, workers=[a.address]), # 40% + c.submit(lambda: "x" * (100 * 2 ** 20), pure=False, workers=[b.address]), # 40% + ] + c.map( + lambda _: "x" * (2 ** 20), range(50), workers=[a.address] + ) # 20% + await wait(futures) + await assert_memory(s, "managed", 250, 251) + await assert_ndata(c, {a.address: 51, b.address: 1}) + await s.rebalance() + await assert_ndata(c, {a.address: 51, b.address: 1}) + s.validate_state() + + +@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) +@gen_cluster( + nthreads=[("127.0.0.1", 1)] * 3, client=True, worker_kwargs={"memory_limit": 0} +) +async def test_rebalance_skip_recipient(client, s, a, b, c): + """A recipient is skipped because it already holds a copy of the key to be sent""" + futures = client.map(lambda _: "x", range(10), workers=[a.address]) + await wait(futures) + await client.replicate(futures[0:2], workers=[a.address, b.address]) + await client.replicate(futures[2:4], workers=[a.address, c.address]) + await assert_ndata(client, {a.address: 10, b.address: 2, c.address: 2}) + await client.rebalance(futures[:2]) + await assert_ndata(client, {a.address: 8, b.address: 2, c.address: 4}) + s.validate_state() + + +@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": 0}) +async def test_rebalance_skip_all_recipients(c, s, a, b): + """All recipients are skipped because they already hold copies""" + futures = c.map(lambda _: "x", range(10), workers=[a.address]) + await wait(futures) + await c.replicate([futures[0]]) + await assert_ndata(c, {a.address: 10, b.address: 1}) + await c.rebalance(futures[:2]) + await assert_ndata(c, {a.address: 9, b.address: 2}) + s.validate_state() + + +@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "250 MiB"}) +async def test_rebalance_sender_below_mean(c, s, a, b): + """A task remains on the sender because moving it would send it below the mean""" + f1 = c.submit(lambda: "x" * (100 * 2 ** 20), workers=[a.address]) + await wait([f1]) + f2 = c.submit(lambda: "x" * (5 * 2 ** 20), workers=[a.address]) + await wait([f2]) + await assert_memory(s, "managed", 105, 106) + await assert_ndata(c, {a.address: 2, b.address: 0}) + await s.rebalance() + assert await c.has_what() == {a.address: (f1.key,), b.address: (f2.key,)} + + +@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "250 MiB"}) +async def test_rebalance_least_recently_inserted_sender_min(c, s, a, b): + """ + 1. keys are picked using a least recently inserted policy + 2. workers below sender-min are never senders + """ + small_futures = c.map(lambda _: "x", range(10), workers=[a.address]) + await wait(small_futures) + await assert_ndata(c, {a.address: 10, b.address: 0}) + await s.rebalance() + await assert_ndata(c, {a.address: 10, b.address: 0}) + large_future = c.submit(lambda: "x" * (75 * 2 ** 20), workers=[a.address]) + await wait([large_future]) + await assert_memory(s, "managed", 75, 76) + await assert_ndata(c, {a.address: 11, b.address: 0}) + await s.rebalance() + await assert_ndata(c, {a.address: 1, b.address: 10}) + assert await c.has_what() == { + a.address: (large_future.key,), + b.address: tuple(f.key for f in small_futures), + } From 37de01f6389b4e27268d0babd2a58c57c606a3f6 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Mon, 17 May 2021 00:40:37 +0100 Subject: [PATCH 26/31] robustness --- distributed/tests/test_scheduler.py | 32 ++++++++++++++--------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index fbd5065d16d..6c74378dcc2 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2618,17 +2618,17 @@ async def test_rebalance_no_workers(s): @set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "250 MiB"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "500 MiB"}) async def test_rebalance_managed_memory(c, s, a, b): - # Generate 100 buffers worth 100 MiB total on worker a. This sends its memory + # Generate 100 buffers worth 200 MiB total on worker a. This sends its memory # utilisation to exactly 40%, ignoring unmanaged, which is above the # distributed.worker.memory.rebalance.sender-min threshold. - futures = c.map(lambda _: "x" * (2 ** 20), range(100), workers=[a.address]) + futures = c.map(lambda _: "x" * (2 ** 21), range(100), workers=[a.address]) await wait(futures) # Even if we're just using managed memory, which is instantaneously accounted for as # soon as the tasks finish, MemoryState.managed is still capped by the process # memory, so we need to wait for the heartbeat. - await assert_memory(s, "managed", 100, 101) + await assert_memory(s, "managed", 200, 201) await assert_ndata(c, {a.address: 100, b.address: 0}) await s.rebalance() # We can expect an exact, stable result because we are completely bypassing the @@ -2658,17 +2658,17 @@ async def test_rebalance_no_limit(c, s, a, b): "distributed.worker.memory.rebalance.recipient-max": 0.4, } ) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "250 MiB"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "500 MiB"}) async def test_rebalance_no_recipients(c, s, a, b): """There are sender workers, but no recipient workers""" futures = [ - c.submit(lambda: "x" * (100 * 2 ** 20), pure=False, workers=[a.address]), # 40% - c.submit(lambda: "x" * (100 * 2 ** 20), pure=False, workers=[b.address]), # 40% + c.submit(lambda: "x" * (200 * 2 ** 20), pure=False, workers=[a.address]), # 40% + c.submit(lambda: "x" * (200 * 2 ** 20), pure=False, workers=[b.address]), # 40% ] + c.map( - lambda _: "x" * (2 ** 20), range(50), workers=[a.address] + lambda _: "x" * (2 ** 21), range(50), workers=[a.address] ) # 20% await wait(futures) - await assert_memory(s, "managed", 250, 251) + await assert_memory(s, "managed", 500, 501) await assert_ndata(c, {a.address: 51, b.address: 1}) await s.rebalance() await assert_ndata(c, {a.address: 51, b.address: 1}) @@ -2705,21 +2705,21 @@ async def test_rebalance_skip_all_recipients(c, s, a, b): @set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "250 MiB"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "500 MiB"}) async def test_rebalance_sender_below_mean(c, s, a, b): """A task remains on the sender because moving it would send it below the mean""" - f1 = c.submit(lambda: "x" * (100 * 2 ** 20), workers=[a.address]) + f1 = c.submit(lambda: "x" * (200 * 2 ** 20), workers=[a.address]) await wait([f1]) - f2 = c.submit(lambda: "x" * (5 * 2 ** 20), workers=[a.address]) + f2 = c.submit(lambda: "x" * (10 * 2 ** 20), workers=[a.address]) await wait([f2]) - await assert_memory(s, "managed", 105, 106) + await assert_memory(s, "managed", 210, 211) await assert_ndata(c, {a.address: 2, b.address: 0}) await s.rebalance() assert await c.has_what() == {a.address: (f1.key,), b.address: (f2.key,)} @set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "250 MiB"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "500 MiB"}) async def test_rebalance_least_recently_inserted_sender_min(c, s, a, b): """ 1. keys are picked using a least recently inserted policy @@ -2730,9 +2730,9 @@ async def test_rebalance_least_recently_inserted_sender_min(c, s, a, b): await assert_ndata(c, {a.address: 10, b.address: 0}) await s.rebalance() await assert_ndata(c, {a.address: 10, b.address: 0}) - large_future = c.submit(lambda: "x" * (75 * 2 ** 20), workers=[a.address]) + large_future = c.submit(lambda: "x" * (150 * 2 ** 20), workers=[a.address]) await wait([large_future]) - await assert_memory(s, "managed", 75, 76) + await assert_memory(s, "managed", 150, 151) await assert_ndata(c, {a.address: 11, b.address: 0}) await s.rebalance() await assert_ndata(c, {a.address: 1, b.address: 10}) From fee3ff8df5f004bef71ffa295b4785a4d64ed6c8 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Mon, 17 May 2021 12:04:38 +0100 Subject: [PATCH 27/31] improve test stability --- distributed/tests/test_scheduler.py | 35 +++++++++++++++-------------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 6c74378dcc2..2c02c64789f 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2618,17 +2618,17 @@ async def test_rebalance_no_workers(s): @set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "500 MiB"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "1000 MiB"}) async def test_rebalance_managed_memory(c, s, a, b): - # Generate 100 buffers worth 200 MiB total on worker a. This sends its memory + # Generate 100 buffers worth 400 MiB total on worker a. This sends its memory # utilisation to exactly 40%, ignoring unmanaged, which is above the # distributed.worker.memory.rebalance.sender-min threshold. - futures = c.map(lambda _: "x" * (2 ** 21), range(100), workers=[a.address]) + futures = c.map(lambda _: "x" * (2 ** 22), range(100), workers=[a.address]) await wait(futures) # Even if we're just using managed memory, which is instantaneously accounted for as # soon as the tasks finish, MemoryState.managed is still capped by the process # memory, so we need to wait for the heartbeat. - await assert_memory(s, "managed", 200, 201) + await assert_memory(s, "managed", 400, 401) await assert_ndata(c, {a.address: 100, b.address: 0}) await s.rebalance() # We can expect an exact, stable result because we are completely bypassing the @@ -2658,20 +2658,20 @@ async def test_rebalance_no_limit(c, s, a, b): "distributed.worker.memory.rebalance.recipient-max": 0.4, } ) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "500 MiB"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "1000 MiB"}) async def test_rebalance_no_recipients(c, s, a, b): """There are sender workers, but no recipient workers""" futures = [ - c.submit(lambda: "x" * (200 * 2 ** 20), pure=False, workers=[a.address]), # 40% - c.submit(lambda: "x" * (200 * 2 ** 20), pure=False, workers=[b.address]), # 40% + c.submit(lambda: "x" * (400 * 2 ** 20), pure=False, workers=[a.address]), # 40% + c.submit(lambda: "x" * (400 * 2 ** 20), pure=False, workers=[b.address]), # 40% ] + c.map( - lambda _: "x" * (2 ** 21), range(50), workers=[a.address] + lambda _: "x" * (2 ** 21), range(100), workers=[a.address] ) # 20% await wait(futures) - await assert_memory(s, "managed", 500, 501) - await assert_ndata(c, {a.address: 51, b.address: 1}) + await assert_memory(s, "managed", 1000, 1001) + await assert_ndata(c, {a.address: 101, b.address: 1}) await s.rebalance() - await assert_ndata(c, {a.address: 51, b.address: 1}) + await assert_ndata(c, {a.address: 101, b.address: 1}) s.validate_state() @@ -2705,21 +2705,21 @@ async def test_rebalance_skip_all_recipients(c, s, a, b): @set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "500 MiB"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "1000 MiB"}) async def test_rebalance_sender_below_mean(c, s, a, b): """A task remains on the sender because moving it would send it below the mean""" - f1 = c.submit(lambda: "x" * (200 * 2 ** 20), workers=[a.address]) + f1 = c.submit(lambda: "x" * (400 * 2 ** 20), workers=[a.address]) await wait([f1]) f2 = c.submit(lambda: "x" * (10 * 2 ** 20), workers=[a.address]) await wait([f2]) - await assert_memory(s, "managed", 210, 211) + await assert_memory(s, "managed", 410, 411) await assert_ndata(c, {a.address: 2, b.address: 0}) await s.rebalance() assert await c.has_what() == {a.address: (f1.key,), b.address: (f2.key,)} @set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "500 MiB"}) +@gen_cluster(client=True, worker_kwargs={"memory_limit": "1000 MiB"}) async def test_rebalance_least_recently_inserted_sender_min(c, s, a, b): """ 1. keys are picked using a least recently inserted policy @@ -2730,9 +2730,10 @@ async def test_rebalance_least_recently_inserted_sender_min(c, s, a, b): await assert_ndata(c, {a.address: 10, b.address: 0}) await s.rebalance() await assert_ndata(c, {a.address: 10, b.address: 0}) - large_future = c.submit(lambda: "x" * (150 * 2 ** 20), workers=[a.address]) + + large_future = c.submit(lambda: "x" * (300 * 2 ** 20), workers=[a.address]) await wait([large_future]) - await assert_memory(s, "managed", 150, 151) + await assert_memory(s, "managed", 300, 301) await assert_ndata(c, {a.address: 11, b.address: 0}) await s.rebalance() await assert_ndata(c, {a.address: 1, b.address: 10}) From 71d08613a10737a468f712c237e027b61f85dde5 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Mon, 17 May 2021 14:02:44 +0100 Subject: [PATCH 28/31] tests stability --- distributed/tests/test_scheduler.py | 60 +++++++++++++++-------------- 1 file changed, 32 insertions(+), 28 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 2c02c64789f..88e535f92ef 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2618,22 +2618,23 @@ async def test_rebalance_no_workers(s): @set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "1000 MiB"}) -async def test_rebalance_managed_memory(c, s, a, b): +@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1000 MiB"}) +async def test_rebalance_managed_memory(c, s, *_): + a, b = s.workers # Generate 100 buffers worth 400 MiB total on worker a. This sends its memory # utilisation to exactly 40%, ignoring unmanaged, which is above the # distributed.worker.memory.rebalance.sender-min threshold. - futures = c.map(lambda _: "x" * (2 ** 22), range(100), workers=[a.address]) + futures = c.map(lambda _: "x" * (2 ** 22), range(100), workers=[a]) await wait(futures) # Even if we're just using managed memory, which is instantaneously accounted for as # soon as the tasks finish, MemoryState.managed is still capped by the process # memory, so we need to wait for the heartbeat. await assert_memory(s, "managed", 400, 401) - await assert_ndata(c, {a.address: 100, b.address: 0}) + await assert_ndata(c, {a: 100, b: 0}) await s.rebalance() # We can expect an exact, stable result because we are completely bypassing the # unpredictability of unmanaged memory. - await assert_ndata(c, {a.address: 62, b.address: 38}) + await assert_ndata(c, {a: 62, b: 38}) s.validate_state() @@ -2658,20 +2659,21 @@ async def test_rebalance_no_limit(c, s, a, b): "distributed.worker.memory.rebalance.recipient-max": 0.4, } ) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "1000 MiB"}) -async def test_rebalance_no_recipients(c, s, a, b): +@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1000 MiB"}) +async def test_rebalance_no_recipients(c, s, *_): """There are sender workers, but no recipient workers""" + a, b = s.workers futures = [ - c.submit(lambda: "x" * (400 * 2 ** 20), pure=False, workers=[a.address]), # 40% - c.submit(lambda: "x" * (400 * 2 ** 20), pure=False, workers=[b.address]), # 40% + c.submit(lambda: "x" * (400 * 2 ** 20), pure=False, workers=[a]), # 40% + c.submit(lambda: "x" * (400 * 2 ** 20), pure=False, workers=[b]), # 40% ] + c.map( - lambda _: "x" * (2 ** 21), range(100), workers=[a.address] + lambda _: "x" * (2 ** 21), range(100), workers=[a] ) # 20% await wait(futures) await assert_memory(s, "managed", 1000, 1001) - await assert_ndata(c, {a.address: 101, b.address: 1}) + await assert_ndata(c, {a: 101, b: 1}) await s.rebalance() - await assert_ndata(c, {a.address: 101, b.address: 1}) + await assert_ndata(c, {a: 101, b: 1}) s.validate_state() @@ -2705,39 +2707,41 @@ async def test_rebalance_skip_all_recipients(c, s, a, b): @set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "1000 MiB"}) -async def test_rebalance_sender_below_mean(c, s, a, b): +@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1000 MiB"}) +async def test_rebalance_sender_below_mean(c, s, *_): """A task remains on the sender because moving it would send it below the mean""" - f1 = c.submit(lambda: "x" * (400 * 2 ** 20), workers=[a.address]) + a, b = s.workers + f1 = c.submit(lambda: "x" * (400 * 2 ** 20), workers=[a]) await wait([f1]) - f2 = c.submit(lambda: "x" * (10 * 2 ** 20), workers=[a.address]) + f2 = c.submit(lambda: "x" * (10 * 2 ** 20), workers=[a]) await wait([f2]) await assert_memory(s, "managed", 410, 411) - await assert_ndata(c, {a.address: 2, b.address: 0}) + await assert_ndata(c, {a: 2, b: 0}) await s.rebalance() - assert await c.has_what() == {a.address: (f1.key,), b.address: (f2.key,)} + assert await c.has_what() == {a: (f1.key,), b: (f2.key,)} @set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": "1000 MiB"}) -async def test_rebalance_least_recently_inserted_sender_min(c, s, a, b): +@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1000 MiB"}) +async def test_rebalance_least_recently_inserted_sender_min(c, s, *_): """ 1. keys are picked using a least recently inserted policy 2. workers below sender-min are never senders """ - small_futures = c.map(lambda _: "x", range(10), workers=[a.address]) + a, b = s.workers + small_futures = c.map(lambda _: "x", range(10), workers=[a]) await wait(small_futures) - await assert_ndata(c, {a.address: 10, b.address: 0}) + await assert_ndata(c, {a: 10, b: 0}) await s.rebalance() - await assert_ndata(c, {a.address: 10, b.address: 0}) + await assert_ndata(c, {a: 10, b: 0}) - large_future = c.submit(lambda: "x" * (300 * 2 ** 20), workers=[a.address]) + large_future = c.submit(lambda: "x" * (300 * 2 ** 20), workers=[a]) await wait([large_future]) await assert_memory(s, "managed", 300, 301) - await assert_ndata(c, {a.address: 11, b.address: 0}) + await assert_ndata(c, {a: 11, b: 0}) await s.rebalance() - await assert_ndata(c, {a.address: 1, b.address: 10}) + await assert_ndata(c, {a: 1, b: 10}) assert await c.has_what() == { - a.address: (large_future.key,), - b.address: tuple(f.key for f in small_futures), + a: (large_future.key,), + b: tuple(f.key for f in small_futures), } From be188dc1bb11231513c20c8a9164f1900da4825c Mon Sep 17 00:00:00 2001 From: crusaderky Date: Mon, 17 May 2021 17:10:41 +0100 Subject: [PATCH 29/31] trivial --- docs/source/memory.rst | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/source/memory.rst b/docs/source/memory.rst index 2ae16698ea8..849b71ec616 100644 --- a/docs/source/memory.rst +++ b/docs/source/memory.rst @@ -166,7 +166,8 @@ 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. +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 From 3f29a813ba4b1b44023359071d69d4895fae1cf1 Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 27 May 2021 12:23:23 +0100 Subject: [PATCH 30/31] reload dask.config on Scheduler.__init__ --- distributed/scheduler.py | 83 +++++++++++++++-------------- distributed/tests/test_scheduler.py | 55 +++++++++++++------ distributed/utils_test.py | 31 ----------- 3 files changed, 81 insertions(+), 88 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 321f587f31f..de3843c17f4 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -163,30 +163,6 @@ def nogil(func): DEFAULT_DATA_SIZE = declare( Py_ssize_t, parse_bytes(dask.config.get("distributed.scheduler.default-data-size")) ) -UNKNOWN_TASK_DURATION = declare( - double, - parse_timedelta(dask.config.get("distributed.scheduler.unknown-task-duration")), -) -MEMORY_RECENT_TO_OLD_TIME = declare( - double, - parse_timedelta(dask.config.get("distributed.worker.memory.recent-to-old-time")), -) -MEMORY_REBALANCE_MEASURE = declare( - str, dask.config.get("distributed.worker.memory.rebalance.measure") -) -MEMORY_REBALANCE_SENDER_MIN = declare( - double, - dask.config.get("distributed.worker.memory.rebalance.sender-min"), -) -MEMORY_REBALANCE_RECIPIENT_MAX = declare( - double, - dask.config.get("distributed.worker.memory.rebalance.recipient-max"), -) -MEMORY_REBALANCE_HALF_GAP = declare( - double, - dask.config.get("distributed.worker.memory.rebalance.sender-recipient-gap") / 2.0, -) - DEFAULT_EXTENSIONS = [ LockExtension, @@ -1800,6 +1776,14 @@ class SchedulerState: _workers: object _workers_dv: dict + # Variables from dask.config, cached by __init__ for performance + UNKNOWN_TASK_DURATION: double + MEMORY_RECENT_TO_OLD_TIME: double + MEMORY_REBALANCE_MEASURE: str + MEMORY_REBALANCE_SENDER_MIN: double + MEMORY_REBALANCE_RECIPIENT_MAX: double + MEMORY_REBALANCE_HALF_GAP: double + def __init__( self, aliases: dict = None, @@ -1874,6 +1858,28 @@ def __init__( else: self._workers = sortedcontainers.SortedDict() self._workers_dv: dict = cast(dict, self._workers) + + # Variables from dask.config, cached by __init__ for performance + self.UNKNOWN_TASK_DURATION = parse_timedelta( + dask.config.get("distributed.scheduler.unknown-task-duration") + ) + self.MEMORY_RECENT_TO_OLD_TIME = parse_timedelta( + dask.config.get("distributed.worker.memory.recent-to-old-time") + ) + self.MEMORY_REBALANCE_MEASURE = dask.config.get( + "distributed.worker.memory.rebalance.measure" + ) + self.MEMORY_REBALANCE_SENDER_MIN = dask.config.get( + "distributed.worker.memory.rebalance.sender-min" + ) + self.MEMORY_REBALANCE_RECIPIENT_MAX = dask.config.get( + "distributed.worker.memory.rebalance.recipient-max" + ) + self.MEMORY_REBALANCE_HALF_GAP = ( + dask.config.get("distributed.worker.memory.rebalance.sender-recipient-gap") + / 2.0 + ) + super().__init__(**kwargs) @property @@ -3094,18 +3100,14 @@ def get_task_duration(self, ts: TaskState, default: double = -1) -> double: (not including any communication cost). """ duration: double = ts._prefix._duration_average - if duration < 0: - s: set = self._unknown_durations.get(ts._prefix._name) - if s is None: - self._unknown_durations[ts._prefix._name] = s = set() - s.add(ts) + if duration >= 0: + return duration - if default < 0: - duration = UNKNOWN_TASK_DURATION - else: - duration = default - - return duration + s: set = self._unknown_durations.get(ts._prefix._name) + if s is None: + self._unknown_durations[ts._prefix._name] = s = set() + s.add(ts) + return default if default >= 0 else self.UNKNOWN_TASK_DURATION @ccall @exceptval(check=False) @@ -3887,7 +3889,7 @@ def heartbeat_worker( # Calculate RSS - dask keys, separating "old" and "new" usage # See MemoryState for details - max_memory_unmanaged_old_hist_age = local_now - MEMORY_RECENT_TO_OLD_TIME + max_memory_unmanaged_old_hist_age = local_now - parent.MEMORY_RECENT_TO_OLD_TIME memory_unmanaged_old = ws._memory_unmanaged_old while ws._memory_other_history: timestamp, size = ws._memory_other_history[0] @@ -5658,6 +5660,7 @@ def _rebalance_find_msgs( - recipient worker - task to be transferred """ + parent: SchedulerState = self ts: TaskState ws: WorkerState @@ -5690,15 +5693,15 @@ def _rebalance_find_msgs( # (distributed.worker.memory.recent-to-old-time). # This lets us ignore temporary spikes caused by task heap usage. memory_by_worker = [ - (ws, getattr(ws.memory, MEMORY_REBALANCE_MEASURE)) for ws in workers + (ws, getattr(ws.memory, parent.MEMORY_REBALANCE_MEASURE)) for ws in workers ] mean_memory = sum(m for _, m in memory_by_worker) // len(memory_by_worker) for ws, ws_memory in memory_by_worker: if ws.memory_limit: - half_gap = int(MEMORY_REBALANCE_HALF_GAP * ws.memory_limit) - sender_min = MEMORY_REBALANCE_SENDER_MIN * ws.memory_limit - recipient_max = MEMORY_REBALANCE_RECIPIENT_MAX * ws.memory_limit + half_gap = int(parent.MEMORY_REBALANCE_HALF_GAP * ws.memory_limit) + sender_min = parent.MEMORY_REBALANCE_SENDER_MIN * ws.memory_limit + recipient_max = parent.MEMORY_REBALANCE_RECIPIENT_MAX * ws.memory_limit else: half_gap = 0 sender_min = 0.0 diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 08016fafd84..1649d3b125c 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -39,7 +39,6 @@ inc, loop, nodebug, - set_config_and_reload, slowadd, slowdec, slowinc, @@ -2626,8 +2625,12 @@ async def test_rebalance_no_workers(s): s.validate_state() -@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1000 MiB"}) +@gen_cluster( + client=True, + Worker=Nanny, + worker_kwargs={"memory_limit": "1000 MiB"}, + config={"distributed.worker.memory.rebalance.measure": "managed"}, +) async def test_rebalance_managed_memory(c, s, *_): a, b = s.workers # Generate 100 buffers worth 400 MiB total on worker a. This sends its memory @@ -2647,8 +2650,11 @@ async def test_rebalance_managed_memory(c, s, *_): s.validate_state() -@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": 0}) +@gen_cluster( + client=True, + worker_kwargs={"memory_limit": 0}, + config={"distributed.worker.memory.rebalance.measure": "managed"}, +) async def test_rebalance_no_limit(c, s, a, b): # See notes in test_rebalance_managed_memory futures = c.map(lambda _: "x", range(100), workers=[a.address]) @@ -2662,13 +2668,15 @@ async def test_rebalance_no_limit(c, s, a, b): s.validate_state() -@set_config_and_reload( - { +@gen_cluster( + client=True, + Worker=Nanny, + worker_kwargs={"memory_limit": "1000 MiB"}, + config={ "distributed.worker.memory.rebalance.measure": "managed", "distributed.worker.memory.rebalance.recipient-max": 0.4, - } + }, ) -@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1000 MiB"}) async def test_rebalance_no_recipients(c, s, *_): """There are sender workers, but no recipient workers""" a, b = s.workers @@ -2686,9 +2694,11 @@ async def test_rebalance_no_recipients(c, s, *_): s.validate_state() -@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) @gen_cluster( - nthreads=[("127.0.0.1", 1)] * 3, client=True, worker_kwargs={"memory_limit": 0} + nthreads=[("127.0.0.1", 1)] * 3, + client=True, + worker_kwargs={"memory_limit": 0}, + config={"distributed.worker.memory.rebalance.measure": "managed"}, ) async def test_rebalance_skip_recipient(client, s, a, b, c): """A recipient is skipped because it already holds a copy of the key to be sent""" @@ -2702,8 +2712,11 @@ async def test_rebalance_skip_recipient(client, s, a, b, c): s.validate_state() -@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, worker_kwargs={"memory_limit": 0}) +@gen_cluster( + client=True, + worker_kwargs={"memory_limit": 0}, + config={"distributed.worker.memory.rebalance.measure": "managed"}, +) async def test_rebalance_skip_all_recipients(c, s, a, b): """All recipients are skipped because they already hold copies""" futures = c.map(lambda _: "x", range(10), workers=[a.address]) @@ -2715,8 +2728,12 @@ async def test_rebalance_skip_all_recipients(c, s, a, b): s.validate_state() -@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1000 MiB"}) +@gen_cluster( + client=True, + Worker=Nanny, + worker_kwargs={"memory_limit": "1000 MiB"}, + config={"distributed.worker.memory.rebalance.measure": "managed"}, +) async def test_rebalance_sender_below_mean(c, s, *_): """A task remains on the sender because moving it would send it below the mean""" a, b = s.workers @@ -2730,8 +2747,12 @@ async def test_rebalance_sender_below_mean(c, s, *_): assert await c.has_what() == {a: (f1.key,), b: (f2.key,)} -@set_config_and_reload({"distributed.worker.memory.rebalance.measure": "managed"}) -@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1000 MiB"}) +@gen_cluster( + client=True, + Worker=Nanny, + worker_kwargs={"memory_limit": "1000 MiB"}, + config={"distributed.worker.memory.rebalance.measure": "managed"}, +) async def test_rebalance_least_recently_inserted_sender_min(c, s, *_): """ 1. keys are picked using a least recently inserted policy diff --git a/distributed/utils_test.py b/distributed/utils_test.py index 5809380ae71..e3f18e5de49 100644 --- a/distributed/utils_test.py +++ b/distributed/utils_test.py @@ -3,7 +3,6 @@ import copy import functools import gc -import importlib import io import itertools import logging @@ -22,7 +21,6 @@ import warnings import weakref from contextlib import contextmanager, nullcontext, suppress -from functools import wraps from glob import glob from time import sleep @@ -1569,32 +1567,3 @@ def transition(self, key, start, finish, **kwargs): ts.metadata["start_time"] = time() elif start == "executing" and finish == "memory": ts.metadata["stop_time"] = time() - - -def set_config_and_reload(cfg): - """Test function decorator that temporarily sets dask.config variables and updates - the cached copy in the global variables of the distributed.scheduler module. - - Tests decorated with this function are skipped when Cython is enabled. - - Note: this decorator must be placed *before* @gen_cluster. - """ - from . import scheduler - - def decorator(f): - @wraps(f) - def wrapper(*args, **kwargs): - try: - with dask.config.set(cfg): - importlib.reload(scheduler) - return f(*args, **kwargs) - finally: - importlib.reload(scheduler) - - if scheduler.COMPILED: - return pytest.mark.skip( - "Monkey-patching module variables doesn't work with Cython" - )(wrapper) - return wrapper - - return decorator From 03e376e25c0c8b5f28999e72ff8cf57495cbfd0b Mon Sep 17 00:00:00 2001 From: crusaderky Date: Thu, 27 May 2021 16:53:03 +0100 Subject: [PATCH 31/31] code review --- distributed/distributed-schema.yaml | 13 +++++-------- distributed/tests/test_client.py | 11 ++++++++++- distributed/tests/test_scheduler.py | 21 +++++++++++++++++---- distributed/tests/test_tls_functional.py | 7 ++++++- 4 files changed, 38 insertions(+), 14 deletions(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index a272aa2ec37..86c00a8f318 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -403,7 +403,6 @@ properties: the worker, ignore it for this long before considering it in non-time-sensitive heuristics. This should be set to be longer than the duration of most dask tasks. - rebalance: type: object description: >- @@ -423,18 +422,16 @@ properties: minimum: 0 maximum: 1 description: >- - Fraction of worker process memory at which we stop potentially - receiving data from other workers. Ignored when max_memory is not - set. - + Fraction of worker process memory at which we start potentially + transferring data to other workers. recipient-max: type: number minimum: 0 maximum: 1 description: >- - Fraction of worker process memory at which we start potentially - transferring data to other workers. - + Fraction of worker process memory at which we stop potentially + receiving data from other workers. Ignored when max_memory is not + set. sender-recipient-gap: type: number minimum: 0 diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index bb981de0fc5..c23a221092a 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -2895,7 +2895,12 @@ def __reduce__(self): await x -@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) +@gen_cluster( + client=True, + Worker=Nanny, + worker_kwargs={"memory_limit": "1 GiB"}, + config={"distributed.worker.memory.rebalance.sender-min": 0.3}, +) async def test_rebalance(c, s, *_): """Test Client.rebalance(). These are just to test the Client wrapper around Scheduler.rebalance(); for more thorough tests on the latter see test_scheduler.py. @@ -2967,6 +2972,7 @@ def test_rebalance_sync(): s = c.cluster.scheduler a, b = [ws.address for ws in s.workers.values()] futures = c.map(lambda _: "x" * (2 ** 29 // 10), range(10), workers=[a]) + wait(futures) # Wait for heartbeat while s.memory.process < 2 ** 29: sleep(0.1) @@ -2984,7 +2990,10 @@ def test_rebalance_sync(): async def test_rebalance_unprepared(c, s, a, b): """Client.rebalance() internally waits for unfinished futures""" futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) + # Let the futures reach the scheduler await asyncio.sleep(0.1) + # We didn't wait enough for futures to complete. However, Client.rebalance() will + # block until all futures are completed before invoking Scheduler.rebalance(). await c.rebalance(futures) s.validate_state() diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 1649d3b125c..f1ddae3615e 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2528,7 +2528,12 @@ async def assert_ndata(client, by_addr, total=None): raise AssertionError(f"Expected {by_addr}, total={total}; got {out}") -@gen_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) +@gen_cluster( + client=True, + Worker=Nanny, + worker_kwargs={"memory_limit": "1 GiB"}, + config={"distributed.worker.memory.rebalance.sender-min": 0.3}, +) async def test_rebalance(c, s, *_): # We used nannies to have separate processes for each worker a, b = s.workers @@ -2593,7 +2598,9 @@ async def test_rebalance_missing_data1(s, a, b): @gen_cluster(client=True) async def test_rebalance_missing_data2(c, s, a, b): - """keys exist but belong to unfinished futures""" + """keys exist but belong to unfinished futures. Unlike Client.rebalance(), + Scheduler.rebalance() does not wait for unfinished futures. + """ futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) await asyncio.sleep(0.1) out = await s.rebalance(keys=[f.key for f in futures]) @@ -2629,7 +2636,10 @@ async def test_rebalance_no_workers(s): client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1000 MiB"}, - config={"distributed.worker.memory.rebalance.measure": "managed"}, + config={ + "distributed.worker.memory.rebalance.measure": "managed", + "distributed.worker.memory.rebalance.sender-min": 0.3, + }, ) async def test_rebalance_managed_memory(c, s, *_): a, b = s.workers @@ -2751,7 +2761,10 @@ async def test_rebalance_sender_below_mean(c, s, *_): client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1000 MiB"}, - config={"distributed.worker.memory.rebalance.measure": "managed"}, + config={ + "distributed.worker.memory.rebalance.measure": "managed", + "distributed.worker.memory.rebalance.sender-min": 0.3, + }, ) async def test_rebalance_least_recently_inserted_sender_min(c, s, *_): """ diff --git a/distributed/tests/test_tls_functional.py b/distributed/tests/test_tls_functional.py index ae685f2c222..84a0f5922d9 100644 --- a/distributed/tests/test_tls_functional.py +++ b/distributed/tests/test_tls_functional.py @@ -99,7 +99,12 @@ async def test_nanny(c, s, a, b): assert result == 11 -@gen_tls_cluster(client=True, Worker=Nanny, worker_kwargs={"memory_limit": "1 GiB"}) +@gen_tls_cluster( + client=True, + Worker=Nanny, + worker_kwargs={"memory_limit": "1 GiB"}, + config={"distributed.worker.memory.rebalance.sender-min": 0.3}, +) async def test_rebalance(c, s, *_): # We used nannies to have separate processes for each worker a, b = s.workers