Skip to content

Commit

Permalink
[Core] Non Unit Instance fractional value fix (ray-project#39293)
Browse files Browse the repository at this point in the history
There's a bug where Ray should've allow non unit instance resource to have fractional value greater than 1 (unit instance resource currently: GPU, TPU, Neuron Core). This PR is to fix that

Signed-off-by: Jonathan Nitisastro <jonathannitisastro@jonathancn-QC69NQYVVG.local.meter>
Signed-off-by: Jonathan Nitisastro <jonathancn@anyscale.com>
Signed-off-by: jonathan-anyscale <144177685+jonathan-anyscale@users.noreply.github.com>
Co-authored-by: Jonathan Nitisastro <jonathannitisastro@jonathancn-QC69NQYVVG.local.meter>
Co-authored-by: angelinalg <122562471+angelinalg@users.noreply.github.com>
Co-authored-by: Jiajun Yao <jeromeyjj@gmail.com>
Signed-off-by: Victor <vctr.y.m@example.com>
  • Loading branch information
4 people authored and Victor committed Oct 11, 2023
1 parent ce0f3d4 commit 9250e1e
Show file tree
Hide file tree
Showing 4 changed files with 35 additions and 11 deletions.
6 changes: 5 additions & 1 deletion doc/source/ray-core/scheduling/resources.rst
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,11 @@ The precision of the fractional resource requirement is 0.0001 so you should avo
:start-after: __specifying_fractional_resource_requirements_start__
:end-before: __specifying_fractional_resource_requirements_end__
.. note::
GPU, TPU, and neuron_cores resource requirements that are greater than 1, need to be whole numbers. For example, ``num_gpus=1.5`` is invalid.
.. tip::
Besides resource requirements, you can also specify an environment for a task or actor to run in,
which can include Python packages, local files, environment variables, and more---see :ref:`Runtime Environments <runtime-environments>` for details.
which can include Python packages, local files, environment variables, and more. See :ref:`Runtime Environments <runtime-environments>` for details.
13 changes: 11 additions & 2 deletions python/ray/_raylet.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -634,6 +634,7 @@ cdef int prepare_resources(
cdef:
unordered_map[c_string, double] out
c_string resource_name
list unit_resources

if resource_dict is None:
raise ValueError("Must provide resource map.")
Expand All @@ -644,10 +645,18 @@ cdef int prepare_resources(
if value < 0:
raise ValueError("Resource quantities may not be negative.")
if value > 0:
unit_resources = (
f"{RayConfig.instance().predefined_unit_instance_resources()\
.decode('utf-8')},"
f"{RayConfig.instance().custom_unit_instance_resources()\
.decode('utf-8')}"
).split(",")

if (value >= 1 and isinstance(value, float)
and not value.is_integer()):
and not value.is_integer() and str(key) in unit_resources):
raise ValueError(
"Resource quantities >1 must be whole numbers.")
f"{key} resource quantities >1 must",
f" be whole numbers. The specified quantity {value} is invalid.")
resource_map[0][key.encode("ascii")] = float(value)
return 0

Expand Down
4 changes: 4 additions & 0 deletions python/ray/includes/ray_config.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,10 @@ cdef extern from "ray/common/ray_config.h" nogil:

c_bool enable_autoscaler_v2() const

c_string predefined_unit_instance_resources() const

c_string custom_unit_instance_resources() const

int64_t nums_py_gcs_reconnect_retry() const

int64_t py_gcs_connect_timeout_s() const
23 changes: 15 additions & 8 deletions python/ray/tests/test_advanced_2.py
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ def method(self):


def test_fractional_resources(shutdown_only):
ray.init(num_cpus=6, num_gpus=3, resources={"Custom": 1})
ray.init(num_cpus=6, num_gpus=3, resources={"Custom": 3, "Custom2": 3, "TPU": 3})

@ray.remote(num_gpus=0.5)
class Foo1:
Expand All @@ -168,7 +168,7 @@ def method(self):
pass

# Create an actor that requires 0.7 of the custom resource.
f1 = Foo2._remote([], {}, resources={"Custom": 0.7})
f1 = Foo2._remote([], {}, resources={"Custom": 2.7})
ray.get(f1.method.remote())
# Make sure that we cannot create an actor that requires 0.7 of the
# custom resource. TODO(rkn): Re-enable this once ray.wait is
Expand All @@ -183,18 +183,25 @@ def method(self):

del f1, f3

# Make sure that we get exceptions if we submit tasks that require a
# fractional number of resources greater than 1.
# Non unit resources (e.g. CPU, ) allow fractional
# number of resources greather than 1.
@ray.remote(num_cpus=1.5, resources={"Custom2": 2.5})
def test_frac_cpu():
return True

@ray.remote(num_cpus=1.5)
def test():
assert ray.get(test_frac_cpu.remote())

# Unit instance resources (GPU, TPU, neuron_core) throw exceptions
# for fractional number of resources greater than 1.
@ray.remote(num_gpus=1.5)
def test_frac_gpu():
pass

with pytest.raises(ValueError):
test.remote()
test_frac_gpu.remote()

with pytest.raises(ValueError):
Foo2._remote([], {}, resources={"Custom": 1.5})
Foo2._remote([], {}, resources={"TPU": 2.5})


def test_fractional_memory_round_down(shutdown_only):
Expand Down

0 comments on commit 9250e1e

Please sign in to comment.