Source code distributed/diagnostics/tests/test_task_stream.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
import os
from time import sleep

import pytest
from tlz import frequencies

from distributed import get_task_stream
from distributed.utils_test import gen_cluster, div, inc, slowinc
from distributed.utils_test import client, loop, cluster_fixture  # noqa: F401
from distributed.client import wait
from distributed.diagnostics.task_stream import TaskStreamPlugin
from distributed.metrics import time


@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3)
async def test_TaskStreamPlugin(c, s, *workers):
    es = TaskStreamPlugin(s)
    assert not es.buffer

    futures = c.map(div, [1] * 10, range(10))
    total = c.submit(sum, futures[1:])
    await wait(total)

    assert len(es.buffer) == 11

    workers = dict()

    rects = es.rectangles(0, 10, workers)
    assert workers
    assert all(n == "div" for n in rects["name"])
    assert all(d > 0 for d in rects["duration"])
    counts = frequencies(rects["color"])
    assert counts["black"] == 1
    assert set(counts.values()) == {9, 1}
    assert len(set(rects["y"])) == 3

    rects = es.rectangles(2, 5, workers)
    assert all(len(L) == 3 for L in rects.values())

    starts = sorted(rects["start"])
    rects = es.rectangles(
        2, 5, workers=workers, start_boundary=(starts[0] + starts[1]) / 2000
    )
    assert set(rects["start"]).issubset(set(starts[1:]))


@gen_cluster(client=True)
async def test_maxlen(c, s, a, b):
    tasks = TaskStreamPlugin(s, maxlen=5)
    futures = c.map(inc, range(10))
    await wait(futures)
    assert len(tasks.buffer) == 5


@gen_cluster(client=True)
async def test_collect(c, s, a, b):
    tasks = TaskStreamPlugin(s)
    start = time()
    futures = c.map(slowinc, range(10), delay=0.1)
    await wait(futures)

    L = tasks.collect()
    assert len(L) == len(futures)
    L = tasks.collect(start=start)
    assert len(L) == len(futures)

    L = tasks.collect(start=start + 0.2)
    assert 4 <= len(L) <= len(futures)

    L = tasks.collect(start="20 s")
    assert len(L) == len(futures)

    L = tasks.collect(start="500ms")
    assert 0 < len(L) <= len(futures)

    L = tasks.collect(count=3)
    assert len(L) == 3
    assert L == list(tasks.buffer)[-3:]

    assert tasks.collect(stop=start + 100, count=3) == tasks.collect(count=3)
    assert tasks.collect(start=start, count=3) == list(tasks.buffer)[:3]


@gen_cluster(client=True)
async def test_client(c, s, a, b):
    L = await c.get_task_stream()
    assert L == ()

    futures = c.map(slowinc, range(10), delay=0.1)
    await wait(futures)

    tasks = [p for p in s.plugins if isinstance(p, TaskStreamPlugin)][0]
    L = await c.get_task_stream()
    assert L == tuple(tasks.buffer)


def test_client_sync(client):
    with get_task_stream(client=client) as ts:
        sleep(0.1)  # to smooth over time differences on the scheduler
        # to smooth over time differences on the scheduler
        futures = client.map(inc, range(10))
        wait(futures)

    assert len(ts.data) == 10


@gen_cluster(client=True)
async def test_get_task_stream_plot(c, s, a, b):
    bokeh = pytest.importorskip("bokeh")
    await c.get_task_stream()

    futures = c.map(slowinc, range(10), delay=0.1)
    await wait(futures)

    data, figure = await c.get_task_stream(plot=True)
    assert isinstance(figure, bokeh.plotting.Figure)


def test_get_task_stream_save(client, tmpdir):
    bokeh = pytest.importorskip("bokeh")
    tmpdir = str(tmpdir)
    fn = os.path.join(tmpdir, "foo.html")

    with get_task_stream(plot="save", filename=fn) as ts:
        wait(client.map(inc, range(10)))
    with open(fn) as f:
        data = f.read()
    assert "inc" in data
    assert "bokeh" in data

    assert isinstance(ts.figure, bokeh.plotting.Figure)