-
Notifications
You must be signed in to change notification settings - Fork 94
/
Copy pathtest_spill.py
412 lines (360 loc) · 13.4 KB
/
test_spill.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
import os
from time import sleep
import pytest
from zict.file import _safe_key as safe_key
import dask
from dask import array as da
from distributed import Client, get_worker, wait
from distributed.metrics import time
from distributed.sizeof import sizeof
from distributed.utils_test import gen_cluster, gen_test, loop # noqa: F401
from distributed.worker import Worker
from dask_cuda import LocalCUDACluster, utils
from dask_cuda.device_host_file import DeviceHostFile
if utils.get_device_total_memory() < 1e10:
pytest.skip("Not enough GPU memory", allow_module_level=True)
def device_host_file_size_matches(
dhf, total_bytes, device_chunk_overhead=0, serialized_chunk_overhead=1024
):
byte_sum = dhf.device_buffer.fast.total_weight
# `dhf.host_buffer.fast` is only available when Worker's `memory_limit != 0`
if hasattr(dhf.host_buffer, "fast"):
byte_sum += dhf.host_buffer.fast.total_weight
else:
byte_sum += sum([sizeof(b) for b in dhf.host_buffer.values()])
# `dhf.disk` is only available when Worker's `memory_limit != 0`
if dhf.disk is not None:
file_path = [
os.path.join(dhf.disk.directory, safe_key(k)) for k in dhf.disk.keys()
]
file_size = [os.path.getsize(f) for f in file_path]
byte_sum += sum(file_size)
# Allow up to chunk_overhead bytes overhead per chunk
device_overhead = len(dhf.device) * device_chunk_overhead
host_overhead = len(dhf.host) * serialized_chunk_overhead
disk_overhead = (
len(dhf.disk) * serialized_chunk_overhead if dhf.disk is not None else 0
)
return (
byte_sum >= total_bytes
and byte_sum <= total_bytes + device_overhead + host_overhead + disk_overhead
)
def assert_device_host_file_size(
dhf, total_bytes, device_chunk_overhead=0, serialized_chunk_overhead=1024
):
assert device_host_file_size_matches(
dhf, total_bytes, device_chunk_overhead, serialized_chunk_overhead
)
def worker_assert(total_size, device_chunk_overhead, serialized_chunk_overhead):
assert_device_host_file_size(
get_worker().data, total_size, device_chunk_overhead, serialized_chunk_overhead
)
def delayed_worker_assert(total_size, device_chunk_overhead, serialized_chunk_overhead):
start = time()
while not device_host_file_size_matches(
get_worker().data, total_size, device_chunk_overhead, serialized_chunk_overhead
):
sleep(0.01)
if time() < start + 3:
assert_device_host_file_size(
get_worker().data,
total_size,
device_chunk_overhead,
serialized_chunk_overhead,
)
# @pytest.mark.xfail(reason="https://github.com/rapidsai/dask-cuda/issues/79")
@pytest.mark.parametrize(
"params",
[
{
"device_memory_limit": 1e9,
"memory_limit": 4e9,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": False,
},
{
"device_memory_limit": 1e9,
"memory_limit": 1e9,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": True,
},
{
"device_memory_limit": 1e9,
"memory_limit": 0,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": False,
},
],
)
def test_cupy_device_spill(params):
@gen_cluster(
client=True,
nthreads=[("127.0.0.1", 1)],
Worker=Worker,
timeout=60,
worker_kwargs={
"memory_limit": params["memory_limit"],
"data": DeviceHostFile(
device_memory_limit=params["device_memory_limit"],
memory_limit=params["memory_limit"],
),
},
config={
"distributed.comm.timeouts.connect": "20s",
"distributed.worker.memory.target": params["host_target"],
"distributed.worker.memory.spill": params["host_spill"],
"distributed.worker.memory.pause": params["host_pause"],
},
)
def test_device_spill(client, scheduler, worker):
cupy = pytest.importorskip("cupy")
rs = da.random.RandomState(RandomState=cupy.random.RandomState)
x = rs.random(int(250e6), chunks=10e6)
xx = x.persist()
yield wait(xx)
# Allow up to 1024 bytes overhead per chunk serialized
yield client.run(worker_assert, x.nbytes, 1024, 1024)
y = client.compute(x.sum())
res = yield y
assert (abs(res / x.size) - 0.5) < 1e-3
yield client.run(worker_assert, x.nbytes, 1024, 1024)
host_chunks = yield client.run(lambda: len(get_worker().data.host))
disk_chunks = yield client.run(lambda: len(get_worker().data.disk or list()))
for hc, dc in zip(host_chunks.values(), disk_chunks.values()):
if params["spills_to_disk"]:
assert dc > 0
else:
assert hc > 0
assert dc == 0
test_device_spill()
@pytest.mark.parametrize(
"params",
[
{
"device_memory_limit": 1e9,
"memory_limit": 4e9,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": False,
},
{
"device_memory_limit": 1e9,
"memory_limit": 1e9,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": True,
},
{
"device_memory_limit": 1e9,
"memory_limit": 0,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": False,
},
],
)
@pytest.mark.asyncio
async def test_cupy_cluster_device_spill(params):
cupy = pytest.importorskip("cupy")
with dask.config.set({"distributed.worker.memory.terminate": False}):
async with LocalCUDACluster(
1,
scheduler_port=0,
processes=True,
silence_logs=False,
dashboard_address=None,
asynchronous=True,
death_timeout=60,
device_memory_limit=params["device_memory_limit"],
memory_limit=params["memory_limit"],
memory_target_fraction=params["host_target"],
memory_spill_fraction=params["host_spill"],
memory_pause_fraction=params["host_pause"],
) as cluster:
async with Client(cluster, asynchronous=True) as client:
rs = da.random.RandomState(RandomState=cupy.random.RandomState)
x = rs.random(int(250e6), chunks=10e6)
await wait(x)
xx = x.persist()
await wait(xx)
# Allow up to 1024 bytes overhead per chunk serialized
await client.run(worker_assert, x.nbytes, 1024, 1024)
y = client.compute(x.sum())
res = await y
assert (abs(res / x.size) - 0.5) < 1e-3
await client.run(worker_assert, x.nbytes, 1024, 1024)
host_chunks = await client.run(lambda: len(get_worker().data.host))
disk_chunks = await client.run(
lambda: len(get_worker().data.disk or list())
)
for hc, dc in zip(host_chunks.values(), disk_chunks.values()):
if params["spills_to_disk"]:
assert dc > 0
else:
assert hc > 0
assert dc == 0
@pytest.mark.xfail(reason="https://github.com/rapidsai/dask-cuda/issues/79")
@pytest.mark.parametrize(
"params",
[
{
"device_memory_limit": 1e9,
"memory_limit": 4e9,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": False,
},
{
"device_memory_limit": 1e9,
"memory_limit": 1e9,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": True,
},
{
"device_memory_limit": 1e9,
"memory_limit": 0,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": False,
},
],
)
def test_cudf_device_spill(params):
@gen_cluster(
client=True,
nthreads=[("127.0.0.1", 1)],
Worker=Worker,
timeout=60,
worker_kwargs={
"memory_limit": params["memory_limit"],
"data": DeviceHostFile(
device_memory_limit=params["device_memory_limit"],
memory_limit=params["memory_limit"],
),
},
config={
"distributed.comm.timeouts.connect": "20s",
"distributed.worker.memory.target": params["host_target"],
"distributed.worker.memory.spill": params["host_spill"],
"distributed.worker.memory.pause": params["host_pause"],
},
)
def test_device_spill(client, scheduler, worker):
cudf = pytest.importorskip("cudf")
# There's a known issue with datetime64:
# https://github.com/numpy/numpy/issues/4983#issuecomment-441332940
# The same error above happens when spilling datetime64 to disk
cdf = (
dask.datasets.timeseries(dtypes={"x": int, "y": float}, freq="20ms")
.reset_index(drop=True)
.map_partitions(cudf.from_pandas)
)
sizes = yield client.compute(cdf.map_partitions(lambda df: df.__sizeof__()))
sizes = sizes.tolist()
nbytes = sum(sizes)
part_index_nbytes = (yield client.compute(cdf.partitions[0].index)).__sizeof__()
cdf2 = cdf.persist()
yield wait(cdf2)
del cdf
host_chunks = yield client.run(lambda: len(get_worker().data.host))
disk_chunks = yield client.run(lambda: len(get_worker().data.disk or list()))
for hc, dc in zip(host_chunks.values(), disk_chunks.values()):
if params["spills_to_disk"]:
assert dc > 0
else:
assert hc > 0
assert dc == 0
yield client.run(worker_assert, nbytes, 32, 2048 + part_index_nbytes)
del cdf2
yield client.run(delayed_worker_assert, 0, 0, 0)
test_device_spill()
@pytest.mark.xfail(reason="https://github.com/rapidsai/dask-cuda/issues/79")
@pytest.mark.parametrize(
"params",
[
{
"device_memory_limit": 1e9,
"memory_limit": 4e9,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": False,
},
{
"device_memory_limit": 1e9,
"memory_limit": 1e9,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": True,
},
{
"device_memory_limit": 1e9,
"memory_limit": 0,
"host_target": 0.0,
"host_spill": 0.0,
"host_pause": None,
"spills_to_disk": False,
},
],
)
@pytest.mark.asyncio
async def test_cudf_cluster_device_spill(params):
cudf = pytest.importorskip("cudf")
with dask.config.set({"distributed.worker.memory.terminate": False}):
async with LocalCUDACluster(
1,
device_memory_limit=params["device_memory_limit"],
memory_limit=params["memory_limit"],
memory_target_fraction=params["host_target"],
memory_spill_fraction=params["host_spill"],
memory_pause_fraction=params["host_pause"],
death_timeout=60,
asynchronous=True,
) as cluster:
async with Client(cluster, asynchronous=True) as client:
# There's a known issue with datetime64:
# https://github.com/numpy/numpy/issues/4983#issuecomment-441332940
# The same error above happens when spilling datetime64 to disk
cdf = (
dask.datasets.timeseries(dtypes={"x": int, "y": float}, freq="20ms")
.reset_index(drop=True)
.map_partitions(cudf.from_pandas)
)
sizes = await client.compute(
cdf.map_partitions(lambda df: df.__sizeof__())
)
sizes = sizes.tolist()
nbytes = sum(sizes)
part_index_nbytes = (
await client.compute(cdf.partitions[0].index)
).__sizeof__()
cdf2 = cdf.persist()
await wait(cdf2)
del cdf
host_chunks = await client.run(lambda: len(get_worker().data.host))
disk_chunks = await client.run(
lambda: len(get_worker().data.disk or list())
)
for hc, dc in zip(host_chunks.values(), disk_chunks.values()):
if params["spills_to_disk"]:
assert dc > 0
else:
assert hc > 0
assert dc == 0
await client.run(worker_assert, nbytes, 32, 2048 + part_index_nbytes)
del cdf2
await client.run(delayed_worker_assert, 0, 0, 0)