Deprecate object_store_memory option (#26252)

Why are these changes needed?
object_store_memory as task option has already been removed and object_store_memory as actor option is actually broken since at least last year (#20344).

Given we now have object spilling and fallback allocation, this option is no longer useful.
This commit is contained in:
Jiajun Yao 2022-07-27 02:07:09 -07:00 committed by GitHub
parent eb71365f32
commit 30ed942aa3
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
8 changed files with 49 additions and 25 deletions

View file

@ -1,7 +1,7 @@
"""Manage, parse and validate options for Ray tasks, actors and actor methods."""
import warnings
from dataclasses import dataclass
from typing import Any, Callable, Dict, Optional, Tuple, Union
import warnings
import ray._private.ray_constants as ray_constants
from ray._private.utils import get_ray_doc_version
@ -261,6 +261,18 @@ def validate_actor_options(options: Dict[str, Any], in_options: bool):
if options.get("get_if_exists") and not options.get("name"):
raise ValueError("The actor name must be specified to use `get_if_exists`.")
if "object_store_memory" in options:
warnings.warn(
"Setting 'object_store_memory'"
" for actors is deprecated since it doesn't actually"
" reserve the required object store memory."
f" Use object spilling that's enabled by default (https://docs.ray.io/en/{get_ray_doc_version()}/ray-core/objects/object-spilling.html) " # noqa: E501
"instead to bypass the object store memory size limitation.",
DeprecationWarning,
stacklevel=1,
)
_check_deprecate_placement_group(options)

View file

@ -2719,7 +2719,6 @@ def remote(
resources: Dict[str, float] = Undefined,
accelerator_type: str = Undefined,
memory: Union[int, float] = Undefined,
object_store_memory: int = Undefined,
max_calls: int = Undefined,
max_restarts: int = Undefined,
max_task_retries: int = Undefined,
@ -2801,7 +2800,6 @@ def remote(*args, **kwargs):
on a node with the specified type of accelerator.
See `ray.accelerators` for accelerator types.
memory: The heap memory request for this task/actor.
object_store_memory: The object store memory request for actors only.
max_calls: Only for *remote functions*. This specifies the
maximum number of times that a given worker can execute
the given remote function before it must exit

View file

@ -583,14 +583,6 @@ cdef execute_task(
actor = worker.actors[core_worker.get_actor_id()]
class_name = actor.__class__.__name__
next_title = f"ray::{class_name}"
pid = os.getpid()
worker_name = f"ray_{class_name}_{pid}"
if c_resources.find(b"object_store_memory") != c_resources.end():
worker.core_worker.set_object_store_client_options(
worker_name,
int(ray_constants.from_memory_units(
dereference(
c_resources.find(b"object_store_memory")).second)))
def function_executor(*arguments, **kwarguments):
function = execution_info.function

View file

@ -303,7 +303,6 @@ class _ActorClassMetadata:
num_gpus: The default number of GPUs required by the actor creation
task.
memory: The heap memory quota for this actor.
object_store_memory: The object store memory quota for this actor.
resources: The default resources required by the actor creation task.
accelerator_type: The specified type of accelerator required for the
node on which this actor runs.
@ -668,8 +667,6 @@ class ActorClass:
num_cpus: The number of CPUs required by the actor creation task.
num_gpus: The number of GPUs required by the actor creation task.
memory: Restrict the heap memory usage of this actor.
object_store_memory: Restrict the object store memory used by
this actor when creating objects.
resources: The custom resources required by the actor creation
task.
max_concurrency: The max number of concurrent calls to allow for

View file

@ -50,7 +50,6 @@ class RemoteFunction:
_num_gpus: The default number of GPUs to use for invocations of this
remote function.
_memory: The heap memory request for this task.
_object_store_memory: The object store memory request for this task.
_resources: The default custom resource requirements for invocations of
this remote function.
_num_returns: The default number of return values for invocations

View file

@ -490,6 +490,28 @@ def test_placement_group_scheduling_warning(ray_start_regular_shared):
assert not w
def test_object_store_memory_deprecation_warning(ray_start_regular_shared):
with warnings.catch_warnings(record=True) as w:
@ray.remote(object_store_memory=1)
class Actor:
pass
Actor.remote()
assert any(
"Setting 'object_store_memory' for actors is deprecated" in str(warning.message)
for warning in w
)
with warnings.catch_warnings(record=True) as w:
ray.util.placement_group([{"object_store_memory": 1}], strategy="STRICT_PACK")
assert any(
"Setting 'object_store_memory' for bundles is deprecated"
in str(warning.message)
for warning in w
)
if __name__ == "__main__":
import os

View file

@ -25,28 +25,20 @@ def test_placement_group_bin_packing_priority(
return Actor.options(
placement_group=pg, placement_group_bundle_index=index, num_cpus=1
).remote()
elif index < 3:
return Actor.options(
placement_group=pg, placement_group_bundle_index=index, num_gpus=1
).remote()
else:
return Actor.options(
placement_group=pg,
placement_group_bundle_index=index,
object_store_memory=1024 * 1024 * 200,
placement_group=pg, placement_group_bundle_index=index, num_gpus=1
).remote()
def add_nodes_to_cluster(cluster):
cluster.add_node(num_cpus=1)
cluster.add_node(num_cpus=2)
cluster.add_node(num_gpus=1)
cluster.add_node(object_store_memory=1024 * 1024 * 250)
default_bundles = [
{"CPU": 1},
{"CPU": 2},
{"CPU": 1, "GPU": 1},
{"CPU": 1, "object_store_memory": 1024 * 1024 * 200},
]
default_num_nodes = len(default_bundles)

View file

@ -1,9 +1,10 @@
import warnings
from typing import Dict, List, Optional, Union
import ray
from ray._private.client_mode_hook import client_mode_should_convert, client_mode_wrap
from ray._private.ray_constants import to_memory_units
from ray._private.utils import hex_to_binary
from ray._private.utils import hex_to_binary, get_ray_doc_version
from ray._raylet import PlacementGroupID
from ray.util.annotations import DeveloperAPI, PublicAPI
@ -191,6 +192,17 @@ def placement_group(
# transformed to memory unit.
to_memory_units(bundle["memory"], True)
if "object_store_memory" in bundle.keys():
warnings.warn(
"Setting 'object_store_memory' for"
" bundles is deprecated since it doesn't actually"
" reserve the required object store memory."
f" Use object spilling that's enabled by default (https://docs.ray.io/en/{get_ray_doc_version()}/ray-core/objects/object-spilling.html) " # noqa: E501
"instead to bypass the object store memory size limitation.",
DeprecationWarning,
stacklevel=1,
)
if lifetime is None:
detached = False
elif lifetime == "detached":