File: test_batched.py

package info (click to toggle)
dask.distributed 2022.12.1%2Bds.1-3
  • links: PTS, VCS
  • area: main
  • in suites: bookworm
  • size: 10,164 kB
  • sloc: python: 81,938; javascript: 1,549; makefile: 228; sh: 100
file content (256 lines) | stat: -rw-r--r-- 6,413 bytes parent folder | download
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
from __future__ import annotations

import asyncio
import random

import pytest
from tlz import assoc

from distributed.batched import BatchedSend
from distributed.core import CommClosedError, connect, listen
from distributed.metrics import time
from distributed.protocol import to_serialize
from distributed.utils import All
from distributed.utils_test import captured_logger, gen_test


class EchoServer:
    count = 0

    async def handle_comm(self, comm):
        while True:
            try:
                msg = await comm.read()
                self.count += 1
                await comm.write(msg)
            except CommClosedError as e:
                return

    async def listen(self):
        listener = await listen("", self.handle_comm)
        self.address = listener.contact_address
        self.stop = listener.stop

    async def __aenter__(self):
        await self.listen()
        return self

    async def __aexit__(self, exc_type, exc_value, traceback):
        self.stop()


@gen_test()
async def test_BatchedSend():
    async with EchoServer() as e:
        comm = await connect(e.address)

        b = BatchedSend(interval=10)
        assert str(len(b.buffer)) in str(b)
        assert str(len(b.buffer)) in repr(b)
        b.start(comm)

        await asyncio.sleep(0.020)

        b.send("hello")
        b.send("hello")
        b.send("world")
        await asyncio.sleep(0.020)
        b.send("HELLO")
        b.send("HELLO")

        result = await comm.read()
        assert result == ("hello", "hello", "world")
        result = await comm.read()
        assert result == ("HELLO", "HELLO")

        assert b.byte_count > 1
        await comm.close()


@gen_test()
async def test_send_before_start():
    async with EchoServer() as e:
        comm = await connect(e.address)

        b = BatchedSend(interval=10)

        b.send("hello")
        b.send("world")

        b.start(comm)
        result = await comm.read()
        assert result == ("hello", "world")
        await comm.close()


@gen_test()
async def test_send_after_stream_start():
    async with EchoServer() as e:
        comm = await connect(e.address)

        b = BatchedSend(interval=10)

        b.start(comm)
        b.send("hello")
        b.send("world")
        result = await comm.read()
        if len(result) < 2:
            result += await comm.read()
        assert result == ("hello", "world")
        await comm.close()


@gen_test()
async def test_send_before_close():
    async with EchoServer() as e:
        comm = await connect(e.address)

        b = BatchedSend(interval=10)
        b.start(comm)

        cnt = int(e.count)
        b.send("hello")
        await b.close()  # close immediately after sending
        assert not b.buffer

        start = time()
        while e.count != cnt + 1:
            await asyncio.sleep(0.01)
            assert time() < start + 5

        with pytest.raises(CommClosedError):
            b.send("123")


@gen_test()
async def test_close_closed():
    async with EchoServer() as e:
        comm = await connect(e.address)

        b = BatchedSend(interval=10)
        b.start(comm)

        b.send(123)
        await comm.close()  # external closing

        await b.close()
        assert "closed" in repr(b)
        assert "closed" in str(b)


@gen_test()
async def test_close_not_started():
    b = BatchedSend(interval=10)
    await b.close()


@gen_test()
async def test_close_twice():
    async with EchoServer() as e:
        comm = await connect(e.address)

        b = BatchedSend(interval=10)
        b.start(comm)
        await b.close()
        await b.close()


@pytest.mark.slow
@gen_test()
async def test_stress():
    async with EchoServer() as e:
        comm = await connect(e.address)
        L = []

        async def send():
            b = BatchedSend(interval=3)
            b.start(comm)
            for i in range(0, 10000, 2):
                b.send(i)
                b.send(i + 1)
                await asyncio.sleep(0.00001 * random.randint(1, 10))

        async def recv():
            while True:
                result = await asyncio.wait_for(comm.read(), 1)
                L.extend(result)
                if result[-1] == 9999:
                    break

        await All([send(), recv()])

        assert L == list(range(0, 10000, 1))
        await comm.close()


async def run_traffic_jam(nsends, nbytes):
    # This test eats `nsends * nbytes` bytes in RAM
    np = pytest.importorskip("numpy")
    from distributed.protocol import to_serialize

    data = bytes(np.random.randint(0, 255, size=(nbytes,)).astype("u1").data)
    async with EchoServer() as e:
        comm = await connect(e.address)

        b = BatchedSend(interval=0.01)
        b.start(comm)

        msg = {"x": to_serialize(data)}
        for i in range(nsends):
            b.send(assoc(msg, "i", i))
            if np.random.random() > 0.5:
                await asyncio.sleep(0.001)

        results = []
        count = 0
        while len(results) < nsends:
            # If this times out then I think it's a backpressure issue
            # Somehow we're able to flood the socket so that the receiving end
            # loses some of our messages
            L = await asyncio.wait_for(comm.read(), 5)
            count += 1
            results.extend(r["i"] for r in L)

        assert count == b.batch_count == e.count
        assert b.message_count == nsends

        assert results == list(range(nsends))

        await comm.close()  # external closing
        await b.close()


@gen_test()
async def test_sending_traffic_jam():
    await run_traffic_jam(50, 300000)


@pytest.mark.slow
@gen_test()
async def test_large_traffic_jam():
    await run_traffic_jam(500, 1500000)


@gen_test()
async def test_serializers():
    async with EchoServer() as e:
        comm = await connect(e.address)

        b = BatchedSend(interval="10ms", serializers=["msgpack"])
        b.start(comm)

        b.send({"x": to_serialize(123)})
        b.send({"x": to_serialize("hello")})
        await asyncio.sleep(0.100)

        b.send({"x": to_serialize(lambda x: x + 1)})

        with captured_logger("distributed.protocol") as sio:
            await asyncio.sleep(0.100)

        value = sio.getvalue()
        assert "serialize" in value
        assert "type" in value
        assert "function" in value

        assert comm.closed()