/
test_adaptive.py
406 lines (310 loc) · 12.9 KB
/
test_adaptive.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
from __future__ import print_function, division, absolute_import
from time import sleep
import pytest
from toolz import frequencies, pluck
from tornado import gen
from tornado.ioloop import IOLoop
from distributed import Client, wait, Adaptive, LocalCluster
from distributed.utils_test import gen_cluster, gen_test, slowinc, inc
from distributed.utils_test import loop, nodebug # noqa: F401
from distributed.metrics import time
def test_get_scale_up_kwargs(loop):
with LocalCluster(0, scheduler_port=0, silence_logs=False,
diagnostics_port=None, loop=loop) as cluster:
alc = Adaptive(cluster.scheduler, cluster, interval=100,
scale_factor=3)
assert alc.get_scale_up_kwargs() == {'n': 1}
with Client(cluster, loop=loop) as c:
future = c.submit(lambda x: x + 1, 1)
assert future.result() == 2
assert c.ncores()
assert alc.get_scale_up_kwargs() == {'n': 3}
@gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 4)
def test_simultaneous_scale_up_and_down(c, s, *workers):
class TestAdaptive(Adaptive):
def get_scale_up_kwargs(self):
assert False
def _retire_workers(self):
assert False
class TestCluster(object):
def scale_up(self, n, **kwargs):
assert False
def scale_down(self, workers):
assert False
cluster = TestCluster()
s.task_duration['a'] = 4
s.task_duration['b'] = 4
s.task_duration['c'] = 1
future = c.map(slowinc, [1, 1, 1], key=['a-4', 'b-4', 'c-1'])
while len(s.rprocessing) < 3:
yield gen.sleep(0.001)
ta = TestAdaptive(s, cluster, interval=100, scale_factor=2)
yield gen.sleep(0.3)
def test_adaptive_local_cluster(loop):
with LocalCluster(0, scheduler_port=0, silence_logs=False,
diagnostics_port=None, loop=loop) as cluster:
alc = Adaptive(cluster.scheduler, cluster, interval=100)
with Client(cluster, loop=loop) as c:
assert not c.ncores()
future = c.submit(lambda x: x + 1, 1)
assert future.result() == 2
assert c.ncores()
sleep(0.1)
assert c.ncores() # still there after some time
del future
start = time()
while cluster.scheduler.ncores:
sleep(0.01)
assert time() < start + 5
assert not c.ncores()
@nodebug
@gen_test(timeout=30)
def test_adaptive_local_cluster_multi_workers():
cluster = yield LocalCluster(0, scheduler_port=0, silence_logs=False,
processes=False, diagnostics_port=None,
asynchronous=True)
try:
cluster.scheduler.allowed_failures = 1000
alc = cluster.adapt(interval=100)
c = yield Client(cluster, asynchronous=True)
futures = c.map(slowinc, range(100), delay=0.01)
start = time()
while not cluster.scheduler.workers:
yield gen.sleep(0.01)
assert time() < start + 15, alc.log
yield c.gather(futures)
del futures
start = time()
# while cluster.workers:
while cluster.scheduler.workers:
yield gen.sleep(0.01)
assert time() < start + 15, alc.log
# assert not cluster.workers
assert not cluster.scheduler.workers
yield gen.sleep(0.2)
# assert not cluster.workers
assert not cluster.scheduler.workers
futures = c.map(slowinc, range(100), delay=0.01)
yield c.gather(futures)
finally:
yield c.close()
yield cluster.close()
@gen_cluster(client=True, ncores=[('127.0.0.1', 1)] * 10, active_rpc_timeout=10)
def test_adaptive_scale_down_override(c, s, *workers):
class TestAdaptive(Adaptive):
def __init__(self, *args, **kwargs):
self.min_size = kwargs.pop("min_size", 0)
Adaptive.__init__(self, *args, **kwargs)
def workers_to_close(self, **kwargs):
num_workers = len(self.scheduler.workers)
to_close = self.scheduler.workers_to_close(**kwargs)
if num_workers - len(to_close) < self.min_size:
to_close = to_close[:num_workers - self.min_size]
return to_close
class TestCluster(object):
def scale_up(self, n, **kwargs):
assert False
def scale_down(self, workers):
assert False
assert len(s.workers) == 10
# Assert that adaptive cycle does not reduce cluster below minimum size
# as determined via override.
cluster = TestCluster()
ta = TestAdaptive(s, cluster, min_size=2, interval=.1, scale_factor=2)
yield gen.sleep(0.3)
assert len(s.workers) == 2
@gen_test(timeout=30)
def test_min_max():
loop = IOLoop.current()
cluster = yield LocalCluster(0, scheduler_port=0, silence_logs=False,
processes=False, diagnostics_port=None,
loop=loop, asynchronous=True)
yield cluster._start()
try:
adapt = Adaptive(cluster.scheduler, cluster, minimum=1, maximum=2,
interval='20 ms', wait_count=10)
c = yield Client(cluster, asynchronous=True, loop=loop)
start = time()
while not cluster.scheduler.workers:
yield gen.sleep(0.01)
assert time() < start + 1
yield gen.sleep(0.2)
assert len(cluster.scheduler.workers) == 1
assert frequencies(pluck(1, adapt.log)) == {'up': 1}
futures = c.map(slowinc, range(100), delay=0.1)
start = time()
while len(cluster.scheduler.workers) < 2:
yield gen.sleep(0.01)
assert time() < start + 1
assert len(cluster.scheduler.workers) == 2
yield gen.sleep(0.5)
assert len(cluster.scheduler.workers) == 2
assert len(cluster.workers) == 2
assert frequencies(pluck(1, adapt.log)) == {'up': 2}
del futures
start = time()
while len(cluster.scheduler.workers) != 1:
yield gen.sleep(0.01)
assert time() < start + 2
assert frequencies(pluck(1, adapt.log)) == {'up': 2, 'down': 1}
finally:
yield c.close()
yield cluster.close()
@gen_test()
def test_avoid_churn():
""" We want to avoid creating and deleting workers frequently
Instead we want to wait a few beats before removing a worker in case the
user is taking a brief pause between work
"""
cluster = yield LocalCluster(0, asynchronous=True, processes=False,
scheduler_port=0, silence_logs=False,
diagnostics_port=None)
client = yield Client(cluster, asynchronous=True)
try:
adapt = Adaptive(cluster.scheduler, cluster, interval='20 ms', wait_count=5)
for i in range(10):
yield client.submit(slowinc, i, delay=0.040)
yield gen.sleep(0.040)
assert frequencies(pluck(1, adapt.log)) == {'up': 1}
finally:
yield client.close()
yield cluster.close()
@gen_test(timeout=None)
def test_adapt_quickly():
""" We want to avoid creating and deleting workers frequently
Instead we want to wait a few beats before removing a worker in case the
user is taking a brief pause between work
"""
cluster = yield LocalCluster(0, asynchronous=True, processes=False,
scheduler_port=0, silence_logs=False,
diagnostics_port=None)
client = yield Client(cluster, asynchronous=True)
adapt = Adaptive(cluster.scheduler, cluster, interval=20, wait_count=5,
maximum=10)
try:
future = client.submit(slowinc, 1, delay=0.100)
yield wait(future)
assert len(adapt.log) == 1
# Scale up when there is plenty of available work
futures = client.map(slowinc, range(1000), delay=0.100)
while frequencies(pluck(1, adapt.log)) == {'up': 1}:
yield gen.sleep(0.01)
assert len(adapt.log) == 2
assert 'up' in adapt.log[-1]
d = [x for x in adapt.log[-1] if isinstance(x, dict)][0]
assert 2 < d['n'] <= adapt.maximum
while len(cluster.scheduler.workers) < adapt.maximum:
yield gen.sleep(0.01)
del futures
while len(cluster.scheduler.workers) > 1:
yield gen.sleep(0.01)
# Don't scale up for large sequential computations
x = yield client.scatter(1)
for i in range(100):
x = client.submit(slowinc, x)
yield gen.sleep(0.1)
assert len(cluster.scheduler.workers) == 1
finally:
yield client.close()
yield cluster.close()
@gen_test(timeout=None)
def test_adapt_down():
""" Ensure that redefining adapt with a lower maximum removes workers """
cluster = yield LocalCluster(0, asynchronous=True, processes=False,
scheduler_port=0, silence_logs=False,
diagnostics_port=None)
client = yield Client(cluster, asynchronous=True)
cluster.adapt(interval='20ms', maximum=5)
try:
futures = client.map(slowinc, range(1000), delay=0.1)
while len(cluster.scheduler.workers) < 5:
yield gen.sleep(0.1)
cluster.adapt(maximum=2)
start = time()
while len(cluster.scheduler.workers) != 2:
yield gen.sleep(0.1)
assert time() < start + 1
finally:
yield client.close()
yield cluster.close()
@pytest.mark.xfail(reason="we currently only judge occupancy, not ntasks")
@gen_test(timeout=30)
def test_no_more_workers_than_tasks():
loop = IOLoop.current()
cluster = yield LocalCluster(0, scheduler_port=0, silence_logs=False,
processes=False, diagnostics_port=None,
loop=loop, asynchronous=True)
yield cluster._start()
try:
adapt = Adaptive(cluster.scheduler, cluster, minimum=0, maximum=4,
interval='10 ms')
client = yield Client(cluster, asynchronous=True, loop=loop)
cluster.scheduler.task_duration['slowinc'] = 1000
yield client.submit(slowinc, 1, delay=0.100)
assert len(cluster.scheduler.workers) <= 1
finally:
yield client.close()
yield cluster.close()
def test_basic_no_loop():
try:
with LocalCluster(0, scheduler_port=0, silence_logs=False,
diagnostics_port=None) as cluster:
with Client(cluster) as client:
cluster.adapt()
future = client.submit(lambda x: x + 1, 1)
assert future.result() == 2
loop = cluster.loop
finally:
loop.add_callback(loop.stop)
@gen_test(timeout=None)
def test_target_duration():
""" Ensure that redefining adapt with a lower maximum removes workers """
cluster = yield LocalCluster(0, asynchronous=True, processes=False,
scheduler_port=0, silence_logs=False,
diagnostics_port=None)
client = yield Client(cluster, asynchronous=True)
adaptive = cluster.adapt(interval='20ms', minimum=2, target_duration='5s')
cluster.scheduler.task_duration['slowinc'] = 1
try:
while len(cluster.scheduler.workers) < 2:
yield gen.sleep(0.01)
futures = client.map(slowinc, range(100), delay=0.3)
while len(adaptive.log) < 2:
yield gen.sleep(0.01)
assert adaptive.log[0][1:] == ('up', {'n': 2})
assert adaptive.log[1][1:] == ('up', {'n': 20})
finally:
yield client.close()
yield cluster.close()
@gen_test(timeout=None)
def test_worker_keys():
""" Ensure that redefining adapt with a lower maximum removes workers """
cluster = yield LocalCluster(0, asynchronous=True, processes=False,
scheduler_port=0, silence_logs=False,
diagnostics_port=None)
try:
yield [cluster.start_worker(name='a-1'),
cluster.start_worker(name='a-2'),
cluster.start_worker(name='b-1'),
cluster.start_worker(name='b-2')]
while len(cluster.scheduler.workers) != 4:
yield gen.sleep(0.01)
def key(ws):
return ws.name.split('-')[0]
cluster._adaptive_options = {'worker_key': key}
adaptive = cluster.adapt(minimum=1)
yield adaptive._adapt()
while len(cluster.scheduler.workers) == 4:
yield gen.sleep(0.01)
names = {ws.name for ws in cluster.scheduler.workers.values()}
assert names == {'a-1', 'a-2'} or names == {'b-1', 'b-2'}
finally:
yield cluster.close()
@gen_cluster(client=True, ncores=[])
def test_without_cluster(c, s):
adapt = Adaptive(scheduler=s)
future = c.submit(inc, 1)
while not s.tasks:
yield gen.sleep(0.01)
response = yield c.scheduler.adaptive_recommendations()
assert response['status'] == 'up'