File: optimize.rst

package info (click to toggle)
dask 2024.12.1%2Bdfsg-2
  • links: PTS, VCS
  • area: main
  • in suites: forky, sid, trixie
  • size: 20,024 kB
  • sloc: python: 105,182; javascript: 1,917; makefile: 159; sh: 88
file content (370 lines) | stat: -rw-r--r-- 12,200 bytes parent folder | download | duplicates (2)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
.. _optimization:

Optimization
============

Performance can be significantly improved in different contexts by making
small optimizations on the Dask graph before calling the scheduler.

The ``dask.optimization`` 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:

1. Simplify computation
2. 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:

.. code-block:: python

    >>> def print_and_return(string):
    ...     print(string)
    ...     return string

    >>> def format_str(count, val, nwords):
    ...     return (f'word list has {count} occurrences of '
    ...             f'{val}, out of {nwords} words')

    >>> 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'),
    ...        'format1': (format_str, 'count1', 'val1', 'nwords'),
    ...        'format2': (format_str, 'count2', 'val2', 'nwords'),
    ...        'format3': (format_str, 'count3', 'val3', 'nwords'),
    ...        'print1': (print_and_return, 'format1'),
    ...        'print2': (print_and_return, 'format2'),
    ...        'print3': (print_and_return, 'format3')}

.. image:: images/optimize_dask1.svg
   :width: 65 %
   :alt: The original non-optimized Dask task graph.

Here we are 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, and then returning the output string.

To perform the computation, we first remove unnecessary components from the
graph using the ``cull`` function and then pass the Dask graph and the desired
output keys to a scheduler ``get`` function:

.. code-block:: python

    >>> from dask.threaded import get
    >>> from dask.optimization import cull

    >>> outputs = ['print1', 'print2']
    >>> dsk1, dependencies = cull(dsk, outputs)  # remove unnecessary tasks from the graph

    >>> results = get(dsk1, outputs)
    word list has 2 occurrences of apple, out of 7 words
    word list has 2 occurrences of orange, out of 7 words

As can be seen above, the scheduler computed only the requested outputs
(``'print3'`` was never computed). This is because we called the
``dask.optimization.cull`` function, which removes the unnecessary tasks from
the graph.

Culling is part of the default optimization pass of almost all collections.
Often you want to call it somewhat early to reduce the amount of work done in
later steps:

.. code-block:: python

    >>> from dask.optimization import cull
    >>> outputs = ['print1', 'print2']
    >>> dsk1, dependencies = cull(dsk, outputs)

.. image:: images/optimize_dask2.svg
   :width: 50 %
   :alt: The Dask task graph after culling tasks for optimization.

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:

.. code-block:: python

    >>> from dask.optimization 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

.. image:: images/optimize_dask3.svg
   :width: 40 %
   :alt: The Dask task graph after inlining for optimization.

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:

.. code-block:: python

    >>> from dask.optimization 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

.. image:: images/optimize_dask4.svg
   :width: 30 %
   :alt: The Dask task graph after inlining functions for optimization.

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:

.. code-block:: python

    >>> from dask.optimization 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

.. image:: images/optimize_dask5.svg
   :width: 30 %
   :alt: The Dask task graph after fusing tasks for optimization.


Putting it all together:

.. code-block:: python

    >>> 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:

1. Removed tasks unnecessary for the desired output using ``cull``
2. Inlined constants using ``inline``
3. Inlined cheap computations using ``inline_functions``, improving parallelism
4. 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.optimization``. 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:

1. ``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``

2. ``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:

1. ``a + a -> 2*a``
2. ``a * a -> a**2``

where ``'a'`` is a variable:

.. code-block:: python

    >>> 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:

.. code-block:: python

    >>> 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:

.. code-block:: python

    # 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:

.. code-block:: python

   def fuse(dsk, keys=None):
       ...

   x.compute(fuse_keys=['x', 'y', 'z'])


Customizing Optimization
------------------------

Dask defines a default optimization strategy for each collection type (Array,
Bag, DataFrame, Delayed).  However, different applications may have different
needs.  To address this variability of needs, you can construct your own custom
optimization function and use it instead of the default.  An optimization
function takes in a task graph and list of desired keys and returns a new
task graph:

.. code-block:: python

   def my_optimize_function(dsk, keys):
       new_dsk = {...}
       return new_dsk

You can then register this optimization class against whichever collection type
you prefer and it will be used instead of the default scheme:

.. code-block:: python

   with dask.config.set(array_optimize=my_optimize_function):
       x, y = dask.compute(x, y)

You can register separate optimization functions for different collections, or
you can register ``None`` if you do not want particular types of collections to
be optimized:

.. code-block:: python

   with dask.config.set(array_optimize=my_optimize_function,
                        dataframe_optimize=None,
                        delayed_optimize=my_other_optimize_function):
       ...

You do not need to specify all collections.  Collections will default to their
standard optimization scheme (which is usually a good choice).


API
---

.. currentmodule:: dask.optimization

**Top level optimizations**

.. autosummary::
   cull
   fuse
   inline
   inline_functions

**Utility functions**

.. autosummary::
   functions_of

**Rewrite Rules**

.. currentmodule:: dask.rewrite

.. autosummary::
    RewriteRule
    RuleSet


Definitions
~~~~~~~~~~~

.. currentmodule:: dask.optimization

.. autofunction:: cull
.. autofunction:: fuse
.. autofunction:: inline
.. autofunction:: inline_functions

.. autofunction:: functions_of

.. currentmodule:: dask.rewrite

.. autofunction:: RewriteRule
.. autofunction:: RuleSet