Diagnostics (distributed)

The Dask distributed scheduler provides live feedback in two forms:

  1. An interactive dashboard containing many plots and tables with live information

  2. A progress bar suitable for interactive use in consoles or notebooks

Dashboard

For information on the Dask dashboard see Dashboard Diagnostics.

Capture diagnostics

get_task_stream([client, plot, filename])

Collect task stream within a context block

Client.profile([key, start, stop, workers, ...])

Collect statistical profiling information about recent work

performance_report([filename, stacklevel, ...])

Gather performance report

You can capture some of the same information that the dashboard presents for offline processing using the get_task_stream and Client.profile functions. These capture the start and stop time of every task and transfer, as well as the results of a statistical profiler.

with get_task_stream(plot='save', filename="task-stream.html") as ts:
    x.compute()

client.profile(filename="dask-profile.html")

history = ts.data

Additionally, Dask can save many diagnostics dashboards at once including the task stream, worker profiles, bandwidths, etc. with the performance_report context manager:

from dask.distributed import performance_report

with performance_report(filename="dask-report.html"):
    ## some dask computation

The following video demonstrates the performance_report context manager in greater detail:

Progress bar

progress(*futures[, notebook, multi, ...])

Track progress of futures

The dask.distributed progress bar differs from the ProgressBar used for local diagnostics. The progress function takes a Dask object that is executing in the background:

# Progress bar on a single-machine scheduler
from dask.diagnostics import ProgressBar

with ProgressBar():
    x.compute()

# Progress bar with the distributed scheduler
from dask.distributed import Client, progress

client = Client()  # use dask.distributed by default

x = x.persist()  # start computation in the background
progress(x)      # watch progress

x.compute()      # convert to final result when done if desired

Connecting to the Dashboard

Some computer networks may restrict access to certain ports or only allow access from certain machines. If you are unable to access the dashboard then you may want to contact your IT administrator.

Some common problems and solutions follow:

Specify an accessible port

Some clusters restrict the ports that are visible to the outside world. These ports may include the default port for the web interface, 8787. There are a few ways to handle this:

  1. Open port 8787 to the outside world. Often this involves asking your cluster administrator.

  2. Use a different port that is publicly accessible using the --dashboard-address :8787 option on the dask-scheduler command.

  3. Use fancier techniques, like Port Forwarding

Port Forwarding

If you have SSH access then one way to gain access to a blocked port is through SSH port forwarding. A typical use case looks like the following:

local$ ssh -L 8000:localhost:8787 user@remote
remote$ dask-scheduler  # now, the web UI is visible at localhost:8000
remote$ # continue to set up dask if needed -- add workers, etc

It is then possible to go to localhost:8000 and see Dask Web UI. This same approach is not specific to dask.distributed, but can be used by any service that operates over a network, such as Jupyter notebooks. For example, if we chose to do this we could forward port 8888 (the default Jupyter port) to port 8001 with ssh -L 8001:localhost:8888 user@remote.

Required Packages

Bokeh must be installed in your scheduler’s environment to run the dashboard. If it’s not the dashboard page will instruct you to install it.

Depending on your configuration, you might also need to install jupyter-server-proxy to access the dashboard.

API

dask.distributed.progress(*futures, notebook=None, multi=True, complete=True, group_by='prefix', **kwargs)[source]

Track progress of futures

This operates differently in the notebook and the console

  • Notebook: This returns immediately, leaving an IPython widget on screen

  • Console: This blocks until the computation completes

Parameters
futuresFutures

A list of futures or keys to track

notebookbool (optional)

Running in the notebook or not (defaults to guess)

multibool (optional)

Track different functions independently (defaults to True)

completebool (optional)

Track all keys (True) or only keys that have not yet run (False) (defaults to True)

group_byCallable | Literal[“spans”] | Literal[“prefix”]

Use spans instead of task key names for grouping tasks (defaults to “prefix”)

Notes

In the notebook, the output of progress must be the last statement in the cell. Typically, this means calling progress at the end of a cell.

Examples

>>> progress(futures)  
[########################################] | 100% Completed |  1.7s
dask.distributed.get_task_stream(client=None, plot=False, filename='task-stream.html')[source]

Collect task stream within a context block

This provides diagnostic information about every task that was run during the time when this block was active.

This must be used as a context manager.

Parameters
plot: boolean, str

If true then also return a Bokeh figure If plot == ‘save’ then save the figure to a file

filename: str (optional)

The filename to save to if you set plot='save'

See also

Client.get_task_stream

Function version of this context manager

Examples

>>> with get_task_stream() as ts:
...     x.compute()
>>> ts.data
[...]

Get back a Bokeh figure and optionally save to a file

>>> with get_task_stream(plot='save', filename='task-stream.html') as ts:
...    x.compute()
>>> ts.figure
<Bokeh Figure>

To share this file with others you may wish to upload and serve it online. A common way to do this is to upload the file as a gist, and then serve it on https://raw.githack.com

$ python -m pip install gist
$ gist task-stream.html
https://gist.github.com/8a5b3c74b10b413f612bb5e250856ceb

You can then navigate to that site, click the “Raw” button to the right of the task-stream.html file, and then provide that URL to https://raw.githack.com . This process should provide a sharable link that others can use to see your task stream plot.