forked from ray-project/ray
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathcomponent_failures_test.py
291 lines (245 loc) · 10.5 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
from __future__ import absolute_import
from __future__ import division
from __future__ import print_function
import os
import ray
import time
import unittest
import pyarrow as pa
class ComponentFailureTest(unittest.TestCase):
def tearDown(self):
ray.shutdown()
# This test checks that when a worker dies in the middle of a get, the
# plasma store and manager will not die.
@unittest.skipIf(
os.environ.get('RAY_USE_NEW_GCS', False),
"Not working with new GCS API.")
def testDyingWorkerGet(self):
obj_id = 20 * b"a"
@ray.remote
def f():
ray.worker.global_worker.plasma_client.get(obj_id)
ray.worker._init(
num_workers=1,
driver_mode=ray.SILENT_MODE,
start_workers_from_local_scheduler=False,
start_ray_local=True,
redirect_output=True)
# Have the worker wait in a get call.
f.remote()
# Kill the worker.
time.sleep(1)
(ray.services.all_processes[ray.services.PROCESS_TYPE_WORKER][0]
.terminate())
time.sleep(0.1)
# Seal the object so the store attempts to notify the worker that the
# get has been fulfilled.
ray.worker.global_worker.plasma_client.create(
pa.plasma.ObjectID(obj_id), 100)
ray.worker.global_worker.plasma_client.seal(pa.plasma.ObjectID(obj_id))
time.sleep(0.1)
# Make sure that nothing has died.
self.assertTrue(
ray.services.all_processes_alive(
exclude=[ray.services.PROCESS_TYPE_WORKER]))
# This test checks that when a worker dies in the middle of a wait, the
# plasma store and manager will not die.
@unittest.skipIf(
os.environ.get('RAY_USE_NEW_GCS', False),
"Not working with new GCS API.")
def testDyingWorkerWait(self):
obj_id = 20 * b"a"
@ray.remote
def f():
ray.worker.global_worker.plasma_client.wait([obj_id])
ray.worker._init(
num_workers=1,
driver_mode=ray.SILENT_MODE,
start_workers_from_local_scheduler=False,
start_ray_local=True,
redirect_output=True)
# Have the worker wait in a get call.
f.remote()
# Kill the worker.
time.sleep(1)
(ray.services.all_processes[ray.services.PROCESS_TYPE_WORKER][0]
.terminate())
time.sleep(0.1)
# Seal the object so the store attempts to notify the worker that the
# get has been fulfilled.
ray.worker.global_worker.plasma_client.create(
pa.plasma.ObjectID(obj_id), 100)
ray.worker.global_worker.plasma_client.seal(pa.plasma.ObjectID(obj_id))
time.sleep(0.1)
# Make sure that nothing has died.
self.assertTrue(
ray.services.all_processes_alive(
exclude=[ray.services.PROCESS_TYPE_WORKER]))
def _testWorkerFailed(self, num_local_schedulers):
@ray.remote
def f(x):
time.sleep(0.5)
return x
num_initial_workers = 4
ray.worker._init(
num_workers=(num_initial_workers * num_local_schedulers),
num_local_schedulers=num_local_schedulers,
start_workers_from_local_scheduler=False,
start_ray_local=True,
num_cpus=[num_initial_workers] * num_local_schedulers,
redirect_output=True)
# 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_local_schedulers)
]
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 worker in (
ray.services.all_processes[ray.services.PROCESS_TYPE_WORKER]):
worker.terminate()
time.sleep(0.1)
# Make sure that we can still get the objects after the executing tasks
# died.
ray.get(object_ids)
def testWorkerFailed(self):
self._testWorkerFailed(1)
def testWorkerFailedMultinode(self):
self._testWorkerFailed(4)
def _testComponentFailed(self, component_type):
"""Kill a component on all worker nodes and check workload succeeds."""
@ray.remote
def f(x, j):
time.sleep(0.2)
return x
# Start with 4 workers and 4 cores.
num_local_schedulers = 4
num_workers_per_scheduler = 8
ray.worker._init(
num_workers=num_workers_per_scheduler,
num_local_schedulers=num_local_schedulers,
start_ray_local=True,
num_cpus=[num_workers_per_scheduler] * num_local_schedulers,
redirect_output=True)
# Submit more tasks than there are workers so that all workers and
# cores are utilized.
object_ids = [
f.remote(i, 0)
for i in range(num_workers_per_scheduler * num_local_schedulers)
]
object_ids += [f.remote(object_id, 1) for object_id in object_ids]
object_ids += [f.remote(object_id, 2) for object_id in object_ids]
# Kill the component on all nodes except the head node as the tasks
# execute.
time.sleep(0.1)
components = ray.services.all_processes[component_type]
for process in components[1:]:
process.terminate()
time.sleep(1)
for process in components[1:]:
process.kill()
process.wait()
self.assertNotEqual(process.poll(), None)
# Make sure that we can still get the objects after the executing tasks
# died.
results = ray.get(object_ids)
expected_results = 4 * list(
range(num_workers_per_scheduler * num_local_schedulers))
self.assertEqual(results, expected_results)
def check_components_alive(self, component_type, check_component_alive):
"""Check that a given component type is alive on all worker nodes.
"""
components = ray.services.all_processes[component_type][1:]
for component in components:
if check_component_alive:
self.assertTrue(component.poll() is None)
else:
print("waiting for " + component_type + " with PID " +
str(component.pid) + "to terminate")
component.wait()
print("done waiting for " + component_type + " with PID " +
str(component.pid) + "to terminate")
self.assertTrue(not component.poll() is None)
@unittest.skipIf(
os.environ.get('RAY_USE_NEW_GCS', False), "Hanging with new GCS API.")
def testLocalSchedulerFailed(self):
# Kill all local schedulers on worker nodes.
self._testComponentFailed(ray.services.PROCESS_TYPE_LOCAL_SCHEDULER)
# The plasma stores and plasma managers should still be alive on the
# worker nodes.
self.check_components_alive(ray.services.PROCESS_TYPE_PLASMA_STORE,
True)
self.check_components_alive(ray.services.PROCESS_TYPE_PLASMA_MANAGER,
True)
self.check_components_alive(ray.services.PROCESS_TYPE_LOCAL_SCHEDULER,
False)
@unittest.skipIf(
os.environ.get('RAY_USE_NEW_GCS', False), "Hanging with new GCS API.")
def testPlasmaManagerFailed(self):
# Kill all plasma managers on worker nodes.
self._testComponentFailed(ray.services.PROCESS_TYPE_PLASMA_MANAGER)
# The plasma stores should still be alive (but unreachable) on the
# worker nodes.
self.check_components_alive(ray.services.PROCESS_TYPE_PLASMA_STORE,
True)
self.check_components_alive(ray.services.PROCESS_TYPE_PLASMA_MANAGER,
False)
self.check_components_alive(ray.services.PROCESS_TYPE_LOCAL_SCHEDULER,
False)
@unittest.skipIf(
os.environ.get('RAY_USE_NEW_GCS', False), "Hanging with new GCS API.")
def testPlasmaStoreFailed(self):
# Kill all plasma stores on worker nodes.
self._testComponentFailed(ray.services.PROCESS_TYPE_PLASMA_STORE)
# No processes should be left alive on the worker nodes.
self.check_components_alive(ray.services.PROCESS_TYPE_PLASMA_STORE,
False)
self.check_components_alive(ray.services.PROCESS_TYPE_PLASMA_MANAGER,
False)
self.check_components_alive(ray.services.PROCESS_TYPE_LOCAL_SCHEDULER,
False)
@unittest.skipIf(
os.environ.get('RAY_USE_NEW_GCS', False),
"Not working with new GCS API.")
def testDriverLivesSequential(self):
ray.worker.init(redirect_output=True)
all_processes = ray.services.all_processes
processes = [
all_processes[ray.services.PROCESS_TYPE_PLASMA_STORE][0],
all_processes[ray.services.PROCESS_TYPE_PLASMA_MANAGER][0],
all_processes[ray.services.PROCESS_TYPE_LOCAL_SCHEDULER][0],
all_processes[ray.services.PROCESS_TYPE_GLOBAL_SCHEDULER][0]
]
# Kill all the components sequentially.
for process in processes:
process.terminate()
time.sleep(0.1)
process.kill()
process.wait()
# If the driver can reach the tearDown method, then it is still alive.
@unittest.skipIf(
os.environ.get('RAY_USE_NEW_GCS', False),
"Not working with new GCS API.")
def testDriverLivesParallel(self):
ray.worker.init(redirect_output=True)
all_processes = ray.services.all_processes
processes = [
all_processes[ray.services.PROCESS_TYPE_PLASMA_STORE][0],
all_processes[ray.services.PROCESS_TYPE_PLASMA_MANAGER][0],
all_processes[ray.services.PROCESS_TYPE_LOCAL_SCHEDULER][0],
all_processes[ray.services.PROCESS_TYPE_GLOBAL_SCHEDULER][0]
]
# Kill all the components in parallel.
for process in processes:
process.terminate()
time.sleep(0.1)
for process in processes:
process.kill()
for process in processes:
process.wait()
# If the driver can reach the tearDown method, then it is still alive.
if __name__ == "__main__":
unittest.main(verbosity=2)