Local Cluster

For convenience you can start a local cluster from your Python session.

>>> from distributed import Client, LocalCluster
>>> cluster = LocalCluster()
LocalCluster("127.0.0.1:8786", workers=8, ncores=8)
>>> client = Client(cluster)
<Client: scheduler=127.0.0.1:8786 processes=8 cores=8>

You can dynamically scale this cluster up and down:

>>> worker = cluster.add_worker()
>>> cluster.remove_worker(worker)

Alternatively, a LocalCluster is made for you automatically if you create an Client with no arguments:

>>> from distributed import Client
>>> client = Client()
>>> client
<Client: scheduler=127.0.0.1:8786 processes=8 cores=8>

API

class distributed.deploy.local.LocalCluster(n_workers=None, threads_per_worker=None, nanny=True, loop=None, start=True, ip=None, scheduler_port=0, silence_logs=50, diagnostics_port=8787, services={}, worker_services={}, **worker_kwargs)[source]

Create local Scheduler and Workers

This creates a “cluster” of a scheduler and workers running on the local machine.

Parameters:

n_workers: int

Number of workers to start

threads_per_worker: int

Number of threads per each worker

nanny: boolean

If true start the workers in separate processes managed by a nanny. If False keep the workers in the main calling process

scheduler_port: int

Port of the scheduler. 8786 by default, use 0 to choose a random port

silence_logs: logging level

Level of logs to print out to stdout. logging.CRITICAL by default. Use a falsey value like False or None for no change.

ip: string

IP address on which the scheduler will listen, defaults to only localhost

kwargs: dict

Extra worker arguments, will be passed to the Worker constructor.

Examples

>>> c = LocalCluster()  # Create a local cluster with as many workers as cores  
>>> c  
LocalCluster("127.0.0.1:8786", workers=8, ncores=8)
>>> c = Client(c)  # connect to local cluster  

Add a new worker to the cluster >>> w = c.start_worker(ncores=2) # doctest: +SKIP

Shut down the extra worker >>> c.remove_worker(w) # doctest: +SKIP

Start a diagnostic web server and open a new browser tab >>> c.start_diagnostics_server(show=True) # doctest: +SKIP

close()[source]

Close the cluster

scale_down(*args, **kwargs)[source]

Remove workers from the cluster

Given a list of worker addresses this function should remove those workers from the cluster. This may require tracking which jobs are associated to which worker address.

This can be implemented either as a function or as a Tornado coroutine.

scale_up(*args, **kwargs)[source]

Bring the total count of workers up to n

This function/coroutine should bring the total number of workers up to the number n.

This can be implemented either as a function or as a Tornado coroutine.

start_diagnostics_server(port=8787, show=False, silence=50)[source]

Start Diagnostics Web Server

This starts a web application to show diagnostics of what is happening on the cluster. This application runs in a separate process and is generally available at the following location:

start_worker(ncores=0, **kwargs)[source]

Add a new worker to the running cluster

Parameters:

port: int (optional)

Port on which to serve the worker, defaults to 0 or random

ncores: int (optional)

Number of threads to use. Defaults to number of logical cores

Returns:

The created Worker or Nanny object. Can be discarded.

Examples

>>> c = LocalCluster()  
>>> c.start_worker(ncores=2)  
stop_worker(w)[source]

Stop a running worker

Examples

>>> c = LocalCluster()  
>>> w = c.start_worker(ncores=2)  
>>> c.stop_worker(w)