ray/test/jenkins_tests/multi_node_tests/many_drivers_test.py
Robert Nishihara 0ac125e9b2 Clean up when a driver disconnects. (#462)
* Clean up state when drivers exit.

* Remove unnecessary field in ActorMapEntry struct.

* Have monitor release GPU resources in Redis when driver exits.

* Enable multiple drivers in multi-node tests and test driver cleanup.

* Make redis GPU allocation a redis transaction and small cleanups.

* Fix multi-node test.

* Small cleanups.

* Make global scheduler take node_ip_address so it appears in the right place in the client table.

* Cleanups.

* Fix linting and cleanups in local scheduler.

* Fix removed_driver_test.

* Fix bug related to vector -> list.

* Fix linting.

* Cleanup.

* Fix multi node tests.

* Fix jenkins tests.

* Add another multi node test with many drivers.

* Fix linting.

* Make the actor creation notification a flatbuffer message.

* Revert "Make the actor creation notification a flatbuffer message."

This reverts commit af99099c8084dbf9177fb4e34c0c9b1a12c78f39.

* Add comment explaining flatbuffer problems.
2017-04-24 18:10:21 -07:00

80 lines
2.6 KiB
Python

from __future__ import absolute_import
from __future__ import division
from __future__ import print_function
import os
import time
import ray
from ray.test.multi_node_tests import (_wait_for_nodes_to_join,
_broadcast_event,
_wait_for_event)
# This test should be run with 5 nodes, which have 0, 0, 5, 6, and 50 GPUs for
# a total of 61 GPUs. It should be run with a large number of drivers (e.g.,
# 100). At most 10 drivers will run at a time, and each driver will use at most
# 5 GPUs (this is ceil(61 / 15), which guarantees that we will always be able
# to make progress).
total_num_nodes = 5
max_concurrent_drivers = 15
num_gpus_per_driver = 5
@ray.actor(num_gpus=1)
class Actor1(object):
def __init__(self):
assert len(ray.get_gpu_ids()) == 1
def check_ids(self):
assert len(ray.get_gpu_ids()) == 1
def driver(redis_address, driver_index):
"""The script for driver 0.
This driver should create five actors that each use one GPU and some actors
that use no GPUs. After a while, it should exit.
"""
ray.init(redis_address=redis_address)
# Wait for all the nodes to join the cluster.
_wait_for_nodes_to_join(total_num_nodes)
# Limit the number of drivers running concurrently.
for i in range(driver_index - max_concurrent_drivers + 1):
_wait_for_event("DRIVER_{}_DONE".format(i), redis_address)
def try_to_create_actor(actor_class, timeout=100):
# Try to create an actor, but allow failures while we wait for the monitor
# to release the resources for the removed drivers.
start_time = time.time()
while time.time() - start_time < timeout:
try:
actor = actor_class()
except Exception as e:
time.sleep(0.1)
else:
return actor
# If we are here, then we timed out while looping.
raise Exception("Timed out while trying to create actor.")
# Create some actors that require one GPU.
actors_one_gpu = []
for _ in range(num_gpus_per_driver):
actors_one_gpu.append(try_to_create_actor(Actor1))
for _ in range(100):
ray.get([actor.check_ids() for actor in actors_one_gpu])
_broadcast_event("DRIVER_{}_DONE".format(driver_index), redis_address)
if __name__ == "__main__":
driver_index = int(os.environ["RAY_DRIVER_INDEX"])
redis_address = os.environ["RAY_REDIS_ADDRESS"]
print("Driver {} started at {}.".format(driver_index, time.time()))
# In this test, all drivers will run the same script.
driver(redis_address, driver_index)
print("Driver {} finished at {}.".format(driver_index, time.time()))