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

[Core] Fix object reconstruction hang on arguments pending creation #47645

Merged
merged 7 commits into from
Sep 27, 2024
Merged
Show file tree
Hide file tree
Changes from 5 commits
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
53 changes: 52 additions & 1 deletion python/ray/tests/test_reconstruction_2.py
Original file line number Diff line number Diff line change
Expand Up @@ -10,10 +10,10 @@
from ray._private.internal_api import memory_summary
from ray._private.test_utils import Semaphore, SignalActor, wait_for_condition
import ray.exceptions
from ray.util.state import list_tasks

# Task status.
WAITING_FOR_DEPENDENCIES = "PENDING_ARGS_AVAIL"
SCHEDULED = "PENDING_NODE_ASSIGNMENT"
FINISHED = "FINISHED"
WAITING_FOR_EXECUTION = "SUBMITTED_TO_WORKER"

Expand Down Expand Up @@ -505,6 +505,57 @@ def dependent_task(x):
ray.get(obj)


def test_object_reconstruction_dead_actor(config, ray_start_cluster):
# Test to make sure that if object reconstruction fails
# due to dead actor, pending_creation is set back to false.
# https://github.com/ray-project/ray/issues/47606
cluster = ray_start_cluster
cluster.add_node(num_cpus=0, _system_config=config)
ray.init(address=cluster.address)
node1 = cluster.add_node(resources={"node1": 1})
node2 = cluster.add_node(resources={"node2": 1})

@ray.remote(max_restarts=0, max_task_retries=-1, resources={"node1": 0.1})
class Worker:
def func_in(self):
return np.random.rand(1024000)

@ray.remote(max_retries=-1, resources={"node2": 0.1})
def func_out(data):
return np.random.rand(1024000)

worker = Worker.remote()

ref_in = worker.func_in.remote()
ref_out = func_out.remote(ref_in)

ray.wait([ref_in, ref_out], num_returns=2, timeout=None, fetch_local=False)

def func_out_resubmitted():
tasks = list_tasks(filters=[("name", "=", "func_out")])
assert len(tasks) == 2
assert (
tasks[0]["state"] == "PENDING_NODE_ASSIGNMENT"
or tasks[1]["state"] == "PENDING_NODE_ASSIGNMENT"
)
return True

cluster.remove_node(node2, allow_graceful=False)
# ref_out will reconstruct, wait for the lease request to reach raylet.
wait_for_condition(func_out_resubmitted)

cluster.remove_node(node1, allow_graceful=False)
# ref_in is lost and the reconstruction will
# fail with ActorDiedError

node1 = cluster.add_node(resources={"node1": 1})
node2 = cluster.add_node(resources={"node2": 1})

with pytest.raises(ray.exceptions.RayTaskError) as exc_info:
ray.get(ref_out)
assert "input arguments for this task could not be computed" in str(exc_info.value)


def test_object_reconstruction_pending_creation(config, ray_start_cluster):
# Test to make sure that an object being reconstructured
# has pending_creation set to true.
Expand Down
2 changes: 1 addition & 1 deletion src/ray/core_worker/test/direct_actor_transport_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -626,7 +626,7 @@ TEST_P(ActorTaskSubmitterTest, TestActorRestartOutOfOrderGcs) {
task = CreateActorTaskHelper(actor_id, worker_id, 4);
EXPECT_CALL(*task_finisher_, FailOrRetryPendingTask(task.TaskId(), _, _, _, _, _))
.Times(1);
ASSERT_FALSE(CheckSubmitTask(task));
ASSERT_TRUE(CheckSubmitTask(task));
jjyao marked this conversation as resolved.
Show resolved Hide resolved
}

TEST_P(ActorTaskSubmitterTest, TestActorRestartFailInflightTasks) {
Expand Down
56 changes: 32 additions & 24 deletions src/ray/core_worker/transport/actor_task_submitter.cc
Original file line number Diff line number Diff line change
Expand Up @@ -227,30 +227,38 @@ Status ActorTaskSubmitter::SubmitTask(TaskSpecification task_spec) {
},
"ActorTaskSubmitter::SubmitTask");
} else {
// Do not hold the lock while calling into task_finisher_.
task_finisher_.MarkTaskCanceled(task_id);
rpc::ErrorType error_type;
rpc::RayErrorInfo error_info;
{
absl::MutexLock lock(&mu_);
const auto queue_it = client_queues_.find(task_spec.ActorId());
const auto &death_cause = queue_it->second.death_cause;
error_info = GetErrorInfoFromActorDeathCause(death_cause);
error_type = error_info.error_type();
}
auto status = Status::IOError("cancelling task of dead actor");
// No need to increment the number of completed tasks since the actor is
// dead.
bool fail_immediately =
error_info.has_actor_died_error() &&
error_info.actor_died_error().has_oom_context() &&
error_info.actor_died_error().oom_context().fail_immediately();
GetTaskFinisherWithoutMu().FailOrRetryPendingTask(task_id,
error_type,
&status,
&error_info,
/*mark_task_object_failed*/ true,
fail_immediately);
// Post to the event loop to maintain the async nature of
// SubmitTask and avoid issues like
// https://github.com/ray-project/ray/issues/47606.
io_service_.post(
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ehh, we do have code that relies on the behavior that this piece of code is executed immediately instead of being posted to the event loop:

def is_shutdown(self) -> bool:
        """Return whether the proxy actor is shutdown.

        If the actor is dead, the health check will return RayActorError.
        """
        try:
            ray.get(self._actor_handle.check_health.remote(), timeout=0)
        except RayActorError:
            # The actor is dead, so it's ready for shutdown.
            return True

        # The actor is still alive, so it's not ready for shutdown.
        return False

[this, task_spec, task_id]() {
// Do not hold the lock while calling into task_finisher_.
task_finisher_.MarkTaskCanceled(task_id);
rpc::ErrorType error_type;
rpc::RayErrorInfo error_info;
{
absl::MutexLock lock(&mu_);
const auto queue_it = client_queues_.find(task_spec.ActorId());
const auto &death_cause = queue_it->second.death_cause;
error_info = GetErrorInfoFromActorDeathCause(death_cause);
error_type = error_info.error_type();
}
auto status = Status::IOError("cancelling task of dead actor");
// No need to increment the number of completed tasks since the actor is
// dead.
bool fail_immediately =
error_info.has_actor_died_error() &&
error_info.actor_died_error().has_oom_context() &&
error_info.actor_died_error().oom_context().fail_immediately();
GetTaskFinisherWithoutMu().FailOrRetryPendingTask(
task_id,
error_type,
&status,
&error_info,
/*mark_task_object_failed*/ true,
fail_immediately);
},
"ActorTaskSubmitter::SubmitTask");
}

// If the task submission subsequently fails, then the client will receive
Expand Down