Futures

Dask supports a real-time task framework that extends Python’s concurrent.futures interface. This interface is good for arbitrary task scheduling, like dask.delayed, but is immediate rather than lazy, which provides some more flexibility in situations where the computations may evolve over time.

These features depend on the second generation task scheduler found in dask.distributed (which, despite its name, runs very well on a single machine).

Start Dask Client

You must start a Client to use the futures interface. This tracks state among the various worker processes or threads.

from dask.distributed import Client

client = Client()  # start local workers as processes
# or
client = Client(processes=False)  # start local workers as threads

If you have Bokeh installed then this starts up a diagnostic dashboard at http://localhost:8786 .

Submit Tasks

Client.submit(func, *args, **kwargs) Submit a function application to the scheduler
Client.map(func, *iterables, **kwargs) Map a function on a sequence of arguments
Future.result

Then you can submit individual tasks using the submit method.

def inc(x):
    return x + 1

def add(x, y):
    return x + y

a = client.submit(inc, 10)  # calls inc(10) in background thread or process
b = client.submit(inc, 20)  # calls inc(20) in background thread or process

Submit returns a Future, which refers to a remote result. This result may not yet be completed:

>>> a
<Future: status: pending, key: inc-b8aaf26b99466a7a1980efa1ade6701d>

Eventually it will complete. The result stays in the remote thread/process/worker until you ask for it back explicitly.

>>> a
<Future: status: finished, type: int, key: inc-b8aaf26b99466a7a1980efa1ade6701d>

>>> a.result()  # blocks until task completes and data arrives
11

You can pass futures as inputs to submit. Dask automatically handles dependency tracking; once all input futures have completed they will be moved onto a single worker (if necessary), and then the computation that depends on them will be started. You do not need to wait for inputs to finish before submitting a new task; Dask will handle this automatically.

c = client.submit(add, a, b)  # calls add on the results of a and b

Similar to Python’s map you can use Client.map to call the same function and many inputs:

futures = client.map(inc, range(1000))

However note that each task comes with about 1ms of overhead. If you want to map a function over a large number of inputs then you might consider dask.bag or dask.dataframe instead.

Move Data

Future.result
Client.gather(futures[, errors, maxsize]) Gather futures from distributed memory
Client.scatter(data[, workers, broadcast, ...]) Scatter data into distributed memory

Given any future you can call the .result method to gather the result. This will block until the future is done computing and then transfer the result back to your local process if necessary.

>>> c.result()
32

You can gather many results concurrently using the Client.gather method. This can be more efficient than calling .result() on each future sequentially.

>>> # results = [future.result() for future in futures]
>>> results = client.gather(futures)  # this can be faster

If you have important local data that you want to include in your computation you can either include it as a normal input to a submit or map call:

>>> df = pd.read_csv('training-data.csv')
>>> future = client.submit(my_function, df)

Or you can scatter it explicitly. Scattering moves your data to a worker and returns a future pointing to that data:

>>> remote_df = client.scatter(df)
>>> remote_df
<Future: status: finished, type: DataFrame, key: bbd0ca93589c56ea14af49cba470006e>

>>> future = client.submit(my_function, remote_df)

Both of these accomplish the same result, but using scatter can sometimes be faster. This is especially true if you use processes or distributed workers (where data transfer is necessary) and you want to use df in many computations. Scattering the data beforehand avoids excessive data movement.

Calling scatter on a list scatters all elements individually. Dask will spread these elements evenly throughout workers in a round-robin fashion:

>>> client.scatter([1, 2, 3])
[<Future: status: finished, type: int, key: c0a8a20f903a4915b94db8de3ea63195>,
 <Future: status: finished, type: int, key: 58e78e1b34eb49a68c65b54815d1b158>,
 <Future: status: finished, type: int, key: d3395e15f605bc35ab1bac6341a285e2>]

References, Cancellation, and Exceptions

Future.cancel
Future.exception
Future.traceback
Client.cancel(futures) Cancel running futures

Dask will only compute and hold onto results for which there are active futures. In this way your local variables define what is active in Dask. When a future is garbage collected by your local Python session, Dask will feel free to delete that data or stop ongoing computations that were trying to produce it.

>>> del future  # deletes remote data once future is garbage collected

You can also explicitly cancel a task using the Future.cancel or Client.cancel methods.

>>> future.cancel()  # deletes data even if other futures point to it

If a future fails, then Dask will raise the remote exceptions and tracebacks if you try to get the result.

def div(x, y):
    return x / y

>>> a = client.submit(div, 1, 0)  # 1 / 0 raises a ZeroDivisionError
>>> a
<Future: status: error, key: div-3601743182196fb56339e584a2bf1039>

>>> a.result()
      1 def div(x, y):
----> 2     return x / y

ZeroDivisionError: division by zero

All futures that depend on an erred future also err with the same exception:

>>> b = client.submit(inc, a)
>>> b
<Future: status: error, key: inc-15e2e4450a0227fa38ede4d6b1a952db>

You can collect the exception or traceback explicitly with the Future.exception or Future.traceback methods.

Waiting on Futures

as_completed(fs) Return futures in the order in which they complete
wait(fs[, timeout, return_when]) Wait until all futures are complete

You can wait on a future or collection of futures using the wait function:

from dask.distributed import wait

>>> wait(futures)

This blocks until all futures are finished or have erred.

You can also iterate over the futures as they complete using the as_completed function:

from dask.distributed import as_completed

futures = client.map(score, x_values)

best = -1
for future in as_completed(futures):
   y = future.result()
   if y > best:
       best = y

For greater efficiency you can also ask as_completed to gather the results in the background.

for future in as_completed(futures, results=True):
   ...

Or collect futures all futures in batches that had arrived since the last iteration

for batch in as_completed(futures, results=True).batches():
   for future in batch:
       ...

Additionally, for iterative algorithms you can add more futures into the as_completed iterator

seq = as_completed(futures)

for future in seq:
    y = future.result()
    if condition(y):
        new_future = client.submit(...)
        seq.add(new_future)  # add back into the loop

Fire and Forget

fire_and_forget

Sometimes we don’t care about gathering the result of a task, and only care about side effects that it might have, like writing a result to a file.

>>> a = client.submit(load, filename)
>>> b = client.submit(process, a)
>>> c = client.submit(write, c, out_filename)

As noted above, Dask will stop work that doesn’t have any active futures. It thinks that because no one has a pointer to this data that no one cares. You can tell Dask to compute a task anyway, even if there are no active futures, using the fire_and_forget function:

from dask.distributed import fire_and_forget

>>> fire_and_forget(c)

This is particularly useful when a future may go out of scope, for example as part of a function:

def process(filename):
    out_filename = 'out-' + filename
    a = client.submit(load, filename)
    b = client.submit(process, a)
    c = client.submit(write, c, out_filename)
    fire_and_forget(c)
    return  # here we lose the reference to c, but that's now ok

for filename in filenames:
    process(filename)

Submit Tasks from Tasks

get_client
secede

Tasks can launch other tasks by getting their own client. This enables complex and highly dynamic workloads.

from dask.distributed import get_client

def my_function(x):
    ...

    # Get locally created client
    client = get_client()

    # Do normal client operations, asking cluster for computation
    a = client.submit(...)
    b = client.submit(...)
    a, b = client.gather([a, b])

    return a + b

It also allows you to set up long running tasks that watch other resources like sockets or physical sensors:

def monitor(device):
   client = get_client()
   while True:
       data = device.read_data()
       future = client.submit(process, data)
       fire_and_forget(future)

for device in devices:
    fire_and_forget(client.submit(monitor))

However, each running task takes up a single thread, and so if you launch many tasks that launch other tasks then it is possible to deadlock the system if you are not careful. You can call the secede function from within a task to have it remove itself from the dedicated thread pool into an administrative thread that does not take up a slot within the Dask worker:

from dask.distributed import get_client, secede

def monitor(device):
   client = get_client()
   secede()
   while True:
       data = device.read_data()
       future = client.submit(process, data)
       fire_and_forget(future)

Coordinate Data Between Clients

Queue
Variable

In the section above we saw that you could have multiple clients running at the same time, each of which generated and manipulated futures. These clients can coordinate with each other using Dask Queue and Variable objects, which can communicate futures or small bits of data between clients sensibly.

Dask queues follow the API for the standard Python Queue, but now move futures or small messages between clients. Queues serialize sensibly and reconnect themselves on remote clients if necessary.

from dask.distributed import Queue

def load_and_submit(filename):
    data = load(filename)
    client = get_client()
    future = client.submit(process, data)
    queue.put(future)

client = Client()

queue = Queue()

for filename in filenames:
    future = client.submit(load_and_submit, filename)
    fire_and_forget(filename)

while True:
    future = queue.get()
    print(future.result())

Queues can also send small pieces of information, anything that is msgpack encodable (ints, strings, bools, lists, dicts, etc..). This can be useful to send back small scores or administrative messages:

def func(x):
    try:
       ...
    except Exception as e:
        error_queue.put(str(e))

error_queue = Queue()

Variables are like Queues in that they communicate futures and small data between clients. However variables hold only a single value. You can get or set that value at any time.

>>> var = Variable('stopping-criterion')
>>> var.set(False)

>>> var.get()
False

This is often used to signal stopping criteria or current parameters, etc. between clients.

If you want to share large pieces of information then scatter the data first

>>> parameters = np.array(...)
>>> future = client.scatter(parameters)
>>> var.set(future)

API

Client

Client([address, start, loop, timeout, ...]) Drive computations on a distributed cluster
Client.cancel(futures) Cancel running futures
Client.compute(collections[, sync, ...]) Compute dask collections on cluster
Client.gather(futures[, errors, maxsize]) Gather futures from distributed memory
Client.get(dsk, keys[, restrictions, ...]) Compute dask graph
Client.get_dataset(name) Get named dataset from the scheduler
Client.get_executor
Client.has_what([workers]) Which keys are held by which workers
Client.list_datasets() List named datasets available on the scheduler
Client.map(func, *iterables, **kwargs) Map a function on a sequence of arguments
Client.ncores([workers]) The number of threads/cores available on each worker node
Client.persist(collections[, ...]) Persist dask collections on cluster
Client.publish_dataset(**kwargs) Publish named datasets to scheduler
Client.rebalance([futures, workers]) Rebalance data within network
Client.replicate(futures[, n, workers, ...]) Set replication of futures within network
Client.restart([environment]) Restart the distributed network
Client.run(function, *args, **kwargs) Run a function on all workers outside of task scheduling system
Client.run_on_scheduler
Client.scatter(data[, workers, broadcast, ...]) Scatter data into distributed memory
Client.shutdown([timeout]) Send shutdown signal and wait until scheduler terminates
Client.scheduler_info() Basic information about the workers in the cluster
Client.shutdown([timeout]) Send shutdown signal and wait until scheduler terminates
Client.start_ipython_workers([workers, ...]) Start IPython kernels on workers
Client.start_ipython_scheduler([magic_name, ...]) Start IPython kernel on the scheduler
Client.submit(func, *args, **kwargs) Submit a function application to the scheduler
Client.unpublish_dataset(name) Remove named datasets from scheduler
Client.upload_file(filename) Upload local package to workers
Client.who_has([futures]) The workers storing each future’s data

Future

Future
Future.add_done_callback
Future.cancel
Future.cancelled
Future.done
Future.exception
Future.result
Future.traceback

Functions

as_completed(fs) Return futures in the order in which they complete
fire_and_forget
get_client
secede
wait(fs[, timeout, return_when]) Wait until all futures are complete
distributed.as_completed(fs)

Return futures in the order in which they complete

This returns an iterator that yields the input future objects in the order in which they complete. Calling next on the iterator will block until the next future completes, irrespective of order.

This function does not return futures in the order in which they are input.

distributed.wait(fs, timeout=None, return_when='ALL_COMPLETED')

Wait until all futures are complete

Parameters:fs: list of futures
Returns:Named tuple of completed, not completed
class distributed.Client(address=None, start=True, loop=None, timeout=3, set_as_default=True)

Drive computations on a distributed cluster

The Client connects users to a distributed compute cluster. It provides an asynchronous user interface around functions and futures. This class resembles executors in concurrent.futures but also allows Future objects within submit/map calls.

Parameters:

address: string, tuple, or ``Scheduler``

This can be the address of a Scheduler server, either as a string '127.0.0.1:8787' or tuple ('127.0.0.1', 8787) or it can be a local Scheduler object.

See also

distributed.scheduler.Scheduler
Internal scheduler

Examples

Provide cluster’s head node address on initialization:

>>> client = Client('127.0.0.1:8787')  

Use submit method to send individual computations to the cluster

>>> a = client.submit(add, 1, 2)  
>>> b = client.submit(add, 10, 20)  

Continue using submit or map on results to build up larger computations

>>> c = client.submit(add, a, b)  

Gather results with the gather method.

>>> client.gather([c])  
33
cancel(futures)

Cancel running futures

This stops future tasks from being scheduled if they have not yet run and deletes them if they have already run. After calling, this result and all dependent results will no longer be accessible

Parameters:futures: list of Futures
compute(collections, sync=False, optimize_graph=True, workers=None, allow_other_workers=False, **kwargs)

Compute dask collections on cluster

Parameters:

collections: iterable of dask objects or single dask object

Collections like dask.array or dataframe or dask.value objects

sync: bool (optional)

Returns Futures if False (default) or concrete values if True

optimize_graph: bool

Whether or not to optimize the underlying graphs

workers: str, list, dict

Which workers can run which parts of the computation If a string a list then the output collections will run on the listed

workers, but other sub-computations can run anywhere

If a dict then keys should be (tuples of) collections and values

should be addresses or lists.

allow_other_workers: bool, list

If True then all restrictions in workers= are considered loose If a list then only the keys for the listed collections are loose

**kwargs:

Options to pass to the graph optimize calls

Returns:

List of Futures if input is a sequence, or a single future otherwise

See also

Client.get
Normal synchronous dask.get function

Examples

>>> from dask import do, value
>>> from operator import add
>>> x = dask.do(add)(1, 2)
>>> y = dask.do(add)(x, x)
>>> xx, yy = client.compute([x, y])  
>>> xx  
<Future: status: finished, key: add-8f6e709446674bad78ea8aeecfee188e>
>>> xx.result()  
3
>>> yy.result()  
6

Also support single arguments

>>> xx = client.compute(x)  
gather(futures, errors='raise', maxsize=0)

Gather futures from distributed memory

Accepts a future, nested container of futures, iterator, or queue. The return type will match the input type.

Returns:Future results

See also

Client.scatter
Send data out to cluster

Examples

>>> from operator import add  
>>> c = Client('127.0.0.1:8787')  
>>> x = c.submit(add, 1, 2)  
>>> c.gather(x)  
3
>>> c.gather([x, [x], x])  # support lists and dicts 
[3, [3], 3]
>>> seq = c.gather(iter([x, x]))  # support iterators 
>>> next(seq)  
3
get(dsk, keys, restrictions=None, loose_restrictions=None, **kwargs)

Compute dask graph

Parameters:

dsk: dict

keys: object, or nested lists of objects

restrictions: dict (optional)

A mapping of {key: {set of worker hostnames}} that restricts where jobs can take place

See also

Client.compute
Compute asynchronous collections

Examples

>>> from operator import add  
>>> c = Client('127.0.0.1:8787')  
>>> c.get({'x': (add, 1, 2)}, 'x')  
3
get_dataset(name)

Get named dataset from the scheduler

get_versions()

Return version info for the scheduler, all workers and myself

Examples

>>> c.get_versions()  
has_what(workers=None)

Which keys are held by which workers

Parameters:

workers: list (optional)

A list of worker addresses, defaults to all

Examples

>>> x, y, z = c.map(inc, [1, 2, 3])  
>>> wait([x, y, z])  
>>> c.has_what()  
{'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea',
                         'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b',
                         'inc-1e297fc27658d7b67b3a758f16bcf47a']}
list_datasets()

List named datasets available on the scheduler

map(func, *iterables, **kwargs)

Map a function on a sequence of arguments

Arguments can be normal objects or Futures

Parameters:

func: callable

iterables: Iterables, Iterators, or Queues

key: str, list

Prefix for task names if string. Explicit names if list.

pure: bool (defaults to True)

Whether or not the function is pure. Set pure=False for impure functions like np.random.random.

workers: set, iterable of sets

A set of worker hostnames on which computations may be performed. Leave empty to default to all workers (common case)

Returns:

List, iterator, or Queue of futures, depending on the type of the

inputs.

See also

Client.submit
Submit a single function

Examples

>>> L = client.map(func, sequence)  
nbytes(keys=None, summary=True)

The bytes taken up by each key on the cluster

This is as measured by sys.getsizeof which may not accurately reflect the true cost.

Parameters:

keys: list (optional)

A list of keys, defaults to all keys

summary: boolean, (optional)

Summarize keys into key types

See also

Client.who_has

Examples

>>> x, y, z = c.map(inc, [1, 2, 3])  
>>> c.nbytes(summary=False)  
{'inc-1c8dd6be1c21646c71f76c16d09304ea': 28,
 'inc-1e297fc27658d7b67b3a758f16bcf47a': 28,
 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b': 28}
>>> c.nbytes(summary=True)  
{'inc': 84}
ncores(workers=None)

The number of threads/cores available on each worker node

Parameters:

workers: list (optional)

A list of workers that we care about specifically. Leave empty to receive information about all workers.

Examples

>>> c.ncores()  
{'192.168.1.141:46784': 8,
 '192.167.1.142:47548': 8,
 '192.167.1.143:47329': 8,
 '192.167.1.144:37297': 8}
normalize_collection(collection)

Replace collection’s tasks by already existing futures if they exist

This normalizes the tasks within a collections task graph against the known futures within the scheduler. It returns a copy of the collection with a task graph that includes the overlapping futures.

See also

Client.persist
trigger computation of collection’s tasks

Examples

>>> len(x.dask)  # x is a dask collection with 100 tasks
100
>>> set(client.futures).intersection(x.dask)  # some overlap exists
10
>>> x = client.normalize_collection(x)
>>> len(x.dask)  # smaller computational graph
20
persist(collections, optimize_graph=True, workers=None, allow_other_workers=None, **kwargs)

Persist dask collections on cluster

Starts computation of the collection on the cluster in the background. Provides a new dask collection that is semantically identical to the previous one, but now based off of futures currently in execution.

Parameters:

collections: sequence or single dask object

Collections like dask.array or dataframe or dask.value objects

optimize_graph: bool

Whether or not to optimize the underlying graphs

workers: str, list, dict

Which workers can run which parts of the computation If a string a list then the output collections will run on the listed

workers, but other sub-computations can run anywhere

If a dict then keys should be (tuples of) collections and values

should be addresses or lists.

allow_other_workers: bool, list

If True then all restrictions in workers= are considered loose If a list then only the keys for the listed collections are loose

kwargs:

Options to pass to the graph optimize calls

Returns:

List of collections, or single collection, depending on type of input.

See also

Client.compute

Examples

>>> xx = client.persist(x)  
>>> xx, yy = client.persist([x, y])  
processing(workers=None)

The tasks currently running on each worker

Parameters:

workers: list (optional)

A list of worker addresses, defaults to all

Examples

>>> x, y, z = c.map(inc, [1, 2, 3])  
>>> c.processing()  
{'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea',
                         'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b',
                         'inc-1e297fc27658d7b67b3a758f16bcf47a']}
publish_dataset(**kwargs)

Publish named datasets to scheduler

This stores a named reference to a dask collection or list of futures on the scheduler. These references are available to other Clients which can download the collection or futures with get_dataset.

Datasets are not immediately computed. You may wish to call Client.persist prior to publishing a dataset.

Parameters:

kwargs: dict

named collections to publish on the scheduler

Returns:

None

Examples

Publishing client:

>>> df = dd.read_csv('s3://...')  
>>> df = c.persist(df) 
>>> c.publish_dataset(my_dataset=df)  

Receiving client:

>>> c.list_datasets()  
['my_dataset']
>>> df2 = c.get_dataset('my_dataset')  
rebalance(futures=None, workers=None)

Rebalance data within network

Move data between workers to roughly balance memory burden. This either affects a subset of the keys/workers or the entire network, depending on keyword arguments.

This operation is generally not well tested against normal operation of the scheduler. It it not recommended to use it while waiting on computations.

Parameters:

futures: list, optional

A list of futures to balance, defaults all data

workers: list, optional

A list of workers on which to balance, defaults to all workers

replicate(futures, n=None, workers=None, branching_factor=2)

Set replication of futures within network

Copy data onto many workers. This helps to broadcast frequently accessed data and it helps to improve resilience.

This performs a tree copy of the data throughout the network individually on each piece of data. This operation blocks until complete. It does not guarantee replication of data to future workers.

Parameters:

futures: list of futures

Futures we wish to replicate

n: int, optional

Number of processes on the cluster on which to replicate the data. Defaults to all.

workers: list of worker addresses

Workers on which we want to restrict the replication. Defaults to all.

branching_factor: int, optional

The number of workers that can copy data in each generation

See also

Client.rebalance

Examples

>>> x = c.submit(func, *args)  
>>> c.replicate([x])  # send to all workers  
>>> c.replicate([x], n=3)  # send to three workers  
>>> c.replicate([x], workers=['alice', 'bob'])  # send to specific  
>>> c.replicate([x], n=1, workers=['alice', 'bob'])  # send to one of specific workers  
>>> c.replicate([x], n=1)  # reduce replications 
restart(environment=None)

Restart the distributed network

This kills all active work, deletes all data on the network, and restarts the worker processes.

run(function, *args, **kwargs)

Run a function on all workers outside of task scheduling system

This calls a function on all currently known workers immediately, blocks until those results come back, and returns the results asynchronously as a dictionary keyed by worker address. This method if generally used for side effects, such and collecting diagnostic information or installing libraries.

Parameters:

function: callable

*args: arguments for remote function

**kwargs: keyword arguments for remote function

workers: list

Workers on which to run the function. Defaults to all known workers.

Examples

>>> c.run(os.getpid)  
{'192.168.0.100:9000': 1234,
 '192.168.0.101:9000': 4321,
 '192.168.0.102:9000': 5555}

Restrict computation to particular workers with the workers= keyword argument.

>>> c.run(os.getpid, workers=['192.168.0.100:9000',
...                           '192.168.0.101:9000'])  
{'192.168.0.100:9000': 1234,
 '192.168.0.101:9000': 4321}
scatter(data, workers=None, broadcast=False, maxsize=0)

Scatter data into distributed memory

This moves data from the local client process into the workers of the distributed scheduler. Note that it is often better to submit jobs to your workers to have them load the data rather than loading data locally and then scattering it out to them.

Parameters:

data: list, iterator, dict, or Queue

Data to scatter out to workers. Output type matches input type.

workers: list of tuples (optional)

Optionally constrain locations of data. Specify workers as hostname/port pairs, e.g. ('127.0.0.1', 8787).

broadcast: bool (defaults to False)

Whether to send each data element to all workers. By default we round-robin based on number of cores.

maxsize: int (optional)

Maximum size of queue if using queues, 0 implies infinite

Returns:

List, dict, iterator, or queue of futures matching the type of input.

See also

Client.gather
Gather data back to local process

Examples

>>> c = Client('127.0.0.1:8787')  
>>> c.scatter([1, 2, 3])  
[<Future: status: finished, key: c0a8a20f903a4915b94db8de3ea63195>,
 <Future: status: finished, key: 58e78e1b34eb49a68c65b54815d1b158>,
 <Future: status: finished, key: d3395e15f605bc35ab1bac6341a285e2>]
>>> c.scatter({'x': 1, 'y': 2, 'z': 3})  
{'x': <Future: status: finished, key: x>,
 'y': <Future: status: finished, key: y>,
 'z': <Future: status: finished, key: z>}

Constrain location of data to subset of workers

>>> c.scatter([1, 2, 3], workers=[('hostname', 8788)])   

Handle streaming sequences of data with iterators or queues

>>> seq = c.scatter(iter([1, 2, 3]))  
>>> next(seq)  
<Future: status: finished, key: c0a8a20f903a4915b94db8de3ea63195>,

Broadcast data to all workers

>>> [future] = c.scatter([element], broadcast=True)  
scheduler_info()

Basic information about the workers in the cluster

Examples

>>> c.scheduler_info()  
{'id': '2de2b6da-69ee-11e6-ab6a-e82aea155996',
 'services': {},
 'type': 'Scheduler',
 'workers': {'127.0.0.1:40575': {'active': 0,
                                 'last-seen': 1472038237.4845693,
                                 'name': '127.0.0.1:40575',
                                 'services': {},
                                 'stored': 0,
                                 'time-delay': 0.0061032772064208984}}}
shutdown(timeout=10)

Send shutdown signal and wait until scheduler terminates

This cancels all currently running tasks, clears the state of the scheduler, and shuts down all workers and scheduler.

You do not need to call this when you finish your session. You only need to call this if you want to take down the distributed cluster.

See also

Client.restart

stacks(workers=None)

The task queues on each worker

Parameters:

workers: list (optional)

A list of worker addresses, defaults to all

Examples

>>> x, y, z = c.map(inc, [1, 2, 3])  
>>> c.stacks()  
{'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea',
                         'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b',
                         'inc-1e297fc27658d7b67b3a758f16bcf47a']}
start(**kwargs)

Start scheduler running in separate thread

start_ipython_scheduler(magic_name='scheduler_if_ipython', qtconsole=False, qtconsole_args=None)

Start IPython kernel on the scheduler

Parameters:

magic_name: str or None (optional)

If defined, register IPython magic with this name for executing code on the scheduler. If not defined, register %scheduler magic if IPython is running.

qtconsole: bool (optional)

If True, launch a Jupyter QtConsole connected to the worker(s).

qtconsole_args: list(str) (optional)

Additional arguments to pass to the qtconsole on startup.

Returns:

connection_info: dict

connection_info dict containing info necessary to connect Jupyter clients to the scheduler.

See also

Client.start_ipython_workers
Start IPython on the workers

Examples

>>> c.start_ipython_scheduler() 
>>> %scheduler scheduler.processing  
{'127.0.0.1:3595': {'inc-1', 'inc-2'},
 '127.0.0.1:53589': {'inc-2', 'add-5'}}
>>> c.start_ipython_scheduler(qtconsole=True) 
start_ipython_workers(workers=None, magic_names=False, qtconsole=False, qtconsole_args=None)

Start IPython kernels on workers

Parameters:

workers: list (optional)

A list of worker addresses, defaults to all

magic_names: str or list(str) (optional)

If defined, register IPython magics with these names for executing code on the workers. If string has asterix then expand asterix into 0, 1, ..., n for n workers

qtconsole: bool (optional)

If True, launch a Jupyter QtConsole connected to the worker(s).

qtconsole_args: list(str) (optional)

Additional arguments to pass to the qtconsole on startup.

Returns:

iter_connection_info: list

List of connection_info dicts containing info necessary to connect Jupyter clients to the workers.

See also

Client.start_ipython_scheduler
start ipython on the scheduler

Examples

>>> info = c.start_ipython_workers() 
>>> %remote info['192.168.1.101:5752'] worker.data  
{'x': 1, 'y': 100}
>>> c.start_ipython_workers('192.168.1.101:5752', magic_names='w') 
>>> %w worker.data  
{'x': 1, 'y': 100}
>>> c.start_ipython_workers('192.168.1.101:5752', qtconsole=True) 

Add asterix * in magic names to add one magic per worker

>>> c.start_ipython_workers(magic_names='w_*') 
>>> %w_0 worker.data  
{'x': 1, 'y': 100}
>>> %w_1 worker.data  
{'z': 5}
submit(func, *args, **kwargs)

Submit a function application to the scheduler

Parameters:

func: callable

*args:

**kwargs:

pure: bool (defaults to True)

Whether or not the function is pure. Set pure=False for impure functions like np.random.random.

workers: set, iterable of sets

A set of worker hostnames on which computations may be performed. Leave empty to default to all workers (common case)

allow_other_workers: bool (defaults to False)

Used with workers. Inidicates whether or not the computations may be performed on workers that are not in the workers set(s).

Returns:

Future

See also

Client.map
Submit on many arguments at once

Examples

>>> c = client.submit(add, a, b)  
unpublish_dataset(name)

Remove named datasets from scheduler

Examples

>>> c.list_datasets()  
['my_dataset']
>>> c.unpublish_datasets('my_dataset')  
>>> c.list_datasets()  
[]
upload_file(filename)

Upload local package to workers

This sends a local file up to all worker nodes. This file is placed into a temporary directory on Python’s system path so any .py or .egg files will be importable.

Parameters:

filename: string

Filename of .py or .egg file to send to workers

Examples

>>> client.upload_file('mylibrary.egg')  
>>> from mylibrary import myfunc  
>>> L = c.map(myfunc, seq)  
who_has(futures=None)

The workers storing each future’s data

Parameters:

futures: list (optional)

A list of futures, defaults to all data

Examples

>>> x, y, z = c.map(inc, [1, 2, 3])  
>>> wait([x, y, z])  
>>> c.who_has()  
{'inc-1c8dd6be1c21646c71f76c16d09304ea': ['192.168.1.141:46784'],
 'inc-1e297fc27658d7b67b3a758f16bcf47a': ['192.168.1.141:46784'],
 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b': ['192.168.1.141:46784']}
>>> c.who_has([x, y])  
{'inc-1c8dd6be1c21646c71f76c16d09304ea': ['192.168.1.141:46784'],
 'inc-1e297fc27658d7b67b3a758f16bcf47a': ['192.168.1.141:46784']}