File: delayed-best-practices.rst

package info (click to toggle)
dask 1.0.0%2Bdfsg-2
  • links: PTS, VCS
  • area: main
  • in suites: buster
  • size: 6,856 kB
  • sloc: python: 51,266; sh: 178; makefile: 142
file content (340 lines) | stat: -rw-r--r-- 7,941 bytes parent folder | download
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
Best Practices
==============

It is easy to get started with Dask delayed, but using it *well* does require
some experience.  This page contains suggestions for best practices, and
includes solutions to common problems.


Call delayed on the function, not the result
--------------------------------------------

Dask delayed operates on functions like ``dask.delayed(f)(x, y)``, not on their results like ``dask.delayed(f(x, y))``.  When you do the latter, Python first calculates ``f(x, y)`` before Dask has a chance to step in.

**Don't**

.. code-block:: python

   dask.delayed(f(x, y))

**Do**

.. code-block:: python

   dask.delayed(f)(x, y)


Compute on lots of computation at once
--------------------------------------

To improve parallelism, you want to include lots of computation in each compute call.
Ideally, you want to make many ``dask.delayed`` calls to define your computation and
then call ``dask.compute`` only at the end.  It is ok to call ``dask.compute``
in the middle of your computation as well, but everything will stop there as
Dask computes those results before moving forward with your code.

**Don't**

.. code-block:: python

   for x in L:
       y = dask.delayed(f)(x)
       y.compute()  # calling compute after every delayed call stops parallelism

**Do**

.. code-block:: python

   results = []
   for x in L:
       y = dask.delayed(f)(x)
       results.append(y)

   results = dask.compute(*results)  # call compute after you have collected many delayed calls


Don't mutate inputs
-------------------

Your functions should not change the inputs directly.

**Don't**

.. code-block:: python

   @dask.delayed
   def f(x):
       x += 1
       return x

**Do**

.. code-block:: python

   @dask.delayed
   def f(x):
       return x + 1

If you need to use a mutable operation, then make a copy within your function first:

.. code-block:: python

   @dask.delayed
   def f(x):
       x = copy(x)
       x += 1
       return x


Avoid global state
------------------

Ideally, your operations shouldn't rely on global state.  Using global state
*might* work if you only use threads, but when you move to multiprocessing or
distributed computing then you will likely encounter confusing errors.

**Don't**

.. code-block:: python

   L = []

   @dask.delayed
   def f(x):
       L.append(x)


Don't rely on side effects
--------------------------

Delayed functions only do something if they are computed.  You will always need
to pass the output to something that eventually calls compute.

**Don't**

.. code-block:: python

   dask.delayed(f)(1, 2, 3)  # this has no effect

**Do**

.. code-block:: python

   x = dask.delayed(f)(1, 2, 3)
   ...
   dask.compute(x, ...)  # need to call compute for something to happen


Break up computations into many pieces
--------------------------------------

Every ``dask.delayed`` function call is a single operation from Dask's perspective.
You achieve parallelism by having many delayed calls, not by using only a
single one: Dask will not look inside a function decorated with ``@dask.delayed``
and parallelize that code internally.  To accomplish that, it needs your help to 
find good places to break up a computation.

**Don't**

.. code-block:: python

   def load(filename):
       ...

   def process(data):
       ...

   def save(data):
       ...

   @dask.delayed
   def f(filenames):
       results = []
       for filename in filenames:
           data = load(filename)
           data = process(data)
           results.append(save(data))
       
       return results

   dask.compute(f(filenames))  # this is only a single task

**Do**

.. code-block:: python

   @dask.delayed
   def load(filename):
       ...

   @dask.delayed
   def process(data):
       ...

   @dask.delayed
   def save(data):
       ...

   def f(filenames):
       results = []
       for filename in filenames:
           data = load(filename)
           data = process(data)
           results.append(save(data))

       return results

   dask.compute(f(filenames))  # this has many tasks and so will parallelize


Avoid too many tasks
--------------------

Every delayed task has an overhead of a few hundred microseconds.  Usually this
is ok, but it can become a problem if you apply ``dask.delayed`` too finely.  In
this case, it's often best to break up your many tasks into batches or use one
of the Dask collections to help you.

**Don't**

.. code-block:: python

   results = []
   for x in range(1000000000):  # Too many dask.delayed calls
       y = dask.delayed(f)(x)
       results.append(y)

**Do**

.. code-block:: python

   # Use collections

   import dask.bag as db
   b = db.from_sequence(1000000000, npartitions=1000)
   b = b.map(f)

.. code-block:: python

   # Or batch manually

   def batch(seq):
       sub_results = []
       for x in seq:
           sub_results.append(f(x))
       return sub_results

   batches = []
   for i in range(0, 1000000000, 1000000):  # in steps of 1000000
       result_batch = dask.delayed(batch, range(i, i + 1000000))
       batches.append(result_batch)


Avoid calling delayed within delayed functions
----------------------------------------------

Often, if you are new to using Dask delayed, you place ``dask.delayed`` calls
everywhere and hope for the best.  While this may actually work, it's usually
slow and results in hard-to-understand solutions.

Usually you never call ``dask.delayed`` within ``dask.delayed`` functions.

**Don't**

.. code-block:: python

   @dask.delayed
   def process_all(L):
       result = []
       for x in L:
           y = dask.delayed(f)(x)
           result.append(y)
        return result

**Do**

Instead, because this function only does delayed work, it is very fast and so
there is no reason to delay it.

.. code-block:: python

   def process_all(L):
       result = []
       for x in L:
           y = dask.delayed(f)(x)
           result.append(y)
        return result


Don't call dask.delayed on other Dask collections
-------------------------------------------------

When you place a Dask array or Dask DataFrame into a delayed call, that function
will receive the NumPy or Pandas equivalent.  Beware that if your array is
large, then this might crash your workers.

Instead, it's more common to use methods like ``da.map_blocks`` or
``df.map_partitions``, or to turn your arrays or DataFrames into *many* delayed
objects.

**Don't**

.. code-block:: python

   import dask.dataframe as dd
   df = dd.read_csv('/path/to/*.csv')

   dask.delayed(train)(df)  # might as well have used Pandas instead

**Do**

.. code-block:: python

   import dask.dataframe as dd
   df = dd.read_csv('/path/to/*.csv')

   df.map_partitions(train)
   # or
   partitions = df.to_delayed()

   delayed_values = [dask.delayed(train)(part) for part in partitions]

However, if you don't mind turning your Dask array/DataFrame into a single
chunk, then this is ok.

.. code-block:: python

   dask.delayed(train)(..., y=df.sum())


Avoid repeatedly putting large inputs into delayed calls
--------------------------------------------------------

Every time you pass a concrete result (anything that isn't delayed) Dask will
hash it by default to give it a name.  This is fairly fast (around 500 MB/s)
but can be slow if you do it over and over again.  Instead, it is better to
delay your data as well.

This is especially important when using a distributed cluster to avoid sending
your data separately for each function call.

**Don't**

.. code-block:: python

   x = np.array(...)  # some large array

   results = [dask.delayed(train)(x, i) for i in range(1000)]


Every call to ``dask.delayed(train)(x, ...)`` has to hash the NumPy array ``x``, which slows things down.


**Do**

.. code-block:: python

   x = np.array(...)  # some large array
   x = dask.delayed(x)  # delay the data, hashing once

   results = [dask.delayed(train)(x, i) for i in range(1000)]