-
-
Notifications
You must be signed in to change notification settings - Fork 27
/
Copy path_expr.py
2396 lines (1897 loc) · 69.5 KB
/
_expr.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 functools
import numbers
import operator
import os
import weakref
from collections import defaultdict
from collections.abc import Generator, Mapping
import dask
import numpy as np
import pandas as pd
import toolz
from dask.base import normalize_token
from dask.core import flatten, ishashable
from dask.dataframe import methods
from dask.dataframe.core import (
_get_divisions_map_partitions,
_get_meta_map_partitions,
apply_and_enforce,
is_dataframe_like,
is_index_like,
is_series_like,
make_meta,
)
from dask.dataframe.dispatch import meta_nonempty
from dask.dataframe.utils import clear_known_categories, drop_by_shallow_copy
from dask.utils import M, apply, funcname, import_required, is_arraylike
from tlz import merge_sorted, unique
from dask_expr._util import _tokenize_deterministic, _tokenize_partial
replacement_rules = []
no_default = "__no_default__"
_object_cache = weakref.WeakValueDictionary()
def normalize_arg(arg):
if isinstance(arg, list):
return tuple(arg)
if isinstance(arg, dict):
return tuple(sorted(arg.items()))
if isinstance(arg, pd.core.base.PandasObject):
return (type(arg), id(arg)) # not quite safe
if isinstance(arg, np.ndarray):
return (type(arg), id(arg)) # not quite safe
return arg
class Expr:
"""Primary class for all Expressions
This mostly includes Dask protocols and various Pandas-like method
definitions to make us look more like a DataFrame.
"""
_parameters = []
_defaults = {}
_is_length_preserving = False
def __new__(cls, *args, **kwargs):
key = (
cls,
tuple(map(normalize_arg, args)),
tuple(sorted(toolz.valmap(normalize_arg, kwargs).items())),
)
try:
return _object_cache[key]
except KeyError:
obj = object.__new__(cls)
cls._init(obj, *args, **kwargs)
_object_cache[key] = obj
return obj
except Exception: # can not hash
obj = object.__new__(cls)
cls._init(obj, *args, **kwargs)
return obj
def _init(self, *args, **kwargs):
operands = list(args)
for parameter in type(self)._parameters[len(operands) :]:
try:
operands.append(kwargs.pop(parameter))
except KeyError:
operands.append(type(self)._defaults[parameter])
assert not kwargs
self.operands = operands
if self._required_attribute:
dep = next(iter(self.dependencies()))._meta
if not hasattr(dep, self._required_attribute):
# Raise a ValueError instead of AttributeError to
# avoid infinite recursion
raise ValueError(f"{dep} has no attribute {self._required_attribute}")
@property
def _required_attribute(self) -> str:
# Specify if the first `dependency` must support
# a specific attribute for valid behavior.
return None
@functools.cached_property
def ndim(self):
meta = self._meta
try:
return meta.ndim
except AttributeError:
return 0
def __str__(self):
s = ", ".join(
str(param) + "=" + str(operand)
for param, operand in zip(self._parameters, self.operands)
if operand != self._defaults.get(param)
)
return f"{type(self).__name__}({s})"
def __repr__(self):
return str(self)
def _tree_repr_lines(self, indent=0, recursive=True):
header = funcname(type(self)) + ":"
lines = []
for i, op in enumerate(self.operands):
if isinstance(op, Expr):
if recursive:
lines.extend(op._tree_repr_lines(2))
else:
try:
param = self._parameters[i]
default = self._defaults[param]
except (IndexError, KeyError):
param = self._parameters[i] if i < len(self._parameters) else ""
default = "--no-default--"
if isinstance(op, pd.core.base.PandasObject):
op = "<pandas>"
elif is_dataframe_like(op):
op = "<dataframe>"
elif is_index_like(op):
op = "<index>"
elif is_series_like(op):
op = "<series>"
elif is_arraylike(op):
op = "<array>"
if repr(op) != repr(default):
if param:
header += f" {param}={repr(op)}"
else:
header += repr(op)
lines = [header] + lines
lines = [" " * indent + line for line in lines]
return lines
def tree_repr(self):
return os.linesep.join(self._tree_repr_lines())
def pprint(self):
for line in self._tree_repr_lines():
print(line)
def __hash__(self):
return hash(self._name)
def __reduce__(self):
return type(self), tuple(self.operands)
def _depth(self):
"""Depth of the expression tree
Returns
-------
depth: int
"""
if not self.dependencies():
return 1
else:
return max(expr._depth() for expr in self.dependencies()) + 1
def __getattr__(self, key):
try:
return object.__getattribute__(self, key)
except AttributeError as err:
if key == "_meta":
# Avoid a recursive loop if/when `self._meta`
# produces an `AttributeError`
raise RuntimeError(
f"Failed to generate metadata for {self}. "
"This operation may not be supported by the current backend."
)
# Allow operands to be accessed as attributes
# as long as the keys are not already reserved
# by existing methods/properties
_parameters = type(self)._parameters
if key in _parameters:
idx = _parameters.index(key)
return self.operands[idx]
if is_dataframe_like(self._meta) and key in self._meta.columns:
return self[key]
link = "https://github.com/dask-contrib/dask-expr/blob/main/README.md#api-coverage"
raise AttributeError(
f"{err}\n\n"
"This often means that you are attempting to use an unsupported "
f"API function. Current API coverage is documented here: {link}."
)
def operand(self, key):
# Access an operand unambiguously
# (e.g. if the key is reserved by a method/property)
return self.operands[type(self)._parameters.index(key)]
def dependencies(self):
# Dependencies are `Expr` operands only
return [operand for operand in self.operands if isinstance(operand, Expr)]
def _task(self, index: int):
"""The task for the i'th partition
Parameters
----------
index:
The index of the partition of this dataframe
Examples
--------
>>> class Add(Expr):
... def _task(self, i):
... return (operator.add, (self.left._name, i), (self.right._name, i))
Returns
-------
task:
The Dask task to compute this partition
See Also
--------
Expr._layer
"""
raise NotImplementedError(
"Expressions should define either _layer (full dictionary) or _task"
" (single task). This expression type defines neither"
)
def _layer(self) -> dict:
"""The graph layer added by this expression
Examples
--------
>>> class Add(Expr):
... def _layer(self):
... return {
... (self._name, i): (operator.add, (self.left._name, i), (self.right._name, i))
... for i in range(self.npartitions)
... }
Returns
-------
layer: dict
The Dask task graph added by this expression
See Also
--------
Expr._task
Expr.__dask_graph__
"""
return {(self._name, i): self._task(i) for i in range(self.npartitions)}
def simplify(self):
"""Simplify expression
This leverages the ``._simplify_down`` method defined on each class
Returns
-------
expr:
output expression
changed:
whether or not any change occured
"""
expr = self
while True:
_continue = False
# Simplify this node
out = expr._simplify_down()
if out is None:
out = expr
if not isinstance(out, Expr):
return out
if out._name != expr._name:
expr = out
continue
# Allow children to simplify their parents
for child in expr.dependencies():
out = child._simplify_up(expr)
if out is None:
out = expr
if not isinstance(out, Expr):
return out
if out is not expr and out._name != expr._name:
expr = out
_continue = True
break
if _continue:
continue
# Simplify all of the children
new_operands = []
changed = False
for operand in expr.operands:
if isinstance(operand, Expr):
new = operand.simplify()
if new._name != operand._name:
changed = True
else:
new = operand
new_operands.append(new)
if changed:
expr = type(expr)(*new_operands)
continue
else:
break
return expr
def _simplify_down(self):
return
def _simplify_up(self, parent):
return
def lower_once(self):
expr = self
# Lower this node
out = expr._lower()
if out is None:
out = expr
if not isinstance(out, Expr):
return out
# Lower all children
new_operands = []
changed = False
for operand in out.operands:
if isinstance(operand, Expr):
new = operand.lower_once()
if new._name != operand._name:
changed = True
else:
new = operand
new_operands.append(new)
if changed:
out = type(out)(*new_operands)
return out
def _lower(self):
return
def combine_similar(
self, root: Expr | None = None, _cache: dict | None = None
) -> Expr:
"""Combine similar expression nodes using global information
This leverages the ``._combine_similar`` method defined
on each class. The global expression-tree traversal will
change IO leaves first, and finish with the root expression.
The primary purpose of this method is to allow column
projections to be "pushed back up" the expression graph
in the case that simlar IO & Blockwise operations can
be captured by the same operations.
Parameters
----------
root:
The root node of the global expression graph. If not
specified, the root is assumed to be ``self``.
_cache:
Optional dictionary to use for caching.
Returns
-------
expr:
output expression
"""
expr = self
update_root = root is None
root = root or self
if _cache is None:
_cache = {}
elif (self._name, root._name) in _cache:
return _cache[(self._name, root._name)]
while True:
changed = False
# Call combine_similar on each dependency
new_operands = []
changed_dependency = False
for operand in expr.operands:
if isinstance(operand, Expr):
new = operand.combine_similar(root=root, _cache=_cache)
if new._name != operand._name:
changed_dependency = True
else:
new = operand
new_operands.append(new)
if changed_dependency:
expr = type(expr)(*new_operands)
changed = True
if update_root:
root = expr
continue
# Execute "_combine_similar" on expr
out = expr._combine_similar(root)
if out is None:
out = expr
if not isinstance(out, Expr):
_cache[(self._name, root._name)] = out
return out
if out._name != expr._name:
changed = True
expr = out
if update_root:
root = expr
if not changed:
break
_cache[(self._name, root._name)] = expr
return expr
def _combine_similar(self, root: Expr):
return
def optimize(self, **kwargs):
return optimize(self, **kwargs)
@property
def index(self):
return Index(self)
@property
def size(self):
return Size(self)
@property
def nbytes(self):
return NBytes(self)
def __getitem__(self, other):
if isinstance(other, Expr):
return Filter(self, other) # df[df.x > 1]
else:
return Projection(self, other) # df[["a", "b", "c"]]
def __add__(self, other):
return Add(self, other)
def __radd__(self, other):
return Add(other, self)
def __sub__(self, other):
return Sub(self, other)
def __rsub__(self, other):
return Sub(other, self)
def __mul__(self, other):
return Mul(self, other)
def __rmul__(self, other):
return Mul(other, self)
def __truediv__(self, other):
return Div(self, other)
def __rtruediv__(self, other):
return Div(other, self)
def __lt__(self, other):
return LT(self, other)
def __rlt__(self, other):
return LT(other, self)
def __gt__(self, other):
return GT(self, other)
def __rgt__(self, other):
return GT(other, self)
def __le__(self, other):
return LE(self, other)
def __rle__(self, other):
return LE(other, self)
def __ge__(self, other):
return GE(self, other)
def __rge__(self, other):
return GE(other, self)
def __eq__(self, other):
return EQ(self, other)
def __ne__(self, other):
return NE(self, other)
def __and__(self, other):
return And(self, other)
def __rand__(self, other):
return And(other, self)
def __or__(self, other):
return Or(self, other)
def __ror__(self, other):
return Or(other, self)
def __xor__(self, other):
return XOr(self, other)
def __rxor__(self, other):
return XOr(other, self)
def __invert__(self):
return Invert(self)
def __neg__(self):
return Neg(self)
def __pos__(self):
return Pos(self)
def sum(self, skipna=True, numeric_only=False, min_count=0):
return Sum(self, skipna, numeric_only, min_count)
def prod(self, skipna=True, numeric_only=False, min_count=0):
return Prod(self, skipna, numeric_only, min_count)
def var(self, axis=0, skipna=True, ddof=1, numeric_only=False):
if axis == 0:
return Var(self, skipna, ddof, numeric_only)
elif axis == 1:
return VarColumns(self, skipna, ddof, numeric_only)
else:
raise ValueError(f"axis={axis} not supported. Please specify 0 or 1")
def std(self, axis=0, skipna=True, ddof=1, numeric_only=False):
return Sqrt(self.var(axis, skipna, ddof, numeric_only))
def mean(self, skipna=True, numeric_only=False, min_count=0):
return Mean(self, skipna=skipna, numeric_only=numeric_only)
def max(self, skipna=True, numeric_only=False, min_count=0):
return Max(self, skipna, numeric_only, min_count)
def any(self, skipna=True):
return Any(self, skipna=skipna)
def all(self, skipna=True):
return All(self, skipna=skipna)
def idxmin(self, skipna=True, numeric_only=False):
return IdxMin(self, skipna=skipna, numeric_only=numeric_only)
def idxmax(self, skipna=True, numeric_only=False):
return IdxMax(self, skipna=skipna, numeric_only=numeric_only)
def mode(self, dropna=True):
return Mode(self, dropna=dropna)
def min(self, skipna=True, numeric_only=False, min_count=0):
return Min(self, skipna, numeric_only, min_count)
def count(self, numeric_only=False):
return Count(self, numeric_only)
def abs(self):
return Abs(self)
def astype(self, dtypes):
return AsType(self, dtypes)
def clip(self, lower=None, upper=None):
return Clip(self, lower=lower, upper=upper)
def combine_first(self, other):
return CombineFirst(self, other=other)
def to_timestamp(self, freq=None, how="start"):
return ToTimestamp(self, freq=freq, how=how)
def isna(self):
return IsNa(self)
def isnull(self):
# These are the same anyway
return IsNa(self)
def round(self, decimals=0):
return Round(self, decimals=decimals)
def apply(self, function, *args, **kwargs):
return Apply(self, function, args, kwargs)
def replace(self, to_replace=None, value=no_default, regex=False):
return Replace(self, to_replace=to_replace, value=value, regex=regex)
def fillna(self, value=None):
return Fillna(self, value=value)
def rename_axis(
self, mapper=no_default, index=no_default, columns=no_default, axis=0
):
return RenameAxis(self, mapper=mapper, index=index, columns=columns, axis=axis)
def align(self, other, join="outer", fill_value=None):
from dask_expr._collection import new_collection
from dask_expr._repartition import Repartition
if are_co_aligned(self, other):
left = self
else:
dfs = [self, other]
if not all(df.known_divisions for df in dfs):
raise ValueError(
"Not all divisions are known, can't align "
"partitions. Please use `set_index` "
"to set the index."
)
divisions = list(unique(merge_sorted(*[df.divisions for df in dfs])))
if len(divisions) == 1: # single value for index
divisions = (divisions[0], divisions[0])
left = Repartition(self, new_divisions=divisions, force=True)
other = Repartition(other, new_divisions=divisions, force=True)
aligned = _Align(left, other, join=join, fill_value=fill_value)
return new_collection(AlignGetitem(aligned, position=0)), new_collection(
AlignGetitem(aligned, position=1)
)
def nunique_approx(self):
return NuniqueApprox(self, b=16)
@functools.cached_property
def divisions(self):
return tuple(self._divisions())
def _divisions(self):
raise NotImplementedError()
@property
def known_divisions(self):
"""Whether divisions are already known"""
return len(self.divisions) > 0 and self.divisions[0] is not None
@property
def npartitions(self):
if "npartitions" in self._parameters:
idx = self._parameters.index("npartitions")
return self.operands[idx]
else:
return len(self.divisions) - 1
@functools.cached_property
def _name(self):
return (
funcname(type(self)).lower() + "-" + _tokenize_deterministic(*self.operands)
)
@property
def columns(self) -> list:
try:
return list(self._meta.columns)
except AttributeError:
return []
@property
def dtypes(self):
return self._meta.dtypes
@property
def _meta(self):
raise NotImplementedError()
def __dask_graph__(self):
"""Traverse expression tree, collect layers"""
stack = [self]
seen = set()
layers = []
while stack:
expr = stack.pop()
if expr._name in seen:
continue
seen.add(expr._name)
layers.append(expr._layer())
for operand in expr.operands:
if isinstance(operand, Expr):
stack.append(operand)
return toolz.merge(layers)
def __dask_keys__(self):
return [(self._name, i) for i in range(self.npartitions)]
def substitute(self, substitutions: dict) -> Expr:
"""Substitute specific `Expr` instances within `self`
Parameters
----------
substitutions:
mapping old terms to new terms. Note that using
non-`Expr` keys may produce unexpected results,
and substituting boolean values is not allowed.
Examples
--------
>>> (df + 10).substitute({10: 20})
df + 20
"""
if not substitutions:
return self
if self in substitutions:
return substitutions[self]
new = []
update = False
for operand in self.operands:
if (
not isinstance(operand, bool)
and ishashable(operand)
and operand in substitutions
):
new.append(substitutions[operand])
update = True
elif isinstance(operand, Expr):
val = operand.substitute(substitutions)
if operand._name != val._name:
update = True
new.append(val)
else:
new.append(operand)
if update: # Only recreate if something changed
return type(self)(*new)
return self
def substitute_parameters(self, substitutions: dict) -> Expr:
"""Substitute specific `Expr` parameters
Parameters
----------
substitutions:
Mapping of parameter keys to new values. Keys that
are not found in ``self._parameters`` will be ignored.
"""
if not substitutions:
return self
changed = False
new_operands = []
for i, operand in enumerate(self.operands):
if i < len(self._parameters) and self._parameters[i] in substitutions:
new_operands.append(substitutions[self._parameters[i]])
changed = True
else:
new_operands.append(operand)
if changed:
return type(self)(*new_operands)
return self
def _find_similar_operations(self, root: Expr, ignore: list | None = None):
# Find operations with the same type and operands.
# Parameter keys specified by `ignore` will not be
# included in the operand comparison
alike = [
op for op in root.find_operations(type(self)) if op._name != self._name
]
if not alike:
# No other operations of the same type. Early return
return []
# Return subset of `alike` with the same "token"
token = _tokenize_partial(self, ignore)
return [item for item in alike if _tokenize_partial(item, ignore) == token]
def _node_label_args(self):
"""Operands to include in the node label by `visualize`"""
return self.dependencies()
def _to_graphviz(
self,
rankdir="BT",
graph_attr=None,
node_attr=None,
edge_attr=None,
**kwargs,
):
from dask.dot import label, name
graphviz = import_required(
"graphviz",
"Drawing dask graphs with the graphviz visualization engine requires the `graphviz` "
"python library and the `graphviz` system library.\n\n"
"Please either conda or pip install as follows:\n\n"
" conda install python-graphviz # either conda install\n"
" python -m pip install graphviz # or pip install and follow installation instructions",
)
graph_attr = graph_attr or {}
node_attr = node_attr or {}
edge_attr = edge_attr or {}
graph_attr["rankdir"] = rankdir
node_attr["shape"] = "box"
node_attr["fontname"] = "helvetica"
graph_attr.update(kwargs)
g = graphviz.Digraph(
graph_attr=graph_attr,
node_attr=node_attr,
edge_attr=edge_attr,
)
stack = [self]
seen = set()
dependencies = {}
while stack:
expr = stack.pop()
if expr._name in seen:
continue
seen.add(expr._name)
dependencies[expr] = set(expr.dependencies())
for dep in expr.dependencies():
stack.append(dep)
cache = {}
for expr in dependencies:
expr_name = name(expr)
attrs = {}
# Make node label
deps = [
funcname(type(dep)) if isinstance(dep, Expr) else str(dep)
for dep in expr._node_label_args()
]
_label = funcname(type(expr))
if deps:
_label = f"{_label}({', '.join(deps)})" if deps else _label
node_label = label(_label, cache=cache)
attrs.setdefault("label", str(node_label))
attrs.setdefault("fontsize", "20")
g.node(expr_name, **attrs)
for expr, deps in dependencies.items():
expr_name = name(expr)
for dep in deps:
dep_name = name(dep)
g.edge(dep_name, expr_name)
return g
def visualize(self, filename="dask-expr.svg", format=None, **kwargs):
"""
Visualize the expression graph.
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 the graph is
rendered in the Jupyter notebook only.
format : {'png', 'pdf', 'dot', 'svg', 'jpeg', 'jpg'}, optional
Format in which to write output file. Default is 'svg'.
**kwargs
Additional keyword arguments to forward to ``to_graphviz``.
"""
from dask.dot import graphviz_to_file
g = self._to_graphviz(**kwargs)
graphviz_to_file(g, filename, format)
return g
def walk(self) -> Generator[Expr]:
"""Iterate through all expressions in the tree
Returns
-------
nodes
Generator of Expr instances in the graph.
Ordering is a depth-first search of the expression tree
"""
stack = [self]
seen = set()
while stack:
node = stack.pop()
if node._name in seen:
continue
seen.add(node._name)
for dep in node.dependencies():
stack.append(dep)
yield node
def find_operations(self, operation: type | tuple[type]) -> Generator[Expr]:
"""Search the expression graph for a specific operation type
Parameters
----------
operation
The operation type to search for.
Returns
-------
nodes
Generator of `operation` instances. Ordering corresponds
to a depth-first search of the expression graph.
"""
assert (
isinstance(operation, tuple)
and all(issubclass(e, Expr) for e in operation)
or issubclass(operation, Expr)
), "`operation` must be`Expr` subclass)"
return (expr for expr in self.walk() if isinstance(expr, operation))
class Literal(Expr):
"""Represent a literal (known) value as an `Expr`"""
_parameters = ["value"]
def _divisions(self):
return (None, None)
@property
def _meta(self):
return make_meta(self.value)
def _task(self, index: int):
assert index == 0
return self.value
class Blockwise(Expr):
"""Super-class for block-wise operations
This is fairly generic, and includes definitions for `_meta`, `divisions`,
`_layer` that are often (but not always) correct. Mostly this helps us
avoid duplication in the future.
Note that `Fused` expressions rely on every `Blockwise`
expression defining a proper `_task` method.
"""
operation = None
_keyword_only = []
_projection_passthrough = False
_filter_passthrough = False
@property
def _required_attribute(self):
if isinstance(self.operation, type(M.method_caller)):
return self.operation.method
return None
@functools.cached_property
def _meta(self):