-
-
Notifications
You must be signed in to change notification settings - Fork 153
/
Copy pathtest_operator.py
195 lines (157 loc) · 7.3 KB
/
test_operator.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
import pytest
import asyncio
from contextlib import asynccontextmanager
import pathlib
import os.path
from dask.distributed import Client
DIR = pathlib.Path(__file__).parent.absolute()
@pytest.fixture()
def gen_cluster(k8s_cluster):
"""Yields an instantiated context manager for creating/deleting a simple cluster."""
@asynccontextmanager
async def cm():
cluster_path = os.path.join(DIR, "resources", "simplecluster.yaml")
cluster_name = "simple-cluster"
# Create cluster resource
k8s_cluster.kubectl("apply", "-f", cluster_path)
while cluster_name not in k8s_cluster.kubectl("get", "daskclusters"):
await asyncio.sleep(0.1)
try:
yield cluster_name
finally:
# Test: remove the wait=True, because I think this is blocking the operator
k8s_cluster.kubectl("delete", "-f", cluster_path)
while cluster_name in k8s_cluster.kubectl("get", "daskclusters"):
await asyncio.sleep(0.1)
yield cm
@pytest.fixture()
def gen_job(k8s_cluster):
"""Yields an instantiated context manager for creating/deleting a simple job."""
@asynccontextmanager
async def cm():
job_path = os.path.join(DIR, "resources", "simplejob.yaml")
job_name = "simple-job"
# Create cluster resource
k8s_cluster.kubectl("apply", "-f", job_path)
while job_name not in k8s_cluster.kubectl("get", "daskjobs"):
await asyncio.sleep(0.1)
try:
yield job_name
finally:
# Test: remove the wait=True, because I think this is blocking the operator
k8s_cluster.kubectl("delete", "-f", job_path)
while job_name in k8s_cluster.kubectl("get", "daskjobs"):
await asyncio.sleep(0.1)
yield cm
def test_customresources(k8s_cluster):
assert "daskclusters.kubernetes.dask.org" in k8s_cluster.kubectl("get", "crd")
assert "daskworkergroups.kubernetes.dask.org" in k8s_cluster.kubectl("get", "crd")
assert "daskjobs.kubernetes.dask.org" in k8s_cluster.kubectl("get", "crd")
def test_operator_runs(kopf_runner):
with kopf_runner as runner:
pass
assert runner.exit_code == 0
assert runner.exception is None
@pytest.mark.asyncio
async def test_scalesimplecluster(k8s_cluster, kopf_runner, gen_cluster):
with kopf_runner as runner:
async with gen_cluster() as cluster_name:
scheduler_pod_name = "simple-cluster-scheduler"
worker_pod_name = "simple-cluster-default-worker-group-worker"
service_name = "simple-cluster-service"
while scheduler_pod_name not in k8s_cluster.kubectl("get", "pods"):
await asyncio.sleep(0.1)
while service_name not in k8s_cluster.kubectl("get", "svc"):
await asyncio.sleep(0.1)
while worker_pod_name not in k8s_cluster.kubectl("get", "pods"):
await asyncio.sleep(0.1)
while "Running" not in k8s_cluster.kubectl(
"get", "pods", scheduler_pod_name
):
await asyncio.sleep(0.1)
with k8s_cluster.port_forward(f"service/{service_name}", 8786) as port:
async with Client(
f"tcp://localhost:{port}", asynchronous=True
) as client:
k8s_cluster.kubectl(
"scale",
"--replicas=5",
"daskworkergroup",
"simple-cluster-default-worker-group",
)
await client.wait_for_workers(5)
k8s_cluster.kubectl(
"scale",
"--replicas=3",
"daskworkergroup",
"simple-cluster-default-worker-group",
)
await client.wait_for_workers(3)
@pytest.mark.timeout(300)
@pytest.mark.asyncio
async def test_simplecluster(k8s_cluster, kopf_runner, gen_cluster):
with kopf_runner as runner:
async with gen_cluster() as cluster_name:
scheduler_pod_name = "simple-cluster-scheduler"
worker_pod_name = "simple-cluster-default-worker-group-worker"
service_name = "simple-cluster-service"
while scheduler_pod_name not in k8s_cluster.kubectl("get", "pods"):
await asyncio.sleep(0.1)
while service_name not in k8s_cluster.kubectl("get", "svc"):
await asyncio.sleep(0.1)
while worker_pod_name not in k8s_cluster.kubectl("get", "pods"):
await asyncio.sleep(0.1)
with k8s_cluster.port_forward(f"service/{service_name}", 8786) as port:
async with Client(
f"tcp://localhost:{port}", asynchronous=True
) as client:
await client.wait_for_workers(2)
# Ensure that inter-worker communication works well
futures = client.map(lambda x: x + 1, range(10))
total = client.submit(sum, futures)
assert (await total) == sum(map(lambda x: x + 1, range(10)))
# Get the the first env value (the only one) of the scheduler
scheduler_env = k8s_cluster.kubectl(
"get",
"pods",
"--selector=dask.org/component=scheduler",
"-o",
"jsonpath='{.items[0].spec.containers[0].env[0]}'",
)
# Just check if its in the string, no need to parse the json
assert "SCHEDULER_ENV" in scheduler_env
# Get the the first env value (the only one) of the first worker
worker_env = k8s_cluster.kubectl(
"get",
"pods",
"--selector=dask.org/component=worker",
"-o",
"jsonpath='{.items[0].spec.containers[0].env[0]}'",
)
# Just check if its in the string, no need to parse the json
assert "WORKER_ENV" in worker_env
assert cluster_name
assert "A DaskCluster has been created" in runner.stdout
assert "A scheduler pod has been created" in runner.stdout
assert "A worker group has been created" in runner.stdout
@pytest.mark.asyncio
async def test_job(k8s_cluster, kopf_runner, gen_job):
with kopf_runner as runner:
async with gen_job() as job:
assert job
cluster_name = f"{job}-cluster"
runner_name = f"{job}-runner"
# Assert that cluster is created
while cluster_name not in k8s_cluster.kubectl("get", "daskclusters"):
await asyncio.sleep(0.1)
# Assert job pod is created
while runner_name not in k8s_cluster.kubectl("get", "po"):
await asyncio.sleep(0.1)
# Assert job pod runs to completion (will fail if doesn't connect to cluster)
while "Completed" not in k8s_cluster.kubectl("get", "po", runner_name):
await asyncio.sleep(0.1)
# Assert cluster is removed on completion
while cluster_name in k8s_cluster.kubectl("get", "daskclusters"):
await asyncio.sleep(0.1)
assert "A DaskJob has been created" in runner.stdout
assert "Job succeeded, deleting Dask cluster." in runner.stdout