File: test_ws.py

package info (click to toggle)
dask.distributed 2024.12.1%2Bds-1
  • links: PTS, VCS
  • area: main
  • in suites: forky, sid, trixie
  • size: 12,588 kB
  • sloc: python: 96,954; javascript: 1,549; sh: 390; makefile: 220
file content (247 lines) | stat: -rw-r--r-- 7,810 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
from __future__ import annotations

import asyncio
import os
import random
import warnings

import pytest

import dask

from distributed import Client, Scheduler, Worker
from distributed.comm import connect, listen, ws
from distributed.comm.core import FatalCommClosedError
from distributed.comm.registry import backends, get_backend
from distributed.comm.tests.test_comms import check_tls_extra
from distributed.security import Security
from distributed.utils_test import (
    gen_cluster,
    gen_test,
    get_client_ssl_context,
    get_server_ssl_context,
    inc,
    xfail_ssl_issue5601,
)

pytestmark = pytest.mark.flaky(reruns=2)


def test_registered():
    assert "ws" in backends
    backend = get_backend("ws")
    assert isinstance(backend, ws.WSBackend)


@gen_test()
async def test_listen_connect():
    comm_closed = asyncio.Event()

    async def handle_comm(comm):
        try:
            while True:
                msg = await comm.read()
                await comm.write(msg)
        finally:
            comm_closed.set()

    async with listen("ws://", handle_comm) as listener:
        comm = await connect(listener.contact_address)
        await comm.write(b"Hello!")
        result = await comm.read()
        assert result == b"Hello!"

        await comm.close()
        await comm_closed.wait()


@gen_test()
async def test_listen_connect_wss():
    comm_closed = asyncio.Event()

    async def handle_comm(comm):
        try:
            while True:
                msg = await comm.read()
                await comm.write(msg)
        finally:
            comm_closed.set()

    server_ctx = get_server_ssl_context()
    client_ctx = get_client_ssl_context()

    async with listen("wss://", handle_comm, ssl_context=server_ctx) as listener:
        comm = await connect(listener.contact_address, ssl_context=client_ctx)
        assert comm.peer_address.startswith("wss://")
        check_tls_extra(comm.extra_info)
        await comm.write(b"Hello!")
        result = await comm.read()
        assert result == b"Hello!"
        await comm.close()
        await comm_closed.wait()


@gen_test()
async def test_expect_ssl_context():
    server_ctx = get_server_ssl_context()

    async with listen("wss://", lambda comm: comm, ssl_context=server_ctx) as listener:
        with pytest.raises(FatalCommClosedError, match="TLS expects a `ssl_context` *"):
            comm = await connect(listener.contact_address)


@gen_test()
async def test_expect_scheduler_ssl_when_sharing_server(tmp_path):
    xfail_ssl_issue5601()
    pytest.importorskip("cryptography")
    security = Security.temporary()
    key_path = os.path.join(str(tmp_path), "dask.pem")
    cert_path = os.path.join(str(tmp_path), "dask.crt")
    with open(key_path, "w") as f:
        f.write(security.tls_scheduler_key)
    with open(cert_path, "w") as f:
        f.write(security.tls_scheduler_cert)
    c = {
        "distributed.scheduler.dashboard.tls.key": key_path,
        "distributed.scheduler.dashboard.tls.cert": cert_path,
    }
    with dask.config.set(c):
        with pytest.raises(RuntimeError):
            async with Scheduler(protocol="ws://", dashboard=True, port=8787):
                pass


@gen_cluster(client=True, scheduler_kwargs={"protocol": "ws://"})
async def test_roundtrip(c, s, a, b):
    assert a.address.startswith("ws://")
    assert b.address.startswith("ws://")
    assert c.scheduler.address.startswith("ws://")
    assert await c.submit(inc, 1) == 2


@gen_cluster(client=True, scheduler_kwargs={"protocol": "ws://"})
async def test_collections(c, s, a, b):
    pytest.importorskip("numpy")
    da = pytest.importorskip("dask.array")
    x = da.random.random((1000, 1000), chunks=(100, 100))
    x = x + x.T
    await x.persist()


@gen_cluster(client=True, scheduler_kwargs={"protocol": "ws://"})
async def test_large_transfer(c, s, a, b):
    x = await c.scatter(random.randbytes(12_000_000))
    await c.gather(x)


@pytest.mark.parametrize(
    "dashboard,protocol,security,port",
    [
        (True, "ws://", None, 8787),
        (True, "wss://", True, 8787),
        (False, "ws://", None, 8787),
        (False, "wss://", True, 8787),
        (True, "ws://", None, 8786),
        (True, "wss://", True, 8786),
        (False, "ws://", None, 8786),
        (False, "wss://", True, 8786),
    ],
)
@gen_test()
async def test_http_and_comm_server(dashboard, protocol, security, port):
    if security:
        xfail_ssl_issue5601()
        pytest.importorskip("cryptography")
        security = Security.temporary()
    async with Scheduler(
        protocol=protocol, dashboard=dashboard, port=port, security=security
    ) as s:
        if port == 8787:
            assert s.http_server is s.listener.server
        else:
            assert s.http_server is not s.listener.server
        async with Worker(s.address, protocol=protocol, security=security) as w:
            async with Client(s.address, asynchronous=True, security=security) as c:
                result = await c.submit(lambda x: x + 1, 10)
                assert result == 11


@pytest.mark.parametrize(
    "protocol,sni",
    [("ws://", True), ("ws://", False), ("wss://", True), ("wss://", False)],
)
@gen_test(clean_kwargs={"instances": False})
async def test_connection_made_with_extra_conn_args(protocol, sni):
    if protocol == "ws://":
        security = Security(
            extra_conn_args={"headers": {"Authorization": "Token abcd"}}
        )
    else:
        xfail_ssl_issue5601()
        pytest.importorskip("cryptography")
        security = Security.temporary(
            extra_conn_args={"headers": {"Authorization": "Token abcd"}}
        )
    async with Scheduler(
        protocol=protocol, security=security, dashboard_address=":0"
    ) as s:
        connection_args = security.get_connection_args("worker")
        if sni:
            connection_args["server_hostname"] = "sni.example.host"
        comm = await connect(s.address, **connection_args)
        assert comm.sock.request.headers.get("Authorization") == "Token abcd"

        await comm.close()


@gen_test(clean_kwargs={"instances": False})
async def test_connection_made_with_sni():
    xfail_ssl_issue5601()
    pytest.importorskip("cryptography")
    security = Security.temporary()
    async with Scheduler(
        protocol="wss://", security=security, dashboard_address=":0"
    ) as s:
        connection_args = security.get_connection_args("worker")
        connection_args["server_hostname"] = "sni.example.host"
        comm = await connect(s.address, **connection_args)
        assert comm.sock.request.headers.get("Host") == "sni.example.host"

        await comm.close()


@gen_test()
async def test_quiet_close():
    with warnings.catch_warnings(record=True) as record:
        async with Client(
            protocol="ws", processes=False, asynchronous=True, dashboard_address=":0"
        ):
            pass

    assert not record, record[0].message


@gen_cluster(client=True, scheduler_kwargs={"protocol": "ws://"})
async def test_ws_roundtrip(c, s, a, b):
    np = pytest.importorskip("numpy")
    x = np.arange(100)
    future = await c.scatter(x)
    y = await future
    assert (x == y).all()


@gen_test()
async def test_wss_roundtrip():
    np = pytest.importorskip("numpy")
    xfail_ssl_issue5601()
    pytest.importorskip("cryptography")
    security = Security.temporary()
    async with Scheduler(
        protocol="wss://", security=security, dashboard_address=":0"
    ) as s:
        async with Worker(s.address, security=security) as w:
            async with Client(s.address, security=security, asynchronous=True) as c:
                x = np.arange(100)
                future = await c.scatter(x)
                y = await future
                assert (x == y).all()