File: test_pickle.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 (222 lines) | stat: -rw-r--r-- 6,414 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
from __future__ import annotations

import pickle
import sys
import weakref
from functools import partial
from operator import add

import cloudpickle
import pytest

from dask.utils import tmpdir

from distributed import profile
from distributed.protocol import deserialize, serialize
from distributed.protocol.pickle import (
    CLOUDPICKLE_GTE_20,
    HIGHEST_PROTOCOL,
    dumps,
    loads,
)
from distributed.utils_test import save_sys_modules


class MemoryviewHolder:
    def __init__(self, mv):
        self.mv = memoryview(mv)

    def __reduce_ex__(self, protocol):
        if protocol >= 5:
            return MemoryviewHolder, (pickle.PickleBuffer(self.mv),)
        else:
            return MemoryviewHolder, (self.mv.tobytes(),)


@pytest.mark.parametrize("protocol", {4, HIGHEST_PROTOCOL})
def test_pickle_data(protocol):
    context = {"pickle-protocol": protocol}

    data = [1, b"123", "123", [123], {}, set()]
    for d in data:
        assert loads(dumps(d, protocol=protocol)) == d
        assert deserialize(*serialize(d, serializers=("pickle",), context=context)) == d


@pytest.mark.parametrize("protocol", {4, HIGHEST_PROTOCOL})
def test_pickle_out_of_band(protocol):
    context = {"pickle-protocol": protocol}

    mv = memoryview(b"123")
    mvh = MemoryviewHolder(mv)

    if protocol >= 5:
        l = []
        d = dumps(mvh, protocol=protocol, buffer_callback=l.append)
        mvh2 = loads(d, buffers=l)

        assert len(l) == 1
        assert isinstance(l[0], pickle.PickleBuffer)
        assert memoryview(l[0]) == mv
    else:
        mvh2 = loads(dumps(mvh, protocol=protocol))

    assert isinstance(mvh2, MemoryviewHolder)
    assert isinstance(mvh2.mv, memoryview)
    assert mvh2.mv == mv

    h, f = serialize(mvh, serializers=("pickle",), context=context)
    mvh3 = deserialize(h, f)

    assert isinstance(mvh3, MemoryviewHolder)
    assert isinstance(mvh3.mv, memoryview)
    assert mvh3.mv == mv

    if protocol >= 5:
        assert len(f) == 2
        assert isinstance(f[0], bytes)
        assert isinstance(f[1], memoryview)
        assert f[1] == mv
    else:
        assert len(f) == 1
        assert isinstance(f[0], bytes)


@pytest.mark.parametrize("protocol", {4, HIGHEST_PROTOCOL})
def test_pickle_empty(protocol):
    context = {"pickle-protocol": protocol}

    x = MemoryviewHolder(bytearray())  # Empty view
    header, frames = serialize(x, serializers=("pickle",), context=context)

    assert header["serializer"] == "pickle"
    assert len(frames) >= 1
    assert isinstance(frames[0], bytes)

    if protocol >= 5:
        assert len(frames) == 2
        assert len(header["writeable"]) == 1

        header["writeable"] = (False,) * len(frames)
    else:
        assert len(frames) == 1
        assert len(header["writeable"]) == 0

    y = deserialize(header, frames)

    assert isinstance(y, MemoryviewHolder)
    assert isinstance(y.mv, memoryview)
    assert y.mv == x.mv
    assert y.mv.nbytes == 0
    assert y.mv.readonly


@pytest.mark.parametrize("protocol", {4, HIGHEST_PROTOCOL})
def test_pickle_numpy(protocol):
    np = pytest.importorskip("numpy")
    context = {"pickle-protocol": protocol}

    x = np.ones(5)
    assert (loads(dumps(x, protocol=protocol)) == x).all()
    assert (
        deserialize(*serialize(x, serializers=("pickle",), context=context)) == x
    ).all()

    x = np.ones(5000)
    assert (loads(dumps(x, protocol=protocol)) == x).all()
    assert (
        deserialize(*serialize(x, serializers=("pickle",), context=context)) == x
    ).all()

    x = np.array([np.arange(3), np.arange(4, 6)], dtype=object)
    x2 = loads(dumps(x, protocol=protocol))
    assert x.shape == x2.shape
    assert x.dtype == x2.dtype
    assert x.strides == x2.strides
    for e_x, e_x2 in zip(x.flat, x2.flat):
        np.testing.assert_equal(e_x, e_x2)
    h, f = serialize(x, serializers=("pickle",), context=context)
    if protocol >= 5:
        assert len(f) == 3
    else:
        assert len(f) == 1
    x3 = deserialize(h, f)
    assert x.shape == x3.shape
    assert x.dtype == x3.dtype
    assert x.strides == x3.strides
    for e_x, e_x3 in zip(x.flat, x3.flat):
        np.testing.assert_equal(e_x, e_x3)

    if protocol >= 5:
        x = np.ones(5000)

        l = []
        d = dumps(x, protocol=protocol, buffer_callback=l.append)
        assert len(l) == 1
        assert isinstance(l[0], pickle.PickleBuffer)
        assert memoryview(l[0]) == memoryview(x)
        assert (loads(d, buffers=l) == x).all()

        h, f = serialize(x, serializers=("pickle",), context=context)
        assert len(f) == 2
        assert isinstance(f[0], bytes)
        assert isinstance(f[1], memoryview)
        assert (deserialize(h, f) == x).all()


@pytest.mark.parametrize("protocol", {4, HIGHEST_PROTOCOL})
def test_pickle_functions(protocol):
    context = {"pickle-protocol": protocol}

    def make_closure():
        value = 1

        def f(x):  # closure
            return x + value

        return f

    def funcs():
        yield make_closure()
        yield (lambda x: x + 1)
        yield partial(add, 1)

    for func in funcs():
        wr = weakref.ref(func)

        func2 = loads(dumps(func, protocol=protocol))
        wr2 = weakref.ref(func2)
        assert func2(1) == func(1)

        func3 = deserialize(*serialize(func, serializers=("pickle",), context=context))
        wr3 = weakref.ref(func3)
        assert func3(1) == func(1)

        del func, func2, func3
        with profile.lock:
            assert wr() is None
            assert wr2() is None
            assert wr3() is None


@pytest.mark.skipif(
    not CLOUDPICKLE_GTE_20, reason="Pickle by value registration not supported"
)
def test_pickle_by_value_when_registered():
    with save_sys_modules():
        with tmpdir() as d:
            try:
                sys.path.insert(0, d)
                module = f"{d}/mymodule.py"
                with open(module, "w") as f:
                    f.write("def myfunc(x):\n    return x + 1")
                import mymodule  # noqa

                assert dumps(mymodule.myfunc) == pickle.dumps(
                    mymodule.myfunc, protocol=HIGHEST_PROTOCOL
                )
                cloudpickle.register_pickle_by_value(mymodule)
                assert len(dumps(mymodule.myfunc)) > len(pickle.dumps(mymodule.myfunc))

            finally:
                sys.path.pop(0)