Futures
Dask supports a real-time task framework that extends Python’sconcurrent.futuresinterface. This interface is good for arbitrary task scheduling likedask.delayed, but is immediate rather than lazy, whichprovides some more flexibility in situations where the computations may evolveover time.
These features depend on the second generation task scheduler found indask.distributed (which,despite its name, runs very well on a single machine).
Examples
Visit https://examples.dask.org/futures.html to see and run examplesusing futures with Dask.
Start Dask Client
You must start a Client
to use the futures interface. This tracks stateamong 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 adiagnostic dashboard at http://localhost:8787 .
Submit Tasks
Client.submit (self, func, *args[, key, …]) | Submit a function application to the scheduler |
Client.map (self, func, *iterables[, key, …]) | Map a function on a sequence of arguments |
Future.result (self[, timeout]) | Wait until computation completes, gather result to local process. |
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
The submit
function returns a Future
, which refers to a remote result. This result maynot yet be completed:
- >>> a
- <Future: status: pending, key: inc-b8aaf26b99466a7a1980efa1ade6701d>
Eventually it will complete. The result stays in the remotethread/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 dependencytracking; once all input futures have completed, they will be moved onto asingle worker (if necessary), and then the computation that depends on themwill be started. You do not need to wait for inputs to finish beforesubmitting 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 samefunction and many inputs:
- futures = client.map(inc, range(1000))
However, note that each task comes with about 1ms of overhead. If you want tomap a function over a large number of inputs, then you might considerdask.bag or dask.dataframe instead.
Move Data
Future.result (self[, timeout]) | Wait until computation completes, gather result to local process. |
Client.gather (self, futures[, errors, …]) | Gather futures from distributed memory |
Client.scatter (self, data[, workers, …]) | 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 resultback 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 futuresequentially:
- >>> # 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 workerand 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 befaster. This is especially true if you use processes or distributed workers(where data transfer is necessary) and you want to use df
in manycomputations. Scattering the data beforehand avoids excessive data movement.
Calling scatter on a list scatters all elements individually. Dask will spreadthese 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 (self, **kwargs) | Cancel request to run this future |
Future.exception (self[, timeout]) | Return the exception of a failed task |
Future.traceback (self[, timeout]) | Return the traceback of a failed task |
Client.cancel (self, futures[, asynchronous, …]) | Cancel running futures |
Dask will only compute and hold onto results for which there are activefutures. In this way, your local variables define what is active in Dask. Whena future is garbage collected by your local Python session, Dask will feel freeto delete that data or stop ongoing computations that were trying to produceit:
- >>> del future # deletes remote data once future is garbage collected
You can also explicitly cancel a task using the Future.cancel
orClient.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 ifyou 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 theFuture.exception
or Future.traceback
methods.
Waiting on Futures
as_completed ([futures, loop, with_results, …]) | Return futures in the order in which they complete |
wait (fs[, timeout, return_when]) | Wait until all/any futures are finished |
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 theas_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 resultsin the background:
- for future, result in as_completed(futures, with_results=True):
- # y = future.result() # don't need this
- ...
Or collect all futures in batches that had arrived since the last iteration:
- for batch in as_completed(futures, with_results=True).batches():
- for future, result in batch:
- ...
Additionally, for iterative algorithms, you can add more futures into theascompleted
iterator _during iteration:
- 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 (obj) | Run tasks at least once, even if we release the futures |
Sometimes we don’t care about gathering the result of a task, and only careabout 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, b, out_filename)
As noted above, Dask will stop work that doesn’t have any active futures. Itthinks that because no one has a pointer to this data that no one cares. Youcan 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, aspart of a function:
- def process(filename):
- out_filename = 'out-' + filename
- a = client.submit(load, filename)
- b = client.submit(process, a)
- c = client.submit(write, b, 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 ([address, timeout, resolve_address]) | Get a client while within a task. |
rejoin () | Have this thread rejoin the ThreadPoolExecutor |
secede () | Have this task secede from the worker’s thread pool |
This is an advanced feature and is rarely necessary in the common case.
Tasks can launch other tasks by getting their own client. This enables complexand 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 likesockets 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 manytasks that launch other tasks, then it is possible to deadlock the system if youare not careful. You can call the secede
function from within a task tohave it remove itself from the dedicated thread pool into an administrativethread 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() # remove this task from the thread pool
- while True:
- data = device.read_data()
- future = client.submit(process, data)
- fire_and_forget(future)
If you intend to do more work in the same thread after waiting on client work,you may want to explicitly block until the thread is able to rejoin thethread pool. This allows some control over the number of threads that arecreated and stops too many threads from being active at once, over-saturatingyour hardware:
- def f(n): # assume that this runs as a task
- client = get_client()
- secede() # secede while we wait for results to come back
- futures = client.map(func, range(n))
- results = client.gather(futures)
- rejoin() # block until a slot is open in the thread pool
- result = analyze(results)
- return result
Alternatively, you can just use the normal compute
function within atask. This will automatically call secede
and rejoin
appropriately:
- def f(name, fn):
- df = dd.read_csv(fn) # note that this is a dask collection
- result = df[df.name == name].count()
- # This calls secede
- # Then runs the computation on the cluster (including this worker)
- # Then blocks on rejoin, and finally delivers the answer
- result = result.compute()
- return result
Coordination Primitives
Queue ([name, client, maxsize]) | Distributed Queue |
Variable ([name, client, maxsize]) | Distributed Global Variable |
Lock ([name, client]) | Distributed Centralized Lock |
Pub (name[, worker, client]) | Publish data with Publish-Subscribe pattern |
Sub (name[, worker, client]) | Subscribe to a Publish/Subscribe topic |
Sometimes situations arise where tasks, workers, or clients need to coordinatewith each other in ways beyond normal task scheduling with futures. In thesecases Dask provides additional primitives to help in complex situations.
Dask provides distributed versions of coordination primitives like locks,queues, global variables, and pub-sub systems that, where appropriate, matchtheir in-memory counterparts. These can be used to control access to externalresources, track progress of ongoing computations, or share data inside-channels between many workers, clients, and tasks sensibly.
These features are rarely necessary for common use of Dask. We recommend thatbeginning users stick with using the simpler futures found above (likeClient.submit
and Client.gather
) rather than embracing needlesslycomplex techniques.
Queues
Queue ([name, client, maxsize]) | Distributed Queue |
Dask queues follow the API for the standard Python Queue, but now move futuresor small messages between clients. Queues serialize sensibly and reconnectthemselves 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(future)
- while True:
- future = queue.get()
- print(future.result())
Queues can also send small pieces of information, anything that is msgpackencodable (ints, strings, bools, lists, dicts, etc.). This can be useful tosend back small scores or administrative messages:
- def func(x):
- try:
- ...
- except Exception as e:
- error_queue.put(str(e))
- error_queue = Queue()
Queues are mediated by the central scheduler, and so they are not ideal forsending large amounts of data (everything you send will be routed through acentral point). They are well suited to move around small bits of metadata, orfutures. These futures may point to much larger pieces of data safely:
- >>> x = ... # my large numpy array
- # Don't do this!
- >>> q.put(x)
- # Do this instead
- >>> future = client.scatter(x)
- >>> q.put(future)
- # Or use futures for metadata
- >>> q.put({'status': 'OK', 'stage=': 1234})
If you’re looking to move large amounts of data between workers, then you mightalso want to consider the Pub/Sub system described a few sections below.
Global Variables
Variable ([name, client, maxsize]) | Distributed Global Variable |
Variables are like Queues in that they communicate futures and small databetween clients. However, variables hold only a single value. You can get orset 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 parametersbetween 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)
Locks
Lock ([name, client]) | Distributed Centralized Lock |
You can also hold onto cluster-wide locks using the Lock
object.Dask Locks have the same API as normal threading.Lock
objects, except thatthey work across the cluster:
- from dask.distributed import Lock
- lock = Lock()
- with lock:
- # access protected resource
You can manage several locks at the same time. Lock can either be given aconsistent name or you can pass the lock object around itself.
Using a consistent name is convenient when you want to lock some known named resource:
- from dask.distributed import Lock
- def load(fn):
- with Lock('the-production-database'):
- # read data from filename using some sensitive source
- return ...
- futures = client.map(load, filenames)
Passing around a lock works as well and is easier when you want to create short-termlocks for a particular situation:
- from dask.distributed import Lock
- lock = Lock()
- def load(fn, lock=None):
- with lock:
- # read data from filename using some sensitive source
- return ...
- futures = client.map(load, filenames, lock=lock)
This can be useful if you want to control concurrent access to some externalresource like a database or un-thread-safe library.
Publish-Subscribe
Pub (name[, worker, client]) | Publish data with Publish-Subscribe pattern |
Sub (name[, worker, client]) | Subscribe to a Publish/Subscribe topic |
Dask implements the Publish Subscribe pattern,providing an additional channel of communication between ongoing tasks.
This allows clients and workers to directly communicate data between eachother with a typical Publish-Subscribe pattern. This involves twocomponents,
Pub objects, into which we put data:
- >>> pub = Pub('my-topic')
- >>> pub.put(123)
And Sub objects, from which we collect data:
- >>> sub = Sub('my-topic')
- >>> sub.get()
- 123
Many Pub and Sub objects can exist for the same topic. All data sent fromany Pub will be sent to all Sub objects on that topic that are currentlyconnected. Pub’s and Sub’s find each other using the scheduler, but theycommunicate directly with each other without coordination from thescheduler.
Pubs and Subs use the central scheduler to find each other, but not tomediate the communication. This means that there is very little additionallatency or overhead, and they are appropriate for very frequent datatransfers. For context, most data transfer first checks with the scheduler to find whichworkers should participate, and then does direct worker-to-workertransfers. This checking in with the scheduler provides some stabilityguarantees, but also adds in a few extra network hops. PubSub doesn’t dothis, and so is faster, but also can easily drop messages if Pubs or Subsdisappear without notice.
When using a Pub or Sub from a Client all communications will be routedthrough the scheduler. This can cause some performance degradation. Pubsand Subs only operate at top-speed when they are both on workers.
Parameters:
- name: object (msgpack serializable)
- The name of the group of Pubs and Subs on which to participate
See also
Examples
- >>> pub = Pub('my-topic')
- >>> sub = Sub('my-topic')
- >>> pub.put([1, 2, 3])
- >>> sub.get()
- [1, 2, 3]
You can also use sub within a for loop:
- >>> for msg in sub: # doctest: +SKIP
- ... print(msg)
or an async for loop
- >>> async for msg in sub: # doctest: +SKIP
- ... print(msg)
Similarly the .get
method will return an awaitable if used by an asyncclient or within the IOLoop thread of a worker
- >>> await sub.get() # doctest: +SKIP
You can see the set of connected worker subscribers by looking at the.subscribers
attribute:
- >>> pub.subscribers
- {'tcp://...': {},
- 'tcp://...': {}}
Actors
Note
This is an advanced feature and is rarely necessary in the common case.
Note
This is an experimental feature and is subject to change without notice.
Actors allow workers to manage rapidly changing state without coordinating withthe central scheduler. This has the advantage of reducing latency(worker-to-worker roundtrip latency is around 1ms), reducing pressure on thecentralized scheduler (workers can coordinate actors entirely among each other),and also enabling workflows that require stateful or in-place memorymanipulation.
However, these benefits come at a cost. The scheduler is unaware of actors andso they don’t benefit from diagnostics, load balancing, or resilience. Once anactor is running on a worker it is forever tied to that worker. If that workerbecomes overburdened or dies, then there is no opportunity to recover theworkload.
Because Actors avoid the central scheduler they can be high-performing, but not resilient.
Example: Counter
An actor is a class containing both state and methods that is submitted to aworker:
- class Counter:
- n = 0
- def __init__(self):
- self.n = 0
- def increment(self):
- self.n += 1
- return self.n
- from dask.distributed import Client
- client = Client()
- future = client.submit(Counter, actor=True)
- counter = future.result()
- >>> counter
- <Actor: Counter, key=Counter-afa1cdfb6b4761e616fa2cfab42398c8>
Method calls on this object produce ActorFutures
, which are similar tonormal Futures, but interact only with the worker holding the Actor:
- >>> future = counter.increment()
- >>> future
- <ActorFuture>
- >>> future.result()
- 1
Attribute access is synchronous and blocking:
- >>> counter.n
- 1
Example: Parameter Server
- import numpy as np
- from dask.distributed import Client
- client = Client(processes=False)
- class ParameterServer:
- def __init__(self):
- self.data = dict()
- def put(self, key, value):
- self.data[key] = value
- def get(self, key):
- return self.data[key]
- ps_future = client.submit(ParameterServer, actor=True)
- ps = ps_future.result()
- ps.put('parameters', np.random.random(1000))
- def train(batch, ps):
- params = ps.get('parameters')
- for batch in batches:
Asynchronous Operation
All operations that require talking to the remote worker are awaitable:
- async def f():
- future = client.submit(Counter, actor=True)
- counter = await future # gather actor object locally
- counter.increment() # send off a request asynchronously
- await counter.increment() # or wait until it was received
- n = await counter.n # attribute access also must be awaited
API
Client
Client ([address, loop, timeout, …]) | Connect to and submit computation to a Dask cluster |
Client.cancel (self, futures[, asynchronous, …]) | Cancel running futures |
Client.compute (self, collections[, sync, …]) | Compute dask collections on cluster |
Client.gather (self, futures[, errors, …]) | Gather futures from distributed memory |
Client.get (self, dsk, keys[, restrictions, …]) | Compute dask graph |
Client.get_dataset (self, name, **kwargs) | Get named dataset from the scheduler |
Client.get_executor (self, **kwargs) | Return a concurrent.futures Executor for submitting tasks on this Client |
Client.has_what (self[, workers]) | Which keys are held by which workers |
Client.list_datasets (self, **kwargs) | List named datasets available on the scheduler |
Client.map (self, func, *iterables[, key, …]) | Map a function on a sequence of arguments |
Client.ncores (self[, workers]) | The number of threads/cores available on each worker node |
Client.persist (self, collections[, …]) | Persist dask collections on cluster |
Client.profile (self[, key, start, stop, …]) | Collect statistical profiling information about recent work |
Client.publish_dataset (self, *args, **kwargs) | Publish named datasets to scheduler |
Client.rebalance (self[, futures, workers]) | Rebalance data within network |
Client.replicate (self, futures[, n, …]) | Set replication of futures within network |
Client.restart (self, **kwargs) | Restart the distributed network |
Client.run (self, function, *args, **kwargs) | Run a function on all workers outside of task scheduling system |
Client.run_on_scheduler (self, function, …) | Run a function on the scheduler process |
Client.scatter (self, data[, workers, …]) | Scatter data into distributed memory |
Client.shutdown (self) | Shut down the connected scheduler and workers |
Client.scheduler_info (self, **kwargs) | Basic information about the workers in the cluster |
Client.shutdown (self) | Shut down the connected scheduler and workers |
Client.start_ipython_workers (self[, …]) | Start IPython kernels on workers |
Client.start_ipython_scheduler (self[, …]) | Start IPython kernel on the scheduler |
Client.submit (self, func, *args[, key, …]) | Submit a function application to the scheduler |
Client.unpublish_dataset (self, name, **kwargs) | Remove named datasets from scheduler |
Client.upload_file (self, filename, **kwargs) | Upload local package to workers |
Client.who_has (self[, futures]) | The workers storing each future’s data |
Future
Future (key[, client, inform, state]) | A remotely running computation |
Future.add_done_callback (self, fn) | Call callback on future when callback has finished |
Future.cancel (self, **kwargs) | Cancel request to run this future |
Future.cancelled (self) | Returns True if the future has been cancelled |
Future.done (self) | Is the computation complete? |
Future.exception (self[, timeout]) | Return the exception of a failed task |
Future.result (self[, timeout]) | Wait until computation completes, gather result to local process. |
Future.traceback (self[, timeout]) | Return the traceback of a failed task |
Functions
as_completed ([futures, loop, with_results, …]) | Return futures in the order in which they complete |
fire_and_forget (obj) | Run tasks at least once, even if we release the futures |
get_client ([address, timeout, resolve_address]) | Get a client while within a task. |
secede () | Have this task secede from the worker’s thread pool |
rejoin () | Have this thread rejoin the ThreadPoolExecutor |
wait (fs[, timeout, return_when]) | Wait until all/any futures are finished |
distributed.
ascompleted
(_futures=None, loop=None, with_results=False, raise_errors=True)- Return futures in the order in which they complete
This returns an iterator that yields the input future objects in the orderin which they complete. Calling next
on the iterator will block untilthe next future completes, irrespective of order.
Additionally, you can also add more futures to this object duringcomputation with the .add
method
Parameters:
- futures: Collection of futures
A list of Future objects to be iterated over in the order in which theycomplete
with_results: bool (False)
Whether to wait and include results of futures as well;in this case as_completed yields a tuple of (future, result)
raise_errors: bool (True)
- Whether we should raise when the result of a future raises an exception;only affects behavior when with_results=True.
Examples
- >>> x, y, z = client.map(inc, [1, 2, 3]) # doctest: +SKIP
- >>> for future in as_completed([x, y, z]): # doctest: +SKIP
- ... print(future.result()) # doctest: +SKIP
- 3
- 2
- 4
Add more futures during computation
- >>> x, y, z = client.map(inc, [1, 2, 3]) # doctest: +SKIP
- >>> ac = as_completed([x, y, z]) # doctest: +SKIP
- >>> for future in ac: # doctest: +SKIP
- ... print(future.result()) # doctest: +SKIP
- ... if random.random() < 0.5: # doctest: +SKIP
- ... ac.add(c.submit(double, future)) # doctest: +SKIP
- 4
- 2
- 8
- 3
- 6
- 12
- 24
Optionally wait until the result has been gathered as well
- >>> ac = as_completed([x, y, z], with_results=True) # doctest: +SKIP
- >>> for future, result in ac: # doctest: +SKIP
- ... print(result) # doctest: +SKIP
- 2
- 4
- 3
Under normal operation Dask will not run any tasks for which there is notan active future (this avoids unnecessary work in many situations).However sometimes you want to just fire off a task, not track its future,and expect it to finish eventually. You can use this function on a futureor collection of futures to ask Dask to complete the task even if no activeclient is tracking it.
The results will not be kept in memory after the task completes (unlessthere is an active future) so this is only useful for tasks that depend onside effects.
Parameters:
- obj: Future, list, dict, dask collection
- The futures that you want to run at least once
Examples
- >>> fire_and_forget(client.submit(func, *args)) # doctest: +SKIP
distributed.
getclient
(_address=None, timeout=3, resolve_address=True)- Get a client while within a task.
This client connects to the same scheduler to which the worker is connected
Parameters:
- address:str, optional
The address of the scheduler to connect to. Defaults to the schedulerthe worker is connected to.
timeout:int, default 3
Timeout (in seconds) for getting the Client
resolve_address:bool, default True
- Whether to resolve address to its canonical form.Returns:
- Client
See also
get_worker
worker_client
secede
Examples
- >>> def f():
- ... client = get_client()
- ... futures = client.map(lambda x: x + 1, range(10)) # spawn many tasks
- ... results = client.gather(futures)
- ... return sum(results)
- >>> future = client.submit(f) # doctest: +SKIP
- >>> future.result() # doctest: +SKIP
- 55
This opens up a new scheduling slot and a new thread for a new task. Thisenables the client to schedule tasks on this node, which isespecially useful while waiting for other jobs to finish (e.g., withclient.gather
).
See also
get_client
get_worker
Examples
- >>> def mytask(x):
- ... # do some work
- ... client = get_client()
- ... futures = client.map(...) # do some remote work
- ... secede() # while that work happens, remove ourself from the pool
- ... return client.gather(futures) # return gathered results
This will block until a new slot opens up in the executor. The next threadto finish a task will leave the pool to allow this one to join.
See also
secede
- leave the thread pool
distributed.
wait
(fs, timeout=None, return_when='ALL_COMPLETED')- Wait until all/any futures are finished
Parameters:
- fs: list of futures
- timeout: number, optional
Time in seconds after which to raise a
dask.distributed.TimeoutError
return_when: str, optional
- One of ALL_COMPLETED or FIRST_COMPLETEDReturns:
- Named tuple of completed, not completed
- class
distributed.
Client
(address=None, loop=None, timeout='no_default', set_as_default=True, scheduler_file=None, security=None, asynchronous=False, name=None, heartbeat_interval=None, serializers=None, deserializers=None, extensions=[], direct_to_workers=None, **kwargs ) - Connect to and submit computation to a Dask cluster
The Client connects users to a Dask cluster. It provides an asynchronoususer interface around functions and futures. This class resemblesexecutors in concurrent.futures
but also allows Future
objectswithin submit/map
calls. When a Client is instantiated it takes overall dask.compute
and dask.persist
calls by default.
It is also common to create a Client without specifying the scheduleraddress , like Client()
. In this case the Client creates aLocalCluster
in the background and connects to that. Any extrakeywords are passed from Client to LocalCluster in this case. See theLocalCluster documentation for more information.
Parameters:
- address: string, or Cluster
This can be the address of a
Scheduler
server like a string'127.0.0.1:8786'
or a cluster object likeLocalCluster()
timeout: int
Timeout duration for initial connection to the scheduler
set_as_default: bool (True)
Claim this scheduler as the global dask scheduler
scheduler_file: string (optional)
Path to a file with scheduler information if available
security: Security or bool, optional
Optional security information. If creating a local cluster can alsopass in
True
, in which case temporary self-signed credentials willbe created automatically.asynchronous: bool (False by default)
Set to True if using this client within async/await functions or withinTornado gen.coroutines. Otherwise this should remain False for normaluse.
name: string (optional)
Gives the client a name that will be included in logs generated onthe scheduler for matters relating to this client
direct_to_workers: bool (optional)
Whether or not to connect directly to the workers, or to askthe scheduler to serve as intermediary.
heartbeat_interval: int
Time in milliseconds between heartbeats to scheduler
**kwargs:
- If you do not pass a scheduler address, Client will create a
LocalCluster
object, passing any extra keyword arguments.
See also
distributed.scheduler.Scheduler
- Internal scheduler
distributed.deploy.local.LocalCluster
Examples
Provide cluster’s scheduler node address on initialization:
- >>> client = Client('127.0.0.1:8786') # doctest: +SKIP
Use submit
method to send individual computations to the cluster
- >>> a = client.submit(add, 1, 2) # doctest: +SKIP
- >>> b = client.submit(add, 10, 20) # doctest: +SKIP
Continue using submit or map on results to build up larger computations
- >>> c = client.submit(add, a, b) # doctest: +SKIP
Gather results with the gather
method.
- >>> client.gather(c) # doctest: +SKIP
- 33
You can also call Client with no arguments in order to create your ownlocal cluster.
- >>> client = Client() # makes your own local "cluster" # doctest: +SKIP
Extra keywords will be passed directly to LocalCluster
- >>> client = Client(processes=False, threads_per_worker=1) # doctest: +SKIP
This is true if the user signaled that we might be when creating theclient as in the following:
- client = Client(asynchronous=True)
However, we override this expectation if we can definitively tell thatwe are running from a thread that is not the event loop. This iscommon when calling get_client() from within a worker task. Eventhough the client was originally created in asynchronous mode we mayfind ourselves in contexts when it is better to operate synchronously.
You can specify data of interest either by providing futures orcollections in the futures=
keyword or a list of explicit keys inthe keys=
keyword. If neither are provided then all call stackswill be returned.
Parameters:
- **futures: list (optional)**
-
List of futures, defaults to all data
- **keys: list (optional)**
-
List of key names, defaults to all data
Examples
- >>> df = dd.read_parquet(...).persist() # doctest: +SKIP
- >>> client.call_stack(df) # call on collections
- >>> client.call_stack() # Or call with no arguments for all activity # doctest: +SKIP
This stops future tasks from being scheduled if they have not yet runand deletes them if they have already run. After calling, this resultand all dependent results will no longer be accessible
Parameters:
- **futures: list of Futures**
-
- **force: boolean (False)**
-
Cancel this future even if other clients desire it
Clients will also close automatically when your Python session ends
If you started a client without arguments like Client()
then thiswill also close the local cluster that was started at the same time.
See also
- [<code>Client.restart</code>](#distributed.Client.restart)
-
compute
(self, collections, sync=False, optimize_graph=True, workers=None, allow_other_workers=False, resources=None, retries=0, priority=0, fifo_timeout='60s', actors=None, traverse=True, **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 computationIf a string a list then the output collections will run on the listedworkers, but other sub-computations can run anywhereIf a dict then keys should be (tuples of) collections and valuesshould be addresses or lists.
- **allow_other_workers: bool, list**
-
If True then all restrictions in workers= are considered looseIf a list then only the keys for the listed collections are loose
- **retries: int (default to 0)**
-
Number of allowed automatic retries if computing a result fails
- **priority: Number**
-
Optional prioritization of task. Zero is default.Higher priorities take precedence
- **fifo_timeout: timedelta str (defaults to ’60s’)**
-
Allowed amount of time between calls to consider the same priority
- **traverse: bool (defaults to True)**
-
By default dask traverses builtin python collections looking fordask objects passed to compute
. For large collections this canbe expensive. If none of the arguments contain any dask objects,set traverse=False
to avoid doing this traversal.
- **resources: dict (defaults to {})**
-
Defines the resources these tasks require on the worker. Canspecify global resources ({'GPU': 2}
), or per-task resources({'x': {'GPU': 1}, 'y': {'SSD': 4}}
), but not both.See worker resources for details on definingresources.
- **actors: bool or dict (default None)**
-
Whether these tasks should exist on the worker as stateful actors.Specified on a global (True/False) or per-task ({'x': True,'y': False}
) basis. See actors for additional details.
- ****kwargs:**
-
Options to pass to the graph optimize callsReturns:
- List of Futures if input is a sequence, or a single future otherwise
-
See also
- [<code>Client.get</code>](#distributed.Client.get)
- Normal synchronous dask.get function
Examples
- >>> from dask import delayed
- >>> from operator import add
- >>> x = delayed(add)(1, 2)
- >>> y = delayed(add)(x, x)
- >>> xx, yy = client.compute([x, y]) # doctest: +SKIP
- >>> xx # doctest: +SKIP
- <Future: status: finished, key: add-8f6e709446674bad78ea8aeecfee188e>
- >>> xx.result() # doctest: +SKIP
- 3
- >>> yy.result() # doctest: +SKIP
- 6
Also support single arguments
- >>> xx = client.compute(x) # doctest: +SKIP
- classmethod
current
() Return global client if one exists, otherwise raise ValueError
gather
(self, futures, errors='raise', direct=None, asynchronous=None)- Gather futures from distributed memory
Accepts a future, nested container of futures, iterator, or queue.The return type will match the input type.
Parameters:
- **futures: Collection of futures**
-
This can be a possibly nested collection of Future objects.Collections can be lists, sets, or dictionaries
- **errors: string**
-
Either ‘raise’ or ‘skip’ if we should raise if a future has erredor skip its inclusion in the output collection
- **direct: boolean**
-
Whether or not to connect directly to the workers, or to askthe scheduler to serve as intermediary. This can also be set whencreating the Client.Returns:
- results: a collection of the same type as the input, but now with
-
- gathered results rather than futures
-
See also
- [<code>Client.scatter</code>](#distributed.Client.scatter)
- Send data out to cluster
Examples
- >>> from operator import add # doctest: +SKIP
- >>> c = Client('127.0.0.1:8787') # doctest: +SKIP
- >>> x = c.submit(add, 1, 2) # doctest: +SKIP
- >>> c.gather(x) # doctest: +SKIP
- 3
- >>> c.gather([x, [x], x]) # support lists and dicts # doctest: +SKIP
- [3, [3], 3]
get
(self, dsk, keys, restrictions=None, loose_restrictions=None, resources=None, sync=True, asynchronous=None, direct=None, retries=None, priority=0, fifo_timeout='60s', actors=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 wherejobs can take place
- **retries: int (default to 0)**
-
Number of allowed automatic retries if computing a result fails
- **priority: Number**
-
Optional prioritization of task. Zero is default.Higher priorities take precedence
- **sync: bool (optional)**
-
Returns Futures if False or concrete values if True (default).
- **direct: bool**
-
Whether or not to connect directly to the workers, or to askthe scheduler to serve as intermediary. This can also be set whencreating the Client.
See also
- [<code>Client.compute</code>](#distributed.Client.compute)
- Compute asynchronous collections
Examples
- >>> from operator import add # doctest: +SKIP
- >>> c = Client('127.0.0.1:8787') # doctest: +SKIP
- >>> c.get({'x': (add, 1, 2)}, 'x') # doctest: +SKIP
- 3
See also
- [<code>Client.publish_dataset</code>](#distributed.Client.publish_dataset)
-
- [<code>Client.list_datasets</code>](#distributed.Client.list_datasets)
-
getexecutor
(_self, **kwargs)- Return a concurrent.futures Executor for submitting tasks on this Client
Parameters:
- ****kwargs:**
-
Any submit()- or map()- compatible arguments, such asworkers or resources.Returns:
- An Executor object that’s fully compatible with the concurrent.futures
-
- API.
-
See set_metadata for the full docstring with examples
Parameters:
- **keys: key or list**
-
Key to access. If a list then gets within a nested collection
- **default: optional**
-
If the key does not exist then return this value instead.If not provided then this raises a KeyError if the key is notpresent
See also
- [<code>Client.set_metadata</code>](#distributed.Client.set_metadata)
-
- classmethod
getrestrictions
(_collections, workers, allow_other_workers) Get restrictions from inputs to compute/persist
- Get logs from scheduler
Parameters:
- **n**:int
-
Number of logs to retrive. Maxes out at 10000 by default,confiruable in config.yaml::log-lengthReturns:
- Logs in reversed order (newest first)
-
gettask_stream
(_self, start=None, stop=None, count=None, plot=False, filename='task-stream.html')- Get task stream data from scheduler
This collects the data present in the diagnostic “Task Stream” plot onthe dashboard. It includes the start, stop, transfer, anddeserialization time of every task for a particular duration.
Note that the task stream diagnostic does not run by default. You maywish to call this function once before you start work to ensure thatthings start recording, and then again after you have completed.
Parameters:
- **start: Number or string**
-
When you want to start recordingIf a number it should be the result of calling time()If a string then it should be a time difference before now,like ’60s’ or ‘500 ms’
- **stop: Number or string**
-
When you want to stop recording
- **count: int**
-
The number of desired records, ignored if both start and stop arespecified
- **plot: boolean, str**
-
If true then also return a Bokeh figureIf plot == ‘save’ then save the figure to a file
- **filename: str (optional)**
-
The filename to save to if you set plot='save'
Returns:
- L: List[Dict]
-
See also
- [<code>get_task_stream</code>](#distributed.Client.get_task_stream)
- a context manager version of this method
Examples
- >>> client.get_task_stream() # prime plugin if not already connected
- >>> x.compute() # do some work
- >>> client.get_task_stream()
- [{'task': ...,
- 'type': ...,
- 'thread': ...,
- ...}]
Pass the plot=True
or plot='save'
keywords to get back a Bokehfigure
- >>> data, figure = client.get_task_stream(plot='save', filename='myfile.html')
Alternatively consider the context manager
- >>> from dask.distributed import get_task_stream
- >>> with get_task_stream() as ts:
- ... x.compute()
- >>> ts.data
- [...]
getversions
(_self, check=False, packages=[])- Return version info for the scheduler, all workers and myself
Parameters:
- **check**:boolean, default False
-
raise ValueError if all required & optional packagesdo not match
- **packages**:List[str]
-
Extra package names to check
Examples
- >>> c.get_versions() # doctest: +SKIP
- >>> c.get_versions(packages=['sklearn', 'geopandas']) # doctest: +SKIP
Parameters:
- **n**:int
-
Number of logs to retrive. Maxes out at 10000 by default,confiruable in config.yaml::log-length
- **workers**:iterable
-
List of worker addresses to retrieve. Gets all workers by default.
- **nanny**:bool, default False
-
Whether to get the logs from the workers (False) or the nannies (True). Ifspecified, the addresses in workers should still be the worker addresses,not the nanny addresses.Returns:
- Dictionary mapping worker address to logs.
-
- Logs are returned in reversed order (newest first)
-
This returns the keys of the data that are held in each worker’smemory.
Parameters:
- **workers: list (optional)**
-
A list of worker addresses, defaults to all
See also
- [<code>Client.who_has</code>](#distributed.Client.who_has)
-
- [<code>Client.nthreads</code>](#distributed.Client.nthreads)
-
- [<code>Client.processing</code>](#distributed.Client.processing)
-
Examples
- >>> x, y, z = c.map(inc, [1, 2, 3]) # doctest: +SKIP
- >>> wait([x, y, z]) # doctest: +SKIP
- >>> c.has_what() # doctest: +SKIP
- {'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea',
- 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b',
- 'inc-1e297fc27658d7b67b3a758f16bcf47a']}
See also
- [<code>Client.publish_dataset</code>](#distributed.Client.publish_dataset)
-
- [<code>Client.get_dataset</code>](#distributed.Client.get_dataset)
-
map
(self, func, *iterables, key=None, workers=None, retries=None, resources=None, priority=0, allow_other_workers=False, fifo_timeout='100 ms', actor=False, actors=False, pure=None, **kwargs)- Map a function on a sequence of arguments
Arguments can be normal objects or Futures
Parameters:
- **func: callable**
-
- **iterables: Iterables**
-
List-like objects to map over. They should have the same length.
- **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
forimpure 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. Indicates whether or not the computationsmay be performed on workers that are not in the workers set(s).
- **retries: int (default to 0)**
-
Number of allowed automatic retries if a task fails
- **priority: Number**
-
Optional prioritization of task. Zero is default.Higher priorities take precedence
- **fifo_timeout: str timedelta (default ‘100ms’)**
-
Allowed amount of time between calls to consider the same priority
- **resources: dict (defaults to {})**
-
Defines the resources each instance of this mapped task requireson the worker; e.g. {'GPU': 2}
. Seeworker resources for details on definingresources.
- **actor: bool (default False)**
-
Whether these tasks should exist on the worker as stateful actors.See actors for additional details.
- **actors: bool (default False)**
-
Alias for actor
- ****kwargs: dict**
-
Extra keywords to send to the function.Large values will be included explicitly in the task graph.Returns:
- List, iterator, or Queue of futures, depending on the type of the
-
- inputs.
-
See also
- [<code>Client.submit</code>](#distributed.Client.submit)
- Submit a single function
Examples
- >>> L = client.map(func, sequence) # doctest: +SKIP
This is as measured by sys.getsizeof
which may not accuratelyreflect 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
- [<code>Client.who_has</code>](#distributed.Client.who_has)
-
Examples
- >>> x, y, z = c.map(inc, [1, 2, 3]) # doctest: +SKIP
- >>> c.nbytes(summary=False) # doctest: +SKIP
- {'inc-1c8dd6be1c21646c71f76c16d09304ea': 28,
- 'inc-1e297fc27658d7b67b3a758f16bcf47a': 28,
- 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b': 28}
- >>> c.nbytes(summary=True) # doctest: +SKIP
- {'inc': 84}
Parameters:
- **workers: list (optional)**
-
A list of workers that we care about specifically.Leave empty to receive information about all workers.
See also
- [<code>Client.who_has</code>](#distributed.Client.who_has)
-
- [<code>Client.has_what</code>](#distributed.Client.has_what)
-
Examples
- >>> c.threads() # doctest: +SKIP
- {'192.168.1.141:46784': 8,
- '192.167.1.142:47548': 8,
- '192.167.1.143:47329': 8,
- '192.167.1.144:37297': 8}
normalizecollection
(_self, collection)- Replace collection’s tasks by already existing futures if they exist
This normalizes the tasks within a collections task graph against theknown futures within the scheduler. It returns a copy of thecollection with a task graph that includes the overlapping futures.
See also
- [<code>Client.persist</code>](#distributed.Client.persist)
- trigger computation of collection’s tasks
Examples
- >>> len(x.__dask_graph__()) # x is a dask collection with 100 tasks # doctest: +SKIP
- 100
- >>> set(client.futures).intersection(x.__dask_graph__()) # some overlap exists # doctest: +SKIP
- 10
- >>> x = client.normalize_collection(x) # doctest: +SKIP
- >>> len(x.__dask_graph__()) # smaller computational graph # doctest: +SKIP
- 20
Parameters:
- **workers: list (optional)**
-
A list of workers that we care about specifically.Leave empty to receive information about all workers.
See also
- [<code>Client.who_has</code>](#distributed.Client.who_has)
-
- [<code>Client.has_what</code>](#distributed.Client.has_what)
-
Examples
- >>> c.threads() # doctest: +SKIP
- {'192.168.1.141:46784': 8,
- '192.167.1.142:47548': 8,
- '192.167.1.143:47329': 8,
- '192.167.1.144:37297': 8}
persist
(self, collections, optimize_graph=True, workers=None, allow_other_workers=None, resources=None, retries=None, priority=0, fifo_timeout='60s', actors=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 theprevious 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 computationIf a string a list then the output collections will run on the listedworkers, but other sub-computations can run anywhereIf a dict then keys should be (tuples of) collections and valuesshould be addresses or lists.
- **allow_other_workers: bool, list**
-
If True then all restrictions in workers= are considered looseIf a list then only the keys for the listed collections are loose
- **retries: int (default to 0)**
-
Number of allowed automatic retries if computing a result fails
- **priority: Number**
-
Optional prioritization of task. Zero is default.Higher priorities take precedence
- **fifo_timeout: timedelta str (defaults to ’60s’)**
-
Allowed amount of time between calls to consider the same priority
- **resources: dict (defaults to {})**
-
Defines the resources these tasks require on the worker. Canspecify global resources ({'GPU': 2}
), or per-task resources({'x': {'GPU': 1}, 'y': {'SSD': 4}}
), but not both.See worker resources for details on definingresources.
- **actors: bool or dict (default None)**
-
Whether these tasks should exist on the worker as stateful actors.Specified on a global (True/False) or per-task ({'x': True,'y': False}
) basis. See actors for additional details.
- ****kwargs:**
-
Options to pass to the graph optimize callsReturns:
- List of collections, or single collection, depending on type of input.
-
See also
- [<code>Client.compute</code>](#distributed.Client.compute)
-
Examples
- >>> xx = client.persist(x) # doctest: +SKIP
- >>> xx, yy = client.persist([x, y]) # doctest: +SKIP
Parameters:
- **workers: list (optional)**
-
A list of worker addresses, defaults to all
See also
- [<code>Client.who_has</code>](#distributed.Client.who_has)
-
- [<code>Client.has_what</code>](#distributed.Client.has_what)
-
- [<code>Client.nthreads</code>](#distributed.Client.nthreads)
-
Examples
- >>> x, y, z = c.map(inc, [1, 2, 3]) # doctest: +SKIP
- >>> c.processing() # doctest: +SKIP
- {'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea',
- 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b',
- 'inc-1e297fc27658d7b67b3a758f16bcf47a']}
profile
(self, key=None, start=None, stop=None, workers=None, merge_workers=True, plot=False, filename=None, server=False, scheduler=False)- Collect statistical profiling information about recent work
Parameters:
- **key: str**
-
Key prefix to select, this is typically a function name like ‘inc’Leave as None to collect all data
- **start: time**
-
- **stop: time**
-
- **workers: list**
-
List of workers to restrict profile information
- **server**:bool
-
If true, return the profile of the worker’s administrative threadrather than the worker threads.This is useful when profiling Dask itself, rather than user code.
- **scheduler: bool**
-
If true, return the profile information from the scheduler’sadministrative thread rather than the workers.This is useful when profiling Dask’s scheduling itself.
- **plot: boolean or string**
-
Whether or not to return a plot object
- **filename: str**
-
Filename to save the plot
Examples
- >>> client.profile() # call on collections
- >>> client.profile(filename='dask-profile.html') # save to html file
This stores a named reference to a dask collection or list of futureson the scheduler. These references are available to other Clientswhich can download the collection or futures with get_dataset
.
Datasets are not immediately computed. You may wish to callClient.persist
prior to publishing a dataset.
Parameters:
- **args**:list of objects to publish as name
-
- **name**:optional name of the dataset to publish
-
- **kwargs: dict**
-
named collections to publish on the schedulerReturns:
- None
-
See also
- [<code>Client.list_datasets</code>](#distributed.Client.list_datasets)
-
- [<code>Client.get_dataset</code>](#distributed.Client.get_dataset)
-
- [<code>Client.unpublish_dataset</code>](#distributed.Client.unpublish_dataset)
-
- [<code>Client.persist</code>](#distributed.Client.persist)
-
Examples
Publishing client:
- >>> df = dd.read_csv('s3://...') # doctest: +SKIP
- >>> df = c.persist(df) # doctest: +SKIP
- >>> c.publish_dataset(my_dataset=df) # doctest: +SKIP
Alternative invocation>>> c.publish_dataset(df, name=’my_dataset’)
Receiving client:
- >>> c.list_datasets() # doctest: +SKIP
- ['my_dataset']
- >>> df2 = c.get_dataset('my_dataset') # doctest: +SKIP
Move data between workers to roughly balance memory burden. Thiseither 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 ofthe scheduler. It it not recommended to use it while waiting oncomputations.
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
registerworker_callbacks
(_self, setup=None)- Registers a setup callback function for all current and future workers.
This registers a new setup function for workers in this cluster. Thefunction will run immediately on all currently connected workers. Itwill also be run upon connection by any workers that are added in thefuture. Multiple setup functions can be registered - these will becalled in the order they were added.
If the function takes an input argument named dask_worker
thenthat variable will be populated with the worker itself.
Parameters:
- **setup**:callable(dask_worker: Worker) -> None
-
Function to register and run on all workers
registerworker_plugin
(_self, plugin=None, name=None)- Registers a lifecycle worker plugin for all current and future workers.
This registers a new object to handle setup, task state transitions andteardown for workers in this cluster. The plugin will instantiate itselfon all currently connected workers. It will also be run on any workerthat connects in the future.
The plugin may include methods setup
, teardown
, andtransition
. See the dask.distributed.WorkerPlugin
class or theexamples below for the interface and docstrings. It must beserializable with the pickle or cloudpickle modules.
If the plugin has a name
attribute, or if the name=
keyword isused then that will control idempotency. A a plugin with that name hasalready registered then any future plugins will not run.
For alternatives to plugins, you may also wish to look into preloadscripts.
Parameters:
- **plugin: WorkerPlugin**
-
The plugin object to pass to the workers
- **name: str, optional**
-
A name for the plugin.Registering a plugin with the same name will have no effect.
See also
- <code>distributed.WorkerPlugin</code>
-
Examples
- >>> class MyPlugin(WorkerPlugin):
- ... def __init__(self, *args, **kwargs):
- ... pass # the constructor is up to you
- ... def setup(self, worker: dask.distributed.Worker):
- ... pass
- ... def teardown(self, worker: dask.distributed.Worker):
- ... pass
- ... def transition(self, key: str, start: str, finish: str, **kwargs):
- ... pass
- >>> plugin = MyPlugin(1, 2, 3)
- >>> client.register_worker_plugin(plugin)
You can get access to the plugin with the get_worker
function
- >>> client.register_worker_plugin(other_plugin, name='my-plugin')
- >>> def f():
- ... worker = get_worker()
- ... plugin = worker.plugins['my-plugin']
- ... return plugin.my_state
- >>> future = client.run(f)
replicate
(self, futures, n=None, workers=None, branching_factor=2, **kwargs)- Set replication of futures within network
Copy data onto many workers. This helps to broadcast frequentlyaccessed data and it helps to improve resilience.
This performs a tree copy of the data throughout the networkindividually on each piece of data. This operation blocks untilcomplete. 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
- [<code>Client.rebalance</code>](#distributed.Client.rebalance)
-
Examples
- >>> x = c.submit(func, *args) # doctest: +SKIP
- >>> c.replicate([x]) # send to all workers # doctest: +SKIP
- >>> c.replicate([x], n=3) # send to three workers # doctest: +SKIP
- >>> c.replicate([x], workers=['alice', 'bob']) # send to specific # doctest: +SKIP
- >>> c.replicate([x], n=1, workers=['alice', 'bob']) # send to one of specific workers # doctest: +SKIP
- >>> c.replicate([x], n=1) # reduce replications # doctest: +SKIP
This kills all active work, deletes all data on the network, andrestarts the worker processes.
retireworkers
(_self, workers=None, close_workers=True, **kwargs)- Retire certain workers on the scheduler
See dask.distributed.Scheduler.retire_workers for the full docstring.
See also
- <code>dask.distributed.Scheduler.retire_workers</code>
-
Examples
You can get information about active workers using the following:>>> workers = client.scheduler_info()[‘workers’]
From that list you may want to select some workers to close>>> client.retire_workers(workers=[‘tcp://address:port’, …])
Parameters:
- **futures: list of Futures**
-
This calls a function on all currently known workers immediately,blocks until those results come back, and returns the resultsasynchronously as a dictionary keyed by worker address. This methodif generally used for side effects, such and collecting diagnosticinformation or installing libraries.
If your function takes an input argument named dask_worker
thenthat variable will be populated with the worker itself.
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.
- **wait: boolean (optional)**
-
If the function is asynchronous whether or not to wait until thatfunction finishes.
- **nanny**:bool, defualt False
-
Whether to run function
on the nanny. By default, the functionis run on the worker process. If specified, the addresses inworkers
should still be the worker addresses, not the nanny addresses.
Examples
- >>> c.run(os.getpid) # doctest: +SKIP
- {'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']) # doctest: +SKIP
- {'192.168.0.100:9000': 1234,
- '192.168.0.101:9000': 4321}
- >>> def get_status(dask_worker):
- ... return dask_worker.status
- >>> c.run(get_hostname) # doctest: +SKIP
- {'192.168.0.100:9000': 'running',
- '192.168.0.101:9000': 'running}
Run asynchronous functions in the background:
- >>> async def print_state(dask_worker): # doctest: +SKIP
- ... while True:
- ... print(dask_worker.status)
- ... await asyncio.sleep(1)
- >>> c.run(print_state, wait=False) # doctest: +SKIP
This spaws a coroutine on all currently known workers and then waitsfor the coroutine on each worker. The coroutines’ results are returnedas a dictionary keyed by worker address.
Parameters:
- **function: a coroutine function**
-
- (typically a function wrapped in gen.coroutine or
-
a Python 3.5+ async function)
- ***args: arguments for remote function**
-
- ****kwargs: keyword arguments for remote function**
-
- **wait: boolean (default True)**
-
Whether to wait for coroutines to end.
- **workers: list**
-
Workers on which to run the function. Defaults to all known workers.
This is typically used for live debugging. The function should take akeyword argument dask_scheduler=
, which will be given the schedulerobject itself.
See also
- [<code>Client.run</code>](#distributed.Client.run)
- Run a function on all workers
- [<code>Client.start_ipython_scheduler</code>](#distributed.Client.start_ipython_scheduler)
- Start an IPython session on scheduler
Examples
- >>> def get_number_of_tasks(dask_scheduler=None):
- ... return len(dask_scheduler.tasks)
- >>> client.run_on_scheduler(get_number_of_tasks) # doctest: +SKIP
- 100
Run asynchronous functions in the background:
- >>> async def print_state(dask_scheduler): # doctest: +SKIP
- ... while True:
- ... print(dask_scheduler.status)
- ... await asyncio.sleep(1)
- >>> c.run(print_state, wait=False) # doctest: +SKIP
scatter
(self, data, workers=None, broadcast=False, direct=None, hash=True, timeout='no_default', asynchronous=None)- Scatter data into distributed memory
This moves data from the local client process into the workers of thedistributed scheduler. Note that it is often better to submit jobs toyour workers to have them load the data rather than loading datalocally and then scattering it out to them.
Parameters:
- **data: list, dict, or object**
-
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.
- **direct: bool (defaults to automatically check)**
-
Whether or not to connect directly to the workers, or to askthe scheduler to serve as intermediary. This can also be set whencreating the Client.
- **hash: bool (optional)**
-
Whether or not to hash data to determine key.If False then this uses a random keyReturns:
- List, dict, iterator, or queue of futures matching the type of input.
-
See also
- [<code>Client.gather</code>](#distributed.Client.gather)
- Gather data back to local process
Examples
- >>> c = Client('127.0.0.1:8787') # doctest: +SKIP
- >>> c.scatter(1) # doctest: +SKIP
- <Future: status: finished, key: c0a8a20f903a4915b94db8de3ea63195>
- >>> c.scatter([1, 2, 3]) # doctest: +SKIP
- [<Future: status: finished, key: c0a8a20f903a4915b94db8de3ea63195>,
- <Future: status: finished, key: 58e78e1b34eb49a68c65b54815d1b158>,
- <Future: status: finished, key: d3395e15f605bc35ab1bac6341a285e2>]
- >>> c.scatter({'x': 1, 'y': 2, 'z': 3}) # doctest: +SKIP
- {'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)]) # doctest: +SKIP
Broadcast data to all workers
- >>> [future] = c.scatter([element], broadcast=True) # doctest: +SKIP
Send scattered data to parallelized function using client futuresinterface
- >>> data = c.scatter(data, broadcast=True) # doctest: +SKIP
- >>> res = [c.submit(func, data, i) for i in range(100)]
Examples
- >>> c.scheduler_info() # doctest: +SKIP
- {'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}}}
This allows you to store small amounts of data on the central schedulerprocess for administrative purposes. Data should be msgpackserializable (ints, strings, lists, dicts)
If the key corresponds to a task then that key will be cleaned up whenthe task is forgotten by the scheduler.
If the key is a list then it will be assumed that you want to indexinto a nested dictionary structure using those keys. For example ifyou call the following:
- >>> client.set_metadata(['a', 'b', 'c'], 123)
Then this is the same as setting
- >>> scheduler.task_metadata['a']['b']['c'] = 123
The lower level dictionaries will be created on demand.
See also
- [<code>get_metadata</code>](#distributed.Client.get_metadata)
-
Examples
- >>> client.set_metadata('x', 123) # doctest: +SKIP
- >>> client.get_metadata('x') # doctest: +SKIP
- 123
- >>> client.set_metadata(['x', 'y'], 123) # doctest: +SKIP
- >>> client.get_metadata('x') # doctest: +SKIP
- {'y': 123}
- >>> client.set_metadata(['x', 'w', 'z'], 456) # doctest: +SKIP
- >>> client.get_metadata('x') # doctest: +SKIP
- {'y': 123, 'w': {'z': 456}}
- >>> client.get_metadata(['x', 'w']) # doctest: +SKIP
- {'z': 456}
Note, this may disrupt other clients that may be using the samescheudler and workers.
See also
- [<code>Client.close</code>](#distributed.Client.close)
- close only this client
start
(self, **kwargs)Start scheduler running in separate thread
startipython_scheduler
(_self, 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 forexecuting 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 necessaryto connect Jupyter clients to the scheduler.
See also
- [<code>Client.start_ipython_workers</code>](#distributed.Client.start_ipython_workers)
- Start IPython on the workers
Examples
- >>> c.start_ipython_scheduler() # doctest: +SKIP
- >>> %scheduler scheduler.processing # doctest: +SKIP
- {'127.0.0.1:3595': {'inc-1', 'inc-2'},
- '127.0.0.1:53589': {'inc-2', 'add-5'}}
- >>> c.start_ipython_scheduler(qtconsole=True) # doctest: +SKIP
startipython_workers
(_self, 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 forexecuting code on the workers. If string has asterix then expandasterix 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 necessaryto connect Jupyter clients to the workers.
See also
- [<code>Client.start_ipython_scheduler</code>](#distributed.Client.start_ipython_scheduler)
- start ipython on the scheduler
Examples
- >>> info = c.start_ipython_workers() # doctest: +SKIP
- >>> %remote info['192.168.1.101:5752'] worker.data # doctest: +SKIP
- {'x': 1, 'y': 100}
- >>> c.start_ipython_workers('192.168.1.101:5752', magic_names='w') # doctest: +SKIP
- >>> %w worker.data # doctest: +SKIP
- {'x': 1, 'y': 100}
- >>> c.start_ipython_workers('192.168.1.101:5752', qtconsole=True) # doctest: +SKIP
Add asterix * in magic names to add one magic per worker
- >>> c.start_ipython_workers(magic_names='w_*') # doctest: +SKIP
- >>> %w_0 worker.data # doctest: +SKIP
- {'x': 1, 'y': 100}
- >>> %w_1 worker.data # doctest: +SKIP
- {'z': 5}
submit
(self, func, *args, key=None, workers=None, resources=None, retries=None, priority=0, fifo_timeout='100 ms', allow_other_workers=False, actor=False, actors=False, pure=None, **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
forimpure 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)
- **key: str**
-
Unique identifier for the task. Defaults to function-name and hash
- **allow_other_workers: bool (defaults to False)**
-
Used with workers. Indicates whether or not the computationsmay be performed on workers that are not in the workers set(s).
- **retries: int (default to 0)**
-
Number of allowed automatic retries if the task fails
- **priority: Number**
-
Optional prioritization of task. Zero is default.Higher priorities take precedence
- **fifo_timeout: str timedelta (default ‘100ms’)**
-
Allowed amount of time between calls to consider the same priority
- **resources: dict (defaults to {})**
-
Defines the resources this job requires on the worker; e.g.{'GPU': 2}
. See worker resources for detailson defining resources.
- **actor: bool (default False)**
-
Whether this task should exist on the worker as a stateful actor.See actors for additional details.
- **actors: bool (default False)**
-
Alias for actorReturns:
- Future
-
See also
- [<code>Client.map</code>](#distributed.Client.map)
- Submit on many arguments at once
Examples
- >>> c = client.submit(add, a, b) # doctest: +SKIP
See also
- [<code>Client.publish_dataset</code>](#distributed.Client.publish_dataset)
-
Examples
- >>> c.list_datasets() # doctest: +SKIP
- ['my_dataset']
- >>> c.unpublish_datasets('my_dataset') # doctest: +SKIP
- >>> c.list_datasets() # doctest: +SKIP
- []
This sends a local file up to all worker nodes. This file is placedinto a temporary directory on Python’s system path so any .py, .eggor .zip files will be importable.
Parameters:
- **filename: string**
-
Filename of .py, .egg or .zip file to send to workers
Examples
- >>> client.upload_file('mylibrary.egg') # doctest: +SKIP
- >>> from mylibrary import myfunc # doctest: +SKIP
- >>> L = c.map(myfunc, seq) # doctest: +SKIP
waitfor_workers
(_self, n_workers=0)Blocking call to wait for n workers before continuing
- The workers storing each future’s data
Parameters:
- **futures: list (optional)**
-
A list of futures, defaults to all data
See also
- [<code>Client.has_what</code>](#distributed.Client.has_what)
-
- [<code>Client.nthreads</code>](#distributed.Client.nthreads)
-
Examples
- >>> x, y, z = c.map(inc, [1, 2, 3]) # doctest: +SKIP
- >>> wait([x, y, z]) # doctest: +SKIP
- >>> c.who_has() # doctest: +SKIP
- {'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]) # doctest: +SKIP
- {'inc-1c8dd6be1c21646c71f76c16d09304ea': ['192.168.1.141:46784'],
- 'inc-1e297fc27658d7b67b3a758f16bcf47a': ['192.168.1.141:46784']}
This facilitates easy sharing of scheduler information using a filesystem. The scheduler file can be used to instantiate a second Clientusing the same scheduler.
Parameters:
- **scheduler_file: str**
-
Path to a write the scheduler file.
Examples
- >>> client = Client() # doctest: +SKIP
- >>> client.write_scheduler_file('scheduler.json') # doctest: +SKIP
- # connect to previous client's scheduler
- >>> client2 = Client(scheduler_file='scheduler.json') # doctest: +SKIP
A Future is a local proxy to a result running on a remote worker. A usermanages future objects in the local Python process to determine whathappens in the larger cluster.
Parameters:
- key: str, or tuple
Key of remote data to which this future refers
client: Client
Client that should own this future. Defaults to _get_global_client()
inform: bool
- Do we inform the scheduler that we need an update on this future
See also
Client
- Creates futures
Examples
Futures typically emerge from Client computations
- >>> my_future = client.submit(add, 1, 2) # doctest: +SKIP
We can track the progress and results of a future
- >>> my_future # doctest: +SKIP
- <Future: status: finished, key: add-8f6e709446674bad78ea8aeecfee188e>
We can get the result or the exception and traceback from the future
- >>> my_future.result() # doctest: +SKIP
The callback fn
should take the future as its only argument. Thiswill be called regardless of if the future completes successfully,errs, or is cancelled
The callback is executed in a separate thread.
See also
- [<code>Client.cancel</code>](#distributed.Client.cancel)
-
cancelled
(self)Returns True if the future has been cancelled
Is the computation complete?
- Return the exception of a failed task
If timeout seconds are elapsed before returning, adask.distributed.TimeoutError
is raised.
See also
- [<code>Future.traceback</code>](#distributed.Future.traceback)
-
If timeout seconds are elapsed before returning, adask.distributed.TimeoutError
is raised.
See also
- [<code>Client.retry</code>](#distributed.Client.retry)
-
This returns a traceback object. You can inspect this object using thetraceback
module. Alternatively if you call future.result()
this traceback will accompany the raised exception.
If timeout seconds are elapsed before returning, adask.distributed.TimeoutError
is raised.
See also
- [<code>Future.exception</code>](#distributed.Future.exception)
-
Examples
- >>> import traceback # doctest: +SKIP
- >>> tb = future.traceback() # doctest: +SKIP
- >>> traceback.format_tb(tb) # doctest: +SKIP
- [...]
This allows multiple clients to share futures or small bits of data betweeneach other with a multi-producer/multi-consumer queue. All metadata issequentialized through the scheduler.
Elements of the Queue must be either Futures or msgpack-encodable data(ints, strings, lists, dicts). All data is sent through the scheduler soit is wise not to send large objects. To share large objects scatter thedata and share the future instead.
Warning
This object is experimental and has known issues in Python 2
See also
Variable
- shared variable between clients
Examples
- >>> from dask.distributed import Client, Queue # doctest: +SKIP
- >>> client = Client() # doctest: +SKIP
- >>> queue = Queue('x') # doctest: +SKIP
- >>> future = client.submit(f, x) # doctest: +SKIP
- >>> queue.put(future) # doctest: +SKIP
Parameters:
- **timeout: Number (optional)**
-
Time in seconds to wait before timing out
- **batch: boolean, int (optional)**
-
If True then return all elements currently waiting in the queue.If an integer than return that many elements from the queueIf False (default) then return one item at a time
put
(self, value, timeout=None, **kwargs)Put data into the queue
- Current number of elements in the queue
This allows multiple clients to share futures and data between each otherwith a single mutable variable. All metadata is sequentialized through thescheduler. Race conditions can occur.
Values must be either Futures or msgpack-encodable data (ints, lists,strings, etc..) All data will be kept and sent through the scheduler, soit is wise not to send too much. If you want to share a large amount ofdata then scatter
it and share the future instead.
Warning
This object is experimental and has known issues in Python 2
See also
Queue
- shared multi-producer/multi-consumer queue between clients
Examples
- >>> from dask.distributed import Client, Variable # doctest: +SKIP
- >>> client = Client() # doctest: +SKIP
- >>> x = Variable('x') # doctest: +SKIP
- >>> x.set(123) # docttest: +SKIP
- >>> x.get() # docttest: +SKIP
- 123
- >>> future = client.submit(f, x) # doctest: +SKIP
- >>> x.set(future) # doctest: +SKIP
Caution, this affects all clients currently pointing to this variable.
Parameters:
- **value: Future or object**
-
Must be either a Future or a msgpack-encodable value
Parameters:
- name: string
- Name of the lock to acquire. Choosing the same name allows twodisconnected processes to coordinate a lock.
Examples
- >>> lock = Lock('x') # doctest: +SKIP
- >>> lock.acquire(timeout=1) # doctest: +SKIP
- >>> # do things with protected resource
- >>> lock.release() # doctest: +SKIP
Parameters:
- **blocking**:bool, optional
-
If false, don’t wait on the lock in the scheduler at all.
- **timeout**:number, optional
-
Seconds to wait on the lock in the scheduler. This does notinclude local coroutine time, network transfer time, etc..It is forbidden to specify a timeout when blocking is false.Returns:
- True or False whether or not it sucessfully acquired the lock
-
Examples
- >>> lock = Lock('x') # doctest: +SKIP
- >>> lock.acquire(timeout=1) # doctest: +SKIP
- class
distributed.
Pub
(name, worker=None, client=None) - Publish data with Publish-Subscribe pattern
This allows clients and workers to directly communicate data between eachother with a typical Publish-Subscribe pattern. This involves twocomponents,
Pub objects, into which we put data:
- >>> pub = Pub('my-topic')
- >>> pub.put(123)
And Sub objects, from which we collect data:
- >>> sub = Sub('my-topic')
- >>> sub.get()
- 123
Many Pub and Sub objects can exist for the same topic. All data sent fromany Pub will be sent to all Sub objects on that topic that are currentlyconnected. Pub’s and Sub’s find each other using the scheduler, but theycommunicate directly with each other without coordination from thescheduler.
Pubs and Subs use the central scheduler to find each other, but not tomediate the communication. This means that there is very little additionallatency or overhead, and they are appropriate for very frequent datatransfers. For context, most data transfer first checks with the scheduler to find whichworkers should participate, and then does direct worker-to-workertransfers. This checking in with the scheduler provides some stabilityguarantees, but also adds in a few extra network hops. PubSub doesn’t dothis, and so is faster, but also can easily drop messages if Pubs or Subsdisappear without notice.
When using a Pub or Sub from a Client all communications will be routedthrough the scheduler. This can cause some performance degradation. Pubsand Subs only operate at top-speed when they are both on workers.
Parameters:
- name: object (msgpack serializable)
- The name of the group of Pubs and Subs on which to participate
See also
Examples
- >>> pub = Pub('my-topic')
- >>> sub = Sub('my-topic')
- >>> pub.put([1, 2, 3])
- >>> sub.get()
- [1, 2, 3]
You can also use sub within a for loop:
- >>> for msg in sub: # doctest: +SKIP
- ... print(msg)
or an async for loop
- >>> async for msg in sub: # doctest: +SKIP
- ... print(msg)
Similarly the .get
method will return an awaitable if used by an asyncclient or within the IOLoop thread of a worker
- >>> await sub.get() # doctest: +SKIP
You can see the set of connected worker subscribers by looking at the.subscribers
attribute:
- >>> pub.subscribers
- {'tcp://...': {},
- 'tcp://...': {}}
put
(self, msg)- Publish a message to all subscribers of this topic
See also
Pub
for full docstring
Get a single message
- Get a single message