-
-
Notifications
You must be signed in to change notification settings - Fork 27
/
Copy path_core.py
778 lines (650 loc) · 23.3 KB
/
_core.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
from __future__ import annotations
import functools
import os
import weakref
from collections import defaultdict
from collections.abc import Generator
import dask
import pandas as pd
import toolz
from dask.dataframe.core import is_dataframe_like, is_index_like, is_series_like
from dask.delayed import Delayed
from dask.utils import funcname, import_required, is_arraylike
from toolz.dicttoolz import merge
from dask_expr._util import _BackendData, _tokenize_deterministic
def _unpack_collections(o):
if isinstance(o, Expr):
return o, o._name
elif hasattr(o, "expr") and not isinstance(o, Delayed):
return o.expr, o.expr._name
else:
return o, None
class Expr:
_parameters = []
_defaults = {}
_instances = weakref.WeakValueDictionary()
_dependents = defaultdict(list)
_seen = set()
def __new__(cls, *args, **kwargs):
operands = list(args)
for parameter in cls._parameters[len(operands) :]:
try:
operands.append(kwargs.pop(parameter))
except KeyError:
operands.append(cls._defaults[parameter])
assert not kwargs, kwargs
parsed_operands = []
children = set()
_subgraphs = []
_subgraph_instances = []
_graph_instances = {}
for o in operands:
expr, name = _unpack_collections(o)
parsed_operands.append(expr)
if name is not None:
children.add(name)
_subgraphs.append(expr._graph)
_subgraph_instances.append(expr._graph_instances)
_graph_instances[name] = expr
inst = object.__new__(cls)
inst.operands = parsed_operands
_name = inst._name
# Graph instances is a mapping name -> Expr instance
# Graph itself is a mapping of dependencies mapping names to a set of names
if _name in Expr._instances:
inst = Expr._instances[_name]
inst._graph_instances.update(merge(_graph_instances, *_subgraph_instances))
inst._graph.update(merge(*_subgraphs))
inst._graph[_name].update(children)
# Probably a bad idea to have a self ref
inst._graph_instances[_name] = inst
else:
Expr._instances[_name] = inst
inst._graph_instances = merge(_graph_instances, *_subgraph_instances)
inst._graph = merge(*_subgraphs)
inst._graph[_name] = children
# Probably a bad idea to have a self ref
inst._graph_instances[_name] = inst
if inst._name in Expr._seen:
# We already registered inst as a dependent of all it's
# dependencies, so we don't need to do it again
return inst
Expr._seen.add(inst._name)
for dep in inst.dependencies():
Expr._dependents[dep._name].append(inst)
return inst
@functools.cached_property
def _dependent_graph(self):
# Reset to clear tracking
Expr._dependents = defaultdict(list)
Expr._seen = set()
rv = Expr._dependents
# This should be O(E)
tmp = defaultdict(set)
for expr, dependencies in self._graph.items():
for dep in dependencies:
tmp[dep].add(expr)
for name, exprs in tmp.items():
rv[name] = [self._graph_instances[e] for e in exprs]
return rv
def __hash__(self):
raise TypeError(
"Expr objects can't be used in sets or dicts or similar, use the _name instead"
)
def _tune_down(self):
return None
def _tune_up(self, parent):
return None
def _cull_down(self):
return None
def _cull_up(self, parent):
return None
def __str__(self):
s = ", ".join(
str(param) + "=" + str(operand)
for param, operand in zip(self._parameters, self.operands)
if isinstance(operand, Expr) or operand != self._defaults.get(param)
)
return f"{type(self).__name__}({s})"
def __repr__(self):
return str(self)
def _tree_repr_argument_construction(self, i, op, header):
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 repr(op) != repr(default):
if param:
header += f" {param}={repr(op)}"
else:
header += repr(op)
return header
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:
if isinstance(op, _BackendData):
op = op._data
# TODO: this stuff is pandas-specific
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>"
header = self._tree_repr_argument_construction(i, op, header)
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):
if dask.config.get("dask-expr-no-serialize", False):
raise RuntimeError(f"Serializing a {type(self)} object")
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 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 dependents(self):
return self._dependent_graph
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 rewrite(self, kind: str):
"""Rewrite an expression
This leverages the ``._{kind}_down`` and ``._{kind}_up``
methods defined on each class
Returns
-------
expr:
output expression
changed:
whether or not any change occured
"""
expr = self
down_name = f"_{kind}_down"
up_name = f"_{kind}_up"
while True:
_continue = False
# Rewrite this node
out = getattr(expr, down_name)()
if out is None:
out = expr
if not isinstance(out, Expr):
return out
if out._name != expr._name:
expr = out
continue
# Allow children to rewrite their parents
for child in expr.dependencies():
out = getattr(child, up_name)(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
# Rewrite all of the children
new_operands = []
changed = False
for operand in expr.operands:
if isinstance(operand, Expr):
new = operand.rewrite(kind=kind)
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_once(self, dependents: defaultdict, simplified: dict):
"""Simplify an expression
This leverages the ``._simplify_down`` and ``._simplify_up``
methods defined on each class
Parameters
----------
dependents: defaultdict[list]
The dependents for every node.
simplified: dict
Cache of simplified expressions for these dependents.
Returns
-------
expr:
output expression
"""
# Check if we've already simplified for these dependents
if self._name in simplified:
return simplified[self._name]
expr = self
while True:
out = expr._simplify_down()
if out is None:
out = expr
if not isinstance(out, Expr):
return out
if out._name != expr._name:
expr = out
# Allow children to simplify their parents
for child in expr.dependencies():
out = child._simplify_up(expr, dependents)
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
break
# Rewrite all of the children
new_operands = []
changed = False
for operand in expr.operands:
if isinstance(operand, Expr):
new = operand.simplify_once(
dependents=dependents, simplified=simplified
)
simplified[operand._name] = new
if new._name != operand._name:
changed = True
else:
new = operand
new_operands.append(new)
if changed:
expr = type(expr)(*new_operands)
break
return expr
def simplify(self) -> Expr:
expr = self
while True:
dependents = expr.dependents()
new = expr.simplify_once(dependents=dependents, simplified={})
if new._name == expr._name:
break
expr = new
return expr
def _simplify_down(self):
return
def _simplify_up(self, parent, dependents):
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_completely(self) -> Expr:
"""Lower an expression completely
This calls the ``lower_once`` method in a loop
until nothing changes. This function does not
apply any other optimizations (like ``simplify``).
Returns
-------
expr:
output expression
See Also
--------
Expr.lower_once
Expr._lower
"""
# Lower until nothing changes
expr = self
while True:
new = expr.lower_once()
if new._name == expr._name:
break
expr = new
return expr
def _lower(self):
return
@functools.cached_property
def _name(self):
return (
funcname(type(self)).lower() + "-" + _tokenize_deterministic(*self.operands)
)
@property
def _meta(self):
raise NotImplementedError()
def __getattr__(self, key):
try:
return object.__getattribute__(self, key)
except AttributeError as err:
if key.startswith("_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 __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.dependencies():
stack.append(operand)
return toolz.merge(layers)
@property
def dask(self):
return self.__dask_graph__()
def substitute(self, old, new) -> Expr:
"""Substitute a specific term within the expression
Note that replacing non-`Expr` terms may produce
unexpected results, and is not recommended.
Substituting boolean values is not allowed.
Parameters
----------
old:
Old term to find and replace.
new:
New term to replace instances of `old` with.
Examples
--------
>>> (df + 10).substitute(10, 20)
df + 20
"""
return self._substitute(old, new, _seen=set())
def _substitute(self, old, new, _seen):
if self._name in _seen:
return self
# Check if we are replacing a literal
if isinstance(old, Expr):
substitute_literal = False
if self._name == old._name:
return new
else:
substitute_literal = True
if isinstance(old, bool):
raise TypeError("Arguments to `substitute` cannot be bool.")
new_exprs = []
update = False
for operand in self.operands:
if isinstance(operand, Expr):
val = operand._substitute(old, new, _seen)
if operand._name != val._name:
update = True
new_exprs.append(val)
elif (
"Fused" in type(self).__name__
and isinstance(operand, list)
and all(isinstance(op, Expr) for op in operand)
):
# Special handling for `Fused`.
# We make no promise to dive through a
# list operand in general, but NEED to
# do so for the `Fused.exprs` operand.
val = []
for op in operand:
val.append(op._substitute(old, new, _seen))
if val[-1]._name != op._name:
update = True
new_exprs.append(val)
elif (
substitute_literal
and not isinstance(operand, bool)
and isinstance(operand, type(old))
and operand == old
):
new_exprs.append(new)
update = True
else:
new_exprs.append(operand)
if update: # Only recreate if something changed
return type(self)(*new_exprs)
else:
_seen.add(self._name)
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 _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))