mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
By default, reconstruction should only be enabled for actor creation. (#6613)
* wip * fix * fix
This commit is contained in:
parent
98689bd263
commit
d3db9e9c1e
8 changed files with 20 additions and 40 deletions
|
@ -768,11 +768,6 @@ def make_actor(cls, num_cpus, num_gpus, memory, object_store_memory, resources,
|
|||
"methods in the `Checkpointable` interface.")
|
||||
|
||||
if max_reconstructions is None:
|
||||
if ray_constants.direct_call_enabled():
|
||||
# Allow the actor creation task to be resubmitted automatically
|
||||
# by default.
|
||||
max_reconstructions = 3
|
||||
else:
|
||||
max_reconstructions = 0
|
||||
|
||||
if not (ray_constants.NO_RECONSTRUCTION <= max_reconstructions <=
|
||||
|
|
|
@ -16,11 +16,8 @@ import time
|
|||
import ray
|
||||
import ray.test_utils
|
||||
import ray.cluster_utils
|
||||
from ray import ray_constants
|
||||
from ray.test_utils import run_string_as_driver
|
||||
|
||||
RAY_FORCE_DIRECT = ray_constants.direct_call_enabled()
|
||||
|
||||
|
||||
def test_actor_init_error_propagated(ray_start_regular):
|
||||
@ray.remote
|
||||
|
@ -810,7 +807,6 @@ def test_exception_raised_when_actor_node_dies(ray_start_cluster_head):
|
|||
@pytest.mark.skipif(
|
||||
os.environ.get("RAY_USE_NEW_GCS") == "on",
|
||||
reason="Hanging with new GCS API.")
|
||||
@pytest.mark.skipif(RAY_FORCE_DIRECT, reason="no ft yet")
|
||||
def test_actor_init_fails(ray_start_cluster_head):
|
||||
cluster = ray_start_cluster_head
|
||||
remote_node = cluster.add_node()
|
||||
|
|
|
@ -12,8 +12,6 @@ import pytest
|
|||
import ray
|
||||
import ray.ray_constants as ray_constants
|
||||
|
||||
RAY_FORCE_DIRECT = ray_constants.direct_call_enabled()
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
"ray_start_cluster", [{
|
||||
|
@ -64,7 +62,6 @@ def test_actor_creation_node_failure(ray_start_cluster):
|
|||
except ray.exceptions.RayActorError:
|
||||
children[i] = Child.remote(death_probability)
|
||||
|
||||
if (RAY_FORCE_DIRECT):
|
||||
children_out = [
|
||||
child.get_probability.remote() for child in children
|
||||
]
|
||||
|
@ -73,9 +70,7 @@ def test_actor_creation_node_failure(ray_start_cluster):
|
|||
# support reconstructing actors that died in the process of
|
||||
# being created.
|
||||
ready, _ = ray.wait(
|
||||
children_out,
|
||||
num_returns=len(children_out),
|
||||
timeout=5 * 60.0)
|
||||
children_out, num_returns=len(children_out), timeout=5 * 60.0)
|
||||
assert len(ready) == len(children_out)
|
||||
|
||||
# Remove a node. Any actor creation tasks that were forwarded to this
|
||||
|
|
|
@ -888,9 +888,6 @@ def test_fill_object_store_exception(ray_start_cluster_head):
|
|||
ray.put(np.zeros(10**8 + 2, dtype=np.uint8))
|
||||
|
||||
|
||||
@pytest.mark.skipif(
|
||||
not RAY_FORCE_DIRECT,
|
||||
reason="raylet path attempts reconstruction for evicted objects")
|
||||
@pytest.mark.parametrize(
|
||||
"ray_start_cluster", [{
|
||||
"num_nodes": 1,
|
||||
|
@ -925,9 +922,6 @@ def test_direct_call_eviction(ray_start_cluster):
|
|||
ray.get(dependent_task.remote(obj))
|
||||
|
||||
|
||||
@pytest.mark.skipif(
|
||||
not RAY_FORCE_DIRECT,
|
||||
reason="raylet path attempts reconstruction for evicted objects")
|
||||
@pytest.mark.parametrize(
|
||||
"ray_start_cluster", [{
|
||||
"num_nodes": 1,
|
||||
|
|
|
@ -82,7 +82,6 @@ def test_object_reconstruction(ray_start_cluster):
|
|||
ray.get(xs)
|
||||
|
||||
|
||||
@pytest.mark.skipif(RAY_FORCE_DIRECT, reason="no actor restart yet")
|
||||
@pytest.mark.parametrize(
|
||||
"ray_start_cluster", [{
|
||||
"num_cpus": 4,
|
||||
|
@ -105,7 +104,7 @@ def test_actor_creation_node_failure(ray_start_cluster):
|
|||
if exit_chance < self.death_probability:
|
||||
sys.exit(-1)
|
||||
|
||||
num_children = 50
|
||||
num_children = 25
|
||||
# Children actors will die about half the time.
|
||||
death_probability = 0.5
|
||||
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
import pytest
|
||||
import time
|
||||
|
||||
from ray import ray_constants
|
||||
import ray
|
||||
import ray.experimental.signal as signal
|
||||
|
||||
|
@ -276,9 +275,6 @@ def test_forget(ray_start_regular):
|
|||
assert len(result_list) == count
|
||||
|
||||
|
||||
@pytest.mark.skipif(
|
||||
ray_constants.direct_call_enabled(),
|
||||
reason="TODO(ekl): this requires reconstruction")
|
||||
def test_signal_on_node_failure(two_node_cluster):
|
||||
"""Test actor checkpointing on a remote node."""
|
||||
|
||||
|
@ -395,6 +391,5 @@ def test_small_receive_timeout(ray_start_regular):
|
|||
|
||||
|
||||
if __name__ == "__main__":
|
||||
import pytest
|
||||
import sys
|
||||
sys.exit(pytest.main(["-v", __file__]))
|
||||
|
|
|
@ -49,6 +49,10 @@ RAY_CONFIG(bool, new_scheduler_enabled, false)
|
|||
// Objects larger than this size will be spilled/promoted to plasma.
|
||||
RAY_CONFIG(int64_t, max_direct_call_object_size, 100 * 1024)
|
||||
|
||||
// The min number of retries for direct actor creation tasks. The actual number
|
||||
// of creation retries will be MAX(actor_creation_min_retries, max_reconstructions).
|
||||
RAY_CONFIG(uint64_t, actor_creation_min_retries, 3)
|
||||
|
||||
/// The initial period for a task execution lease. The lease will expire this
|
||||
/// many milliseconds after the first acquisition of the lease. Nodes that
|
||||
/// require an object will not try to reconstruct the task until at least
|
||||
|
|
|
@ -650,8 +650,10 @@ Status CoreWorker::CreateActor(const RayFunction &function,
|
|||
*return_actor_id = actor_id;
|
||||
TaskSpecification task_spec = builder.Build();
|
||||
if (actor_creation_options.is_direct_call) {
|
||||
task_manager_->AddPendingTask(GetCallerId(), rpc_address_, task_spec,
|
||||
actor_creation_options.max_reconstructions);
|
||||
task_manager_->AddPendingTask(
|
||||
GetCallerId(), rpc_address_, task_spec,
|
||||
std::max(RayConfig::instance().actor_creation_min_retries(),
|
||||
actor_creation_options.max_reconstructions));
|
||||
return direct_task_submitter_->SubmitTask(task_spec);
|
||||
} else {
|
||||
return local_raylet_client_->SubmitTask(task_spec);
|
||||
|
|
Loading…
Add table
Reference in a new issue