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 allowsFuture
objects withinsubmit/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 localScheduler
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
See also
-
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
See also
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
See also
-
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 likenp.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
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.
See also
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
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
See also
Client.stacks
,Client.who_has
,Client.has_what
,Client.ncores
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
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
-
stacks
(workers=None)¶ The task queues on each worker
Parameters: workers: list (optional)
A list of worker addresses, defaults to all
See also
Client.processing
,Client.who_has
,Client.has_what
,Client.ncores
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 likenp.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
See also
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
See also
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']}