Skip to content

Commit

Permalink
General attribute-based heterogeneity support with hard and soft cons…
Browse files Browse the repository at this point in the history
…traints (ray-project#248)

* attribute-based heterogeneity-awareness in global scheduler and photon

* minor post-rebase fix

* photon: enforce dynamic capacity constraint on task dispatch

* globalsched: cap the number of times we try to schedule a task in round robin

* propagating ability to specify resource capacity to ray.init

* adding resources to remote function export and fetch/register

* globalsched: remove unused functions; update cached photon resource capacity (until next photon heartbeat)

* Add some integration tests.

* globalsched: cleanup + factor out constraint checking

* lots of style

* task_spec_required_resource: global refactor

* clang format

* clang format + comment update in photon

* clang format photon comment

* valgrind

* reduce verbosity for Travis

* Add test for scheduler load balancing.

* addressing comments

* refactoring global scheduler algorithm

* Minor cleanups.

* Linting.

* Fix array_test.py and linting.

* valgrind fix for photon tests

* Attempt to fix stress tests.

* fix hashmap free

* fix hashmap free comment

* memset photon resource vectors to 0 in case they get used before the first heartbeat

* More whitespace changes.

* Undo whitespace error I introduced.
  • Loading branch information
atumanov authored and robertnishihara committed Feb 9, 2017
1 parent 1a7e1c4 commit dfb6107
Show file tree
Hide file tree
Showing 22 changed files with 1,035 additions and 224 deletions.
97 changes: 65 additions & 32 deletions python/global_scheduler/test/test.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
USE_VALGRIND = False
PLASMA_STORE_MEMORY = 1000000000
ID_SIZE = 20
NUM_CLUSTER_NODES = 2

# These constants must match the scheduling state enum in task.h.
TASK_STATUS_WAITING = 1
Expand All @@ -43,13 +44,16 @@ def random_task_id():
def random_function_id():
return photon.ObjectID(np.random.bytes(ID_SIZE))

def random_object_id():
return photon.ObjectID(np.random.bytes(ID_SIZE))

def new_port():
return random.randint(10000, 65535)

class TestGlobalScheduler(unittest.TestCase):

def setUp(self):
# Start a Redis server.
# Start one Redis server and N pairs of (plasma, photon)
redis_path = os.path.join(os.path.abspath(os.path.dirname(__file__)), "../../core/src/common/thirdparty/redis/src/redis-server")
redis_module = os.path.join(os.path.dirname(os.path.abspath(__file__)), "../../core/src/common/redis_module/libray_redis_module.so")
assert os.path.isfile(redis_path)
Expand All @@ -61,29 +65,47 @@ def setUp(self):
time.sleep(0.1)
# Create a Redis client.
self.redis_client = redis.StrictRedis(host=node_ip_address, port=redis_port)
# Start the global scheduler.
# Start one global scheduler.
self.p1 = global_scheduler.start_global_scheduler(redis_address, use_valgrind=USE_VALGRIND)
# Start the Plasma store.
plasma_store_name, self.p2 = plasma.start_plasma_store()
# Start the Plasma manager.
plasma_manager_name, self.p3, plasma_manager_port = plasma.start_plasma_manager(plasma_store_name, redis_address)
self.plasma_address = "{}:{}".format(node_ip_address, plasma_manager_port)
self.plasma_client = plasma.PlasmaClient(plasma_store_name, plasma_manager_name)
# Start the local scheduler.
local_scheduler_name, self.p4 = photon.start_local_scheduler(
plasma_store_name,
plasma_manager_name=plasma_manager_name,
plasma_address=self.plasma_address,
redis_address=redis_address)
# Connect to the scheduler.
self.photon_client = photon.PhotonClient(local_scheduler_name)
self.plasma_store_pids = []
self.plasma_manager_pids = []
self.local_scheduler_pids = []
self.plasma_clients = []
self.photon_clients = []

for i in range(NUM_CLUSTER_NODES):
# Start the Plasma store. Plasma store name is randomly generated.
plasma_store_name, p2 = plasma.start_plasma_store()
self.plasma_store_pids.append(p2)
# Start the Plasma manager.
# Assumption: Plasma manager name and port are randomly generated by the plasma module.
plasma_manager_name, p3, plasma_manager_port = plasma.start_plasma_manager(plasma_store_name, redis_address)
self.plasma_manager_pids.append(p3)
plasma_address = "{}:{}".format(node_ip_address, plasma_manager_port)
plasma_client = plasma.PlasmaClient(plasma_store_name, plasma_manager_name)
self.plasma_clients.append(plasma_client)
# Start the local scheduler.
local_scheduler_name, p4 = photon.start_local_scheduler(
plasma_store_name,
plasma_manager_name=plasma_manager_name,
plasma_address=plasma_address,
redis_address=redis_address,
static_resource_list=[None, 0])
# Connect to the scheduler.
photon_client = photon.PhotonClient(local_scheduler_name)
self.photon_clients.append(photon_client)
self.local_scheduler_pids.append(p4)

def tearDown(self):
# Check that the processes are still alive.
self.assertEqual(self.p1.poll(), None)
self.assertEqual(self.p2.poll(), None)
self.assertEqual(self.p3.poll(), None)
self.assertEqual(self.p4.poll(), None)
for p2 in self.plasma_store_pids:
self.assertEqual(p2.poll(), None)
for p3 in self.plasma_manager_pids:
self.assertEqual(p3.poll(), None)
for p4 in self.local_scheduler_pids:
self.assertEqual(p4.poll(), None)

self.assertEqual(self.redis_process.poll(), None)

# Kill the global scheduler.
Expand All @@ -94,9 +116,10 @@ def tearDown(self):
os._exit(-1)
else:
self.p1.kill()
self.p2.kill()
self.p3.kill()
self.p4.kill()
# Kill local schedulers, plasma managers, and plasma stores.
map(subprocess.Popen.kill, self.local_scheduler_pids)
map(subprocess.Popen.kill, self.plasma_manager_pids)
map(subprocess.Popen.kill, self.plasma_store_pids)
# Kill Redis. In the event that we are using valgrind, this needs to happen
# after we kill the global scheduler.
self.redis_process.kill()
Expand All @@ -123,15 +146,21 @@ def get_plasma_manager_id(self):

return db_client_id

def test_task_default_resources(self):
task1 = photon.Task(random_driver_id(), random_function_id(), [random_object_id()], 0, random_task_id(), 0)
self.assertEqual(task1.required_resources(), [1.0, 0.0])
task2 = photon.Task(random_driver_id(), random_function_id(), [random_object_id()], 0, random_task_id(), 0, [1.0, 2.0])
self.assertEqual(task2.required_resources(), [1.0, 2.0])

def test_redis_only_single_task(self):
"""
Tests global scheduler functionality by interacting with Redis and checking
task state transitions in Redis only. TODO(atumanov): implement.
"""
# Check precondition for this test:
# There should be three db clients, the global scheduler, the local
# scheduler, and the plasma manager.
self.assertEqual(len(self.redis_client.keys("{}*".format(DB_CLIENT_PREFIX))), 3)
# There should be 2n+1 db clients: the global scheduler + one photon and one plasma per node.
self.assertEqual(len(self.redis_client.keys("{}*".format(DB_CLIENT_PREFIX))),
2 * NUM_CLUSTER_NODES + 1)
db_client_id = self.get_plasma_manager_id()
assert(db_client_id != None)
assert(db_client_id.startswith(b"CL:"))
Expand All @@ -140,21 +169,23 @@ def test_redis_only_single_task(self):
def test_integration_single_task(self):
# There should be three db clients, the global scheduler, the local
# scheduler, and the plasma manager.
self.assertEqual(len(self.redis_client.keys("{}*".format(DB_CLIENT_PREFIX))), 3)
self.assertEqual(len(self.redis_client.keys("{}*".format(DB_CLIENT_PREFIX))),
2 * NUM_CLUSTER_NODES + 1)

num_return_vals = [0, 1, 2, 3, 5, 10]
# There should not be anything else in Redis yet.
self.assertEqual(len(self.redis_client.keys("*")), 3)
self.assertEqual(len(self.redis_client.keys("*")), 2 * NUM_CLUSTER_NODES + 1)
# Insert the object into Redis.
data_size = 0xf1f0
metadata_size = 0x40
object_dep, memory_buffer, metadata = create_object(self.plasma_client, data_size, metadata_size, seal=True)
plasma_client = self.plasma_clients[0]
object_dep, memory_buffer, metadata = create_object(plasma_client, data_size, metadata_size, seal=True)

# Sleep before submitting task to photon.
time.sleep(0.1)
# Submit a task to Redis.
task = photon.Task(random_driver_id(), random_function_id(), [photon.ObjectID(object_dep)], num_return_vals[0], random_task_id(), 0)
self.photon_client.submit(task)
self.photon_clients[0].submit(task)
time.sleep(0.1)
# There should now be a task in Redis, and it should get assigned to the
# local scheduler
Expand Down Expand Up @@ -184,7 +215,8 @@ def test_integration_single_task(self):
def integration_many_tasks_helper(self, timesync=True):
# There should be three db clients, the global scheduler, the local
# scheduler, and the plasma manager.
self.assertEqual(len(self.redis_client.keys("{}*".format(DB_CLIENT_PREFIX))), 3)
self.assertEqual(len(self.redis_client.keys("{}*".format(DB_CLIENT_PREFIX))),
2 * NUM_CLUSTER_NODES + 1)
num_return_vals = [0, 1, 2, 3, 5, 10]

# Submit a bunch of tasks to Redis.
Expand All @@ -193,12 +225,13 @@ def integration_many_tasks_helper(self, timesync=True):
# Create a new object for each task.
data_size = np.random.randint(1 << 20)
metadata_size = np.random.randint(1 << 10)
object_dep, memory_buffer, metadata = create_object(self.plasma_client, data_size, metadata_size, seal=True)
plasma_client = self.plasma_clients[0]
object_dep, memory_buffer, metadata = create_object(plasma_client, data_size, metadata_size, seal=True)
if timesync:
# Give 10ms for object info handler to fire (long enough to yield CPU).
time.sleep(0.010)
task = photon.Task(random_driver_id(), random_function_id(), [photon.ObjectID(object_dep)], num_return_vals[0], random_task_id(), 0)
self.photon_client.submit(task)
self.photon_clients[0].submit(task)
# Check that there are the correct number of tasks in Redis and that they
# all get assigned to the local scheduler.
num_retries = 10
Expand Down
24 changes: 23 additions & 1 deletion python/photon/photon_services.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@
from __future__ import division
from __future__ import print_function

import multiprocessing
import os
import random
import subprocess
Expand All @@ -14,7 +15,7 @@ def start_local_scheduler(plasma_store_name, plasma_manager_name=None,
worker_path=None, plasma_address=None,
node_ip_address="127.0.0.1", redis_address=None,
use_valgrind=False, use_profiler=False,
redirect_output=False):
redirect_output=False, static_resource_list=None):
"""Start a local scheduler process.
Args:
Expand All @@ -37,6 +38,9 @@ def start_local_scheduler(plasma_store_name, plasma_manager_name=None,
profiler. If this is True, use_valgrind must be False.
redirect_output (bool): True if stdout and stderr should be redirected to
/dev/null.
static_resource_list (list): A list of integers specifying the local
scheduler's resource capacities. The resources should appear in an order
matching the order defined in task.h.
Return:
A tuple of the name of the local scheduler socket and the process ID of the
Expand Down Expand Up @@ -71,6 +75,24 @@ def start_local_scheduler(plasma_store_name, plasma_manager_name=None,
command += ["-r", redis_address]
if plasma_address is not None:
command += ["-a", plasma_address]
# We want to be able to support independently setting capacity for each of the
# supported resource types. Thus, the list can be None or contain any number
# of None values.
if static_resource_list is None:
static_resource_list = [None, None]
if static_resource_list[0] is None:
# By default, use the number of hardware execution threads for the number of
# cores.
static_resource_list[0] = multiprocessing.cpu_count()
if static_resource_list[1] is None:
# By default, do not configure any GPUs on this node.
static_resource_list[1] = 0
# Pass the resource capacity string to the photon scheduler in all cases.
# Sanity check to make sure all resource capacities in the list are numeric
# (int or float).
assert(all([x == int or x == float for x in map(type, static_resource_list)]))
command += ["-c", ",".join(map(str, static_resource_list))]

with open(os.devnull, "w") as FNULL:
stdout = FNULL if redirect_output else None
stderr = FNULL if redirect_output else None
Expand Down
2 changes: 2 additions & 0 deletions python/plasma/plasma.py
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,8 @@ def __init__(self, store_socket_name, manager_socket_name=None, release_delay=64
store_socket_name (str): Name of the socket the plasma store is listening at.
manager_socket_name (str): Name of the socket the plasma manager is listening at.
"""
self.store_socket_name = store_socket_name
self.manager_socket_name = manager_socket_name
self.alive = True

if manager_socket_name is not None:
Expand Down
44 changes: 36 additions & 8 deletions python/ray/services.py
Original file line number Diff line number Diff line change
Expand Up @@ -264,7 +264,8 @@ def start_global_scheduler(redis_address, cleanup=True, redirect_output=False):

def start_local_scheduler(redis_address, node_ip_address, plasma_store_name,
plasma_manager_name, worker_path, plasma_address=None,
cleanup=True, redirect_output=False):
cleanup=True, redirect_output=False,
static_resource_list=None):
"""Start a local scheduler process.
Args:
Expand All @@ -281,6 +282,8 @@ def start_local_scheduler(redis_address, node_ip_address, plasma_store_name,
that imported services exits.
redirect_output (bool): True if stdout and stderr should be redirected to
/dev/null.
static_resource_list (list): An ordered list of the configured resource
capacities for this local scheduler.
Return:
The name of the local scheduler socket.
Expand All @@ -292,7 +295,8 @@ def start_local_scheduler(redis_address, node_ip_address, plasma_store_name,
redis_address=redis_address,
plasma_address=plasma_address,
use_profiler=RUN_PHOTON_PROFILER,
redirect_output=redirect_output)
redirect_output=redirect_output,
static_resource_list=static_resource_list)
if cleanup:
all_processes[PROCESS_TYPE_LOCAL_SCHEDULER].append(p)
return local_scheduler_name
Expand Down Expand Up @@ -386,7 +390,9 @@ def start_ray_processes(address_info=None,
cleanup=True,
redirect_output=False,
include_global_scheduler=False,
include_redis=False):
include_redis=False,
num_cpus=None,
num_gpus=None):
"""Helper method to start Ray processes.
Args:
Expand All @@ -411,11 +417,22 @@ def start_ray_processes(address_info=None,
start a global scheduler process.
include_redis (bool): If include_redis is True, then start a Redis server
process.
num_cpus: A list of length num_local_schedulers containing the number of
CPUs each local scheduler should be configured with.
num_gpus: A list of length num_local_schedulers containing the number of
GPUs each local scheduler should be configured with.
Returns:
A dictionary of the address information for the processes that were
started.
"""
if not isinstance(num_cpus, list):
num_cpus = num_local_schedulers * [num_cpus]
if not isinstance(num_gpus, list):
num_gpus = num_local_schedulers * [num_gpus]
assert len(num_cpus) == num_local_schedulers
assert len(num_gpus) == num_local_schedulers

if address_info is None:
address_info = {}
address_info["node_ip_address"] = node_ip_address
Expand Down Expand Up @@ -486,7 +503,8 @@ def start_ray_processes(address_info=None,
worker_path,
plasma_address=plasma_address,
cleanup=cleanup,
redirect_output=redirect_output)
redirect_output=redirect_output,
static_resource_list=[num_cpus[i], num_gpus[i]])
local_scheduler_socket_names.append(local_scheduler_name)
time.sleep(0.1)

Expand Down Expand Up @@ -517,7 +535,9 @@ def start_ray_node(node_ip_address,
num_local_schedulers=1,
worker_path=None,
cleanup=True,
redirect_output=False):
redirect_output=False,
num_cpus=None,
num_gpus=None):
"""Start the Ray processes for a single node.
This assumes that the Ray processes on some master node have already been
Expand Down Expand Up @@ -550,15 +570,19 @@ def start_ray_node(node_ip_address,
num_local_schedulers=num_local_schedulers,
worker_path=worker_path,
cleanup=cleanup,
redirect_output=redirect_output)
redirect_output=redirect_output,
num_cpus=num_cpus,
num_gpus=num_gpus)

def start_ray_head(address_info=None,
node_ip_address="127.0.0.1",
num_workers=0,
num_local_schedulers=1,
worker_path=None,
cleanup=True,
redirect_output=False):
redirect_output=False,
num_cpus=None,
num_gpus=None):
"""Start Ray in local mode.
Args:
Expand All @@ -579,6 +603,8 @@ def start_ray_head(address_info=None,
method exits.
redirect_output (bool): True if stdout and stderr should be redirected to
/dev/null.
num_cpus (int): number of cpus to configure the local scheduler with.
num_gpus (int): number of gpus to configure the local scheduler with.
Returns:
A dictionary of the address information for the processes that were
Expand All @@ -592,4 +618,6 @@ def start_ray_head(address_info=None,
cleanup=cleanup,
redirect_output=redirect_output,
include_global_scheduler=True,
include_redis=True)
include_redis=True,
num_cpus=num_cpus,
num_gpus=num_gpus)
Loading

0 comments on commit dfb6107

Please sign in to comment.