forked from ray-project/ray
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathcomponent_failures_test.py
472 lines (381 loc) · 14.3 KB
/
component_failures_test.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
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
from __future__ import absolute_import
from __future__ import division
from __future__ import print_function
import os
import json
import signal
import sys
import time
import numpy as np
import pytest
import ray
import ray.ray_constants as ray_constants
from ray.test.cluster_utils import Cluster
from ray.test.test_utils import run_string_as_driver_nonblocking
@pytest.fixture
def shutdown_only():
yield None
# The code after the yield will run as teardown code.
ray.shutdown()
@pytest.fixture
def ray_start_cluster():
node_args = {
"num_cpus": 4,
"_internal_config": json.dumps({
"initial_reconstruction_timeout_milliseconds": 1000,
"num_heartbeats_timeout": 10
})
}
# Start with 3 worker nodes and 4 cores each.
cluster = Cluster(
initialize_head=True, connect=True, head_node_args=node_args)
workers = []
for _ in range(3):
workers.append(cluster.add_node(**node_args))
cluster.wait_for_nodes()
yield cluster
ray.shutdown()
cluster.shutdown()
# This test checks that when a worker dies in the middle of a get, the plasma
# store and raylet will not die.
@pytest.mark.skipif(
os.environ.get("RAY_USE_NEW_GCS") == "on",
reason="Not working with new GCS API.")
def test_dying_worker_get(shutdown_only):
# Start the Ray processes.
ray.init(num_cpus=2)
@ray.remote
def sleep_forever():
time.sleep(10**6)
@ray.remote
def get_worker_pid():
return os.getpid()
x_id = sleep_forever.remote()
time.sleep(0.01) # Try to wait for the sleep task to get scheduled.
# Get the PID of the other worker.
worker_pid = ray.get(get_worker_pid.remote())
@ray.remote
def f(id_in_a_list):
ray.get(id_in_a_list[0])
# Have the worker wait in a get call.
result_id = f.remote([x_id])
time.sleep(1)
# Make sure the task hasn't finished.
ready_ids, _ = ray.wait([result_id], timeout=0)
assert len(ready_ids) == 0
# Kill the worker.
os.kill(worker_pid, signal.SIGKILL)
time.sleep(0.1)
# Make sure the sleep task hasn't finished.
ready_ids, _ = ray.wait([x_id], timeout=0)
assert len(ready_ids) == 0
# Seal the object so the store attempts to notify the worker that the
# get has been fulfilled.
ray.worker.global_worker.put_object(x_id, 1)
time.sleep(0.1)
# Make sure that nothing has died.
assert ray.services.remaining_processes_alive()
# This test checks that when a driver dies in the middle of a get, the plasma
# store and raylet will not die.
@pytest.mark.skipif(
os.environ.get("RAY_USE_NEW_GCS") == "on",
reason="Not working with new GCS API.")
def test_dying_driver_get(shutdown_only):
# Start the Ray processes.
address_info = ray.init(num_cpus=1)
@ray.remote
def sleep_forever():
time.sleep(10**6)
x_id = sleep_forever.remote()
driver = """
import ray
ray.init("{}")
ray.get(ray.ObjectID(ray.utils.hex_to_binary("{}")))
""".format(address_info["redis_address"], x_id.hex())
p = run_string_as_driver_nonblocking(driver)
# Make sure the driver is running.
time.sleep(1)
assert p.poll() is None
# Kill the driver process.
p.kill()
p.wait()
time.sleep(0.1)
# Make sure the original task hasn't finished.
ready_ids, _ = ray.wait([x_id], timeout=0)
assert len(ready_ids) == 0
# Seal the object so the store attempts to notify the worker that the
# get has been fulfilled.
ray.worker.global_worker.put_object(x_id, 1)
time.sleep(0.1)
# Make sure that nothing has died.
assert ray.services.remaining_processes_alive()
# This test checks that when a worker dies in the middle of a wait, the plasma
# store and raylet will not die.
@pytest.mark.skipif(
os.environ.get("RAY_USE_NEW_GCS") == "on",
reason="Not working with new GCS API.")
def test_dying_worker_wait(shutdown_only):
ray.init(num_cpus=2)
@ray.remote
def sleep_forever():
time.sleep(10**6)
@ray.remote
def get_pid():
return os.getpid()
x_id = sleep_forever.remote()
# Get the PID of the worker that block_in_wait will run on (sleep a little
# to make sure that sleep_forever has already started).
time.sleep(0.1)
worker_pid = ray.get(get_pid.remote())
@ray.remote
def block_in_wait(object_id_in_list):
ray.wait(object_id_in_list)
# Have the worker wait in a wait call.
block_in_wait.remote([x_id])
time.sleep(0.1)
# Kill the worker.
os.kill(worker_pid, signal.SIGKILL)
time.sleep(0.1)
# Create the object.
ray.worker.global_worker.put_object(x_id, 1)
time.sleep(0.1)
# Make sure that nothing has died.
assert ray.services.remaining_processes_alive()
# This test checks that when a driver dies in the middle of a wait, the plasma
# store and raylet will not die.
@pytest.mark.skipif(
os.environ.get("RAY_USE_NEW_GCS") == "on",
reason="Not working with new GCS API.")
def test_dying_driver_wait(shutdown_only):
# Start the Ray processes.
address_info = ray.init(num_cpus=1)
@ray.remote
def sleep_forever():
time.sleep(10**6)
x_id = sleep_forever.remote()
driver = """
import ray
ray.init("{}")
ray.wait([ray.ObjectID(ray.utils.hex_to_binary("{}"))])
""".format(address_info["redis_address"], x_id.hex())
p = run_string_as_driver_nonblocking(driver)
# Make sure the driver is running.
time.sleep(1)
assert p.poll() is None
# Kill the driver process.
p.kill()
p.wait()
time.sleep(0.1)
# Make sure the original task hasn't finished.
ready_ids, _ = ray.wait([x_id], timeout=0)
assert len(ready_ids) == 0
# Seal the object so the store attempts to notify the worker that the
# wait can return.
ray.worker.global_worker.put_object(x_id, 1)
time.sleep(0.1)
# Make sure that nothing has died.
assert ray.services.remaining_processes_alive()
@pytest.fixture(params=[(1, 4), (4, 4)])
def ray_start_workers_separate_multinode(request):
num_nodes = request.param[0]
num_initial_workers = request.param[1]
# Start the Ray processes.
cluster = Cluster()
for _ in range(num_nodes):
cluster.add_node(num_cpus=num_initial_workers)
ray.init(redis_address=cluster.redis_address)
yield num_nodes, num_initial_workers
# The code after the yield will run as teardown code.
ray.shutdown()
cluster.shutdown()
def test_worker_failed(ray_start_workers_separate_multinode):
num_nodes, num_initial_workers = (ray_start_workers_separate_multinode)
@ray.remote
def get_pids():
time.sleep(0.25)
return os.getpid()
start_time = time.time()
pids = set()
while len(pids) < num_nodes * num_initial_workers:
new_pids = ray.get([
get_pids.remote()
for _ in range(2 * num_nodes * num_initial_workers)
])
for pid in new_pids:
pids.add(pid)
if time.time() - start_time > 60:
raise Exception("Timed out while waiting to get worker PIDs.")
@ray.remote
def f(x):
time.sleep(0.5)
return x
# Submit more tasks than there are workers so that all workers and
# cores are utilized.
object_ids = [f.remote(i) for i in range(num_initial_workers * num_nodes)]
object_ids += [f.remote(object_id) for object_id in object_ids]
# Allow the tasks some time to begin executing.
time.sleep(0.1)
# Kill the workers as the tasks execute.
for pid in pids:
os.kill(pid, signal.SIGKILL)
time.sleep(0.1)
# Make sure that we either get the object or we get an appropriate
# exception.
for object_id in object_ids:
try:
ray.get(object_id)
except (ray.exceptions.RayTaskError, ray.exceptions.RayWorkerError):
pass
@pytest.fixture
def ray_initialize_cluster():
# Start with 4 workers and 4 cores.
num_nodes = 4
num_workers_per_scheduler = 8
cluster = Cluster()
for _ in range(num_nodes):
cluster.add_node(
num_cpus=num_workers_per_scheduler,
_internal_config=json.dumps({
"initial_reconstruction_timeout_milliseconds": 1000,
"num_heartbeats_timeout": 10,
}))
ray.init(redis_address=cluster.redis_address)
yield cluster
ray.shutdown()
cluster.shutdown()
def _test_component_failed(cluster, component_type):
"""Kill a component on all worker nodes and check workload succeeds."""
# Submit many tasks with many dependencies.
@ray.remote
def f(x):
return x
@ray.remote
def g(*xs):
return 1
# Kill the component on all nodes except the head node as the tasks
# execute. Do this in a loop while submitting tasks between each
# component failure.
time.sleep(0.1)
worker_nodes = cluster.list_all_nodes()[1:]
assert len(worker_nodes) > 0
for node in worker_nodes:
process = node.all_processes[component_type][0].process
# Submit a round of tasks with many dependencies.
x = 1
for _ in range(1000):
x = f.remote(x)
xs = [g.remote(1)]
for _ in range(100):
xs.append(g.remote(*xs))
xs.append(g.remote(1))
# Kill a component on one of the nodes.
process.terminate()
time.sleep(1)
process.kill()
process.wait()
assert not process.poll() is None
# Make sure that we can still get the objects after the
# executing tasks died.
ray.get(x)
ray.get(xs)
def check_components_alive(cluster, component_type, check_component_alive):
"""Check that a given component type is alive on all worker nodes."""
worker_nodes = cluster.list_all_nodes()[1:]
assert len(worker_nodes) > 0
for node in worker_nodes:
process = node.all_processes[component_type][0].process
if check_component_alive:
assert process.poll() is None
else:
print("waiting for " + component_type + " with PID " +
str(process.pid) + "to terminate")
process.wait()
print("done waiting for " + component_type + " with PID " +
str(process.pid) + "to terminate")
assert not process.poll() is None
def test_raylet_failed(ray_initialize_cluster):
cluster = ray_initialize_cluster
# Kill all local schedulers on worker nodes.
_test_component_failed(cluster, ray_constants.PROCESS_TYPE_RAYLET)
# The plasma stores should still be alive on the worker nodes.
check_components_alive(cluster, ray_constants.PROCESS_TYPE_PLASMA_STORE,
True)
@pytest.mark.skipif(
os.environ.get("RAY_USE_NEW_GCS") == "on",
reason="Hanging with new GCS API.")
def test_plasma_store_failed(ray_initialize_cluster):
cluster = ray_initialize_cluster
# Kill all plasma stores on worker nodes.
_test_component_failed(cluster, ray_constants.PROCESS_TYPE_PLASMA_STORE)
# No processes should be left alive on the worker nodes.
check_components_alive(cluster, ray_constants.PROCESS_TYPE_PLASMA_STORE,
False)
check_components_alive(cluster, ray_constants.PROCESS_TYPE_RAYLET, False)
def test_actor_creation_node_failure(ray_start_cluster):
# TODO(swang): Refactor test_raylet_failed, etc to reuse the below code.
cluster = ray_start_cluster
@ray.remote
class Child(object):
def __init__(self, death_probability):
self.death_probability = death_probability
def ping(self):
# Exit process with some probability.
exit_chance = np.random.rand()
if exit_chance < self.death_probability:
sys.exit(-1)
num_children = 50
# Children actors will die about half the time.
death_probability = 0.5
children = [Child.remote(death_probability) for _ in range(num_children)]
while len(cluster.list_all_nodes()) > 1:
for j in range(2):
# Submit some tasks on the actors. About half of the actors will
# fail.
children_out = [child.ping.remote() for child in children]
# Wait a while for all the tasks to complete. This should trigger
# reconstruction for any actor creation tasks that were forwarded
# to nodes that then failed.
ready, _ = ray.wait(
children_out, num_returns=len(children_out), timeout=5 * 60.0)
assert len(ready) == len(children_out)
# Replace any actors that died.
for i, out in enumerate(children_out):
try:
ray.get(out)
except ray.exceptions.RayActorError:
children[i] = Child.remote(death_probability)
# Remove a node. Any actor creation tasks that were forwarded to this
# node must be reconstructed.
cluster.remove_node(cluster.list_all_nodes()[-1])
@pytest.mark.skipif(
os.environ.get("RAY_USE_NEW_GCS") == "on",
reason="Hanging with new GCS API.")
def test_driver_lives_sequential(shutdown_only):
ray.init(num_cpus=1)
ray.worker._global_node.kill_raylet()
ray.worker._global_node.kill_plasma_store()
ray.worker._global_node.kill_log_monitor()
ray.worker._global_node.kill_monitor()
ray.worker._global_node.kill_raylet_monitor()
# If the driver can reach the tearDown method, then it is still alive.
@pytest.mark.skipif(
os.environ.get("RAY_USE_NEW_GCS") == "on",
reason="Hanging with new GCS API.")
def test_driver_lives_parallel(shutdown_only):
ray.init(num_cpus=1)
all_processes = ray.worker._global_node.all_processes
process_infos = (all_processes[ray_constants.PROCESS_TYPE_PLASMA_STORE] +
all_processes[ray_constants.PROCESS_TYPE_RAYLET] +
all_processes[ray_constants.PROCESS_TYPE_LOG_MONITOR] +
all_processes[ray_constants.PROCESS_TYPE_MONITOR] +
all_processes[ray_constants.PROCESS_TYPE_RAYLET_MONITOR])
assert len(process_infos) == 5
# Kill all the components in parallel.
for process_info in process_infos:
process_info.process.terminate()
time.sleep(0.1)
for process_info in process_infos:
process_info.process.kill()
for process_info in process_infos:
process_info.process.wait()
# If the driver can reach the tearDown method, then it is still alive.