/
base.py
1528 lines (1254 loc) · 50.8 KB
/
base.py
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
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
from __future__ import annotations
import dataclasses
import datetime
import hashlib
import inspect
import os
import pickle
import threading
import uuid
import warnings
from collections import OrderedDict
from collections.abc import Callable, Iterator, Mapping
from concurrent.futures import Executor
from contextlib import contextmanager
from enum import Enum
from functools import partial
from numbers import Integral, Number
from operator import getitem
from typing import Literal
from packaging.version import parse as parse_version
from tlz import curry, groupby, identity, merge
from tlz.functoolz import Compose
from dask import config, local
from dask.compatibility import _EMSCRIPTEN, _PY_VERSION
from dask.context import thread_state
from dask.core import flatten
from dask.core import get as simple_get
from dask.core import literal, quote
from dask.hashing import hash_buffer_hex
from dask.system import CPU_COUNT
from dask.typing import SchedulerGetCallable
from dask.utils import Dispatch, apply, ensure_dict, key_split
__all__ = (
"DaskMethodsMixin",
"annotate",
"is_dask_collection",
"compute",
"persist",
"optimize",
"visualize",
"tokenize",
"normalize_token",
"get_collection_names",
"get_name_from_key",
"replace_name_in_key",
"clone_key",
)
@contextmanager
def annotate(**annotations):
"""Context Manager for setting HighLevelGraph Layer annotations.
Annotations are metadata or soft constraints associated with
tasks that dask schedulers may choose to respect: They signal intent
without enforcing hard constraints. As such, they are
primarily designed for use with the distributed scheduler.
Almost any object can serve as an annotation, but small Python objects
are preferred, while large objects such as NumPy arrays are discouraged.
Callables supplied as an annotation should take a single *key* argument and
produce the appropriate annotation. Individual task keys in the annotated collection
are supplied to the callable.
Parameters
----------
**annotations : key-value pairs
Examples
--------
All tasks within array A should have priority 100 and be retried 3 times
on failure.
>>> import dask
>>> import dask.array as da
>>> with dask.annotate(priority=100, retries=3):
... A = da.ones((10000, 10000))
Prioritise tasks within Array A on flattened block ID.
>>> nblocks = (10, 10)
>>> with dask.annotate(priority=lambda k: k[1]*nblocks[1] + k[2]):
... A = da.ones((1000, 1000), chunks=(100, 100))
Annotations may be nested.
>>> with dask.annotate(priority=1):
... with dask.annotate(retries=3):
... A = da.ones((1000, 1000))
... B = A + 1
"""
# Sanity check annotations used in place of
# legacy distributed Client.{submit, persist, compute} keywords
if "workers" in annotations:
if isinstance(annotations["workers"], (list, set, tuple)):
annotations["workers"] = list(annotations["workers"])
elif isinstance(annotations["workers"], str):
annotations["workers"] = [annotations["workers"]]
elif callable(annotations["workers"]):
pass
else:
raise TypeError(
"'workers' annotation must be a sequence of str, a str or a callable, but got %s."
% annotations["workers"]
)
if (
"priority" in annotations
and not isinstance(annotations["priority"], Number)
and not callable(annotations["priority"])
):
raise TypeError(
"'priority' annotation must be a Number or a callable, but got %s"
% annotations["priority"]
)
if (
"retries" in annotations
and not isinstance(annotations["retries"], Number)
and not callable(annotations["retries"])
):
raise TypeError(
"'retries' annotation must be a Number or a callable, but got %s"
% annotations["retries"]
)
if (
"resources" in annotations
and not isinstance(annotations["resources"], dict)
and not callable(annotations["resources"])
):
raise TypeError(
"'resources' annotation must be a dict, but got %s"
% annotations["resources"]
)
if (
"allow_other_workers" in annotations
and not isinstance(annotations["allow_other_workers"], bool)
and not callable(annotations["allow_other_workers"])
):
raise TypeError(
"'allow_other_workers' annotations must be a bool or a callable, but got %s"
% annotations["allow_other_workers"]
)
prev_annotations = config.get("annotations", {})
new_annotations = {
**prev_annotations,
**{f"annotations.{k}": v for k, v in annotations.items()},
}
with config.set(new_annotations):
yield
def is_dask_collection(x) -> bool:
"""Returns ``True`` if ``x`` is a dask collection.
Parameters
----------
x : Any
Object to test.
Returns
-------
result : bool
``True`` if `x` is a Dask collection.
Notes
-----
The DaskCollection typing.Protocol implementation defines a Dask
collection as a class that returns a Mapping from the
``__dask_graph__`` method. This helper function existed before the
implementation of the protocol.
"""
try:
return x.__dask_graph__() is not None
except (AttributeError, TypeError):
return False
class DaskMethodsMixin:
"""A mixin adding standard dask collection methods"""
__slots__ = ()
def visualize(self, filename="mydask", format=None, optimize_graph=False, **kwargs):
"""Render the computation of this object's task graph using graphviz.
Requires ``graphviz`` to be installed.
Parameters
----------
filename : str or None, optional
The name of the file to write to disk. If the provided `filename`
doesn't include an extension, '.png' will be used by default.
If `filename` is None, no file will be written, and we communicate
with dot using only pipes.
format : {'png', 'pdf', 'dot', 'svg', 'jpeg', 'jpg'}, optional
Format in which to write output file. Default is 'png'.
optimize_graph : bool, optional
If True, the graph is optimized before rendering. Otherwise,
the graph is displayed as is. Default is False.
color: {None, 'order'}, optional
Options to color nodes. Provide ``cmap=`` keyword for additional
colormap
**kwargs
Additional keyword arguments to forward to ``to_graphviz``.
Examples
--------
>>> x.visualize(filename='dask.pdf') # doctest: +SKIP
>>> x.visualize(filename='dask.pdf', color='order') # doctest: +SKIP
Returns
-------
result : IPython.diplay.Image, IPython.display.SVG, or None
See dask.dot.dot_graph for more information.
See Also
--------
dask.base.visualize
dask.dot.dot_graph
Notes
-----
For more information on optimization see here:
https://docs.dask.org/en/latest/optimize.html
"""
return visualize(
self,
filename=filename,
format=format,
optimize_graph=optimize_graph,
**kwargs,
)
def persist(self, **kwargs):
"""Persist this dask collection into memory
This turns a lazy Dask collection into a Dask collection with the same
metadata, but now with the results fully computed or actively computing
in the background.
The action of function differs significantly depending on the active
task scheduler. If the task scheduler supports asynchronous computing,
such as is the case of the dask.distributed scheduler, then persist
will return *immediately* and the return value's task graph will
contain Dask Future objects. However if the task scheduler only
supports blocking computation then the call to persist will *block*
and the return value's task graph will contain concrete Python results.
This function is particularly useful when using distributed systems,
because the results will be kept in distributed memory, rather than
returned to the local process as with compute.
Parameters
----------
scheduler : string, optional
Which scheduler to use like "threads", "synchronous" or "processes".
If not provided, the default is to check the global settings first,
and then fall back to the collection defaults.
optimize_graph : bool, optional
If True [default], the graph is optimized before computation.
Otherwise the graph is run as is. This can be useful for debugging.
**kwargs
Extra keywords to forward to the scheduler function.
Returns
-------
New dask collections backed by in-memory data
See Also
--------
dask.base.persist
"""
(result,) = persist(self, traverse=False, **kwargs)
return result
def compute(self, **kwargs):
"""Compute this dask collection
This turns a lazy Dask collection into its in-memory equivalent.
For example a Dask array turns into a NumPy array and a Dask dataframe
turns into a Pandas dataframe. The entire dataset must fit into memory
before calling this operation.
Parameters
----------
scheduler : string, optional
Which scheduler to use like "threads", "synchronous" or "processes".
If not provided, the default is to check the global settings first,
and then fall back to the collection defaults.
optimize_graph : bool, optional
If True [default], the graph is optimized before computation.
Otherwise the graph is run as is. This can be useful for debugging.
kwargs
Extra keywords to forward to the scheduler function.
See Also
--------
dask.base.compute
"""
(result,) = compute(self, traverse=False, **kwargs)
return result
def __await__(self):
try:
from distributed import futures_of, wait
except ImportError as e:
raise ImportError(
"Using async/await with dask requires the `distributed` package"
) from e
from tornado import gen
@gen.coroutine
def f():
if futures_of(self):
yield wait(self)
raise gen.Return(self)
return f().__await__()
def compute_as_if_collection(cls, dsk, keys, scheduler=None, get=None, **kwargs):
"""Compute a graph as if it were of type cls.
Allows for applying the same optimizations and default scheduler."""
from dask.highlevelgraph import HighLevelGraph
schedule = get_scheduler(scheduler=scheduler, cls=cls, get=get)
dsk2 = optimization_function(cls)(dsk, keys, **kwargs)
# see https://github.com/dask/dask/issues/8991.
# This merge should be removed once the underlying issue is fixed.
dsk2 = HighLevelGraph.merge(dsk2)
return schedule(dsk2, keys, **kwargs)
def dont_optimize(dsk, keys, **kwargs):
return dsk
def optimization_function(x):
return getattr(x, "__dask_optimize__", dont_optimize)
def collections_to_dsk(collections, optimize_graph=True, optimizations=(), **kwargs):
"""
Convert many collections into a single dask graph, after optimization
"""
from dask.highlevelgraph import HighLevelGraph
optimizations = tuple(optimizations) + tuple(config.get("optimizations", ()))
if optimize_graph:
groups = groupby(optimization_function, collections)
graphs = []
for opt, val in groups.items():
dsk, keys = _extract_graph_and_keys(val)
dsk = opt(dsk, keys, **kwargs)
for opt_inner in optimizations:
dsk = opt_inner(dsk, keys, **kwargs)
graphs.append(dsk)
# Merge all graphs
if any(isinstance(graph, HighLevelGraph) for graph in graphs):
dsk = HighLevelGraph.merge(*graphs)
else:
dsk = merge(*map(ensure_dict, graphs))
else:
dsk, _ = _extract_graph_and_keys(collections)
return dsk
def _extract_graph_and_keys(vals):
"""Given a list of dask vals, return a single graph and a list of keys such
that ``get(dsk, keys)`` is equivalent to ``[v.compute() for v in vals]``."""
from dask.highlevelgraph import HighLevelGraph
graphs, keys = [], []
for v in vals:
graphs.append(v.__dask_graph__())
keys.append(v.__dask_keys__())
if any(isinstance(graph, HighLevelGraph) for graph in graphs):
graph = HighLevelGraph.merge(*graphs)
else:
graph = merge(*map(ensure_dict, graphs))
return graph, keys
def unpack_collections(*args, traverse=True):
"""Extract collections in preparation for compute/persist/etc...
Intended use is to find all collections in a set of (possibly nested)
python objects, do something to them (compute, etc...), then repackage them
in equivalent python objects.
Parameters
----------
*args
Any number of objects. If it is a dask collection, it's extracted and
added to the list of collections returned. By default, python builtin
collections are also traversed to look for dask collections (for more
information see the ``traverse`` keyword).
traverse : bool, optional
If True (default), builtin python collections are traversed looking for
any dask collections they might contain.
Returns
-------
collections : list
A list of all dask collections contained in ``args``
repack : callable
A function to call on the transformed collections to repackage them as
they were in the original ``args``.
"""
collections = []
repack_dsk = {}
collections_token = uuid.uuid4().hex
def _unpack(expr):
if is_dask_collection(expr):
tok = tokenize(expr)
if tok not in repack_dsk:
repack_dsk[tok] = (getitem, collections_token, len(collections))
collections.append(expr)
return tok
tok = uuid.uuid4().hex
if not traverse:
tsk = quote(expr)
else:
# Treat iterators like lists
typ = list if isinstance(expr, Iterator) else type(expr)
if typ in (list, tuple, set):
tsk = (typ, [_unpack(i) for i in expr])
elif typ in (dict, OrderedDict):
tsk = (typ, [[_unpack(k), _unpack(v)] for k, v in expr.items()])
elif dataclasses.is_dataclass(expr) and not isinstance(expr, type):
tsk = (
apply,
typ,
(),
(
dict,
[
[f.name, _unpack(getattr(expr, f.name))]
for f in dataclasses.fields(expr)
],
),
)
else:
return expr
repack_dsk[tok] = tsk
return tok
out = uuid.uuid4().hex
repack_dsk[out] = (tuple, [_unpack(i) for i in args])
def repack(results):
dsk = repack_dsk.copy()
dsk[collections_token] = quote(results)
return simple_get(dsk, out)
return collections, repack
def optimize(*args, traverse=True, **kwargs):
"""Optimize several dask collections at once.
Returns equivalent dask collections that all share the same merged and
optimized underlying graph. This can be useful if converting multiple
collections to delayed objects, or to manually apply the optimizations at
strategic points.
Note that in most cases you shouldn't need to call this method directly.
Parameters
----------
*args : objects
Any number of objects. If a dask object, its graph is optimized and
merged with all those of all other dask objects before returning an
equivalent dask collection. Non-dask arguments are passed through
unchanged.
traverse : bool, optional
By default dask traverses builtin python collections looking for dask
objects passed to ``optimize``. For large collections this can be
expensive. If none of the arguments contain any dask objects, set
``traverse=False`` to avoid doing this traversal.
optimizations : list of callables, optional
Additional optimization passes to perform.
**kwargs
Extra keyword arguments to forward to the optimization passes.
Examples
--------
>>> import dask as d
>>> import dask.array as da
>>> a = da.arange(10, chunks=2).sum()
>>> b = da.arange(10, chunks=2).mean()
>>> a2, b2 = d.optimize(a, b)
>>> a2.compute() == a.compute()
True
>>> b2.compute() == b.compute()
True
"""
collections, repack = unpack_collections(*args, traverse=traverse)
if not collections:
return args
dsk = collections_to_dsk(collections, **kwargs)
postpersists = []
for a in collections:
r, s = a.__dask_postpersist__()
postpersists.append(r(dsk, *s))
return repack(postpersists)
def compute(
*args, traverse=True, optimize_graph=True, scheduler=None, get=None, **kwargs
):
"""Compute several dask collections at once.
Parameters
----------
args : object
Any number of objects. If it is a dask object, it's computed and the
result is returned. By default, python builtin collections are also
traversed to look for dask objects (for more information see the
``traverse`` keyword). Non-dask arguments are passed through unchanged.
traverse : bool, optional
By default dask traverses builtin python collections looking for dask
objects passed to ``compute``. For large collections this can be
expensive. If none of the arguments contain any dask objects, set
``traverse=False`` to avoid doing this traversal.
scheduler : string, optional
Which scheduler to use like "threads", "synchronous" or "processes".
If not provided, the default is to check the global settings first,
and then fall back to the collection defaults.
optimize_graph : bool, optional
If True [default], the optimizations for each collection are applied
before computation. Otherwise the graph is run as is. This can be
useful for debugging.
get : ``None``
Should be left to ``None`` The get= keyword has been removed.
kwargs
Extra keywords to forward to the scheduler function.
Examples
--------
>>> import dask as d
>>> import dask.array as da
>>> a = da.arange(10, chunks=2).sum()
>>> b = da.arange(10, chunks=2).mean()
>>> d.compute(a, b)
(45, 4.5)
By default, dask objects inside python collections will also be computed:
>>> d.compute({'a': a, 'b': b, 'c': 1})
({'a': 45, 'b': 4.5, 'c': 1},)
"""
collections, repack = unpack_collections(*args, traverse=traverse)
if not collections:
return args
schedule = get_scheduler(
scheduler=scheduler,
collections=collections,
get=get,
)
dsk = collections_to_dsk(collections, optimize_graph, **kwargs)
keys, postcomputes = [], []
for x in collections:
keys.append(x.__dask_keys__())
postcomputes.append(x.__dask_postcompute__())
results = schedule(dsk, keys, **kwargs)
return repack([f(r, *a) for r, (f, a) in zip(results, postcomputes)])
def visualize(
*args,
filename="mydask",
traverse=True,
optimize_graph=False,
maxval=None,
engine: Literal["cytoscape", "ipycytoscape", "graphviz"] | None = None,
**kwargs,
):
"""
Visualize several dask graphs simultaneously.
Requires ``graphviz`` to be installed. All options that are not the dask
graph(s) should be passed as keyword arguments.
Parameters
----------
args : object
Any number of objects. If it is a dask collection (for example, a
dask DataFrame, Array, Bag, or Delayed), its associated graph
will be included in the output of visualize. By default, python builtin
collections are also traversed to look for dask objects (for more
information see the ``traverse`` keyword). Arguments lacking an
associated graph will be ignored.
filename : str or None, optional
The name of the file to write to disk. If the provided `filename`
doesn't include an extension, '.png' will be used by default.
If `filename` is None, no file will be written, and we communicate
with dot using only pipes.
format : {'png', 'pdf', 'dot', 'svg', 'jpeg', 'jpg'}, optional
Format in which to write output file. Default is 'png'.
traverse : bool, optional
By default, dask traverses builtin python collections looking for dask
objects passed to ``visualize``. For large collections this can be
expensive. If none of the arguments contain any dask objects, set
``traverse=False`` to avoid doing this traversal.
optimize_graph : bool, optional
If True, the graph is optimized before rendering. Otherwise,
the graph is displayed as is. Default is False.
color : {None, 'order', 'ages', 'freed', 'memoryincreases', 'memorydecreases', 'memorypressure'}, optional
Options to color nodes. colormap:
- None, the default, no colors.
- 'order', colors the nodes' border based on the order they appear in the graph.
- 'ages', how long the data of a node is held.
- 'freed', the number of dependencies released after running a node.
- 'memoryincreases', how many more outputs are held after the lifetime of a node.
Large values may indicate nodes that should have run later.
- 'memorydecreases', how many fewer outputs are held after the lifetime of a node.
Large values may indicate nodes that should have run sooner.
- 'memorypressure', the number of data held when the node is run (circle), or
the data is released (rectangle).
maxval : {int, float}, optional
Maximum value for colormap to normalize form 0 to 1.0. Default is ``None``
will make it the max number of values
collapse_outputs : bool, optional
Whether to collapse output boxes, which often have empty labels.
Default is False.
verbose : bool, optional
Whether to label output and input boxes even if the data aren't chunked.
Beware: these labels can get very long. Default is False.
engine : {"graphviz", "ipycytoscape", "cytoscape"}, optional.
The visualization engine to use. If not provided, this checks the dask config
value "visualization.engine". If that is not set, it tries to import ``graphviz``
and ``ipycytoscape``, using the first one to succeed.
**kwargs
Additional keyword arguments to forward to the visualization engine.
Examples
--------
>>> x.visualize(filename='dask.pdf') # doctest: +SKIP
>>> x.visualize(filename='dask.pdf', color='order') # doctest: +SKIP
Returns
-------
result : IPython.diplay.Image, IPython.display.SVG, or None
See dask.dot.dot_graph for more information.
See Also
--------
dask.dot.dot_graph
Notes
-----
For more information on optimization see here:
https://docs.dask.org/en/latest/optimize.html
"""
args, _ = unpack_collections(*args, traverse=traverse)
dsk = dict(collections_to_dsk(args, optimize_graph=optimize_graph))
color = kwargs.get("color")
if color in {
"order",
"order-age",
"order-freed",
"order-memoryincreases",
"order-memorydecreases",
"order-memorypressure",
"age",
"freed",
"memoryincreases",
"memorydecreases",
"memorypressure",
}:
import matplotlib.pyplot as plt
from dask.order import diagnostics, order
o = order(dsk)
try:
cmap = kwargs.pop("cmap")
except KeyError:
cmap = plt.cm.plasma
if isinstance(cmap, str):
import matplotlib.pyplot as plt
cmap = getattr(plt.cm, cmap)
def label(x):
return str(values[x])
data_values = None
if color != "order":
info = diagnostics(dsk, o)[0]
if color.endswith("age"):
values = {key: val.age for key, val in info.items()}
elif color.endswith("freed"):
values = {key: val.num_dependencies_freed for key, val in info.items()}
elif color.endswith("memorypressure"):
values = {key: val.num_data_when_run for key, val in info.items()}
data_values = {
key: val.num_data_when_released for key, val in info.items()
}
elif color.endswith("memoryincreases"):
values = {
key: max(0, val.num_data_when_released - val.num_data_when_run)
for key, val in info.items()
}
else: # memorydecreases
values = {
key: max(0, val.num_data_when_run - val.num_data_when_released)
for key, val in info.items()
}
if color.startswith("order-"):
def label(x):
return str(o[x]) + "-" + str(values[x])
else:
values = o
if maxval is None:
maxval = max(1, max(values.values()))
colors = {k: _colorize(cmap(v / maxval, bytes=True)) for k, v in values.items()}
if data_values is None:
data_values = values
data_colors = colors
else:
data_colors = {
k: _colorize(cmap(v / maxval, bytes=True))
for k, v in data_values.items()
}
kwargs["function_attributes"] = {
k: {"color": v, "label": label(k)} for k, v in colors.items()
}
kwargs["data_attributes"] = {k: {"color": v} for k, v in data_colors.items()}
elif color:
raise NotImplementedError("Unknown value color=%s" % color)
# Determine which engine to dispatch to, first checking the kwarg, then config,
# then whichever of graphviz or ipycytoscape are installed, in that order.
engine = engine or config.get("visualization.engine", None)
if not engine:
try:
import graphviz # noqa: F401
engine = "graphviz"
except ImportError:
try:
import ipycytoscape # noqa: F401
engine = "cytoscape"
except ImportError:
pass
if engine == "graphviz":
from dask.dot import dot_graph
return dot_graph(dsk, filename=filename, **kwargs)
elif engine in ("cytoscape", "ipycytoscape"):
from dask.dot import cytoscape_graph
return cytoscape_graph(dsk, filename=filename, **kwargs)
elif engine is None:
raise RuntimeError(
"No visualization engine detected, please install graphviz or ipycytoscape"
)
else:
raise ValueError(f"Visualization engine {engine} not recognized")
def persist(*args, traverse=True, optimize_graph=True, scheduler=None, **kwargs):
"""Persist multiple Dask collections into memory
This turns lazy Dask collections into Dask collections with the same
metadata, but now with their results fully computed or actively computing
in the background.
For example a lazy dask.array built up from many lazy calls will now be a
dask.array of the same shape, dtype, chunks, etc., but now with all of
those previously lazy tasks either computed in memory as many small :class:`numpy.array`
(in the single-machine case) or asynchronously running in the
background on a cluster (in the distributed case).
This function operates differently if a ``dask.distributed.Client`` exists
and is connected to a distributed scheduler. In this case this function
will return as soon as the task graph has been submitted to the cluster,
but before the computations have completed. Computations will continue
asynchronously in the background. When using this function with the single
machine scheduler it blocks until the computations have finished.
When using Dask on a single machine you should ensure that the dataset fits
entirely within memory.
Examples
--------
>>> df = dd.read_csv('/path/to/*.csv') # doctest: +SKIP
>>> df = df[df.name == 'Alice'] # doctest: +SKIP
>>> df['in-debt'] = df.balance < 0 # doctest: +SKIP
>>> df = df.persist() # triggers computation # doctest: +SKIP
>>> df.value().min() # future computations are now fast # doctest: +SKIP
-10
>>> df.value().max() # doctest: +SKIP
100
>>> from dask import persist # use persist function on multiple collections
>>> a, b = persist(a, b) # doctest: +SKIP
Parameters
----------
*args: Dask collections
scheduler : string, optional
Which scheduler to use like "threads", "synchronous" or "processes".
If not provided, the default is to check the global settings first,
and then fall back to the collection defaults.
traverse : bool, optional
By default dask traverses builtin python collections looking for dask
objects passed to ``persist``. For large collections this can be
expensive. If none of the arguments contain any dask objects, set
``traverse=False`` to avoid doing this traversal.
optimize_graph : bool, optional
If True [default], the graph is optimized before computation.
Otherwise the graph is run as is. This can be useful for debugging.
**kwargs
Extra keywords to forward to the scheduler function.
Returns
-------
New dask collections backed by in-memory data
"""
collections, repack = unpack_collections(*args, traverse=traverse)
if not collections:
return args
schedule = get_scheduler(scheduler=scheduler, collections=collections)
if inspect.ismethod(schedule):
try:
from distributed.client import default_client
except ImportError:
pass
else:
try:
client = default_client()
except ValueError:
pass
else:
if client.get == schedule:
results = client.persist(
collections, optimize_graph=optimize_graph, **kwargs
)
return repack(results)
dsk = collections_to_dsk(collections, optimize_graph, **kwargs)
keys, postpersists = [], []
for a in collections:
a_keys = list(flatten(a.__dask_keys__()))
rebuild, state = a.__dask_postpersist__()
keys.extend(a_keys)
postpersists.append((rebuild, a_keys, state))
results = schedule(dsk, keys, **kwargs)
d = dict(zip(keys, results))
results2 = [r({k: d[k] for k in ks}, *s) for r, ks, s in postpersists]
return repack(results2)
############
# Tokenize #
############
# Pass `usedforsecurity=False` for Python 3.9+ to support FIPS builds of Python
_md5: Callable
if _PY_VERSION >= parse_version("3.9"):
def _md5(x, _hashlib_md5=hashlib.md5):
return _hashlib_md5(x, usedforsecurity=False)
else:
_md5 = hashlib.md5
def tokenize(*args, **kwargs):
"""Deterministic token
>>> tokenize([1, 2, '3'])
'7d6a880cd9ec03506eee6973ff551339'
>>> tokenize('Hello') == tokenize('Hello')
True
"""
hasher = _md5(str(tuple(map(normalize_token, args))).encode())
if kwargs:
hasher.update(str(normalize_token(kwargs)).encode())
return hasher.hexdigest()
normalize_token = Dispatch()
normalize_token.register(
(
int,
float,
str,
bytes,
type(None),
type,
slice,
complex,
type(Ellipsis),
datetime.date,
datetime.timedelta,
),
identity,
)
@normalize_token.register(dict)
def normalize_dict(d):
return normalize_token(sorted(d.items(), key=str))
@normalize_token.register(OrderedDict)
def normalize_ordered_dict(d):
return type(d).__name__, normalize_token(list(d.items()))
@normalize_token.register(set)
def normalize_set(s):
return normalize_token(sorted(s, key=str))
def _normalize_seq_func(seq):
# Defined outside normalize_seq to avoid unnecessary redefinitions and
# therefore improving computation times.
try:
return list(map(normalize_token, seq))
except RecursionError:
if not config.get("tokenize.ensure-deterministic"):
return uuid.uuid4().hex
raise RuntimeError(
f"Sequence {str(seq)} cannot be deterministically hashed. Please, see "
"https://docs.dask.org/en/latest/custom-collections.html#implementing-deterministic-hashing "
"for more information"
)
@normalize_token.register((tuple, list))
def normalize_seq(seq):
return type(seq).__name__, _normalize_seq_func(seq)
@normalize_token.register(literal)
def normalize_literal(lit):
return "literal", normalize_token(lit())
@normalize_token.register(range)
def normalize_range(r):