-
-
Notifications
You must be signed in to change notification settings - Fork 1.7k
/
test_typing.py
214 lines (147 loc) · 5.48 KB
/
test_typing.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
from __future__ import annotations
from collections.abc import Hashable, Mapping, Sequence
from typing import Any
import pytest
import dask
import dask.threaded
from dask.base import DaskMethodsMixin, dont_optimize, tokenize
from dask.context import globalmethod
from dask.delayed import Delayed, delayed
from dask.typing import (
DaskCollection,
HLGDaskCollection,
PostComputeCallable,
PostPersistCallable,
)
try:
from IPython.display import DisplayObject
except ImportError:
DisplayObject = Any
da = pytest.importorskip("dask.array")
db = pytest.importorskip("dask.bag")
dds = pytest.importorskip("dask.datasets")
dd = pytest.importorskip("dask.dataframe")
def finalize(x: Sequence[Any]) -> Any:
return x[0]
def get1(dsk: Mapping, keys: Sequence[Hashable] | Hashable, **kwargs: Any) -> Any:
return dask.threaded.get(dsk, keys, **kwargs)
def get2(dsk: Mapping, keys: Sequence[Hashable] | Hashable, **kwargs: Any) -> Any:
return dask.get(dsk, keys, **kwargs)
class Inheriting(DaskCollection):
def __init__(self, based_on: DaskCollection) -> None:
self.based_on = based_on
def __dask_graph__(self) -> Mapping:
return self.based_on.__dask_graph__()
def __dask_keys__(self) -> list[Hashable]:
return self.based_on.__dask_keys__()
def __dask_postcompute__(self) -> tuple[PostComputeCallable, tuple]:
return finalize, ()
def __dask_postpersist__(self) -> tuple[PostPersistCallable, tuple]:
return self.based_on.__dask_postpersist__()
def __dask_tokenize__(self) -> Hashable:
return tokenize(self.based_on)
__dask_scheduler__ = staticmethod(dask.threaded.get)
__dask_optimize__ = globalmethod(
dont_optimize,
key="hlgcollection_optim",
falsey=dont_optimize,
)
def compute(self, **kwargs) -> Any:
return dask.compute(self, **kwargs)
def persist(self, **kwargs) -> Inheriting:
return Inheriting(self.based_on.persist(**kwargs))
def visualize(
self,
filename: str = "mydask",
format: str | None = None,
optimize_graph: bool = False,
**kwargs: Any,
) -> DisplayObject | None:
return dask.visualize(
self,
filename=filename,
format=format,
optimize_graph=optimize_graph,
**kwargs,
)
class HLGCollection(DaskMethodsMixin):
def __init__(self, based_on: HLGDaskCollection) -> None:
self.based_on = based_on
def __dask_graph__(self) -> Mapping:
return self.based_on.__dask_graph__()
def __dask_layers__(self) -> Sequence[str]:
return self.based_on.__dask_layers__()
def __dask_keys__(self) -> list[Hashable]:
return self.based_on.__dask_keys__()
def __dask_postcompute__(self) -> tuple[PostComputeCallable, tuple]:
return finalize, ()
def __dask_postpersist__(self) -> tuple[PostPersistCallable, tuple]:
return self.based_on.__dask_postpersist__()
def __dask_tokenize__(self) -> Hashable:
return tokenize(self.based_on)
__dask_scheduler__ = staticmethod(get1)
__dask_optimize__ = globalmethod(
dont_optimize,
key="hlgcollection_optim",
falsey=dont_optimize,
)
class NotHLGCollection(DaskMethodsMixin):
def __init__(self, based_on: DaskCollection) -> None:
self.based_on = based_on
def __dask_graph__(self) -> Mapping:
return self.based_on.__dask_graph__()
def __dask_keys__(self) -> list[Hashable]:
return self.based_on.__dask_keys__()
def __dask_postcompute__(self) -> tuple[PostComputeCallable, tuple]:
return finalize, ()
def __dask_postpersist__(self) -> tuple[PostPersistCallable, tuple]:
return self.based_on.__dask_postpersist__()
def __dask_tokenize__(self) -> Hashable:
return tokenize(self.based_on)
__dask_scheduler__ = staticmethod(get2)
__dask_optimize__ = globalmethod(
dont_optimize,
key="collection_optim",
falsey=dont_optimize,
)
def increment_(x: int) -> int:
return x + 1
increment: Delayed = delayed(increment_)
def assert_isinstance(coll: DaskCollection, protocol: Any) -> None:
assert isinstance(coll, protocol)
@pytest.mark.parametrize("protocol", [DaskCollection, HLGDaskCollection])
def test_isinstance_core(protocol):
arr = da.ones(10)
bag = db.from_sequence([1, 2, 3, 4, 5], npartitions=2)
df = dds.timeseries()
dobj = increment(2)
assert_isinstance(arr, protocol)
assert_isinstance(bag, protocol)
assert_isinstance(df, protocol)
assert_isinstance(dobj, protocol)
def test_isinstance_custom() -> None:
a = da.ones(10)
hlgc = HLGCollection(a)
nhlgc = NotHLGCollection(a)
assert isinstance(hlgc, DaskCollection)
assert isinstance(nhlgc, DaskCollection)
assert isinstance(nhlgc, DaskCollection)
assert not isinstance(nhlgc, HLGDaskCollection)
def compute(coll: DaskCollection) -> Any:
return coll.compute()
def compute2(coll: DaskCollection) -> Any:
return coll.compute()
def test_parameter_passing() -> None:
from dask.array import Array
a: Delayed = increment(2)
hlgc = HLGCollection(a)
assert compute(hlgc) == 3
assert compute2(hlgc) == 3
d: Delayed = increment(3)
assert compute(d) == 4
assert compute2(d) == 4
array: Array = da.ones(10)
assert compute(array).shape == (10,)
assert compute2(array).shape == (10,)
def test_inheriting_class() -> Inheriting:
return Inheriting(increment(2))