-
Notifications
You must be signed in to change notification settings - Fork 5.8k
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
Warn on resource deadlock; improve object store error messages #5555
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -336,6 +336,7 @@ void NodeManager::Heartbeat() { | |
static_cast<int64_t>(now_ms - last_debug_dump_at_ms_) > debug_dump_period_) { | ||
DumpDebugState(); | ||
RecordMetrics(); | ||
WarnResourceDeadlock(); | ||
last_debug_dump_at_ms_ = now_ms; | ||
} | ||
|
||
|
@@ -347,6 +348,69 @@ void NodeManager::Heartbeat() { | |
}); | ||
} | ||
|
||
void NodeManager::WarnResourceDeadlock() { | ||
// Check if any progress is being made on this raylet. | ||
for (const auto &task : local_queues_.GetTasks(TaskState::RUNNING)) { | ||
// Ignore blocked tasks. | ||
if (local_queues_.GetBlockedTaskIds().count(task.GetTaskSpecification().TaskId())) { | ||
continue; | ||
} | ||
// Progress is being made, don't warn. | ||
resource_deadlock_warned_ = false; | ||
return; | ||
} | ||
|
||
// suppress duplicates warning messages | ||
if (resource_deadlock_warned_) { | ||
return; | ||
} | ||
|
||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe add an additional check that there are no resources available on the local node. The There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I thought about that but it could be the actor is not placeable due to its size of resources. We could try checking for that too, but it seems simpler to rely on the periodic delay to avoid the warning firing too often. |
||
// The node is full of actors and no progress has been made for some time. | ||
// If there are any pending tasks, build a warning. | ||
ericl marked this conversation as resolved.
Show resolved
Hide resolved
|
||
std::ostringstream error_message; | ||
ray::Task exemplar; | ||
bool should_warn = false; | ||
int pending_actor_creations = 0; | ||
int pending_tasks = 0; | ||
|
||
// See if any tasks are blocked trying to acquire resources. | ||
for (const auto &task : local_queues_.GetTasks(TaskState::READY)) { | ||
const TaskSpecification &spec = task.GetTaskSpecification(); | ||
if (spec.IsActorCreationTask()) { | ||
pending_actor_creations += 1; | ||
} else { | ||
pending_tasks += 1; | ||
} | ||
if (!should_warn) { | ||
exemplar = task; | ||
should_warn = true; | ||
} | ||
} | ||
|
||
// Push an warning to the driver that a task is blocked trying to acquire resources. | ||
if (should_warn) { | ||
const auto &my_client_id = gcs_client_->client_table().GetLocalClientId(); | ||
SchedulingResources &local_resources = cluster_resource_map_[my_client_id]; | ||
error_message | ||
<< "The actor or task with ID " << exemplar.GetTaskSpecification().TaskId() | ||
<< " is pending and cannot currently be scheduled. It requires " | ||
<< exemplar.GetTaskSpecification().GetRequiredResources().ToString() | ||
<< " for execution and " | ||
<< exemplar.GetTaskSpecification().GetRequiredPlacementResources().ToString() | ||
<< " for placement, but this node only has remaining " | ||
<< local_resources.GetAvailableResources().ToString() << ". In total there are " | ||
<< pending_tasks << " pending tasks and " << pending_actor_creations | ||
<< " pending actors on this node. " | ||
<< "This is likely due to all cluster resources being claimed by actors. " | ||
<< "To resolve the issue, consider creating fewer actors or increase the " | ||
ericl marked this conversation as resolved.
Show resolved
Hide resolved
|
||
<< "resources available to this Ray cluster."; | ||
RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( | ||
exemplar.GetTaskSpecification().JobId(), "resource_deadlock", error_message.str(), | ||
current_time_ms())); | ||
resource_deadlock_warned_ = true; | ||
} | ||
} | ||
|
||
void NodeManager::GetObjectManagerProfileInfo() { | ||
int64_t start_time_ms = current_time_ms(); | ||
|
||
|
@@ -1330,12 +1394,15 @@ void NodeManager::ScheduleTasks( | |
std::string type = "infeasible_task"; | ||
std::ostringstream error_message; | ||
error_message | ||
<< "The task with ID " << task.GetTaskSpecification().TaskId() | ||
<< " is infeasible and cannot currently be executed. It requires " | ||
<< "The actor or task with ID " << task.GetTaskSpecification().TaskId() | ||
<< " is infeasible and cannot currently be scheduled. It requires " | ||
<< task.GetTaskSpecification().GetRequiredResources().ToString() | ||
<< " for execution and " | ||
<< task.GetTaskSpecification().GetRequiredPlacementResources().ToString() | ||
<< " for placement. Check the client table to view node resources."; | ||
<< " for placement, however there are no nodes in the cluster that can " | ||
<< "provide the requested resources. To resolve this issue, consider " | ||
<< "reducing the resource requests of this task or add nodes that " | ||
<< "can fit the task."; | ||
RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( | ||
task.GetTaskSpecification().JobId(), type, error_message.str(), | ||
current_time_ms())); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's call this in
DispatchTasks
instead, and only call it if we weren't able to dispatch any tasks. Otherwise, we'll end up pushing an error every heartbeat for as long as the deadlock is happening (which is probably forever).There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I tried doing this, but it ended up printing out too many false positives. The issue is that you only want to fire the warning if there has been a significant delay, and right after DispatchTasks is not it (though if you wait even a tiny bit of time resources could immediately free up, like if a task returns right after creating an actor).
I think printing forever is fine -- it is deadlocked after all.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm, maybe I'm misunderstanding how DispatchTasks works, but if DispatchTasks doesn't succeed in scheduling anything, and the conditions that you check are true (no running tasks), then doesn't that mean it'll never succeed again? I think when this happens, it can only be because there are no available workers or if all the cores are taken up by actors. We can make sure it's the second case if we also check that there are no resources available.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not necessarily, a block/unblock could free up resources, here is the example I was testing: