Skip to content

Commit

Permalink
Adding basic support for a user-interpretable resource label (ray-pro…
Browse files Browse the repository at this point in the history
…ject#761)

* adding support for the user-interpretable label(UIR)

* more plumbing for num_uirs further upstream; set to infty when specified on cmd line

* pass default num_uirs for actors; update GlobalStateAPI

* support num_uirs in ray.init()

* local scheduler resource accounting: support num_uirs; prep for vectorized resource accounting

* global scheduler test updated

* Fix bug introduced by rebase.

* Rename UIR -> CustomResource and add test.

* Small changes and use constexpr instead of macros.

* Linting and some renaming.

* Reorder some code.

* Remove cpus_in_use and fix bug.

* Add another test and make a small change.

* Rephrase documentation about feature stability.
  • Loading branch information
atumanov authored and pcmoritz committed Aug 8, 2017
1 parent 03f2325 commit fc885bd
Show file tree
Hide file tree
Showing 14 changed files with 242 additions and 82 deletions.
5 changes: 4 additions & 1 deletion python/ray/actor.py
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,7 @@ def temporary_actor_method(*xs):
FunctionProperties(num_return_vals=1,
num_cpus=1,
num_gpus=0,
num_custom_resource=0,
max_calls=0))
worker.num_task_executions[driver_id][function_id] = 0

Expand Down Expand Up @@ -174,6 +175,7 @@ def export_actor(actor_id, class_id, actor_method_names, num_cpus, num_gpus,
FunctionProperties(num_return_vals=1,
num_cpus=1,
num_gpus=0,
num_custom_resource=0,
max_calls=0))

# Select a local scheduler for the actor.
Expand Down Expand Up @@ -259,7 +261,8 @@ def hex_to_object_id(hex_id):
hex_to_object_id(task_spec_info["ActorID"]),
task_spec_info["ActorCounter"],
[task_spec_info["RequiredResources"]["CPUs"],
task_spec_info["RequiredResources"]["GPUs"]])
task_spec_info["RequiredResources"]["GPUs"],
task_spec_info["RequiredResources"]["CustomResource"]])

# Verify that the return object IDs are the same as they were the
# first time.
Expand Down
13 changes: 10 additions & 3 deletions python/ray/experimental/state.py
Original file line number Diff line number Diff line change
Expand Up @@ -257,9 +257,13 @@ def _task_table(self, task_id):
args.append(binary_to_object_id(arg.ObjectId()))
else:
args.append(pickle.loads(arg.Data()))
assert task_spec_message.RequiredResourcesLength() == 2
required_resources = {"CPUs": task_spec_message.RequiredResources(0),
"GPUs": task_spec_message.RequiredResources(1)}
# TODO(atumanov): Instead of hard coding these indices, we should use
# the flatbuffer constants.
assert task_spec_message.RequiredResourcesLength() == 3
required_resources = {
"CPUs": task_spec_message.RequiredResources(0),
"GPUs": task_spec_message.RequiredResources(1),
"CustomResource": task_spec_message.RequiredResources(2)}
task_spec_info = {
"DriverID": binary_to_hex(task_spec_message.DriverId()),
"TaskID": binary_to_hex(task_spec_message.TaskId()),
Expand Down Expand Up @@ -351,6 +355,9 @@ def client_table(self):
if b"num_gpus" in client_info:
client_info_parsed["NumGPUs"] = float(
decode(client_info[b"num_gpus"]))
if b"num_custom_resource" in client_info:
client_info_parsed["NumCustomResource"] = float(
decode(client_info[b"num_custom_resource"]))
if b"local_scheduler_socket_name" in client_info:
client_info_parsed["LocalSchedulerSocketName"] = decode(
client_info[b"local_scheduler_socket_name"])
Expand Down
6 changes: 3 additions & 3 deletions python/ray/global_scheduler/test/test.py
Original file line number Diff line number Diff line change
Expand Up @@ -166,12 +166,12 @@ def test_task_default_resources(self):
task1 = local_scheduler.Task(random_driver_id(), random_function_id(),
[random_object_id()], 0, random_task_id(),
0)
self.assertEqual(task1.required_resources(), [1.0, 0.0])
self.assertEqual(task1.required_resources(), [1.0, 0.0, 0.0])
task2 = local_scheduler.Task(random_driver_id(), random_function_id(),
[random_object_id()], 0, random_task_id(),
0, local_scheduler.ObjectID(NIL_ACTOR_ID),
0, [1.0, 2.0])
self.assertEqual(task2.required_resources(), [1.0, 2.0])
0, [1.0, 2.0, 0.0])
self.assertEqual(task2.required_resources(), [1.0, 2.0, 0.0])

def test_redis_only_single_task(self):
# Tests global scheduler functionality by interacting with Redis and
Expand Down
9 changes: 7 additions & 2 deletions python/ray/scripts/scripts.py
Original file line number Diff line number Diff line change
Expand Up @@ -59,12 +59,15 @@ def cli():
help="the number of CPUs on this node")
@click.option("--num-gpus", required=False, type=int,
help="the number of GPUs on this node")
@click.option("--num-custom-resource", required=False, type=int,
help="the amount of a user-defined custom resource on this node")
@click.option("--head", is_flag=True, default=False,
help="provide this argument for the head node")
@click.option("--block", is_flag=True, default=False,
help="provide this argument to block forever in this command")
def start(node_ip_address, redis_address, redis_port, num_redis_shards,
object_manager_port, num_workers, num_cpus, num_gpus, head, block):
object_manager_port, num_workers, num_cpus, num_gpus,
num_custom_resource, head, block):
# Note that we redirect stdout and stderr to /dev/null because otherwise
# attempts to print may cause exceptions if a process is started inside of
# an SSH connection and the SSH connection dies. TODO(rkn): This is a
Expand Down Expand Up @@ -99,6 +102,7 @@ def start(node_ip_address, redis_address, redis_port, num_redis_shards,
redirect_output=True,
num_cpus=num_cpus,
num_gpus=num_gpus,
num_custom_resource=num_custom_resource,
num_redis_shards=num_redis_shards)
print(address_info)
print("\nStarted Ray on this node. You can add additional nodes to "
Expand Down Expand Up @@ -144,7 +148,8 @@ def start(node_ip_address, redis_address, redis_port, num_redis_shards,
cleanup=False,
redirect_output=True,
num_cpus=num_cpus,
num_gpus=num_gpus)
num_gpus=num_gpus,
num_custom_resource=num_custom_resource)
print(address_info)
print("\nStarted Ray on this node. If you wish to terminate the "
"processes that have been started, run\n\n"
Expand Down
30 changes: 24 additions & 6 deletions python/ray/services.py
Original file line number Diff line number Diff line change
Expand Up @@ -512,6 +512,7 @@ def start_local_scheduler(redis_address,
cleanup=True,
num_cpus=None,
num_gpus=None,
num_custom_resource=None,
num_workers=0):
"""Start a local scheduler process.
Expand All @@ -536,6 +537,8 @@ def start_local_scheduler(redis_address,
with.
num_gpus: The number of GPUs the local scheduler should be configured
with.
num_custom_resource: The quantity of a user-defined custom resource
that the local scheduler should be configured with.
num_workers (int): The number of workers that the local scheduler
should start.
Expand All @@ -549,8 +552,11 @@ def start_local_scheduler(redis_address,
if num_gpus is None:
# By default, assume this node has no GPUs.
num_gpus = 0
print("Starting local scheduler with {} CPUs and {} GPUs."
.format(num_cpus, num_gpus))
if num_custom_resource is None:
# By default, assume this node has none of the custom resource.
num_custom_resource = 0
print("Starting local scheduler with {} CPUs, {} GPUs"
.format(num_cpus, num_gpus, num_custom_resource))
local_scheduler_name, p = ray.local_scheduler.start_local_scheduler(
plasma_store_name,
plasma_manager_name,
Expand All @@ -561,7 +567,7 @@ def start_local_scheduler(redis_address,
use_profiler=RUN_LOCAL_SCHEDULER_PROFILER,
stdout_file=stdout_file,
stderr_file=stderr_file,
static_resource_list=[num_cpus, num_gpus],
static_resource_list=[num_cpus, num_gpus, num_custom_resource],
num_workers=num_workers)
if cleanup:
all_processes[PROCESS_TYPE_LOCAL_SCHEDULER].append(p)
Expand Down Expand Up @@ -752,7 +758,8 @@ def start_ray_processes(address_info=None,
include_webui=False,
start_workers_from_local_scheduler=True,
num_cpus=None,
num_gpus=None):
num_gpus=None,
num_custom_resource=None):
"""Helper method to start Ray processes.
Args:
Expand Down Expand Up @@ -796,6 +803,9 @@ def start_ray_processes(address_info=None,
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.
num_custom_resource: A list of length num_local_schedulers containing
the quantity of a user-defined custom resource that each local
scheduler should be configured with.
Returns:
A dictionary of the address information for the processes that were
Expand All @@ -805,8 +815,11 @@ def start_ray_processes(address_info=None,
num_cpus = num_local_schedulers * [num_cpus]
if not isinstance(num_gpus, list):
num_gpus = num_local_schedulers * [num_gpus]
if not isinstance(num_custom_resource, list):
num_custom_resource = num_local_schedulers * [num_custom_resource]
assert len(num_cpus) == num_local_schedulers
assert len(num_gpus) == num_local_schedulers
assert len(num_custom_resource) == num_local_schedulers

if num_workers is not None:
workers_per_local_scheduler = num_local_schedulers * [num_workers]
Expand Down Expand Up @@ -940,6 +953,7 @@ def start_ray_processes(address_info=None,
cleanup=cleanup,
num_cpus=num_cpus[i],
num_gpus=num_gpus[i],
num_custom_resource=num_custom_resource[i],
num_workers=num_local_scheduler_workers)
local_scheduler_socket_names.append(local_scheduler_name)
time.sleep(0.1)
Expand Down Expand Up @@ -991,7 +1005,8 @@ def start_ray_node(node_ip_address,
cleanup=True,
redirect_output=False,
num_cpus=None,
num_gpus=None):
num_gpus=None,
num_custom_resource=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 @@ -1030,7 +1045,8 @@ def start_ray_node(node_ip_address,
cleanup=cleanup,
redirect_output=redirect_output,
num_cpus=num_cpus,
num_gpus=num_gpus)
num_gpus=num_gpus,
num_custom_resource=num_custom_resource)


def start_ray_head(address_info=None,
Expand All @@ -1045,6 +1061,7 @@ def start_ray_head(address_info=None,
start_workers_from_local_scheduler=True,
num_cpus=None,
num_gpus=None,
num_custom_resource=None,
num_redis_shards=None):
"""Start Ray in local mode.
Expand Down Expand Up @@ -1102,6 +1119,7 @@ def start_ray_head(address_info=None,
start_workers_from_local_scheduler=start_workers_from_local_scheduler,
num_cpus=num_cpus,
num_gpus=num_gpus,
num_custom_resource=num_custom_resource,
num_redis_shards=num_redis_shards)


Expand Down
1 change: 1 addition & 0 deletions python/ray/utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ def hex_to_binary(hex_identifier):
["num_return_vals",
"num_cpus",
"num_gpus",
"num_custom_resource",
"max_calls"])
"""FunctionProperties: A named tuple storing remote functions information."""

Expand Down
Loading

0 comments on commit fc885bd

Please sign in to comment.