Optimization¶
Performance can be significantly improved in different contexts by making small optimizations on the dask graph before calling the scheduler.
The dask.optimize
module contains several functions to transform graphs in
a variety of useful ways. In most cases, users won’t need to interact with
these functions directly, as specialized subsets of these transforms are done
automatically in the dask collections (dask.array
, dask.bag
, and
dask.dataframe
). However, users working with custom graphs or computations
may find that applying these methods results in substantial speedups.
In general, there are two goals when doing graph optimizations:
- Simplify computation
- Improve parallelism.
Simplifying computation can be done on a graph level by removing unnecessary
tasks (cull
), or on a task level by replacing expensive operations with
cheaper ones (RewriteRule
).
Parallelism can be improved by reducing
inter-task communication, whether by fusing many tasks into one (fuse
), or
by inlining cheap operations (inline
, inline_functions
).
Below, we show an example walking through the use of some of these to optimize a task graph.
Example¶
Suppose you had a custom dask graph for doing a word counting task:
>>> from __future__ import print_function
>>> def print_and_return(string):
... print(string)
... return string
>>> def format_str(count, val, nwords):
... return ('word list has {0} occurrences of {1}, '
... 'out of {2} words').format(count, val, nwords)
>>> dsk = {'words': 'apple orange apple pear orange pear pear',
... 'nwords': (len, (str.split, 'words')),
... 'val1': 'orange',
... 'val2': 'apple',
... 'val3': 'pear',
... 'count1': (str.count, 'words', 'val1'),
... 'count2': (str.count, 'words', 'val2'),
... 'count3': (str.count, 'words', 'val3'),
... 'out1': (format_str, 'count1', 'val1', 'nwords'),
... 'out2': (format_str, 'count2', 'val2', 'nwords'),
... 'out3': (format_str, 'count3', 'val3', 'nwords'),
... 'print1': (print_and_return, 'out1'),
... 'print2': (print_and_return, 'out2'),
... 'print3': (print_and_return, 'out3')}

Here we’re counting the occurrence of the words 'orange
, 'apple'
, and
'pear'
in the list of words, formatting an output string reporting the
results, printing the output, then returning the output string.
To perform the computation, we pass the dask graph and the desired output keys
to a scheduler get
function:
>>> from dask.threaded import get
>>> outputs = ['print1', 'print2']
>>> results = get(dsk, outputs)
word list has 2 occurrences of apple, out of 7 words
word list has 2 occurrences of orange, out of 7 words
>>> results
('word list has 2 occurrences of orange, out of 7 words',
'word list has 2 occurrences of apple, out of 7 words')
As can be seen above, the scheduler computed only the requested outputs
('print3'
was never computed). This is because the scheduler internally
calls cull
, which removes the unnecessary tasks from the graph. Even though
this is done internally in the scheduler, it can be beneficial to call it at
the start of a series of optimizations to reduce the amount of work done in
later steps:
>>> from dask.optimize import cull
>>> dsk1, dependencies = cull(dsk, outputs)

Looking at the task graph above, there are multiple accesses to constants such
as 'val1'
or 'val2'
in the dask graph. These can be inlined into the
tasks to improve efficiency using the inline
function. For example:
>>> from dask.optimize import inline
>>> dsk2 = inline(dsk1, dependencies=dependencies)
>>> results = get(dsk2, outputs)
word list has 2 occurrences of apple, out of 7 words
word list has 2 occurrences of orange, out of 7 words

Now we have two sets of almost linear task chains. The only link between them
is the word counting function. For cheap operations like this, the
serialization cost may be larger than the actual computation, so it may be
faster to do the computation more than once, rather than passing the results to
all nodes. To perform this function inlining, the inline_functions
function
can be used:
>>> from dask.optimize import inline_functions
>>> dsk3 = inline_functions(dsk2, outputs, [len, str.split],
... dependencies=dependencies)
>>> results = get(dsk3, outputs)
word list has 2 occurrences of apple, out of 7 words
word list has 2 occurrences of orange, out of 7 words

Now we have a set of purely linear tasks. We’d like to have the scheduler run
all of these on the same worker to reduce data serialization between workers.
One option is just to merge these linear chains into one big task using the
fuse
function:
>>> from dask.optimize import fuse
>>> dsk4, dependencies = fuse(dsk3)
>>> results = get(dsk4, outputs)
word list has 2 occurrences of apple, out of 7 words
word list has 2 occurrences of orange, out of 7 words

Putting it all together:
>>> def optimize_and_get(dsk, keys):
... dsk1, deps = cull(dsk, keys)
... dsk2 = inline(dsk1, dependencies=deps)
... dsk3 = inline_functions(dsk2, keys, [len, str.split],
... dependencies=deps)
... dsk4, deps = fuse(dsk3)
... return get(dsk4, keys)
>>> optimize_and_get(dsk, outputs)
word list has 2 occurrences of apple, out of 7 words
word list has 2 occurrences of orange, out of 7 words
In summary, the above operations accomplish the following:
- Removed tasks unnecessary for the desired output using
cull
. - Inlined constants using
inline
. - Inlined cheap computations using
inline_functions
, improving parallelism. - Fused linear tasks together to ensure they run on the same worker using
fuse
.
As stated previously, these optimizations are already performed automatically in the dask collections. Users not working with custom graphs or computations should rarely need to directly interact with them.
These are just a few of the optimizations provided in dask.optimize
. For
more information, see the API below.
Rewrite Rules¶
For context based optimizations, dask.rewrite
provides functionality for
pattern matching and term rewriting. This is useful for replacing expensive
computations with equivalent, cheaper computations. For example, dask.array
uses the rewrite functionality to replace series of array slicing operations
with a more efficient single slice.
The interface to the rewrite system consists of two classes:
RewriteRule(lhs, rhs, vars)
Given a left-hand-side (
lhs
), a right-hand-side (rhs
), and a set of variables (vars
), a rewrite rule declaratively encodes the following operation:lhs -> rhs if task matches lhs over variables
RuleSet(*rules)
A collection of rewrite rules. The design of
RuleSet
class allows for efficient “many-to-one” pattern matching, meaning that there is minimal overhead for rewriting with multiple rules in a rule set.
Example¶
Here we create two rewrite rules expressing the following mathematical transformations:
a + a -> 2*a
a * a -> a**2
where 'a'
is a variable:
>>> from dask.rewrite import RewriteRule, RuleSet
>>> from operator import add, mul, pow
>>> variables = ('a',)
>>> rule1 = RewriteRule((add, 'a', 'a'), (mul, 'a', 2), variables)
>>> rule2 = RewriteRule((mul, 'a', 'a'), (pow, 'a', 2), variables)
>>> rs = RuleSet(rule1, rule2)
The RewriteRule
objects describe the desired transformations in a
declarative way, and the RuleSet
builds an efficient automata for applying
that transformation. Rewriting can then be done using the rewrite
method:
>>> rs.rewrite((add, 5, 5))
(mul, 5, 2)
>>> rs.rewrite((mul, 5, 5))
(pow, 5, 2)
>>> rs.rewrite((mul, (add, 3, 3), (add, 3, 3)))
(pow, (mul, 3, 2), 2)
The whole task is traversed by default. If you only want to apply a transform
to the top-level of the task, you can pass in strategy='top_level'
as shown:
# Transforms whole task
>>> rs.rewrite((sum, [(add, 3, 3), (mul, 3, 3)]))
(sum, [(mul, 3, 2), (pow, 3, 2)])
# Only applies to top level, no transform occurs
>>> rs.rewrite((sum, [(add, 3, 3), (mul, 3, 3)]), strategy='top_level')
(sum, [(add, 3, 3), (mul, 3, 3)])
The rewriting system provides a powerful abstraction for transforming computations at a task level. Again, for many users, directly interacting with these transformations will be unnecessary.
Keyword Arguments¶
Some optimizations take optional keyword arguments. To pass keywords from the
compute call down to the right optimization, prepend the keyword with the name
of the optimization. For example to send a keys=
keyword argument to the
fuse
optimization from a compute call, use the fuse_keys=
keyword:
def fuse(dsk, keys=None):
...
x.compute(fuse_keys=['x', 'y', 'z'])
API¶
Top level optimizations
cull (dsk, keys) |
Return new dask with only the tasks required to calculate keys. |
fuse (dsk[, keys, dependencies, ...]) |
Return new dask graph with linear sequence of tasks fused together. |
inline (dsk[, keys, inline_constants, ...]) |
Return new dask with the given keys inlined with their values. |
inline_functions (dsk, output[, ...]) |
Inline cheap functions into larger operations |
Utility functions
functions_of (task) |
Set of functions contained within nested task |
Rewrite Rules
RewriteRule (lhs, rhs[, vars]) |
A rewrite rule. |
RuleSet (*rules) |
A set of rewrite rules. |
Definitions¶
-
dask.optimize.
cull
(dsk, keys)¶ Return new dask with only the tasks required to calculate keys.
In other words, remove unnecessary tasks from dask.
keys
may be a single key or list of keys.Returns: dsk: culled dask graph
dependencies: Dict mapping {key: [deps]}. Useful side effect to accelerate
other optimizations, notably fuse.
Examples
>>> d = {'x': 1, 'y': (inc, 'x'), 'out': (add, 'x', 10)} >>> dsk, dependencies = cull(d, 'out') >>> dsk {'x': 1, 'out': (add, 'x', 10)} >>> dependencies {'x': set(), 'out': set(['x'])}
-
dask.optimize.
fuse
(dsk, keys=None, dependencies=None, rename_fused_keys=True)¶ Return new dask graph with linear sequence of tasks fused together.
If specified, the keys in
keys
keyword argument are not fused. Supplydependencies
from output ofcull
if available to avoid recomputing dependencies.Parameters: dsk: dict
keys: list
dependencies: dict, optional
{key: [list-of-keys]}. Must be a list to provide count of each key This optional input often comes from
cull
rename_fused_keys: bool or func, optional
Whether to rename the fused keys with
default_fused_keys_renamer
or not. Renaming fused keys can keep the graph more understandable and comprehensive, but it comes at the cost of additional processing. If False, then the top-most key will be used. For advanced usage, a func is also accepted,new_key = rename_fused_keys(fused_key_list)
.Returns: dsk: output graph with keys fused
dependencies: dict mapping dependencies after fusion. Useful side effect
to accelerate other downstream optimizations.
Examples
>>> d = {'a': 1, 'b': (inc, 'a'), 'c': (inc, 'b')} >>> dsk, dependencies = fuse(d) >>> dsk {'a-b-c': (inc, (inc, 1)), 'c': 'a-b-c'} >>> dsk, dependencies = fuse(d, rename_fused_keys=False) >>> dsk {'c': (inc, (inc, 1))} >>> dsk, dependencies = fuse(d, keys=['b'], rename_fused_keys=False) >>> dsk {'b': (inc, 1), 'c': (inc, 'b')}
-
dask.optimize.
inline
(dsk, keys=None, inline_constants=True, dependencies=None)¶ Return new dask with the given keys inlined with their values.
Inlines all constants if
inline_constants
keyword is True. Note that the constant keys will remain in the graph, to remove them followinline
withcull
.Examples
>>> d = {'x': 1, 'y': (inc, 'x'), 'z': (add, 'x', 'y')} >>> inline(d) {'x': 1, 'y': (inc, 1), 'z': (add, 1, 'y')}
>>> inline(d, keys='y') {'x': 1, 'y': (inc, 1), 'z': (add, 1, (inc, 1))}
>>> inline(d, keys='y', inline_constants=False) {'x': 1, 'y': (inc, 1), 'z': (add, 'x', (inc, 'x'))}
-
dask.optimize.
inline_functions
(dsk, output, fast_functions=None, inline_constants=False, dependencies=None)¶ Inline cheap functions into larger operations
Examples
>>> dsk = {'out': (add, 'i', 'd'), ... 'i': (inc, 'x'), ... 'd': (double, 'y'), ... 'x': 1, 'y': 1} >>> inline_functions(dsk, [], [inc]) {'out': (add, (inc, 'x'), 'd'), 'd': (double, 'y'), 'x': 1, 'y': 1}
Protect output keys. In the example below
i
is not inlined because it is marked as an output key.>>> inline_functions(dsk, ['i', 'out'], [inc, double]) {'out': (add, 'i', (double, 'y')), 'i': (inc, 'x'), 'x': 1, 'y': 1}
-
dask.optimize.
functions_of
(task)¶ Set of functions contained within nested task
Examples
>>> task = (add, (mul, 1, 2), (inc, 3)) >>> functions_of(task) set([add, mul, inc])
-
dask.rewrite.
RewriteRule
(lhs, rhs, vars=())¶ A rewrite rule.
Expresses lhs -> rhs, for variables vars.
Parameters: lhs : task
The left-hand-side of the rewrite rule.
rhs : task or function
The right-hand-side of the rewrite rule. If it’s a task, variables in rhs will be replaced by terms in the subject that match the variables in lhs. If it’s a function, the function will be called with a dict of such matches.
vars: tuple, optional
Tuple of variables found in the lhs. Variables can be represented as any hashable object; a good convention is to use strings. If there are no variables, this can be omitted.
Examples
Here’s a RewriteRule to replace all nested calls to list, so that (list, (list, ‘x’)) is replaced with (list, ‘x’), where ‘x’ is a variable.
>>> lhs = (list, (list, 'x')) >>> rhs = (list, 'x') >>> variables = ('x',) >>> rule = RewriteRule(lhs, rhs, variables)
Here’s a more complicated rule that uses a callable right-hand-side. A callable rhs takes in a dictionary mapping variables to their matching values. This rule replaces all occurrences of (list, ‘x’) with ‘x’ if ‘x’ is a list itself.
>>> lhs = (list, 'x') >>> def repl_list(sd): ... x = sd['x'] ... if isinstance(x, list): ... return x ... else: ... return (list, x) >>> rule = RewriteRule(lhs, repl_list, variables)
-
dask.rewrite.
RuleSet
(*rules)¶ A set of rewrite rules.
Forms a structure for fast rewriting over a set of rewrite rules. This allows for syntactic matching of terms to patterns for many patterns at the same time.
Examples
>>> def f(*args): pass >>> def g(*args): pass >>> def h(*args): pass >>> from operator import add
>>> rs = RuleSet( # Make RuleSet with two Rules ... RewriteRule((add, 'x', 0), 'x', ('x',)), ... RewriteRule((f, (g, 'x'), 'y'), ... (h, 'x', 'y'), ... ('x', 'y')))
>>> rs.rewrite((add, 2, 0)) # Apply ruleset to single task 2
>>> rs.rewrite((f, (g, 'a', 3))) (h, 'a', 3)
>>> dsk = {'a': (add, 2, 0), # Apply ruleset to full dask graph ... 'b': (f, (g, 'a', 3))}
>>> from toolz import valmap >>> valmap(rs.rewrite, dsk) {'a': 2, 'b': (h, 'a', 3)}
Attributes
rules (list) A list of RewriteRule`s included in the `RuleSet.