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
|
from __future__ import annotations
import ast
import pytest
import dask
import dask.array as da
import dask.dataframe as dd
from distributed.diagnostics import SchedulerPlugin
from distributed.utils_test import gen_cluster
np = pytest.importorskip("numpy")
pd = pytest.importorskip("pandas")
from numpy.testing import assert_array_equal
@gen_cluster(client=True)
async def test_combo_of_layer_types(c, s, a, b):
"""Check pack/unpack of a HLG that has everything!"""
def add(x, y, z, extra_arg):
return x + y + z + extra_arg
y = c.submit(lambda x: x, 2)
z = c.submit(lambda x: x, 3)
x = da.blockwise(
add,
"x",
da.zeros((3,), chunks=(1,)),
"x",
da.ones((3,), chunks=(1,)),
"x",
y,
None,
concatenate=False,
dtype=int,
extra_arg=z,
)
df = dd.from_pandas(pd.DataFrame({"a": np.arange(3)}), npartitions=3)
df = df.shuffle("a", shuffle="tasks")
df = df["a"].to_dask_array()
res = x.sum() + df.sum()
res = await c.compute(res, optimize_graph=False)
assert res == 21
@gen_cluster(client=True)
async def test_blockwise(c, s, a, b):
"""Check pack/unpack of blockwise layer"""
def add(x, y, z, extra_arg):
return x + y + z + extra_arg
y = c.submit(lambda x: x, 10)
z = c.submit(lambda x: x, 3)
x = da.blockwise(
add,
"x",
da.zeros((3,), chunks=(1,)),
"x",
da.ones((3,), chunks=(1,)),
"x",
y,
None,
concatenate=False,
dtype=int,
extra_arg=z,
)
res = await c.compute(x.sum(), optimize_graph=False)
assert res == 42
@gen_cluster(client=True)
async def test_shuffle(c, s, a, b):
"""Check pack/unpack of a shuffled dataframe"""
df = dd.from_pandas(
pd.DataFrame(
{"a": np.arange(10, dtype=int), "b": np.arange(10, 0, -1, dtype=float)}
),
npartitions=5,
)
df = df.shuffle("a", shuffle="tasks", max_branch=2)
df = df["a"] + df["b"]
res = await c.compute(df, optimize_graph=False)
assert res.dtypes == np.float64
assert (res == 10.0).all()
class ExampleAnnotationPlugin(SchedulerPlugin):
def __init__(self, priority_fn=None, qux="", resource="", retries=0):
self.priority_fn = priority_fn or (lambda k: 0)
self.qux = qux
self.resource = resource
self.retries = retries
self.priority_matches = 0
self.resource_matches = 0
self.retry_matches = 0
self.qux_matches = 0
def update_graph(self, scheduler, dsk=None, keys=None, restrictions=None, **kwargs):
annots = kwargs["annotations"]
if "priority" in annots:
self.priority_matches = sum(
int(self.priority_fn(ast.literal_eval(k)) == p)
for k, p in annots["priority"].items()
)
if "qux" in annots:
self.qux_matches = sum(int(self.qux == v) for v in annots["qux"].values())
if "custom_resource" in annots:
self.resource_matches = sum(
int(self.resource == v) for v in annots["custom_resource"].values()
)
if "retries" in annots:
self.retry_matches = sum(
int(self.retries == v) for v in annots["retries"].values()
)
@gen_cluster(client=True)
async def test_array_annotations(c, s, a, b):
def fn(k):
return k[1] * 5 + k[2]
qux = "baz"
resource = "widget"
plugin = ExampleAnnotationPlugin(priority_fn=fn, qux=qux, resource=resource)
s.add_plugin(plugin)
assert plugin in s.plugins.values()
with dask.annotate(priority=fn, qux=qux):
A = da.ones((10, 10), chunks=(2, 2))
with dask.annotate(custom_resource=resource):
B = A + 1
with dask.config.set(optimization__fuse__active=False):
result = await c.compute(B)
assert_array_equal(result, 2)
# There are annotation matches per array chunk (i.e. task)
assert plugin.qux_matches == A.npartitions
assert plugin.priority_matches == A.npartitions
assert plugin.resource_matches == B.npartitions
@gen_cluster(client=True)
async def test_dataframe_annotations(c, s, a, b):
retries = 5
plugin = ExampleAnnotationPlugin(retries=retries)
s.add_plugin(plugin)
assert plugin in s.plugins.values()
df = dd.from_pandas(
pd.DataFrame(
{"a": np.arange(10, dtype=int), "b": np.arange(10, 0, -1, dtype=float)}
),
npartitions=5,
)
df = df.shuffle("a", shuffle="tasks", max_branch=2)
acol = df["a"]
bcol = df["b"]
with dask.annotate(retries=retries):
df = acol + bcol
with dask.config.set(optimization__fuse__active=False):
rdf = await c.compute(df)
assert rdf.dtypes == np.float64
assert (rdf == 10.0).all()
# There is an annotation match per partition (i.e. task)
assert plugin.retry_matches == df.npartitions
|