Source code distributed/deploy/tests/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
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
import asyncio
import gc
import math
from time import sleep

import dask
import pytest

from distributed import Client, wait, Adaptive, LocalCluster, SpecCluster, Worker
from distributed.utils_test import gen_test, slowinc, clean
from distributed.utils_test import loop, nodebug, cleanup  # noqa: F401
from distributed.metrics import time


@pytest.mark.asyncio
async def test_simultaneous_scale_up_and_down(cleanup):
    class TestAdaptive(Adaptive):
        def get_scale_up_kwargs(self):
            assert False

        def _retire_workers(self):
            assert False

    class TestCluster(LocalCluster):
        def scale_up(self, n, **kwargs):
            assert False

        def scale_down(self, workers):
            assert False

    with dask.config.set(
        {"distributed.scheduler.default-task-durations": {"a": 4, "b": 4, "c": 1}}
    ):
        async with TestCluster(
            n_workers=4, processes=False, asynchronous=True
        ) as cluster:
            async with Client(cluster, asynchronous=True) as c:
                s = cluster.scheduler

                future = c.map(slowinc, [1, 1, 1], key=["a-4", "b-4", "c-1"])

                while len(s.rprocessing) < 3:
                    await asyncio.sleep(0.001)

                ta = cluster.adapt(
                    interval="100 ms", scale_factor=2, Adaptive=TestAdaptive
                )

                await asyncio.sleep(0.3)


def test_adaptive_local_cluster(loop):
    with LocalCluster(
        0, scheduler_port=0, silence_logs=False, dashboard_address=None, loop=loop
    ) as cluster:
        alc = cluster.adapt(interval="100 ms")
        with Client(cluster, loop=loop) as c:
            assert not c.nthreads()
            future = c.submit(lambda x: x + 1, 1)
            assert future.result() == 2
            assert c.nthreads()

            sleep(0.1)
            assert c.nthreads()  # still there after some time

            del future

            start = time()
            while cluster.scheduler.nthreads:
                sleep(0.01)
                assert time() < start + 5

            assert not c.nthreads()


@pytest.mark.asyncio
async def test_adaptive_local_cluster_multi_workers(cleanup):
    async with LocalCluster(
        0,
        scheduler_port=0,
        silence_logs=False,
        processes=False,
        dashboard_address=None,
        asynchronous=True,
    ) as cluster:

        cluster.scheduler.allowed_failures = 1000
        adapt = cluster.adapt(interval="100 ms")
        async with Client(cluster, asynchronous=True) as c:
            futures = c.map(slowinc, range(100), delay=0.01)

            start = time()
            while not cluster.scheduler.workers:
                await asyncio.sleep(0.01)
                assert time() < start + 15, adapt.log

            await c.gather(futures)
            del futures

            start = time()
            # while cluster.workers:
            while cluster.scheduler.workers:
                await asyncio.sleep(0.01)
                assert time() < start + 15, adapt.log

            # no workers for a while
            for i in range(10):
                assert not cluster.scheduler.workers
                await asyncio.sleep(0.05)

            futures = c.map(slowinc, range(100), delay=0.01)
            await c.gather(futures)


@pytest.mark.xfail(reason="changed API")
@pytest.mark.asyncio
async def test_adaptive_scale_down_override(cleanup):
    class TestAdaptive(Adaptive):
        def __init__(self, *args, **kwargs):
            self.min_size = kwargs.pop("min_size", 0)
            Adaptive.__init__(self, *args, **kwargs)

        async def workers_to_close(self, **kwargs):
            num_workers = len(self.cluster.workers)
            to_close = await 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(LocalCluster):
        def scale_up(self, n, **kwargs):
            assert False

    async with TestCluster(n_workers=10, processes=False, asynchronous=True) as cluster:
        ta = cluster.adapt(
            min_size=2, interval=0.1, scale_factor=2, Adaptive=TestAdaptive
        )
        await asyncio.sleep(0.3)

        # Assert that adaptive cycle does not reduce cluster below minimum size
        # as determined via override.
        assert len(cluster.scheduler.workers) == 2


@gen_test()
async def test_min_max():
    cluster = await LocalCluster(
        0,
        scheduler_port=0,
        silence_logs=False,
        processes=False,
        dashboard_address=None,
        asynchronous=True,
        threads_per_worker=1,
    )
    try:
        adapt = cluster.adapt(minimum=1, maximum=2, interval="20 ms", wait_count=10)
        c = await Client(cluster, asynchronous=True)

        start = time()
        while not cluster.scheduler.workers:
            await asyncio.sleep(0.01)
            assert time() < start + 1

        await asyncio.sleep(0.2)
        assert len(cluster.scheduler.workers) == 1
        assert len(adapt.log) == 1 and adapt.log[-1][1] == {"status": "up", "n": 1}

        futures = c.map(slowinc, range(100), delay=0.1)

        start = time()
        while len(cluster.scheduler.workers) < 2:
            await asyncio.sleep(0.01)
            assert time() < start + 1

        assert len(cluster.scheduler.workers) == 2
        await asyncio.sleep(0.5)
        assert len(cluster.scheduler.workers) == 2
        assert len(cluster.workers) == 2
        assert len(adapt.log) == 2 and all(d["status"] == "up" for _, d in adapt.log)

        del futures
        gc.collect()

        start = time()
        while len(cluster.scheduler.workers) != 1:
            await asyncio.sleep(0.01)
            assert time() < start + 2
        assert adapt.log[-1][1]["status"] == "down"
    finally:
        await c.close()
        await cluster.close()


@pytest.mark.asyncio
async def test_avoid_churn(cleanup):
    """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
    """
    async with LocalCluster(
        0,
        asynchronous=True,
        processes=False,
        scheduler_port=0,
        silence_logs=False,
        dashboard_address=None,
    ) as cluster:
        async with Client(cluster, asynchronous=True) as client:
            adapt = cluster.adapt(interval="20 ms", wait_count=5)

            for i in range(10):
                await client.submit(slowinc, i, delay=0.040)
                await asyncio.sleep(0.040)

            assert len(adapt.log) == 1


@pytest.mark.asyncio
async 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 = await LocalCluster(
        0,
        asynchronous=True,
        processes=False,
        scheduler_port=0,
        silence_logs=False,
        dashboard_address=None,
    )
    client = await Client(cluster, asynchronous=True)
    adapt = cluster.adapt(interval="20 ms", wait_count=5, maximum=10)
    try:
        future = client.submit(slowinc, 1, delay=0.100)
        await 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 len(adapt.log) == 1:
            await asyncio.sleep(0.01)
        assert len(adapt.log) == 2
        assert adapt.log[-1][1]["status"] == "up"
        d = [x for x in adapt.log[-1] if isinstance(x, dict)][0]
        assert 2 < d["n"] <= adapt.maximum

        while len(cluster.workers) < adapt.maximum:
            await asyncio.sleep(0.01)

        del futures

        while len(cluster.scheduler.tasks) > 1:
            await asyncio.sleep(0.01)

        await cluster

        while len(cluster.scheduler.workers) > 1 or len(cluster.worker_spec) > 1:
            await asyncio.sleep(0.01)

        # Don't scale up for large sequential computations
        x = await client.scatter(1)
        log = list(cluster._adaptive.log)
        for i in range(100):
            x = client.submit(slowinc, x)

        await asyncio.sleep(0.1)
        assert len(cluster.workers) == 1
    finally:
        await client.close()
        await cluster.close()


@gen_test(timeout=None)
async def test_adapt_down():
    """ Ensure that redefining adapt with a lower maximum removes workers """
    async with LocalCluster(
        0,
        asynchronous=True,
        processes=False,
        scheduler_port=0,
        silence_logs=False,
        dashboard_address=None,
    ) as cluster:
        async with Client(cluster, asynchronous=True) as client:
            cluster.adapt(interval="20ms", maximum=5)

            futures = client.map(slowinc, range(1000), delay=0.1)
            while len(cluster.scheduler.workers) < 5:
                await asyncio.sleep(0.1)

            cluster.adapt(maximum=2)

            start = time()
            while len(cluster.scheduler.workers) != 2:
                await asyncio.sleep(0.1)
                assert time() < start + 3


@gen_test(timeout=30)
async def test_no_more_workers_than_tasks():
    with dask.config.set(
        {"distributed.scheduler.default-task-durations": {"slowinc": 1000}}
    ):
        async with LocalCluster(
            0,
            scheduler_port=0,
            silence_logs=False,
            processes=False,
            dashboard_address=None,
            asynchronous=True,
        ) as cluster:
            adapt = cluster.adapt(minimum=0, maximum=4, interval="10 ms")
            async with Client(cluster, asynchronous=True) as client:
                await client.submit(slowinc, 1, delay=0.100)
                assert len(cluster.scheduler.workers) <= 1


def test_basic_no_loop(loop):
    with clean(threads=False):
        try:
            with LocalCluster(
                0, scheduler_port=0, silence_logs=False, dashboard_address=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)


@pytest.mark.asyncio
async def test_target_duration():
    """ Ensure that redefining adapt with a lower maximum removes workers """
    with dask.config.set(
        {"distributed.scheduler.default-task-durations": {"slowinc": 1}}
    ):
        async with LocalCluster(
            0,
            asynchronous=True,
            processes=False,
            scheduler_port=0,
            silence_logs=False,
            dashboard_address=None,
        ) as cluster:
            adapt = cluster.adapt(interval="20ms", minimum=2, target_duration="5s")
            async with Client(cluster, asynchronous=True) as client:
                while len(cluster.scheduler.workers) < 2:
                    await asyncio.sleep(0.01)

                futures = client.map(slowinc, range(100), delay=0.3)

                while len(adapt.log) < 2:
                    await asyncio.sleep(0.01)

                assert adapt.log[0][1] == {"status": "up", "n": 2}
                assert adapt.log[1][1] == {"status": "up", "n": 20}


@pytest.mark.asyncio
async def test_worker_keys(cleanup):
    """ Ensure that redefining adapt with a lower maximum removes workers """
    async with SpecCluster(
        workers={
            "a-1": {"cls": Worker},
            "a-2": {"cls": Worker},
            "b-1": {"cls": Worker},
            "b-2": {"cls": Worker},
        },
        asynchronous=True,
    ) as cluster:

        def key(ws):
            return ws.name.split("-")[0]

        cluster._adaptive_options = {"worker_key": key}

        adaptive = cluster.adapt(minimum=1)
        await adaptive.adapt()

        while len(cluster.scheduler.workers) == 4:
            await asyncio.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"}


@pytest.mark.asyncio
async def test_adapt_cores_memory(cleanup):
    async with LocalCluster(
        0,
        threads_per_worker=2,
        memory_limit="3 GB",
        scheduler_port=0,
        silence_logs=False,
        processes=False,
        dashboard_address=None,
        asynchronous=True,
    ) as cluster:
        adapt = cluster.adapt(minimum_cores=3, maximum_cores=9)
        assert adapt.minimum == 2
        assert adapt.maximum == 4

        adapt = cluster.adapt(minimum_memory="7GB", maximum_memory="20 GB")
        assert adapt.minimum == 3
        assert adapt.maximum == 6

        adapt = cluster.adapt(
            minimum_cores=1,
            minimum_memory="7GB",
            maximum_cores=10,
            maximum_memory="1 TB",
        )
        assert adapt.minimum == 3
        assert adapt.maximum == 5


def test_adaptive_config():
    with dask.config.set(
        {"distributed.adaptive.minimum": 10, "distributed.adaptive.wait-count": 8}
    ):
        adapt = Adaptive(interval="5s")
        assert adapt.minimum == 10
        assert adapt.maximum == math.inf
        assert adapt.interval == 5
        assert adapt.wait_count == 8


@pytest.mark.asyncio
async def test_update_adaptive(cleanup):
    async with LocalCluster(
        0,
        threads_per_worker=2,
        memory_limit="3 GB",
        scheduler_port=0,
        silence_logs=False,
        processes=False,
        dashboard_address=None,
        asynchronous=True,
    ) as cluster:
        first = cluster.adapt(maxmimum=1)
        second = cluster.adapt(maxmimum=2)
        await asyncio.sleep(0.2)
        assert first.periodic_callback is None
        assert second.periodic_callback.is_running()