Adaptive deployments

Motivation

Most Dask deployments are static with a single scheduler and a fixed number of workers. This results in predictable behavior, but is wasteful of resources in two situations:

  1. The user may not be using the cluster, or perhaps they are busy interpreting a recent result or plot, and so the workers sit idly, taking up valuable shared resources from other potential users

  2. The user may be very active, and is limited by their original allocation.

Particularly efficient users may learn to manually add and remove workers during their session, but this is rare. Instead, we would like the size of a Dask cluster to match the computational needs at any given time. This is the goal of the adaptive deployments discussed in this document.


Dask adaptive scaling

These are particularly helpful for interactive workloads, which are characterized by long periods of inactivity interrupted with short bursts of heavy activity. Adaptive deployments can result in both faster analyses that give users much more power, but with much less pressure on computational resources.

Adaptive

To make setting up adaptive deployments easy, some Dask deployment solutions offer an .adapt() method. Here is an example with dask_kubernetes.KubeCluster.

from dask_kubernetes import KubeCluster

cluster = KubeCluster()
cluster.adapt(minimum=0, maximum=100)  # scale between 0 and 100 workers

For more keyword options, see the Adaptive class below:

Adaptive(cluster[, interval, minimum, ...])

Adaptively allocate workers based on scheduler load.

Dependence on a Resource Manager

The Dask scheduler does not know how to launch workers on its own. Instead, it relies on an external resource scheduler like Kubernetes above, or Yarn, SGE, SLURM, Mesos, or some other in-house system (see how to deploy Dask clusters for options). In order to use adaptive deployments, you must provide some mechanism for the scheduler to launch new workers. Typically, this is done by using one of the solutions listed in the how to deploy Dask clusters, or by subclassing from the Cluster superclass and implementing that API.

Cluster([asynchronous, loop, quiet, name, ...])

Superclass for cluster objects

Scaling Heuristics

The Dask scheduler tracks a variety of information that is useful to correctly allocate the number of workers:

  1. The historical runtime of every function and task that it has seen, and all of the functions that it is currently able to run for users

  2. The amount of memory used and available on each worker

  3. Which workers are idle or saturated for various reasons, like the presence of specialized hardware

From these, it is able to determine a target number of workers by dividing the cumulative expected runtime of all pending tasks by the target_duration parameter (defaults to five seconds). This number of workers serves as a baseline request for the resource manager. This number can be altered for a variety of reasons:

  1. If the cluster needs more memory, then it will choose either the target number of workers or twice the current number of workers (whichever is larger)

  2. If the target is outside of the range of the minimum and maximum values, then it is clipped to fit within that range

Additionally, when scaling down, Dask preferentially chooses those workers that are idle and have the least data in memory. It moves that data to other machines before retiring the worker. To avoid rapid cycling of the cluster up and down in size, we only retire a worker after a few cycles have gone by where it has consistently been a good idea to retire it (controlled by the wait_count and interval parameters).

API

class distributed.deploy.Adaptive(cluster: Cluster, interval: str | float | timedelta | None = None, minimum: int | None = None, maximum: int | float | None = None, wait_count: int | None = None, target_duration: str | float | timedelta | None = None, worker_key: Callable[[WorkerState], Hashable] | None = None, **kwargs: Any)[source]

Adaptively allocate workers based on scheduler load. A superclass.

Contains logic to dynamically resize a Dask cluster based on current use. This class needs to be paired with a system that can create and destroy Dask workers using a cluster resource manager. Typically it is built into already existing solutions, rather than used directly by users. It is most commonly used from the .adapt(...) method of various Dask cluster classes.

Parameters
cluster: object

Must have scale and scale_down methods/coroutines

intervaltimedelta or str, default “1000 ms”

Milliseconds between checks

wait_count: int, default 3

Number of consecutive times that a worker should be suggested for removal before we remove it.

target_duration: timedelta or str, default “5s”

Amount of time we want a computation to take. This affects how aggressively we scale up.

worker_key: Callable[WorkerState]

Function to group workers together when scaling down See Scheduler.workers_to_close for more information

minimum: int

Minimum number of workers to keep around

maximum: int

Maximum number of workers to keep around

**kwargs:

Extra parameters to pass to Scheduler.workers_to_close

Notes

Subclasses can override Adaptive.target() and Adaptive.workers_to_close() to control when the cluster should be resized. The default implementation checks if there are too many tasks per worker or too little memory available (see distributed.Scheduler.adaptive_target()). The values for interval, min, max, wait_count and target_duration can be specified in the dask config under the distributed.adaptive key.

Examples

This is commonly used from existing Dask classes, like KubeCluster

>>> from dask_kubernetes import KubeCluster
>>> cluster = KubeCluster()
>>> cluster.adapt(minimum=10, maximum=100)

Alternatively you can use it from your own Cluster class by subclassing from Dask’s Cluster superclass

>>> from distributed.deploy import Cluster
>>> class MyCluster(Cluster):
...     def scale_up(self, n):
...         """ Bring worker count up to n """
...     def scale_down(self, workers):
...        """ Remove worker addresses from cluster """
>>> cluster = MyCluster()
>>> cluster.adapt(minimum=10, maximum=100)
class distributed.deploy.Cluster(asynchronous=False, loop=None, quiet=False, name=None, scheduler_sync_interval=1)[source]

Superclass for cluster objects

This class contains common functionality for Dask Cluster manager classes.

To implement this class, you must provide

  1. A scheduler_comm attribute, which is a connection to the scheduler following the distributed.core.rpc API.

  2. Implement scale, which takes an integer and scales the cluster to that many workers, or else set _supports_scaling to False

For that, you should get the following:

  1. A standard __repr__

  2. A live IPython widget

  3. Adaptive scaling

  4. Integration with dask-labextension

  5. A scheduler_info attribute which contains an up-to-date copy of Scheduler.identity(), which is used for much of the above

  6. Methods to gather logs