Source code distributed/cli/tests/test_dask_scheduler.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
import pytest

pytest.importorskip("requests")

import os
import requests
import socket
import shutil
import sys
import tempfile
from time import sleep

from click.testing import CliRunner

import distributed
from distributed import Scheduler, Client
from distributed.utils import get_ip, get_ip_interface, tmpfile
from distributed.utils_test import (
    popen,
    assert_can_connect_from_everywhere_4_6,
    assert_can_connect_locally_4,
)
from distributed.utils_test import loop  # noqa: F401
from distributed.metrics import time
import distributed.cli.dask_scheduler


def test_defaults(loop):
    with popen(["dask-scheduler", "--no-dashboard"]) as proc:

        async def f():
            # Default behaviour is to listen on all addresses
            await assert_can_connect_from_everywhere_4_6(8786, timeout=5.0)

        with Client("127.0.0.1:%d" % Scheduler.default_port, loop=loop) as c:
            c.sync(f)

        response = requests.get("http://127.0.0.1:8787/status/")
        assert response.status_code == 404

    with pytest.raises(Exception):
        response = requests.get("http://127.0.0.1:9786/info.json")


def test_hostport(loop):
    with popen(["dask-scheduler", "--no-dashboard", "--host", "127.0.0.1:8978"]):

        async def f():
            # The scheduler's main port can't be contacted from the outside
            await assert_can_connect_locally_4(8978, timeout=5.0)

        with Client("127.0.0.1:8978", loop=loop) as c:
            assert len(c.nthreads()) == 0
            c.sync(f)


def test_no_dashboard(loop):
    pytest.importorskip("bokeh")
    with popen(["dask-scheduler", "--no-dashboard"]) as proc:
        with Client("127.0.0.1:%d" % Scheduler.default_port, loop=loop) as c:
            response = requests.get("http://127.0.0.1:8787/status/")
            assert response.status_code == 404


def test_dashboard(loop):
    pytest.importorskip("bokeh")

    with popen(["dask-scheduler"]) as proc:
        for line in proc.stderr:
            if b"dashboard at" in line:
                dashboard_port = int(line.decode().split(":")[-1].strip())
                break

        with Client("127.0.0.1:%d" % Scheduler.default_port, loop=loop) as c:
            pass

        names = ["localhost", "127.0.0.1", get_ip()]
        if "linux" in sys.platform:
            names.append(socket.gethostname())

        start = time()
        while True:
            try:
                # All addresses should respond
                for name in names:
                    uri = "http://%s:%d/status/" % (name, dashboard_port)
                    response = requests.get(uri)
                    assert response.ok
                break
            except Exception as f:
                print("got error on %r: %s" % (uri, f))
                sleep(0.1)
                assert time() < start + 10

    with pytest.raises(Exception):
        requests.get("http://127.0.0.1:%d/status/" % dashboard_port)


def test_dashboard_non_standard_ports(loop):
    pytest.importorskip("bokeh")

    with popen(
        ["dask-scheduler", "--port", "3448", "--dashboard-address", ":4832"]
    ) as proc:
        with Client("127.0.0.1:3448", loop=loop) as c:
            pass

        start = time()
        while True:
            try:
                response = requests.get("http://localhost:4832/status/")
                assert response.ok
                break
            except Exception:
                sleep(0.1)
                assert time() < start + 20
    with pytest.raises(Exception):
        requests.get("http://localhost:4832/status/")


@pytest.mark.skipif(
    not sys.platform.startswith("linux"), reason="Need 127.0.0.2 to mean localhost"
)
def test_dashboard_whitelist(loop):
    pytest.importorskip("bokeh")
    with pytest.raises(Exception):
        requests.get("http://localhost:8787/status/").ok

    with popen(["dask-scheduler"]) as proc:
        with Client("127.0.0.1:%d" % Scheduler.default_port, loop=loop) as c:
            pass

        start = time()
        while True:
            try:
                for name in ["127.0.0.2", "127.0.0.3"]:
                    response = requests.get("http://%s:8787/status/" % name)
                    assert response.ok
                break
            except Exception as f:
                print(f)
                sleep(0.1)
                assert time() < start + 20


def test_multiple_workers(loop):
    with popen(["dask-scheduler", "--no-dashboard"]) as s:
        with popen(["dask-worker", "localhost:8786", "--no-dashboard"]) as a:
            with popen(["dask-worker", "localhost:8786", "--no-dashboard"]) as b:
                with Client("127.0.0.1:%d" % Scheduler.default_port, loop=loop) as c:
                    start = time()
                    while len(c.nthreads()) < 2:
                        sleep(0.1)
                        assert time() < start + 10


def test_interface(loop):
    psutil = pytest.importorskip("psutil")
    if_names = sorted(psutil.net_if_addrs())
    for if_name in if_names:
        try:
            ipv4_addr = get_ip_interface(if_name)
        except ValueError:
            pass
        else:
            if ipv4_addr == "127.0.0.1":
                break
    else:
        pytest.skip(
            "Could not find loopback interface. "
            "Available interfaces are: %s." % (if_names,)
        )

    with popen(["dask-scheduler", "--no-dashboard", "--interface", if_name]) as s:
        with popen(
            ["dask-worker", "127.0.0.1:8786", "--no-dashboard", "--interface", if_name]
        ) as a:
            with Client("tcp://127.0.0.1:%d" % Scheduler.default_port, loop=loop) as c:
                start = time()
                while not len(c.nthreads()):
                    sleep(0.1)
                    assert time() - start < 5
                info = c.scheduler_info()
                assert "tcp://127.0.0.1" in info["address"]
                assert all("127.0.0.1" == d["host"] for d in info["workers"].values())


@pytest.mark.skipif(
    sys.version_info < (3, 7), reason="Intermittent failure on old Python version"
)
def test_pid_file(loop):
    def check_pidfile(proc, pidfile):
        start = time()
        while not os.path.exists(pidfile):
            sleep(0.01)
            assert time() < start + 5

        text = False
        start = time()
        while not text:
            sleep(0.01)
            assert time() < start + 5
            with open(pidfile) as f:
                text = f.read()
        pid = int(text)
        if sys.platform.startswith("win"):
            # On Windows, `dask-XXX` invokes the dask-XXX.exe
            # shim, but the PID is written out by the child Python process
            assert pid
        else:
            assert proc.pid == pid

    with tmpfile() as s:
        with popen(["dask-scheduler", "--pid-file", s, "--no-dashboard"]) as sched:
            check_pidfile(sched, s)

        with tmpfile() as w:
            with popen(
                ["dask-worker", "127.0.0.1:8786", "--pid-file", w, "--no-dashboard"]
            ) as worker:
                check_pidfile(worker, w)


def test_scheduler_port_zero(loop):
    with tmpfile() as fn:
        with popen(
            ["dask-scheduler", "--no-dashboard", "--scheduler-file", fn, "--port", "0"]
        ) as sched:
            with Client(scheduler_file=fn, loop=loop) as c:
                assert c.scheduler.port
                assert c.scheduler.port != 8786


def test_dashboard_port_zero(loop):
    pytest.importorskip("bokeh")
    with tmpfile() as fn:
        with popen(["dask-scheduler", "--dashboard-address", ":0"]) as proc:
            count = 0
            while count < 1:
                line = proc.stderr.readline()
                if b"dashboard" in line.lower():
                    sleep(0.01)
                    count += 1
                    assert b":0" not in line


PRELOAD_TEXT = """
_scheduler_info = {}

def dask_setup(scheduler):
    _scheduler_info['address'] = scheduler.address
    scheduler.foo = "bar"

def get_scheduler_address():
    return _scheduler_info['address']
"""


def test_preload_file(loop):
    def check_scheduler():
        import scheduler_info

        return scheduler_info.get_scheduler_address()

    tmpdir = tempfile.mkdtemp()
    try:
        path = os.path.join(tmpdir, "scheduler_info.py")
        with open(path, "w") as f:
            f.write(PRELOAD_TEXT)
        with tmpfile() as fn:
            with popen(["dask-scheduler", "--scheduler-file", fn, "--preload", path]):
                with Client(scheduler_file=fn, loop=loop) as c:
                    assert c.run_on_scheduler(check_scheduler) == c.scheduler.address
    finally:
        shutil.rmtree(tmpdir)


def test_preload_module(loop):
    def check_scheduler():
        import scheduler_info

        return scheduler_info.get_scheduler_address()

    tmpdir = tempfile.mkdtemp()
    try:
        path = os.path.join(tmpdir, "scheduler_info.py")
        with open(path, "w") as f:
            f.write(PRELOAD_TEXT)
        env = os.environ.copy()
        if "PYTHONPATH" in env:
            env["PYTHONPATH"] = tmpdir + ":" + env["PYTHONPATH"]
        else:
            env["PYTHONPATH"] = tmpdir
        with tmpfile() as fn:
            with popen(
                [
                    "dask-scheduler",
                    "--scheduler-file",
                    fn,
                    "--preload",
                    "scheduler_info",
                ],
                env=env,
            ):
                with Client(scheduler_file=fn, loop=loop) as c:
                    assert c.run_on_scheduler(check_scheduler) == c.scheduler.address
    finally:
        shutil.rmtree(tmpdir)


def test_preload_remote_module(loop, tmp_path):
    with open(tmp_path / "scheduler_info.py", "w") as f:
        f.write(PRELOAD_TEXT)

    with popen([sys.executable, "-m", "http.server", "9382"], cwd=tmp_path):
        with popen(
            [
                "dask-scheduler",
                "--scheduler-file",
                str(tmp_path / "scheduler-file.json"),
                "--preload",
                "http://localhost:9382/scheduler_info.py",
            ]
        ) as proc:
            with Client(
                scheduler_file=tmp_path / "scheduler-file.json", loop=loop
            ) as c:
                assert (
                    c.run_on_scheduler(
                        lambda dask_scheduler: getattr(dask_scheduler, "foo", None)
                    )
                    == "bar"
                )


PRELOAD_COMMAND_TEXT = """
import click
_config = {}

@click.command()
@click.option("--passthrough", type=str, default="default")
def dask_setup(scheduler, passthrough):
    _config["passthrough"] = passthrough

def get_passthrough():
    return _config["passthrough"]
"""


def test_preload_command(loop):
    def check_passthrough():
        import passthrough_info

        return passthrough_info.get_passthrough()

    tmpdir = tempfile.mkdtemp()
    try:
        path = os.path.join(tmpdir, "passthrough_info.py")
        with open(path, "w") as f:
            f.write(PRELOAD_COMMAND_TEXT)

        with tmpfile() as fn:
            print(fn)
            with popen(
                [
                    "dask-scheduler",
                    "--scheduler-file",
                    fn,
                    "--preload",
                    path,
                    "--passthrough",
                    "foobar",
                ]
            ):
                with Client(scheduler_file=fn, loop=loop) as c:
                    assert c.run_on_scheduler(check_passthrough) == "foobar"
    finally:
        shutil.rmtree(tmpdir)


def test_preload_command_default(loop):
    def check_passthrough():
        import passthrough_info

        return passthrough_info.get_passthrough()

    tmpdir = tempfile.mkdtemp()
    try:
        path = os.path.join(tmpdir, "passthrough_info.py")
        with open(path, "w") as f:
            f.write(PRELOAD_COMMAND_TEXT)

        with tmpfile() as fn2:
            print(fn2)
            with popen(
                ["dask-scheduler", "--scheduler-file", fn2, "--preload", path],
                stdout=sys.stdout,
                stderr=sys.stderr,
            ):
                with Client(scheduler_file=fn2, loop=loop) as c:
                    assert c.run_on_scheduler(check_passthrough) == "default"

    finally:
        shutil.rmtree(tmpdir)


def test_version_option():
    runner = CliRunner()
    result = runner.invoke(distributed.cli.dask_scheduler.main, ["--version"])
    assert result.exit_code == 0


@pytest.mark.slow
def test_idle_timeout(loop):
    start = time()
    runner = CliRunner()
    result = runner.invoke(
        distributed.cli.dask_scheduler.main, ["--idle-timeout", "1s"]
    )
    stop = time()
    assert 1 < stop - start < 10


def test_multiple_workers(loop):
    text = """
def dask_setup(worker):
    worker.foo = 'setup'
"""
    with popen(["dask-scheduler", "--no-dashboard"]) as s:
        with popen(
            [
                "dask-worker",
                "localhost:8786",
                "--no-dashboard",
                "--preload",
                text,
                "--preload-nanny",
                text,
            ]
        ) as a:
            with Client("127.0.0.1:8786", loop=loop) as c:
                c.wait_for_workers(1)
                [foo] = c.run(lambda dask_worker: dask_worker.foo).values()
                assert foo == "setup"
                [foo] = c.run(lambda dask_worker: dask_worker.foo, nanny=True).values()
                assert foo == "setup"