Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Provide experimental API for changing number of return values and res… #1735

Merged
merged 2 commits into from
Mar 19, 2018
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Remove code duplication and add tests.
  • Loading branch information
robertnishihara committed Mar 18, 2018
commit e1f09f879fa65e06bd1bc28cba3658959925d6ba
42 changes: 15 additions & 27 deletions python/ray/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -507,7 +507,7 @@ def submit_task(self, function_id, args, actor_id=None,
is_actor_checkpoint_method=False, actor_creation_id=None,
actor_creation_dummy_object_id=None,
execution_dependencies=None, num_return_vals=None,
resources=None):
num_cpus=None, num_gpus=None, resources=None):
"""Submit a remote task to the scheduler.

Tell the scheduler to schedule the execution of the function with ID
Expand All @@ -531,6 +531,8 @@ def submit_task(self, function_id, args, actor_id=None,
execution_dependencies: The execution dependencies for this task.
num_return_vals: The number of return values this function should
have.
num_cpus: The number of CPUs required by this task.
num_gpus: The number of GPUs required by this task.
resources: The resource requirements for this task.

Returns:
Expand Down Expand Up @@ -578,8 +580,15 @@ def submit_task(self, function_id, args, actor_id=None,
if num_return_vals is None:
num_return_vals = function_properties.num_return_vals

if resources is None:
if resources is None and num_cpus is None and num_gpus is None:
resources = function_properties.resources
else:
resources = {} if resources is None else resources
if "CPU" in resources or "GPU" in resources:
raise ValueError("The resources dictionary must not "
"contain the keys 'CPU' or 'GPU'")
resources["CPU"] = num_cpus
resources["GPU"] = num_gpus

# Submit the task to local scheduler.
task = ray.local_scheduler.Task(
Expand Down Expand Up @@ -2576,38 +2585,16 @@ def remote_function_decorator(func, function_properties):

def func_call(*args, **kwargs):
"""This runs immediately when a remote function is called."""
check_connected()
check_main_thread()
args = signature.extend_args(function_signature, args, kwargs)
return _submit(args=args, kwargs=kwargs)

if _mode() == PYTHON_MODE:
# In PYTHON_MODE, remote calls simply execute the function.
# We copy the arguments to prevent the function call from
# mutating them and to match the usual behavior of
# immutable remote objects.
result = func(*copy.deepcopy(args))
return result
object_ids = _submit_task(function_id, args)
if len(object_ids) == 1:
return object_ids[0]
elif len(object_ids) > 1:
return object_ids

def _submit(args, kwargs=None, num_return_vals=1, num_cpus=1,
num_gpus=0, resources=None):
def _submit(args=None, kwargs=None, num_return_vals=None,
num_cpus=None, num_gpus=None, resources=None):
"""An experimental alternate way to submit remote functions."""
check_connected()
check_main_thread()
kwargs = {} if kwargs is None else kwargs
args = signature.extend_args(function_signature, args, kwargs)

resources = {} if resources is None else resources
if "CPU" in resources or "GPU" in resources:
raise ValueError("The resources dictionary must not "
"contain the keys 'CPU' or 'GPU'")
resources["CPU"] = num_cpus
resources["GPU"] = num_gpus

if _mode() == PYTHON_MODE:
# In PYTHON_MODE, remote calls simply execute the function.
# We copy the arguments to prevent the function call from
Expand All @@ -2617,6 +2604,7 @@ def _submit(args, kwargs=None, num_return_vals=1, num_cpus=1,
return result
object_ids = _submit_task(function_id, args,
num_return_vals=num_return_vals,
num_cpus=num_cpus, num_gpus=num_gpus,
resources=resources)
if len(object_ids) == 1:
return object_ids[0]
Expand Down
20 changes: 20 additions & 0 deletions test/runtest.py
Original file line number Diff line number Diff line change
Expand Up @@ -685,6 +685,26 @@ def m(x):
self.assertEqual(ray.get(k2.remote(1)), 2)
self.assertEqual(ray.get(m.remote(1)), 2)

def testSubmitAPI(self):
self.init_ray(num_gpus=1, resources={"Custom": 1}, num_workers=1)

@ray.remote
def f(n):
return list(range(n))

@ray.remote
def g():
return ray.get_gpu_ids()

assert f._submit([0], num_return_vals=0) is None
assert ray.get(f._submit(args=[1], num_return_vals=1)) == [0]
assert ray.get(f._submit(args=[2], num_return_vals=2)) == [0, 1]
assert ray.get(f._submit(args=[3], num_return_vals=3)) == [0, 1, 2]
assert ray.get(g._submit(args=[],
num_cpus=1,
num_gpus=1,
resources={"Custom": 1})) == [0]

def testGetMultiple(self):
self.init_ray()
object_ids = [ray.put(i) for i in range(10)]
Expand Down