mirror of
https://github.com/vale981/ray
synced 2025-03-06 02:21:39 -05:00
Trigger global GC when resources may be occupied by deleted actors
This commit is contained in:
parent
f2f118df9e
commit
413db0902d
3 changed files with 47 additions and 9 deletions
|
@ -129,6 +129,32 @@ def test_global_gc_when_full(shutdown_only):
|
|||
gc.enable()
|
||||
|
||||
|
||||
def test_global_gc_actors(shutdown_only):
|
||||
ray.init(num_cpus=1)
|
||||
|
||||
try:
|
||||
gc.disable()
|
||||
|
||||
@ray.remote(num_cpus=1)
|
||||
class A:
|
||||
def f(self):
|
||||
return "Ok"
|
||||
|
||||
# Try creating 3 actors. Unless python GC is triggered to break
|
||||
# reference cycles, this won't be possible. Note this test takes 20s
|
||||
# to run due to the 10s delay before checking of infeasible tasks.
|
||||
for i in range(3):
|
||||
a = A.remote()
|
||||
cycle = [a]
|
||||
cycle.append(cycle)
|
||||
ray.get(a.f.remote())
|
||||
print("iteration", i)
|
||||
del a
|
||||
del cycle
|
||||
finally:
|
||||
gc.enable()
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
import sys
|
||||
sys.exit(pytest.main(["-v", __file__]))
|
||||
|
|
|
@ -440,16 +440,11 @@ void NodeManager::WarnResourceDeadlock() {
|
|||
return;
|
||||
}
|
||||
|
||||
// suppress duplicates warning messages
|
||||
if (resource_deadlock_warned_) {
|
||||
return;
|
||||
}
|
||||
|
||||
// The node is full of actors and no progress has been made for some time.
|
||||
// If there are any pending tasks, build a warning.
|
||||
std::ostringstream error_message;
|
||||
ray::Task exemplar;
|
||||
bool should_warn = false;
|
||||
bool any_pending = false;
|
||||
int pending_actor_creations = 0;
|
||||
int pending_tasks = 0;
|
||||
|
||||
|
@ -461,14 +456,23 @@ void NodeManager::WarnResourceDeadlock() {
|
|||
} else {
|
||||
pending_tasks += 1;
|
||||
}
|
||||
if (!should_warn) {
|
||||
if (!any_pending) {
|
||||
exemplar = task;
|
||||
should_warn = true;
|
||||
any_pending = true;
|
||||
}
|
||||
}
|
||||
|
||||
// Push an warning to the driver that a task is blocked trying to acquire resources.
|
||||
if (should_warn) {
|
||||
if (any_pending) {
|
||||
// Actor references may be caught in cycles, preventing them from being deleted.
|
||||
// Trigger global GC to hopefully free up resource slots.
|
||||
TriggerGlobalGC();
|
||||
|
||||
// Suppress duplicates warning messages.
|
||||
if (resource_deadlock_warned_) {
|
||||
return;
|
||||
}
|
||||
|
||||
SchedulingResources &local_resources = cluster_resource_map_[self_node_id_];
|
||||
error_message
|
||||
<< "The actor or task with ID " << exemplar.GetTaskSpecification().TaskId()
|
||||
|
@ -3683,6 +3687,10 @@ void NodeManager::HandleFormatGlobalMemoryInfo(
|
|||
void NodeManager::HandleGlobalGC(const rpc::GlobalGCRequest &request,
|
||||
rpc::GlobalGCReply *reply,
|
||||
rpc::SendReplyCallback send_reply_callback) {
|
||||
TriggerGlobalGC();
|
||||
}
|
||||
|
||||
void NodeManager::TriggerGlobalGC() {
|
||||
RAY_LOG(WARNING) << "Broadcasting global GC request to all raylets.";
|
||||
should_global_gc_ = true;
|
||||
// We won't see our own request, so trigger local GC in the next heartbeat.
|
||||
|
|
|
@ -605,6 +605,10 @@ class NodeManager : public rpc::NodeManagerServiceHandler {
|
|||
rpc::FormatGlobalMemoryInfoReply *reply,
|
||||
rpc::SendReplyCallback send_reply_callback) override;
|
||||
|
||||
/// Trigger global GC across the cluster to free up references to actors or
|
||||
/// object ids.
|
||||
void TriggerGlobalGC();
|
||||
|
||||
/// Trigger local GC on each worker of this raylet.
|
||||
void DoLocalGC();
|
||||
|
||||
|
|
Loading…
Add table
Reference in a new issue