From aef1757fb7a327866027e854bb307e1c0283360f Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Wed, 1 May 2024 17:26:34 -0700 Subject: [PATCH 01/40] [Core] Better propogate node death information Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 24 +++++++++++++++++++ src/ray/gcs/gcs_client/accessor.h | 2 ++ .../gcs_autoscaler_state_manager.cc | 1 + src/ray/gcs/gcs_server/gcs_node_manager.cc | 20 ++++++++++++++++ src/ray/gcs/gcs_server/gcs_node_manager.h | 5 ++++ src/ray/protobuf/gcs.proto | 2 ++ src/ray/protobuf/gcs_service.proto | 12 ++++++++++ src/ray/raylet/node_manager.cc | 14 +++++++++-- .../placement_group_resource_manager_test.cc | 3 ++- src/ray/raylet/raylet.cc | 6 ++++- .../scheduling/cluster_resource_scheduler.cc | 11 ++++++--- .../scheduling/cluster_resource_scheduler.h | 4 +++- .../cluster_resource_scheduler_test.cc | 3 ++- .../scheduling/cluster_task_manager_test.cc | 12 ++++++---- .../scheduling/local_resource_manager.cc | 18 ++++++++++++-- .../scheduling/local_resource_manager.h | 9 ++++++- .../scheduling/local_resource_manager_test.cc | 6 +++-- src/ray/rpc/gcs_server/gcs_rpc_client.h | 8 ++++++- src/ray/rpc/gcs_server/gcs_rpc_server.h | 4 ++++ 19 files changed, 145 insertions(+), 19 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 3a0178d399d21..2206d32cf18bc 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -441,6 +441,30 @@ Status NodeInfoAccessor::RegisterSelf(const GcsNodeInfo &local_node_info, return Status::OK(); } +Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_info) { + if (local_node_id_.IsNil()) { + RAY_LOG(INFO) << "The node is already unregistered."; + // This node is already unregistered. + return Status::OK(); + } + auto node_id = NodeID::FromBinary(local_node_info_.node_id()); + RAY_CHECK(!node_id.IsNil()) << "Node ID cannnot be nil."; + rpc::UnregisterNodeRequest request; + request.set_node_id(local_node_info_.node_id()); + request.mutable_node_death_info()->CopyFrom(node_death_info); + client_impl_->GetGcsRpcClient().UnregisterNode( + request, + [this, node_id](const Status &status, const rpc::UnregisterNodeReply &reply) { + if (status.ok()) { + local_node_info_.set_state(GcsNodeInfo::DEAD); + local_node_id_ = NodeID::Nil(); + } + RAY_LOG(DEBUG) << "Finished unregistering node info, status = " << status + << ", node id = " << node_id; + }); + return Status::OK(); +} + Status NodeInfoAccessor::DrainSelf() { if (local_node_id_.IsNil()) { RAY_LOG(INFO) << "The node is already drained."; diff --git a/src/ray/gcs/gcs_client/accessor.h b/src/ray/gcs/gcs_client/accessor.h index 195b4b0724933..cabc58c3d2f2a 100644 --- a/src/ray/gcs/gcs_client/accessor.h +++ b/src/ray/gcs/gcs_client/accessor.h @@ -312,6 +312,8 @@ class NodeInfoAccessor { virtual Status AsyncRegister(const rpc::GcsNodeInfo &node_info, const StatusCallback &callback); + virtual Status UnregisterSelf(const rpc::NodeDeathInfo &node_death_info); + /// Send a check alive request to GCS for the liveness of some node. /// /// \param callback The callback function once the request is finished. diff --git a/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.cc b/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.cc index 16a3aa2deb856..4d79a412c12f7 100644 --- a/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.cc @@ -395,6 +395,7 @@ void GcsAutoscalerStateManager::HandleDrainNode( } else { death_info->set_reason(rpc::NodeDeathInfo::AUTOSCALER_DRAIN_IDLE); } + death_info->set_reason_message(request.reason_message()); if (RayConfig::instance().enable_reap_actor_death()) { gcs_actor_manager_.SetPreemptedAndPublish(node_id); } diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.cc b/src/ray/gcs/gcs_server/gcs_node_manager.cc index 0c7ba7eec63f5..cacfeeb2fe796 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_node_manager.cc @@ -108,6 +108,26 @@ void GcsNodeManager::HandleCheckAlive(rpc::CheckAliveRequest request, GCS_RPC_SEND_REPLY(send_reply_callback, reply, Status::OK()); } +void GcsNodeManager::HandleUnregisterNode(rpc::UnregisterNodeRequest request, + rpc::UnregisterNodeReply *reply, + rpc::SendReplyCallback send_reply_callback) { + NodeID node_id = NodeID::FromBinary(request.node_id()); + auto node = RemoveNode(node_id, /* is_intended = */ true); + if (!node) { + RAY_LOG(INFO) << "Node " << node_id << " is already removed"; + return; + } + node->set_state(rpc::GcsNodeInfo::DEAD); + node->set_end_time_ms(current_sys_time_ms()); + AddDeadNodeToCache(node); + auto node_info_delta = std::make_shared(); + node_info_delta->set_node_id(node->node_id()); + node_info_delta->set_state(node->state()); + node_info_delta->set_end_time_ms(node->end_time_ms()); + // Update node state to DEAD instead of deleting it. + RAY_CHECK_OK(gcs_table_storage_->NodeTable().Put(node_id, *node, nullptr)); +} + void GcsNodeManager::HandleDrainNode(rpc::DrainNodeRequest request, rpc::DrainNodeReply *reply, rpc::SendReplyCallback send_reply_callback) { diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.h b/src/ray/gcs/gcs_server/gcs_node_manager.h index e2375a8caf7fe..aad591d934e7d 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.h +++ b/src/ray/gcs/gcs_server/gcs_node_manager.h @@ -62,6 +62,11 @@ class GcsNodeManager : public rpc::NodeInfoHandler { rpc::RegisterNodeReply *reply, rpc::SendReplyCallback send_reply_callback) override; + /// Handle unregister rpc request come from raylet. + void HandleUnregisterNode(rpc::UnregisterNodeRequest request, + rpc::UnregisterNodeReply *reply, + rpc::SendReplyCallback send_reply_callback) override; + /// Handle unregister rpc request come from raylet. void HandleDrainNode(rpc::DrainNodeRequest request, rpc::DrainNodeReply *reply, diff --git a/src/ray/protobuf/gcs.proto b/src/ray/protobuf/gcs.proto index 34e0a2a4e1e5d..652310c3b0bf4 100644 --- a/src/ray/protobuf/gcs.proto +++ b/src/ray/protobuf/gcs.proto @@ -328,6 +328,8 @@ message NodeDeathInfo { AUTOSCALER_DRAIN_IDLE = 4; } Reason reason = 1; + // A message describing the reason for the node death. + string reason_message = 2; } message GcsNodeInfo { diff --git a/src/ray/protobuf/gcs_service.proto b/src/ray/protobuf/gcs_service.proto index 3963ce8452961..fd184a5f93879 100644 --- a/src/ray/protobuf/gcs_service.proto +++ b/src/ray/protobuf/gcs_service.proto @@ -193,6 +193,16 @@ message RegisterNodeReply { GcsStatus status = 1; } +message UnregisterNodeRequest { + // The node id of the node to be unregistered. + bytes node_id = 1; + NodeDeathInfo node_death_info = 2; +} + +message UnregisterNodeReply { + GcsStatus status = 1; +} + message GetAllNodeInfoRequest {} message GetAllNodeInfoReply { @@ -661,6 +671,8 @@ service NodeInfoGcsService { rpc GetClusterId(GetClusterIdRequest) returns (GetClusterIdReply); // Register a node to GCS Service. rpc RegisterNode(RegisterNodeRequest) returns (RegisterNodeReply); + // Unregister a node to GCS Service. + rpc UnregisterNode(UnregisterNodeRequest) returns (UnregisterNodeReply); // Drain a node from GCS Service. // When the RPC is replied, // - It is guaranteed that the requested nodes are going to be drained eventually. diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index e86ea9ea64141..557b7e899a23a 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -309,6 +309,10 @@ NodeManager::NodeManager(instrumented_io_context &io_service, }, /*get_pull_manager_at_capacity*/ [this]() { return object_manager_.PullManagerHasPullsQueued(); }, + [this](rpc::NodeDeathInfo node_death_info) { + Status status = gcs_client_->Nodes().UnregisterSelf(node_death_info); + return status.ok(); + }, /*labels*/ config.labels); @@ -1945,8 +1949,11 @@ void NodeManager::HandleDrainRaylet(rpc::DrainRayletRequest request, const bool is_idle = cluster_resource_scheduler_->GetLocalResourceManager().IsLocalNodeIdle(); if (is_idle) { + rpc::NodeDeathInfo node_death_info; + node_death_info.set_reason(rpc::NodeDeathInfo::AUTOSCALER_DRAIN_IDLE); + node_death_info.set_reason_message(request.reason_message()); cluster_resource_scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - request.deadline_timestamp_ms()); + request.deadline_timestamp_ms(), node_death_info); reply->set_is_accepted(true); } else { reply->set_is_accepted(false); @@ -1957,8 +1964,11 @@ void NodeManager::HandleDrainRaylet(rpc::DrainRayletRequest request, // Non-rejectable draining request. RAY_CHECK_EQ(request.reason(), rpc::autoscaler::DrainNodeReason::DRAIN_NODE_REASON_PREEMPTION); + rpc::NodeDeathInfo node_death_info; + node_death_info.set_reason(rpc::NodeDeathInfo::AUTOSCALER_DRAIN_PREEMPTED); + node_death_info.set_reason_message(request.reason_message()); cluster_resource_scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - request.deadline_timestamp_ms()); + request.deadline_timestamp_ms(), node_death_info); reply->set_is_accepted(true); } diff --git a/src/ray/raylet/placement_group_resource_manager_test.cc b/src/ray/raylet/placement_group_resource_manager_test.cc index 721003974c321..a46694fe76a3a 100644 --- a/src/ray/raylet/placement_group_resource_manager_test.cc +++ b/src/ray/raylet/placement_group_resource_manager_test.cc @@ -184,8 +184,9 @@ TEST_F(NewPlacementGroupResourceManagerTest, TestNewPrepareBundleDuringDraining) ASSERT_TRUE(new_placement_group_resource_manager_->PrepareBundles(bundle1_specs)); // Drain the node, new bundle prepare will fail. + rpc::NodeDeathInfo node_death_info; // TODO: fill in proper values cluster_resource_scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max()); + std::numeric_limits::max(), node_death_info); ASSERT_FALSE(new_placement_group_resource_manager_->PrepareBundles(bundle2_specs)); // Prepared bundles can still be committed. new_placement_group_resource_manager_->CommitBundles(bundle1_specs); diff --git a/src/ray/raylet/raylet.cc b/src/ray/raylet/raylet.cc index ca4ddfcddafbc..5f8f4f9d45624 100644 --- a/src/ray/raylet/raylet.cc +++ b/src/ray/raylet/raylet.cc @@ -117,7 +117,11 @@ void Raylet::Start() { } void Raylet::Stop() { - RAY_CHECK_OK(gcs_client_->Nodes().DrainSelf()); + // RAY_CHECK_OK(gcs_client_->Nodes().DrainSelf()); + rpc::NodeDeathInfo node_death_info; + node_death_info.set_reason(rpc::NodeDeathInfo::EXPECTED_TERMINATION); + node_death_info.set_reason_message("Raylet stop"); + RAY_CHECK_OK(gcs_client_->Nodes().UnregisterSelf(node_death_info)); node_manager_.Stop(); acceptor_.close(); } diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler.cc b/src/ray/raylet/scheduling/cluster_resource_scheduler.cc index 518cb185b51f6..3102942426093 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler.cc +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler.cc @@ -35,7 +35,8 @@ ClusterResourceScheduler::ClusterResourceScheduler( Init(io_service, local_node_resources, /*get_used_object_store_memory=*/nullptr, - /*get_pull_manager_at_capacity=*/nullptr); + /*get_pull_manager_at_capacity=*/nullptr, + /*unregister_self=*/nullptr); } ClusterResourceScheduler::ClusterResourceScheduler( @@ -45,6 +46,7 @@ ClusterResourceScheduler::ClusterResourceScheduler( std::function is_node_available_fn, std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, + std::function unregister_self, const absl::flat_hash_map &local_node_labels) : local_node_id_(local_node_id), is_node_available_fn_(is_node_available_fn) { NodeResources node_resources = ResourceMapToNodeResources( @@ -52,20 +54,23 @@ ClusterResourceScheduler::ClusterResourceScheduler( Init(io_service, node_resources, get_used_object_store_memory, - get_pull_manager_at_capacity); + get_pull_manager_at_capacity, + unregister_self); } void ClusterResourceScheduler::Init( instrumented_io_context &io_service, const NodeResources &local_node_resources, std::function get_used_object_store_memory, - std::function get_pull_manager_at_capacity) { + std::function get_pull_manager_at_capacity, + std::function unregister_self) { cluster_resource_manager_ = std::make_unique(io_service); local_resource_manager_ = std::make_unique( local_node_id_, local_node_resources, get_used_object_store_memory, get_pull_manager_at_capacity, + unregister_self, [this](const NodeResources &local_resource_update) { cluster_resource_manager_->AddOrUpdateNode(local_node_id_, local_resource_update); }); diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler.h b/src/ray/raylet/scheduling/cluster_resource_scheduler.h index 6841ac644c5bf..524e65ac5a7b0 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler.h +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler.h @@ -63,6 +63,7 @@ class ClusterResourceScheduler { std::function is_node_available_fn, std::function get_used_object_store_memory = nullptr, std::function get_pull_manager_at_capacity = nullptr, + std::function unregister_self = nullptr, const absl::flat_hash_map &local_node_labels = {}); /// Schedule the specified resources to the cluster nodes. @@ -133,7 +134,8 @@ class ClusterResourceScheduler { void Init(instrumented_io_context &io_service, const NodeResources &local_node_resources, std::function get_used_object_store_memory, - std::function get_pull_manager_at_capacity); + std::function get_pull_manager_at_capacity, + std::function unregister_self); bool NodeAvailable(scheduling::NodeID node_id) const; diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler_test.cc b/src/ray/raylet/scheduling/cluster_resource_scheduler_test.cc index 792eaf790f192..d9c3b67b4c6f1 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler_test.cc +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler_test.cc @@ -362,8 +362,9 @@ TEST_F(ClusterResourceSchedulerTest, NodeAffinitySchedulingStrategyTest) { ASSERT_TRUE(resource_scheduler.GetLocalResourceManager().AllocateLocalTaskResources( resource_request, task_allocation)); // Drain the local node so that it's not schedulable for new tasks. + rpc::NodeDeathInfo node_death_info; resource_scheduler.GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max()); + std::numeric_limits::max(), node_death_info); scheduling_strategy.mutable_node_affinity_scheduling_strategy()->set_node_id( local_node_id.Binary()); diff --git a/src/ray/raylet/scheduling/cluster_task_manager_test.cc b/src/ray/raylet/scheduling/cluster_task_manager_test.cc index 149c95677c37a..ef7ceccf130e4 100644 --- a/src/ray/raylet/scheduling/cluster_task_manager_test.cc +++ b/src/ray/raylet/scheduling/cluster_task_manager_test.cc @@ -743,8 +743,9 @@ TEST_F(ClusterTaskManagerTest, DrainingWhileResolving) { ASSERT_EQ(pool_.workers.size(), 1); // Drain the local node. + rpc::NodeDeathInfo node_death_info; // TODO: fill in values scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max()); + std::numeric_limits::max(), node_death_info); // Arg is resolved. missing_objects_.erase(missing_arg); @@ -1077,8 +1078,9 @@ TEST_F(ClusterTaskManagerTest, NotOKPopWorkerAfterDrainingTest) { AddNode(remote_node_id, 5); // Drain the local node. + rpc::NodeDeathInfo node_death_info; // TODO: fill in values scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max()); + std::numeric_limits::max(), node_death_info); pool_.callbacks[task1.GetTaskSpecification().GetRuntimeEnvHash()].front()( nullptr, PopWorkerStatus::WorkerPendingRegistration, ""); @@ -2620,8 +2622,9 @@ TEST_F(ClusterTaskManagerTest, PopWorkerBeforeDraining) { task_manager_.QueueAndScheduleTask(task, false, false, &reply, callback); // Drain the local node. + rpc::NodeDeathInfo node_death_info; // TODO: fill in values scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max()); + std::numeric_limits::max(), node_death_info); std::shared_ptr worker = std::make_shared(WorkerID::FromRandom(), 1234); @@ -2659,8 +2662,9 @@ TEST_F(ClusterTaskManagerTest, UnscheduleableWhileDraining) { AddNode(remote_node_id, 5); // Drain the local node. + rpc::NodeDeathInfo node_death_info; // TODO: fill in values scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max()); + std::numeric_limits::max(), node_death_info); RayTask spillback_task = CreateTask({{ray::kCPU_ResourceLabel, 1}}); rpc::RequestWorkerLeaseReply spillback_reply; diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 09d72d1e03564..0a7c10d651d0a 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -28,10 +28,12 @@ LocalResourceManager::LocalResourceManager( const NodeResources &node_resources, std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, + std::function unregister_self, std::function resource_change_subscriber) : local_node_id_(local_node_id), get_used_object_store_memory_(get_used_object_store_memory), get_pull_manager_at_capacity_(get_pull_manager_at_capacity), + unregister_self_(unregister_self), resource_change_subscriber_(resource_change_subscriber) { RAY_CHECK(node_resources.total == node_resources.available); local_resources_.available = NodeResourceInstanceSet(node_resources.total); @@ -382,7 +384,17 @@ std::optional LocalResourceManager::CreateSyncMessage( void LocalResourceManager::OnResourceOrStateChanged() { if (IsLocalNodeDraining() && IsLocalNodeIdle()) { // The node is drained. - RAY_LOG(INFO) << "The node is drained, exiting..."; + if (unregister_self_ != nullptr) { + RAY_LOG(INFO) << "The node is drained, unregister itself ..."; + if (!unregister_self_(node_death_info_)) { + RAY_LOG(ERROR) << "Failed to unregister the node."; + // Still shutdown? + } else { + RAY_LOG(INFO) << "The node is unregistered, exiting ..."; + } + } else { + RAY_LOG(INFO) << "The node is drained, exiting ..."; + } raylet::ShutdownRayletGracefully(); } @@ -444,10 +456,12 @@ void LocalResourceManager::RecordMetrics() const { } } -void LocalResourceManager::SetLocalNodeDraining(int64_t draining_deadline_timestamp_ms) { +void LocalResourceManager::SetLocalNodeDraining( + int64_t draining_deadline_timestamp_ms, const rpc::NodeDeathInfo &node_death_info) { RAY_CHECK_GE(draining_deadline_timestamp_ms, 0); is_local_node_draining_ = true; local_node_draining_deadline_timestamp_ms_ = draining_deadline_timestamp_ms; + node_death_info_ = node_death_info; OnResourceOrStateChanged(); } diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index 7d98c4e7b0e00..1dbbb6cb004e4 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -58,6 +58,7 @@ class LocalResourceManager : public syncer::ReporterInterface { const NodeResources &node_resources, std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, + std::function unregister_self, std::function resource_change_subscriber); scheduling::NodeID GetNodeId() const { return local_node_id_; } @@ -153,7 +154,8 @@ class LocalResourceManager : public syncer::ReporterInterface { /// Change the local node to the draining state. /// After that, no new tasks can be scheduled onto the local node. - void SetLocalNodeDraining(int64_t draining_deadline_timestamp_ms); + void SetLocalNodeDraining(int64_t draining_deadline_timestamp_ms, + const rpc::NodeDeathInfo &node_death_info); bool IsLocalNodeDraining() const { return is_local_node_draining_; } @@ -215,6 +217,9 @@ class LocalResourceManager : public syncer::ReporterInterface { std::function get_used_object_store_memory_; /// Function to get whether the pull manager is at capacity. std::function get_pull_manager_at_capacity_; + /// Function to unregister the local node. + std::function unregister_self_; + /// Subscribes to resource changes. std::function resource_change_subscriber_; @@ -228,6 +233,8 @@ class LocalResourceManager : public syncer::ReporterInterface { // 0 if there is no deadline. int64_t local_node_draining_deadline_timestamp_ms_ = -1; + rpc::NodeDeathInfo node_death_info_; + FRIEND_TEST(ClusterResourceSchedulerTest, SchedulingUpdateTotalResourcesTest); FRIEND_TEST(ClusterResourceSchedulerTest, AvailableResourceInstancesOpsTest); FRIEND_TEST(ClusterResourceSchedulerTest, TaskResourceInstancesTest); diff --git a/src/ray/raylet/scheduling/local_resource_manager_test.cc b/src/ray/raylet/scheduling/local_resource_manager_test.cc index ea79aabd15d04..0c2319904b0ad 100644 --- a/src/ray/raylet/scheduling/local_resource_manager_test.cc +++ b/src/ray/raylet/scheduling/local_resource_manager_test.cc @@ -153,7 +153,8 @@ TEST_F(LocalResourceManagerTest, NodeDrainingTest) { manager->AllocateLocalTaskResources(resource_request, task_allocation); } - manager->SetLocalNodeDraining(std::numeric_limits::max()); + rpc::NodeDeathInfo node_death_info; // TODO: fill in values + manager->SetLocalNodeDraining(std::numeric_limits::max(), node_death_info); ASSERT_TRUE(manager->IsLocalNodeDraining()); // Make the node idle so that the node is drained and terminated. @@ -178,7 +179,8 @@ TEST_F(LocalResourceManagerTest, ObjectStoreMemoryDrainingTest) { *used_object_store = 1; manager->UpdateAvailableObjectStoreMemResource(); - manager->SetLocalNodeDraining(std::numeric_limits::max()); + rpc::NodeDeathInfo node_death_info; // TODO: fill in values + manager->SetLocalNodeDraining(std::numeric_limits::max(), node_death_info); ASSERT_TRUE(manager->IsLocalNodeDraining()); // Free object store memory so that the node is drained and terminated. diff --git a/src/ray/rpc/gcs_server/gcs_rpc_client.h b/src/ray/rpc/gcs_server/gcs_rpc_client.h index 67bdc43ad42e2..e1ecd6ad16adb 100644 --- a/src/ray/rpc/gcs_server/gcs_rpc_client.h +++ b/src/ray/rpc/gcs_server/gcs_rpc_client.h @@ -326,12 +326,18 @@ class GcsRpcClient { node_info_grpc_client_, /*method_timeout_ms*/ -1, ) - /// Unregister a node from GCS Service. + /// Drain a node from GCS Service. VOID_GCS_RPC_CLIENT_METHOD(NodeInfoGcsService, DrainNode, node_info_grpc_client_, /*method_timeout_ms*/ -1, ) + /// Unregister a node from GCS Service. + VOID_GCS_RPC_CLIENT_METHOD(NodeInfoGcsService, + UnregisterNode, + node_info_grpc_client_, + /*method_timeout_ms*/ -1, ) + /// Get information of all nodes from GCS Service. VOID_GCS_RPC_CLIENT_METHOD(NodeInfoGcsService, GetAllNodeInfo, diff --git a/src/ray/rpc/gcs_server/gcs_rpc_server.h b/src/ray/rpc/gcs_server/gcs_rpc_server.h index 16c94c23f4bfd..acf677ef7a261 100644 --- a/src/ray/rpc/gcs_server/gcs_rpc_server.h +++ b/src/ray/rpc/gcs_server/gcs_rpc_server.h @@ -300,6 +300,10 @@ class NodeInfoGcsServiceHandler { RegisterNodeReply *reply, SendReplyCallback send_reply_callback) = 0; + virtual void HandleUnregisterNode(UnregisterNodeRequest request, + UnregisterNodeReply *reply, + SendReplyCallback send_reply_callback) = 0; + virtual void HandleCheckAlive(CheckAliveRequest request, CheckAliveReply *reply, SendReplyCallback send_reply_callback) = 0; From 5d46d4519eaebaaf203a347f6d647696f7094ae7 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Fri, 3 May 2024 18:27:24 -0700 Subject: [PATCH 02/40] WIP: to fix compilation error Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 6 ++- src/ray/raylet/agent_manager.cc | 5 +- src/ray/raylet/agent_manager.h | 3 ++ src/ray/raylet/main.cc | 52 +++++++++++-------- src/ray/raylet/node_manager.cc | 16 ++++-- src/ray/raylet/node_manager.h | 5 +- src/ray/raylet/raylet.cc | 12 ++--- src/ray/raylet/raylet.h | 3 +- src/ray/raylet/runtime_env_agent_client.cc | 10 +++- .../scheduling/cluster_resource_scheduler.cc | 11 ++-- .../scheduling/cluster_resource_scheduler.h | 4 +- .../scheduling/local_resource_manager.cc | 28 +++++----- .../scheduling/local_resource_manager.h | 3 ++ 13 files changed, 104 insertions(+), 54 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 2206d32cf18bc..debeb838d8ca8 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -452,17 +452,19 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf rpc::UnregisterNodeRequest request; request.set_node_id(local_node_info_.node_id()); request.mutable_node_death_info()->CopyFrom(node_death_info); + std::promise promise; client_impl_->GetGcsRpcClient().UnregisterNode( request, - [this, node_id](const Status &status, const rpc::UnregisterNodeReply &reply) { + [this, node_id, &promise](const Status &status, const rpc::UnregisterNodeReply &reply) { if (status.ok()) { local_node_info_.set_state(GcsNodeInfo::DEAD); local_node_id_ = NodeID::Nil(); } RAY_LOG(DEBUG) << "Finished unregistering node info, status = " << status << ", node id = " << node_id; + promise.set_value(status); }); - return Status::OK(); + return promise.get_future().get(); } Status NodeInfoAccessor::DrainSelf() { diff --git a/src/ray/raylet/agent_manager.cc b/src/ray/raylet/agent_manager.cc index 38e176faac839..a055085569913 100644 --- a/src/ray/raylet/agent_manager.cc +++ b/src/ray/raylet/agent_manager.cc @@ -96,7 +96,10 @@ void AgentManager::StartAgent() { "https://docs.ray.io/en/master/ray-observability/user-guides/" "configure-logging.html#logging-directory-structure.\n" "- The agent is killed by the OS (e.g., out of memory)."; - ShutdownRayletGracefully(); + rpc::NodeDeathInfo node_death_info; + node_death_info.set_reason(rpc::NodeDeathInfo::UNEXPECTED_TERMINATION); + node_death_info.set_reason_message("Ray agent failed"); // TODO: add more info + shutdown_raylet_gracefully_(node_death_info); // If the process is not terminated within 10 seconds, forcefully kill raylet // itself. delay_executor_([]() { QuickExit(); }, /*ms*/ 10000); diff --git a/src/ray/raylet/agent_manager.h b/src/ray/raylet/agent_manager.h index 4594f0087d63d..30856c63d2dee 100644 --- a/src/ray/raylet/agent_manager.h +++ b/src/ray/raylet/agent_manager.h @@ -56,9 +56,11 @@ class AgentManager { explicit AgentManager(Options options, DelayExecutorFn delay_executor, + std::function shutdown_raylet_gracefully, bool start_agent = true /* for test */) : options_(std::move(options)), delay_executor_(std::move(delay_executor)), + shutdown_raylet_gracefully_(std::move(shutdown_raylet_gracefully)), fate_shares_(options_.fate_shares) { if (options_.agent_name.empty()) { RAY_LOG(FATAL) << "AgentManager agent_name must not be empty."; @@ -79,6 +81,7 @@ class AgentManager { const Options options_; Process process_; DelayExecutorFn delay_executor_; + std::function shutdown_raylet_gracefully_; // If true, when the agent dies, raylet kills itself. std::atomic fate_shares_; std::unique_ptr monitor_thread_; diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index e6c05c0fb17b0..7b106bf906a6b 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -224,6 +224,29 @@ int main(int argc, char *argv[]) { #endif }; + auto shutted_down = std::make_shared>(false); + + // Destroy the Raylet on a SIGTERM. The pointer to main_service is + // guaranteed to be valid since this function will run the event loop + // instead of returning immediately. + // We should stop the service and remove the local socket file. + auto shutdown_raylet_gracefully = [&main_service, &raylet_socket_name, &raylet, &gcs_client, shutted_down]( + const rpc::NodeDeathInfo &node_death_info) { + // Make the shutdown handler idempotent since graceful shutdown can be triggered + // by many places. + if (*shutted_down) { + RAY_LOG(INFO) << "Raylet shutdown already triggered, ignoring this request."; + return; + } + RAY_LOG(INFO) << "Raylet shutdown triggered, shutting down..."; + *shutted_down = true; + raylet->Stop(); + gcs_client->Disconnect(); + ray::stats::Shutdown(); + main_service.stop(); + remove(raylet_socket_name.c_str()); + }; + RAY_CHECK_OK(gcs_client->Nodes().AsyncGetInternalConfig( [&](::ray::Status status, const boost::optional &stored_raylet_config) { @@ -382,7 +405,8 @@ int main(int argc, char *argv[]) { object_manager_config, gcs_client, metrics_export_port, - is_head_node); + is_head_node, + shutdown_raylet_gracefully); // Initialize event framework. if (RayConfig::instance().event_log_reporter_enabled() && !log_dir.empty()) { @@ -396,27 +420,13 @@ int main(int argc, char *argv[]) { raylet->Start(); })); - auto shutted_down = std::make_shared>(false); - - // Destroy the Raylet on a SIGTERM. The pointer to main_service is - // guaranteed to be valid since this function will run the event loop - // instead of returning immediately. - // We should stop the service and remove the local socket file. - auto handler = [&main_service, &raylet_socket_name, &raylet, &gcs_client, shutted_down]( + auto signal_handler = [&main_service, &raylet_socket_name, &raylet, &gcs_client, shutted_down]( const boost::system::error_code &error, int signal_number) { - // Make the shutdown handler idempotent since graceful shutdown can be triggered - // by many places. - if (*shutted_down) { - RAY_LOG(INFO) << "Raylet already received SIGTERM. It will ignore the request."; - return; - } RAY_LOG(INFO) << "Raylet received SIGTERM, shutting down..."; - *shutted_down = true; - raylet->Stop(); - gcs_client->Disconnect(); - ray::stats::Shutdown(); - main_service.stop(); - remove(raylet_socket_name.c_str()); + rpc::NodeDeathInfo node_death_info; + node_death_info.set_reason(rpc::NodeDeathInfo::EXPECTED_TERMINATION); + node_death_info.set_reason_message("SIGTERM received"); + shutdown_raylet_gracefully(node_death_info); }; boost::asio::signal_set signals(main_service); #ifdef _WIN32 @@ -424,7 +434,7 @@ int main(int argc, char *argv[]) { #else signals.add(SIGTERM); #endif - signals.async_wait(handler); + signals.async_wait(signal_handler); main_service.run(); } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 557b7e899a23a..9da7e4fd845c6 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -104,11 +104,13 @@ NodeManager::NodeManager(instrumented_io_context &io_service, const std::string &self_node_name, const NodeManagerConfig &config, const ObjectManagerConfig &object_manager_config, - std::shared_ptr gcs_client) + std::shared_ptr gcs_client, + std::function shutdown_raylet_gracefully) : self_node_id_(self_node_id), self_node_name_(self_node_name), io_service_(io_service), gcs_client_(gcs_client), + shutdown_raylet_gracefully_(shutdown_raylet_gracefully), worker_pool_( io_service, self_node_id_, @@ -313,6 +315,7 @@ NodeManager::NodeManager(instrumented_io_context &io_service, Status status = gcs_client_->Nodes().UnregisterSelf(node_death_info); return status.ok(); }, + shutdown_raylet_gracefully, /*labels*/ config.labels); @@ -1994,7 +1997,10 @@ void NodeManager::HandleShutdownRaylet(rpc::ShutdownRayletRequest request, // Note that the callback is posted to the io service after the shutdown GRPC request // is replied. Otherwise, the RPC might not be replied to GCS before it shutsdown // itself. - ShutdownRayletGracefully(); + rpc::NodeDeathInfo node_death_info; + node_death_info.reason = rpc::NodeDeathInfo::EXPECTED_TERMINATION; + node_death_info.reason_message = "ShutdownRaylet RPC has been received."; + shutdown_raylet_gracefully_(node_death_info); }; is_shutdown_request_received_ = true; send_reply_callback(Status::OK(), shutdown_after_reply, shutdown_after_reply); @@ -3070,7 +3076,8 @@ std::unique_ptr NodeManager::CreateDashboardAgentManager( /*delay_executor=*/ [this](std::function task, uint32_t delay_ms) { return execute_after(io_service_, task, std::chrono::milliseconds(delay_ms)); - }); + }, + shutdown_raylet_gracefully_); } std::unique_ptr NodeManager::CreateRuntimeEnvAgentManager( @@ -3101,7 +3108,8 @@ std::unique_ptr NodeManager::CreateRuntimeEnvAgentManager( /*delay_executor=*/ [this](std::function task, uint32_t delay_ms) { return execute_after(io_service_, task, std::chrono::milliseconds(delay_ms)); - }); + }, + shutdown_raylet_gracefully_); } } // namespace raylet diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index 43608aa8334e2..aef652ea8f011 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -135,7 +135,8 @@ class NodeManager : public rpc::NodeManagerServiceHandler, const std::string &self_node_name, const NodeManagerConfig &config, const ObjectManagerConfig &object_manager_config, - std::shared_ptr gcs_client); + std::shared_ptr gcs_client, + std::function shutdown_raylet_gracefully); /// Process a new client connection. /// @@ -884,6 +885,8 @@ class NodeManager : public rpc::NodeManagerServiceHandler, std::unique_ptr memory_monitor_; std::unique_ptr mutable_object_provider_; + + std::function shutdown_raylet_gracefully_; }; } // namespace raylet diff --git a/src/ray/raylet/raylet.cc b/src/ray/raylet/raylet.cc index 5f8f4f9d45624..959fb34f325e5 100644 --- a/src/ray/raylet/raylet.cc +++ b/src/ray/raylet/raylet.cc @@ -65,7 +65,8 @@ Raylet::Raylet(instrumented_io_context &main_service, const ObjectManagerConfig &object_manager_config, std::shared_ptr gcs_client, int metrics_export_port, - bool is_head_node) + bool is_head_node, + std::function shutdown_raylet_gracefully) : self_node_id_(self_node_id), gcs_client_(gcs_client), node_manager_(main_service, @@ -73,7 +74,8 @@ Raylet::Raylet(instrumented_io_context &main_service, node_name, node_manager_config, object_manager_config, - gcs_client_), + gcs_client_, + shutdown_raylet_gracefully), socket_name_(socket_name), acceptor_(main_service, ParseUrlEndpoint(socket_name)), socket_(main_service) { @@ -116,11 +118,7 @@ void Raylet::Start() { DoAccept(); } -void Raylet::Stop() { - // RAY_CHECK_OK(gcs_client_->Nodes().DrainSelf()); - rpc::NodeDeathInfo node_death_info; - node_death_info.set_reason(rpc::NodeDeathInfo::EXPECTED_TERMINATION); - node_death_info.set_reason_message("Raylet stop"); +void Raylet::Stop(const rpc::NodeDeathInfo &node_death_info) { RAY_CHECK_OK(gcs_client_->Nodes().UnregisterSelf(node_death_info)); node_manager_.Stop(); acceptor_.close(); diff --git a/src/ray/raylet/raylet.h b/src/ray/raylet/raylet.h index 93144abebcb59..f1ddbc8f3c3e9 100644 --- a/src/ray/raylet/raylet.h +++ b/src/ray/raylet/raylet.h @@ -58,7 +58,8 @@ class Raylet { const ObjectManagerConfig &object_manager_config, std::shared_ptr gcs_client, int metrics_export_port, - bool is_head_node); + bool is_head_node, + std::function shutdown_raylet_gracefully); /// Start this raylet. void Start(); diff --git a/src/ray/raylet/runtime_env_agent_client.cc b/src/ray/raylet/runtime_env_agent_client.cc index bae9676f92cb9..4c2f45cb16e86 100644 --- a/src/ray/raylet/runtime_env_agent_client.cc +++ b/src/ray/raylet/runtime_env_agent_client.cc @@ -256,6 +256,7 @@ class HttpRuntimeEnvAgentClient : public RuntimeEnvAgentClient { int port, std::function( std::function, uint32_t delay_ms)> delay_executor, + std::function shutdown_raylet_gracefully, uint32_t agent_register_timeout_ms, uint32_t agent_manager_retry_interval_ms, uint32_t session_pool_size = 10) @@ -264,6 +265,7 @@ class HttpRuntimeEnvAgentClient : public RuntimeEnvAgentClient { address_(address), port_str_(std::to_string(port)), delay_executor_(delay_executor), + shutdown_raylet_gracefully_(shutdown_raylet_gracefully), agent_register_timeout_ms_(agent_register_timeout_ms), agent_manager_retry_interval_ms_(agent_manager_retry_interval_ms) {} ~HttpRuntimeEnvAgentClient() = default; @@ -283,7 +285,10 @@ class HttpRuntimeEnvAgentClient : public RuntimeEnvAgentClient { "file structure here " "https://docs.ray.io/en/master/ray-observability/user-guides/" "configure-logging.html#logging-directory-structure.\n"; - ShutdownRayletGracefully(); + rpc::NodeDeathInfo node_death_info; + node_death_info.set_reason(rpc::NodeDeathInfo_Reason::NodeDeathInfo_Reason_UNEXPECTED_TERMINATION); + node_death_info.set_reason_message("Runtime Env Agent timed out"); + shutdown_raylet_gracefully_(node_death_info); // If the process is not terminated within 10 seconds, forcefully kill itself. delay_executor_([]() { QuickExit(); }, /*ms*/ 10000); } @@ -500,6 +505,7 @@ class HttpRuntimeEnvAgentClient : public RuntimeEnvAgentClient { std::function(std::function, uint32_t delay_ms)> delay_executor_; + std::function shutdown_raylet_gracefully_; const uint32_t agent_register_timeout_ms_; const uint32_t agent_manager_retry_interval_ms_; }; @@ -511,12 +517,14 @@ std::shared_ptr RuntimeEnvAgentClient::Create( int port, std::function( std::function, uint32_t delay_ms)> delay_executor, + std::function shutdown_raylet_gracefully,, uint32_t agent_register_timeout_ms, uint32_t agent_manager_retry_interval_ms) { return std::make_shared(io_context, address, port, delay_executor, + shutdown_raylet_gracefully, agent_register_timeout_ms, agent_manager_retry_interval_ms); } diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler.cc b/src/ray/raylet/scheduling/cluster_resource_scheduler.cc index 3102942426093..234b0fe9bb169 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler.cc +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler.cc @@ -36,7 +36,8 @@ ClusterResourceScheduler::ClusterResourceScheduler( local_node_resources, /*get_used_object_store_memory=*/nullptr, /*get_pull_manager_at_capacity=*/nullptr, - /*unregister_self=*/nullptr); + /*unregister_self=*/nullptr, + /*shutdown_raylet_gracefully=*/nullptr); } ClusterResourceScheduler::ClusterResourceScheduler( @@ -47,6 +48,7 @@ ClusterResourceScheduler::ClusterResourceScheduler( std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, std::function unregister_self, + std::function shutdown_raylet_gracefully, const absl::flat_hash_map &local_node_labels) : local_node_id_(local_node_id), is_node_available_fn_(is_node_available_fn) { NodeResources node_resources = ResourceMapToNodeResources( @@ -55,7 +57,8 @@ ClusterResourceScheduler::ClusterResourceScheduler( node_resources, get_used_object_store_memory, get_pull_manager_at_capacity, - unregister_self); + unregister_self, + shutdown_raylet_gracefully); } void ClusterResourceScheduler::Init( @@ -63,7 +66,8 @@ void ClusterResourceScheduler::Init( const NodeResources &local_node_resources, std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, - std::function unregister_self) { + std::function unregister_self, + std::function shutdown_raylet_gracefully) { cluster_resource_manager_ = std::make_unique(io_service); local_resource_manager_ = std::make_unique( local_node_id_, @@ -71,6 +75,7 @@ void ClusterResourceScheduler::Init( get_used_object_store_memory, get_pull_manager_at_capacity, unregister_self, + shutdown_raylet_gracefully, [this](const NodeResources &local_resource_update) { cluster_resource_manager_->AddOrUpdateNode(local_node_id_, local_resource_update); }); diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler.h b/src/ray/raylet/scheduling/cluster_resource_scheduler.h index 524e65ac5a7b0..b68f874e38324 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler.h +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler.h @@ -64,6 +64,7 @@ class ClusterResourceScheduler { std::function get_used_object_store_memory = nullptr, std::function get_pull_manager_at_capacity = nullptr, std::function unregister_self = nullptr, + std::function shutdown_raylet_gracefully = nullptr, const absl::flat_hash_map &local_node_labels = {}); /// Schedule the specified resources to the cluster nodes. @@ -135,7 +136,8 @@ class ClusterResourceScheduler { const NodeResources &local_node_resources, std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, - std::function unregister_self); + std::function unregister_self, + std::function shutdown_raylet_gracefully); bool NodeAvailable(scheduling::NodeID node_id) const; diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 0a7c10d651d0a..42e39b6b9b870 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -29,11 +29,13 @@ LocalResourceManager::LocalResourceManager( std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, std::function unregister_self, + std::function shutdown_raylet_gracefully, std::function resource_change_subscriber) : local_node_id_(local_node_id), get_used_object_store_memory_(get_used_object_store_memory), get_pull_manager_at_capacity_(get_pull_manager_at_capacity), unregister_self_(unregister_self), + shutdown_raylet_gracefully_(shutdown_raylet_gracefully), resource_change_subscriber_(resource_change_subscriber) { RAY_CHECK(node_resources.total == node_resources.available); local_resources_.available = NodeResourceInstanceSet(node_resources.total); @@ -384,18 +386,20 @@ std::optional LocalResourceManager::CreateSyncMessage( void LocalResourceManager::OnResourceOrStateChanged() { if (IsLocalNodeDraining() && IsLocalNodeIdle()) { // The node is drained. - if (unregister_self_ != nullptr) { - RAY_LOG(INFO) << "The node is drained, unregister itself ..."; - if (!unregister_self_(node_death_info_)) { - RAY_LOG(ERROR) << "Failed to unregister the node."; - // Still shutdown? - } else { - RAY_LOG(INFO) << "The node is unregistered, exiting ..."; - } - } else { - RAY_LOG(INFO) << "The node is drained, exiting ..."; - } - raylet::ShutdownRayletGracefully(); + + // TODO: remove the following and clean up unregister_self_. + // if (unregister_self_ != nullptr) { + // RAY_LOG(INFO) << "The node is drained, unregister itself ..."; + // if (!unregister_self_(node_death_info_)) { + // RAY_LOG(ERROR) << "Failed to unregister the node."; + // // Still shutdown? + // } else { + // RAY_LOG(INFO) << "The node is unregistered, exiting ..."; + // } + // } else { + // RAY_LOG(INFO) << "The node is drained, exiting ..."; + // } + shutdown_raylet_gracefully_(node_death_info_); } ++version_; diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index 1dbbb6cb004e4..9c8f17f190d08 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -59,6 +59,7 @@ class LocalResourceManager : public syncer::ReporterInterface { std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, std::function unregister_self, + std::function shutdown_raylet_gracefully, std::function resource_change_subscriber); scheduling::NodeID GetNodeId() const { return local_node_id_; } @@ -219,6 +220,8 @@ class LocalResourceManager : public syncer::ReporterInterface { std::function get_pull_manager_at_capacity_; /// Function to unregister the local node. std::function unregister_self_; + /// Function to shutdown the raylet gracefully. + std::function shutdown_raylet_gracefully_; /// Subscribes to resource changes. std::function resource_change_subscriber_; From 4bca6207a4abe9eb81748bbbc00e2f1f6b90adf7 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Mon, 6 May 2024 08:39:33 -0700 Subject: [PATCH 03/40] fix compilation Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 3 +- src/ray/raylet/agent_manager.cc | 2 +- src/ray/raylet/agent_manager.h | 10 +++-- src/ray/raylet/main.cc | 42 ++++++++++--------- src/ray/raylet/node_manager.cc | 27 ++++++------ src/ray/raylet/node_manager.h | 4 +- src/ray/raylet/raylet.h | 2 +- src/ray/raylet/runtime_env_agent_client.cc | 24 ++++++----- src/ray/raylet/runtime_env_agent_client.h | 2 + .../scheduling/cluster_resource_scheduler.h | 3 +- 10 files changed, 66 insertions(+), 53 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index debeb838d8ca8..8e7b6bca2289d 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -455,7 +455,8 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf std::promise promise; client_impl_->GetGcsRpcClient().UnregisterNode( request, - [this, node_id, &promise](const Status &status, const rpc::UnregisterNodeReply &reply) { + [this, node_id, &promise](const Status &status, + const rpc::UnregisterNodeReply &reply) { if (status.ok()) { local_node_info_.set_state(GcsNodeInfo::DEAD); local_node_id_ = NodeID::Nil(); diff --git a/src/ray/raylet/agent_manager.cc b/src/ray/raylet/agent_manager.cc index a055085569913..55a9a0920ee5b 100644 --- a/src/ray/raylet/agent_manager.cc +++ b/src/ray/raylet/agent_manager.cc @@ -98,7 +98,7 @@ void AgentManager::StartAgent() { "- The agent is killed by the OS (e.g., out of memory)."; rpc::NodeDeathInfo node_death_info; node_death_info.set_reason(rpc::NodeDeathInfo::UNEXPECTED_TERMINATION); - node_death_info.set_reason_message("Ray agent failed"); // TODO: add more info + node_death_info.set_reason_message("Ray agent failed"); // TODO: add more info shutdown_raylet_gracefully_(node_death_info); // If the process is not terminated within 10 seconds, forcefully kill raylet // itself. diff --git a/src/ray/raylet/agent_manager.h b/src/ray/raylet/agent_manager.h index 30856c63d2dee..9d5fea55965dd 100644 --- a/src/ray/raylet/agent_manager.h +++ b/src/ray/raylet/agent_manager.h @@ -23,6 +23,7 @@ #include "ray/common/id.h" #include "ray/util/process.h" +#include "src/ray/protobuf/gcs.pb.h" namespace ray { namespace raylet { @@ -54,10 +55,11 @@ class AgentManager { bool fate_shares; }; - explicit AgentManager(Options options, - DelayExecutorFn delay_executor, - std::function shutdown_raylet_gracefully, - bool start_agent = true /* for test */) + explicit AgentManager( + Options options, + DelayExecutorFn delay_executor, + std::function shutdown_raylet_gracefully, + bool start_agent = true /* for test */) : options_(std::move(options)), delay_executor_(std::move(delay_executor)), shutdown_raylet_gracefully_(std::move(shutdown_raylet_gracefully)), diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index 7b106bf906a6b..a7bdf25e928e8 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -31,6 +31,7 @@ #include "ray/util/event.h" #include "ray/util/process.h" #include "ray/util/subreaper.h" +#include "src/ray/protobuf/gcs.pb.h" using json = nlohmann::json; @@ -230,22 +231,23 @@ int main(int argc, char *argv[]) { // guaranteed to be valid since this function will run the event loop // instead of returning immediately. // We should stop the service and remove the local socket file. - auto shutdown_raylet_gracefully = [&main_service, &raylet_socket_name, &raylet, &gcs_client, shutted_down]( - const rpc::NodeDeathInfo &node_death_info) { - // Make the shutdown handler idempotent since graceful shutdown can be triggered - // by many places. - if (*shutted_down) { - RAY_LOG(INFO) << "Raylet shutdown already triggered, ignoring this request."; - return; - } - RAY_LOG(INFO) << "Raylet shutdown triggered, shutting down..."; - *shutted_down = true; - raylet->Stop(); - gcs_client->Disconnect(); - ray::stats::Shutdown(); - main_service.stop(); - remove(raylet_socket_name.c_str()); - }; + auto shutdown_raylet_gracefully = + [&main_service, &raylet_socket_name, &raylet, &gcs_client, shutted_down]( + const ray::rpc::NodeDeathInfo &node_death_info) { + // Make the shutdown handler idempotent since graceful shutdown can be triggered + // by many places. + if (*shutted_down) { + RAY_LOG(INFO) << "Raylet shutdown already triggered, ignoring this request."; + return; + } + RAY_LOG(INFO) << "Raylet shutdown triggered, shutting down..."; + *shutted_down = true; + raylet->Stop(node_death_info); + gcs_client->Disconnect(); + ray::stats::Shutdown(); + main_service.stop(); + remove(raylet_socket_name.c_str()); + }; RAY_CHECK_OK(gcs_client->Nodes().AsyncGetInternalConfig( [&](::ray::Status status, @@ -420,11 +422,11 @@ int main(int argc, char *argv[]) { raylet->Start(); })); - auto signal_handler = [&main_service, &raylet_socket_name, &raylet, &gcs_client, shutted_down]( - const boost::system::error_code &error, int signal_number) { + auto signal_handler = [shutdown_raylet_gracefully]( + const boost::system::error_code &error, int signal_number) { RAY_LOG(INFO) << "Raylet received SIGTERM, shutting down..."; - rpc::NodeDeathInfo node_death_info; - node_death_info.set_reason(rpc::NodeDeathInfo::EXPECTED_TERMINATION); + ray::rpc::NodeDeathInfo node_death_info; + node_death_info.set_reason(ray::rpc::NodeDeathInfo::EXPECTED_TERMINATION); node_death_info.set_reason_message("SIGTERM received"); shutdown_raylet_gracefully(node_death_info); }; diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 9da7e4fd845c6..88a9f32399d2f 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -99,13 +99,14 @@ void NodeManagerConfig::AddDefaultLabels(const std::string &self_node_id) { labels[kLabelKeyNodeID] = self_node_id; } -NodeManager::NodeManager(instrumented_io_context &io_service, - const NodeID &self_node_id, - const std::string &self_node_name, - const NodeManagerConfig &config, - const ObjectManagerConfig &object_manager_config, - std::shared_ptr gcs_client, - std::function shutdown_raylet_gracefully) +NodeManager::NodeManager( + instrumented_io_context &io_service, + const NodeID &self_node_id, + const std::string &self_node_name, + const NodeManagerConfig &config, + const ObjectManagerConfig &object_manager_config, + std::shared_ptr gcs_client, + std::function shutdown_raylet_gracefully) : self_node_id_(self_node_id), self_node_name_(self_node_name), io_service_(io_service), @@ -387,7 +388,8 @@ NodeManager::NodeManager(instrumented_io_context &io_service, config.runtime_env_agent_port, /*delay_executor=*/ [this](std::function task, uint32_t delay_ms) { return execute_after(io_service_, task, std::chrono::milliseconds(delay_ms)); - }); + }, + shutdown_raylet_gracefully_); worker_pool_.SetRuntimeEnvAgentClient(runtime_env_agent_client_); worker_pool_.Start(); @@ -1992,15 +1994,16 @@ void NodeManager::HandleShutdownRaylet(rpc::ShutdownRayletRequest request, "request RPC is ignored."; return; } - auto shutdown_after_reply = []() { + auto shutdown_after_reply = [shutdown_raylet_gracefully = + shutdown_raylet_gracefully_]() { rpc::DrainServerCallExecutor(); // Note that the callback is posted to the io service after the shutdown GRPC request // is replied. Otherwise, the RPC might not be replied to GCS before it shutsdown // itself. rpc::NodeDeathInfo node_death_info; - node_death_info.reason = rpc::NodeDeathInfo::EXPECTED_TERMINATION; - node_death_info.reason_message = "ShutdownRaylet RPC has been received."; - shutdown_raylet_gracefully_(node_death_info); + node_death_info.set_reason(rpc::NodeDeathInfo::EXPECTED_TERMINATION); + node_death_info.set_reason_message("ShutdownRaylet RPC has been received."); + shutdown_raylet_gracefully(node_death_info); }; is_shutdown_request_received_ = true; send_reply_callback(Status::OK(), shutdown_after_reply, shutdown_after_reply); diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index aef652ea8f011..2de4d8d831d65 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -718,6 +718,8 @@ class NodeManager : public rpc::NodeManagerServiceHandler, instrumented_io_context &io_service_; /// A client connection to the GCS. std::shared_ptr gcs_client_; + /// The function to shutdown raylet gracefully. + std::function shutdown_raylet_gracefully_; /// A pool of workers. WorkerPool worker_pool_; /// The `ClientCallManager` object that is shared by all `NodeManagerClient`s @@ -885,8 +887,6 @@ class NodeManager : public rpc::NodeManagerServiceHandler, std::unique_ptr memory_monitor_; std::unique_ptr mutable_object_provider_; - - std::function shutdown_raylet_gracefully_; }; } // namespace raylet diff --git a/src/ray/raylet/raylet.h b/src/ray/raylet/raylet.h index f1ddbc8f3c3e9..c522723fcf5e3 100644 --- a/src/ray/raylet/raylet.h +++ b/src/ray/raylet/raylet.h @@ -65,7 +65,7 @@ class Raylet { void Start(); /// Stop this raylet. - void Stop(); + void Stop(const rpc::NodeDeathInfo &node_death_info); /// Destroy the NodeServer. ~Raylet(); diff --git a/src/ray/raylet/runtime_env_agent_client.cc b/src/ray/raylet/runtime_env_agent_client.cc index 4c2f45cb16e86..d0b400e35e11a 100644 --- a/src/ray/raylet/runtime_env_agent_client.cc +++ b/src/ray/raylet/runtime_env_agent_client.cc @@ -251,15 +251,16 @@ inline constexpr std::string_view HTTP_PATH_DELETE_RUNTIME_ENV_IF_POSSIBLE = class HttpRuntimeEnvAgentClient : public RuntimeEnvAgentClient { public: - HttpRuntimeEnvAgentClient(instrumented_io_context &io_context, - const std::string &address, - int port, - std::function( - std::function, uint32_t delay_ms)> delay_executor, - std::function shutdown_raylet_gracefully, - uint32_t agent_register_timeout_ms, - uint32_t agent_manager_retry_interval_ms, - uint32_t session_pool_size = 10) + HttpRuntimeEnvAgentClient( + instrumented_io_context &io_context, + const std::string &address, + int port, + std::function( + std::function, uint32_t delay_ms)> delay_executor, + std::function shutdown_raylet_gracefully, + uint32_t agent_register_timeout_ms, + uint32_t agent_manager_retry_interval_ms, + uint32_t session_pool_size = 10) : io_context_(io_context), session_pool_(session_pool_size), address_(address), @@ -286,7 +287,8 @@ class HttpRuntimeEnvAgentClient : public RuntimeEnvAgentClient { "https://docs.ray.io/en/master/ray-observability/user-guides/" "configure-logging.html#logging-directory-structure.\n"; rpc::NodeDeathInfo node_death_info; - node_death_info.set_reason(rpc::NodeDeathInfo_Reason::NodeDeathInfo_Reason_UNEXPECTED_TERMINATION); + node_death_info.set_reason( + rpc::NodeDeathInfo_Reason::NodeDeathInfo_Reason_UNEXPECTED_TERMINATION); node_death_info.set_reason_message("Runtime Env Agent timed out"); shutdown_raylet_gracefully_(node_death_info); // If the process is not terminated within 10 seconds, forcefully kill itself. @@ -517,7 +519,7 @@ std::shared_ptr RuntimeEnvAgentClient::Create( int port, std::function( std::function, uint32_t delay_ms)> delay_executor, - std::function shutdown_raylet_gracefully,, + std::function shutdown_raylet_gracefully, uint32_t agent_register_timeout_ms, uint32_t agent_manager_retry_interval_ms) { return std::make_shared(io_context, diff --git a/src/ray/raylet/runtime_env_agent_client.h b/src/ray/raylet/runtime_env_agent_client.h index 70055f9844753..21ca93e4a960b 100644 --- a/src/ray/raylet/runtime_env_agent_client.h +++ b/src/ray/raylet/runtime_env_agent_client.h @@ -23,6 +23,7 @@ #include "ray/common/asio/instrumented_io_context.h" #include "ray/common/id.h" #include "ray/common/ray_config.h" +#include "src/ray/protobuf/gcs.pb.h" #include "src/ray/protobuf/runtime_env_common.pb.h" namespace ray { @@ -51,6 +52,7 @@ class RuntimeEnvAgentClient { // Not using typedef to avoid conflict with agent_manager.h std::function( std::function, uint32_t delay_ms)> delay_executor, + std::function shutdown_raylet_gracefully, uint32_t agent_register_timeout_ms = RayConfig::instance().agent_register_timeout_ms(), uint32_t agent_manager_retry_interval_ms = diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler.h b/src/ray/raylet/scheduling/cluster_resource_scheduler.h index b68f874e38324..0aeca60fa5487 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler.h +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler.h @@ -64,7 +64,8 @@ class ClusterResourceScheduler { std::function get_used_object_store_memory = nullptr, std::function get_pull_manager_at_capacity = nullptr, std::function unregister_self = nullptr, - std::function shutdown_raylet_gracefully = nullptr, + std::function shutdown_raylet_gracefully = + nullptr, const absl::flat_hash_map &local_node_labels = {}); /// Schedule the specified resources to the cluster nodes. From 729770687ea4665665a6d461199a4501ad9a3e1b Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Mon, 6 May 2024 10:04:00 -0700 Subject: [PATCH 04/40] cleanup Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 6 +++-- src/ray/raylet/main.cc | 8 ++++--- src/ray/raylet/node_manager.cc | 4 ---- .../scheduling/cluster_resource_scheduler.cc | 5 ---- .../scheduling/cluster_resource_scheduler.h | 2 -- .../scheduling/local_resource_manager.cc | 23 +++++++------------ .../scheduling/local_resource_manager.h | 3 --- 7 files changed, 17 insertions(+), 34 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 8e7b6bca2289d..03124f7c842b9 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -448,10 +448,12 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf return Status::OK(); } auto node_id = NodeID::FromBinary(local_node_info_.node_id()); - RAY_CHECK(!node_id.IsNil()) << "Node ID cannnot be nil."; + RAY_LOG(INFO) << "Unregistering node info, node id = " << node_id; rpc::UnregisterNodeRequest request; request.set_node_id(local_node_info_.node_id()); request.mutable_node_death_info()->CopyFrom(node_death_info); + // Unregister the node synchronously because this method is only called in the + // raylet shutdown path: waiting is not an issue and actually desired. std::promise promise; client_impl_->GetGcsRpcClient().UnregisterNode( request, @@ -461,7 +463,7 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf local_node_info_.set_state(GcsNodeInfo::DEAD); local_node_id_ = NodeID::Nil(); } - RAY_LOG(DEBUG) << "Finished unregistering node info, status = " << status + RAY_LOG(INFO) << "Finished unregistering node info, status = " << status << ", node id = " << node_id; promise.set_value(status); }); diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index a7bdf25e928e8..ba65beab09eda 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -227,7 +227,7 @@ int main(int argc, char *argv[]) { auto shutted_down = std::make_shared>(false); - // Destroy the Raylet on a SIGTERM. The pointer to main_service is + // Shut down raylet gracefully. The pointer to main_service is // guaranteed to be valid since this function will run the event loop // instead of returning immediately. // We should stop the service and remove the local socket file. @@ -240,7 +240,10 @@ int main(int argc, char *argv[]) { RAY_LOG(INFO) << "Raylet shutdown already triggered, ignoring this request."; return; } - RAY_LOG(INFO) << "Raylet shutdown triggered, shutting down..."; + RAY_LOG(INFO) << "Raylet graceful shutdown triggered, " + << "reason = " << node_death_info.reason() << ", " + << "reason message = " << node_death_info.reason_message(); + RAY_LOG(INFO) << "Shutting down..."; *shutted_down = true; raylet->Stop(node_death_info); gcs_client->Disconnect(); @@ -424,7 +427,6 @@ int main(int argc, char *argv[]) { auto signal_handler = [shutdown_raylet_gracefully]( const boost::system::error_code &error, int signal_number) { - RAY_LOG(INFO) << "Raylet received SIGTERM, shutting down..."; ray::rpc::NodeDeathInfo node_death_info; node_death_info.set_reason(ray::rpc::NodeDeathInfo::EXPECTED_TERMINATION); node_death_info.set_reason_message("SIGTERM received"); diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 88a9f32399d2f..b90843b5296b7 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -312,10 +312,6 @@ NodeManager::NodeManager( }, /*get_pull_manager_at_capacity*/ [this]() { return object_manager_.PullManagerHasPullsQueued(); }, - [this](rpc::NodeDeathInfo node_death_info) { - Status status = gcs_client_->Nodes().UnregisterSelf(node_death_info); - return status.ok(); - }, shutdown_raylet_gracefully, /*labels*/ config.labels); diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler.cc b/src/ray/raylet/scheduling/cluster_resource_scheduler.cc index 234b0fe9bb169..501cb3ecbe658 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler.cc +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler.cc @@ -36,7 +36,6 @@ ClusterResourceScheduler::ClusterResourceScheduler( local_node_resources, /*get_used_object_store_memory=*/nullptr, /*get_pull_manager_at_capacity=*/nullptr, - /*unregister_self=*/nullptr, /*shutdown_raylet_gracefully=*/nullptr); } @@ -47,7 +46,6 @@ ClusterResourceScheduler::ClusterResourceScheduler( std::function is_node_available_fn, std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, - std::function unregister_self, std::function shutdown_raylet_gracefully, const absl::flat_hash_map &local_node_labels) : local_node_id_(local_node_id), is_node_available_fn_(is_node_available_fn) { @@ -57,7 +55,6 @@ ClusterResourceScheduler::ClusterResourceScheduler( node_resources, get_used_object_store_memory, get_pull_manager_at_capacity, - unregister_self, shutdown_raylet_gracefully); } @@ -66,7 +63,6 @@ void ClusterResourceScheduler::Init( const NodeResources &local_node_resources, std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, - std::function unregister_self, std::function shutdown_raylet_gracefully) { cluster_resource_manager_ = std::make_unique(io_service); local_resource_manager_ = std::make_unique( @@ -74,7 +70,6 @@ void ClusterResourceScheduler::Init( local_node_resources, get_used_object_store_memory, get_pull_manager_at_capacity, - unregister_self, shutdown_raylet_gracefully, [this](const NodeResources &local_resource_update) { cluster_resource_manager_->AddOrUpdateNode(local_node_id_, local_resource_update); diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler.h b/src/ray/raylet/scheduling/cluster_resource_scheduler.h index 0aeca60fa5487..fbefe86a6b3b2 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler.h +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler.h @@ -63,7 +63,6 @@ class ClusterResourceScheduler { std::function is_node_available_fn, std::function get_used_object_store_memory = nullptr, std::function get_pull_manager_at_capacity = nullptr, - std::function unregister_self = nullptr, std::function shutdown_raylet_gracefully = nullptr, const absl::flat_hash_map &local_node_labels = {}); @@ -137,7 +136,6 @@ class ClusterResourceScheduler { const NodeResources &local_node_resources, std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, - std::function unregister_self, std::function shutdown_raylet_gracefully); bool NodeAvailable(scheduling::NodeID node_id) const; diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 42e39b6b9b870..5e04b83f8f2d2 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -28,13 +28,11 @@ LocalResourceManager::LocalResourceManager( const NodeResources &node_resources, std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, - std::function unregister_self, std::function shutdown_raylet_gracefully, std::function resource_change_subscriber) : local_node_id_(local_node_id), get_used_object_store_memory_(get_used_object_store_memory), get_pull_manager_at_capacity_(get_pull_manager_at_capacity), - unregister_self_(unregister_self), shutdown_raylet_gracefully_(shutdown_raylet_gracefully), resource_change_subscriber_(resource_change_subscriber) { RAY_CHECK(node_resources.total == node_resources.available); @@ -387,19 +385,14 @@ void LocalResourceManager::OnResourceOrStateChanged() { if (IsLocalNodeDraining() && IsLocalNodeIdle()) { // The node is drained. - // TODO: remove the following and clean up unregister_self_. - // if (unregister_self_ != nullptr) { - // RAY_LOG(INFO) << "The node is drained, unregister itself ..."; - // if (!unregister_self_(node_death_info_)) { - // RAY_LOG(ERROR) << "Failed to unregister the node."; - // // Still shutdown? - // } else { - // RAY_LOG(INFO) << "The node is unregistered, exiting ..."; - // } - // } else { - // RAY_LOG(INFO) << "The node is drained, exiting ..."; - // } - shutdown_raylet_gracefully_(node_death_info_); + if (shutdown_raylet_gracefully_ == nullptr) { + RAY_LOG(ERROR) << "shutdown_raylet_gracefully_ callback is not set, " + << "please check the LocalResourceManager constructor." + << "Using ShutdownRayletGracefully (sending SIGTERM) instead."; + raylet::ShutdownRayletGracefully(); + } else { + shutdown_raylet_gracefully_(node_death_info_); + } } ++version_; diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index 9c8f17f190d08..c70c90eb0e390 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -58,7 +58,6 @@ class LocalResourceManager : public syncer::ReporterInterface { const NodeResources &node_resources, std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, - std::function unregister_self, std::function shutdown_raylet_gracefully, std::function resource_change_subscriber); @@ -218,8 +217,6 @@ class LocalResourceManager : public syncer::ReporterInterface { std::function get_used_object_store_memory_; /// Function to get whether the pull manager is at capacity. std::function get_pull_manager_at_capacity_; - /// Function to unregister the local node. - std::function unregister_self_; /// Function to shutdown the raylet gracefully. std::function shutdown_raylet_gracefully_; From 0889057fd531d9a2307b41d4ced7d3e09b75e075 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Mon, 6 May 2024 10:43:19 -0700 Subject: [PATCH 05/40] format Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 2 +- src/ray/raylet/main.cc | 4 ++-- src/ray/raylet/scheduling/local_resource_manager.cc | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 03124f7c842b9..f74dd50eccbea 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -464,7 +464,7 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf local_node_id_ = NodeID::Nil(); } RAY_LOG(INFO) << "Finished unregistering node info, status = " << status - << ", node id = " << node_id; + << ", node id = " << node_id; promise.set_value(status); }); return promise.get_future().get(); diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index ba65beab09eda..0da52186dc733 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -241,8 +241,8 @@ int main(int argc, char *argv[]) { return; } RAY_LOG(INFO) << "Raylet graceful shutdown triggered, " - << "reason = " << node_death_info.reason() << ", " - << "reason message = " << node_death_info.reason_message(); + << "reason = " << node_death_info.reason() << ", " + << "reason message = " << node_death_info.reason_message(); RAY_LOG(INFO) << "Shutting down..."; *shutted_down = true; raylet->Stop(node_death_info); diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 5e04b83f8f2d2..301f16bbef2a2 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -387,8 +387,8 @@ void LocalResourceManager::OnResourceOrStateChanged() { if (shutdown_raylet_gracefully_ == nullptr) { RAY_LOG(ERROR) << "shutdown_raylet_gracefully_ callback is not set, " - << "please check the LocalResourceManager constructor." - << "Using ShutdownRayletGracefully (sending SIGTERM) instead."; + << "please check the LocalResourceManager constructor." + << "Using ShutdownRayletGracefully (sending SIGTERM) instead."; raylet::ShutdownRayletGracefully(); } else { shutdown_raylet_gracefully_(node_death_info_); From 440d3199ceac872fabf0b61c969fcf32f7e0acba Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 7 May 2024 12:51:39 -0700 Subject: [PATCH 06/40] update Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 11 ++++++----- src/ray/gcs/gcs_client/accessor.h | 8 ++++++-- src/ray/gcs/gcs_server/gcs_node_manager.cc | 10 ++++++++++ src/ray/raylet/main.cc | 3 ++- src/ray/rpc/gcs_server/gcs_rpc_server.h | 1 + 5 files changed, 25 insertions(+), 8 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index f74dd50eccbea..3fade029b813e 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -454,20 +454,21 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf request.mutable_node_death_info()->CopyFrom(node_death_info); // Unregister the node synchronously because this method is only called in the // raylet shutdown path: waiting is not an issue and actually desired. - std::promise promise; + // std::promise promise; client_impl_->GetGcsRpcClient().UnregisterNode( request, - [this, node_id, &promise](const Status &status, - const rpc::UnregisterNodeReply &reply) { + [this, node_id /*, &promise */](const Status &status, + const rpc::UnregisterNodeReply &reply) { if (status.ok()) { local_node_info_.set_state(GcsNodeInfo::DEAD); local_node_id_ = NodeID::Nil(); } RAY_LOG(INFO) << "Finished unregistering node info, status = " << status << ", node id = " << node_id; - promise.set_value(status); + // promise.set_value(status); }); - return promise.get_future().get(); + // return promise.get_future().get(); + return Status::OK(); } Status NodeInfoAccessor::DrainSelf() { diff --git a/src/ray/gcs/gcs_client/accessor.h b/src/ray/gcs/gcs_client/accessor.h index cabc58c3d2f2a..6e461fcbc9d87 100644 --- a/src/ray/gcs/gcs_client/accessor.h +++ b/src/ray/gcs/gcs_client/accessor.h @@ -286,6 +286,12 @@ class NodeInfoAccessor { virtual Status RegisterSelf(const rpc::GcsNodeInfo &local_node_info, const StatusCallback &callback); + /// Unregister local node to GCS synchronously + /// + /// \param node_death_info The death information of the node to unregister from GCS. + /// \return Status + virtual Status UnregisterSelf(const rpc::NodeDeathInfo &node_death_info); + /// Drain (remove the information of the node from the cluster) the local node from GCS /// synchronously. /// @@ -312,8 +318,6 @@ class NodeInfoAccessor { virtual Status AsyncRegister(const rpc::GcsNodeInfo &node_info, const StatusCallback &callback); - virtual Status UnregisterSelf(const rpc::NodeDeathInfo &node_death_info); - /// Send a check alive request to GCS for the liveness of some node. /// /// \param callback The callback function once the request is finished. diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.cc b/src/ray/gcs/gcs_server/gcs_node_manager.cc index cacfeeb2fe796..b8f3ed2e44f1e 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_node_manager.cc @@ -112,6 +112,7 @@ void GcsNodeManager::HandleUnregisterNode(rpc::UnregisterNodeRequest request, rpc::UnregisterNodeReply *reply, rpc::SendReplyCallback send_reply_callback) { NodeID node_id = NodeID::FromBinary(request.node_id()); + RAY_LOG(DEBUG) << "HandleUnregisterNode, node id = " << node_id; auto node = RemoveNode(node_id, /* is_intended = */ true); if (!node) { RAY_LOG(INFO) << "Node " << node_id << " is already removed"; @@ -126,6 +127,15 @@ void GcsNodeManager::HandleUnregisterNode(rpc::UnregisterNodeRequest request, node_info_delta->set_end_time_ms(node->end_time_ms()); // Update node state to DEAD instead of deleting it. RAY_CHECK_OK(gcs_table_storage_->NodeTable().Put(node_id, *node, nullptr)); + RAY_LOG(ERROR) << "Published node info, node id = " << node_id; + // GCS_RPC_SEND_REPLY(send_reply_callback, reply, Status::OK()); + auto status = Status::OK(); + reply->mutable_status()->set_code((int)status.code()); + reply->mutable_status()->set_message(status.message()); + send_reply_callback( + status, + []() { RAY_LOG(DEBUG) << "HandleUnregisterNode reply successed"; }, + []() { RAY_LOG(DEBUG) << "HandleUnregisterNode reply failed"; }); } void GcsNodeManager::HandleDrainNode(rpc::DrainNodeRequest request, diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index 0da52186dc733..3e96afdd6b758 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -241,7 +241,8 @@ int main(int argc, char *argv[]) { return; } RAY_LOG(INFO) << "Raylet graceful shutdown triggered, " - << "reason = " << node_death_info.reason() << ", " + << "reason = " + << NodeDeathInfo_Reason_Name(node_death_info.reason()) << ", " << "reason message = " << node_death_info.reason_message(); RAY_LOG(INFO) << "Shutting down..."; *shutted_down = true; diff --git a/src/ray/rpc/gcs_server/gcs_rpc_server.h b/src/ray/rpc/gcs_server/gcs_rpc_server.h index acf677ef7a261..cfbccfec4a098 100644 --- a/src/ray/rpc/gcs_server/gcs_rpc_server.h +++ b/src/ray/rpc/gcs_server/gcs_rpc_server.h @@ -346,6 +346,7 @@ class NodeInfoGrpcService : public GrpcService { RayConfig::instance().gcs_max_active_rpcs_per_handler(), AuthType::EMPTY_AUTH); NODE_INFO_SERVICE_RPC_HANDLER(RegisterNode); + NODE_INFO_SERVICE_RPC_HANDLER(UnregisterNode); NODE_INFO_SERVICE_RPC_HANDLER(DrainNode); NODE_INFO_SERVICE_RPC_HANDLER(GetAllNodeInfo); NODE_INFO_SERVICE_RPC_HANDLER(GetInternalConfig); From ddc9491a6d0fcd71623e30bfdb7635eb4fb07a13 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 7 May 2024 14:17:30 -0700 Subject: [PATCH 07/40] update Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 17 ++--------------- src/ray/raylet/node_manager.cc | 5 ++--- 2 files changed, 4 insertions(+), 18 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 3fade029b813e..2472d42083701 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -454,21 +454,8 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf request.mutable_node_death_info()->CopyFrom(node_death_info); // Unregister the node synchronously because this method is only called in the // raylet shutdown path: waiting is not an issue and actually desired. - // std::promise promise; - client_impl_->GetGcsRpcClient().UnregisterNode( - request, - [this, node_id /*, &promise */](const Status &status, - const rpc::UnregisterNodeReply &reply) { - if (status.ok()) { - local_node_info_.set_state(GcsNodeInfo::DEAD); - local_node_id_ = NodeID::Nil(); - } - RAY_LOG(INFO) << "Finished unregistering node info, status = " << status - << ", node id = " << node_id; - // promise.set_value(status); - }); - // return promise.get_future().get(); - return Status::OK(); + rpc::UnregisterNodeReply reply; + return client_impl_->GetGcsRpcClient().SyncUnregisterNode(request, &reply); } Status NodeInfoAccessor::DrainSelf() { diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index b90843b5296b7..ca589e135287b 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1990,8 +1990,7 @@ void NodeManager::HandleShutdownRaylet(rpc::ShutdownRayletRequest request, "request RPC is ignored."; return; } - auto shutdown_after_reply = [shutdown_raylet_gracefully = - shutdown_raylet_gracefully_]() { + auto shutdown_after_reply = [&]() { rpc::DrainServerCallExecutor(); // Note that the callback is posted to the io service after the shutdown GRPC request // is replied. Otherwise, the RPC might not be replied to GCS before it shutsdown @@ -1999,7 +1998,7 @@ void NodeManager::HandleShutdownRaylet(rpc::ShutdownRayletRequest request, rpc::NodeDeathInfo node_death_info; node_death_info.set_reason(rpc::NodeDeathInfo::EXPECTED_TERMINATION); node_death_info.set_reason_message("ShutdownRaylet RPC has been received."); - shutdown_raylet_gracefully(node_death_info); + shutdown_raylet_gracefully_(node_death_info); }; is_shutdown_request_received_ = true; send_reply_callback(Status::OK(), shutdown_after_reply, shutdown_after_reply); From 9f2786505a9048ca8715426b1a54b26957cb7d63 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Wed, 8 May 2024 11:56:14 -0700 Subject: [PATCH 08/40] async Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 2472d42083701..28ce9475a5027 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -454,8 +454,29 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf request.mutable_node_death_info()->CopyFrom(node_death_info); // Unregister the node synchronously because this method is only called in the // raylet shutdown path: waiting is not an issue and actually desired. - rpc::UnregisterNodeReply reply; - return client_impl_->GetGcsRpcClient().SyncUnregisterNode(request, &reply); + + // rpc::UnregisterNodeReply reply; + // Status status = client_impl_->GetGcsRpcClient().SyncUnregisterNode(request, &reply); + // RAY_LOG(INFO) << "Finished unregistering node info, status = " << status + // << ", node id = " << node_id; + // if (status.ok()) { + // local_node_info_.set_state(GcsNodeInfo::DEAD); + // local_node_id_ = NodeID::Nil(); + // } + // return status; + + client_impl_->GetGcsRpcClient().UnregisterNode( + request, + [this, node_id](const Status &status, + const rpc::UnregisterNodeReply &reply) { + if (status.ok()) { + local_node_info_.set_state(GcsNodeInfo::DEAD); + local_node_id_ = NodeID::Nil(); + } + RAY_LOG(INFO) << "Finished unregistering node info, status = " << status + << ", node id = " << node_id; + }); + return Status::OK(); } Status NodeInfoAccessor::DrainSelf() { From bb60589c8ef8ffe1e39e23830ac797c460bf90b4 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Wed, 8 May 2024 15:03:12 -0700 Subject: [PATCH 09/40] async shutdown_gracefully Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 19 +++++++++---------- src/ray/raylet/agent_manager.cc | 3 ++- src/ray/raylet/main.cc | 12 +++++++++--- 3 files changed, 20 insertions(+), 14 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 28ce9475a5027..4f1687dfe0e6a 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -466,16 +466,15 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf // return status; client_impl_->GetGcsRpcClient().UnregisterNode( - request, - [this, node_id](const Status &status, - const rpc::UnregisterNodeReply &reply) { - if (status.ok()) { - local_node_info_.set_state(GcsNodeInfo::DEAD); - local_node_id_ = NodeID::Nil(); - } - RAY_LOG(INFO) << "Finished unregistering node info, status = " << status - << ", node id = " << node_id; - }); + request, + [this, node_id](const Status &status, const rpc::UnregisterNodeReply &reply) { + if (status.ok()) { + local_node_info_.set_state(GcsNodeInfo::DEAD); + local_node_id_ = NodeID::Nil(); + } + RAY_LOG(INFO) << "Finished unregistering node info, status = " << status + << ", node id = " << node_id; + }); return Status::OK(); } diff --git a/src/ray/raylet/agent_manager.cc b/src/ray/raylet/agent_manager.cc index 55a9a0920ee5b..8710003f1dad0 100644 --- a/src/ray/raylet/agent_manager.cc +++ b/src/ray/raylet/agent_manager.cc @@ -98,7 +98,8 @@ void AgentManager::StartAgent() { "- The agent is killed by the OS (e.g., out of memory)."; rpc::NodeDeathInfo node_death_info; node_death_info.set_reason(rpc::NodeDeathInfo::UNEXPECTED_TERMINATION); - node_death_info.set_reason_message("Ray agent failed"); // TODO: add more info + node_death_info.set_reason_message( + "Ray agent failed and raylet fate-shares with it."); shutdown_raylet_gracefully_(node_death_info); // If the process is not terminated within 10 seconds, forcefully kill raylet // itself. diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index 3e96afdd6b758..2d4a5efc0fc38 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -231,7 +231,7 @@ int main(int argc, char *argv[]) { // guaranteed to be valid since this function will run the event loop // instead of returning immediately. // We should stop the service and remove the local socket file. - auto shutdown_raylet_gracefully = + auto shutdown_raylet_gracefully_sync = [&main_service, &raylet_socket_name, &raylet, &gcs_client, shutted_down]( const ray::rpc::NodeDeathInfo &node_death_info) { // Make the shutdown handler idempotent since graceful shutdown can be triggered @@ -253,6 +253,12 @@ int main(int argc, char *argv[]) { remove(raylet_socket_name.c_str()); }; + auto shutdown_raylet_gracefully = [&main_service, shutdown_raylet_gracefully_sync]( + const ray::rpc::NodeDeathInfo &node_death_info) { + main_service.post([shutdown_raylet_gracefully_sync, node_death_info]() {}, + "shutdown_raylet_gracefully_sync"); + }; + RAY_CHECK_OK(gcs_client->Nodes().AsyncGetInternalConfig( [&](::ray::Status status, const boost::optional &stored_raylet_config) { @@ -426,12 +432,12 @@ int main(int argc, char *argv[]) { raylet->Start(); })); - auto signal_handler = [shutdown_raylet_gracefully]( + auto signal_handler = [shutdown_raylet_gracefully_sync]( const boost::system::error_code &error, int signal_number) { ray::rpc::NodeDeathInfo node_death_info; node_death_info.set_reason(ray::rpc::NodeDeathInfo::EXPECTED_TERMINATION); node_death_info.set_reason_message("SIGTERM received"); - shutdown_raylet_gracefully(node_death_info); + shutdown_raylet_gracefully_sync(node_death_info); }; boost::asio::signal_set signals(main_service); #ifdef _WIN32 From 5b5ad48c869ddb83b725f0d4351cf2396e8e8793 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Wed, 8 May 2024 16:13:34 -0700 Subject: [PATCH 10/40] fix Signed-off-by: Rui Qiao --- src/ray/raylet/main.cc | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index 2d4a5efc0fc38..7cb54d508ced5 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -255,8 +255,11 @@ int main(int argc, char *argv[]) { auto shutdown_raylet_gracefully = [&main_service, shutdown_raylet_gracefully_sync]( const ray::rpc::NodeDeathInfo &node_death_info) { - main_service.post([shutdown_raylet_gracefully_sync, node_death_info]() {}, - "shutdown_raylet_gracefully_sync"); + main_service.post( + [shutdown_raylet_gracefully_sync, node_death_info]() { + shutdown_raylet_gracefully_sync(node_death_info); + }, + "shutdown_raylet_gracefully_sync"); }; RAY_CHECK_OK(gcs_client->Nodes().AsyncGetInternalConfig( From 96ebaa755206e6f565afeef4a8984b2b40d23569 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Wed, 8 May 2024 18:29:10 -0700 Subject: [PATCH 11/40] unregister callback Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 9 +++++++-- src/ray/gcs/gcs_client/accessor.h | 6 ++++-- src/ray/raylet/main.cc | 16 +++++++++++----- src/ray/raylet/raylet.cc | 9 +++++++-- src/ray/raylet/raylet.h | 6 +++++- 5 files changed, 34 insertions(+), 12 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 4f1687dfe0e6a..610d742798a9c 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -441,7 +441,9 @@ Status NodeInfoAccessor::RegisterSelf(const GcsNodeInfo &local_node_info, return Status::OK(); } -Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_info) { +Status NodeInfoAccessor::UnregisterSelf( + const rpc::NodeDeathInfo &node_death_info, + const std::function &unregister_done_callback) { if (local_node_id_.IsNil()) { RAY_LOG(INFO) << "The node is already unregistered."; // This node is already unregistered. @@ -467,13 +469,16 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf client_impl_->GetGcsRpcClient().UnregisterNode( request, - [this, node_id](const Status &status, const rpc::UnregisterNodeReply &reply) { + [this, node_id, &unregister_done_callback](const Status &status, + const rpc::UnregisterNodeReply &reply) { if (status.ok()) { local_node_info_.set_state(GcsNodeInfo::DEAD); local_node_id_ = NodeID::Nil(); } RAY_LOG(INFO) << "Finished unregistering node info, status = " << status << ", node id = " << node_id; + unregister_done_callback(); + RAY_LOG(INFO) << "Finished executing unregister_done_callback."; }); return Status::OK(); } diff --git a/src/ray/gcs/gcs_client/accessor.h b/src/ray/gcs/gcs_client/accessor.h index 6e461fcbc9d87..56d6585aefda3 100644 --- a/src/ray/gcs/gcs_client/accessor.h +++ b/src/ray/gcs/gcs_client/accessor.h @@ -289,8 +289,10 @@ class NodeInfoAccessor { /// Unregister local node to GCS synchronously /// /// \param node_death_info The death information of the node to unregister from GCS. - /// \return Status - virtual Status UnregisterSelf(const rpc::NodeDeathInfo &node_death_info); + /// \param unregister_done_callback Callback that will be called when unregistration is + /// done. \return Status + virtual Status UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, + const std::function &unregister_done_callback); /// Drain (remove the information of the node from the cluster) the local node from GCS /// synchronously. diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index 7cb54d508ced5..fcc9974101809 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -246,11 +246,17 @@ int main(int argc, char *argv[]) { << "reason message = " << node_death_info.reason_message(); RAY_LOG(INFO) << "Shutting down..."; *shutted_down = true; - raylet->Stop(node_death_info); - gcs_client->Disconnect(); - ray::stats::Shutdown(); - main_service.stop(); - remove(raylet_socket_name.c_str()); + + auto unregister_done_callback = + [&main_service, &raylet_socket_name, &raylet, &gcs_client]() { + raylet->Stop(); + gcs_client->Disconnect(); + ray::stats::Shutdown(); + main_service.stop(); + remove(raylet_socket_name.c_str()); + }; + + raylet->UnregisterSelf(node_death_info, unregister_done_callback); }; auto shutdown_raylet_gracefully = [&main_service, shutdown_raylet_gracefully_sync]( diff --git a/src/ray/raylet/raylet.cc b/src/ray/raylet/raylet.cc index 959fb34f325e5..7638190a09fdc 100644 --- a/src/ray/raylet/raylet.cc +++ b/src/ray/raylet/raylet.cc @@ -118,8 +118,13 @@ void Raylet::Start() { DoAccept(); } -void Raylet::Stop(const rpc::NodeDeathInfo &node_death_info) { - RAY_CHECK_OK(gcs_client_->Nodes().UnregisterSelf(node_death_info)); +void Raylet::UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, + const std::function &unregister_done_callback) { + RAY_CHECK_OK( + gcs_client_->Nodes().UnregisterSelf(node_death_info, unregister_done_callback)); +} + +void Raylet::Stop() { node_manager_.Stop(); acceptor_.close(); } diff --git a/src/ray/raylet/raylet.h b/src/ray/raylet/raylet.h index c522723fcf5e3..e8b590768df06 100644 --- a/src/ray/raylet/raylet.h +++ b/src/ray/raylet/raylet.h @@ -65,7 +65,11 @@ class Raylet { void Start(); /// Stop this raylet. - void Stop(const rpc::NodeDeathInfo &node_death_info); + void Stop(); + + /// Unregister this raylet from the GCS. + void UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, + const std::function &unregister_done_callback); /// Destroy the NodeServer. ~Raylet(); From 78138aaa9f9c9b1bf4a84ae08e6c48afe1e0dc88 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Thu, 9 May 2024 09:47:52 -0700 Subject: [PATCH 12/40] fix Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 4 ++-- src/ray/gcs/gcs_client/accessor.h | 2 +- src/ray/raylet/main.cc | 21 +++++++++++---------- src/ray/raylet/raylet.cc | 2 +- src/ray/raylet/raylet.h | 2 +- 5 files changed, 16 insertions(+), 15 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 610d742798a9c..1f4fbbce2a2a8 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -443,7 +443,7 @@ Status NodeInfoAccessor::RegisterSelf(const GcsNodeInfo &local_node_info, Status NodeInfoAccessor::UnregisterSelf( const rpc::NodeDeathInfo &node_death_info, - const std::function &unregister_done_callback) { + std::function unregister_done_callback) { if (local_node_id_.IsNil()) { RAY_LOG(INFO) << "The node is already unregistered."; // This node is already unregistered. @@ -469,7 +469,7 @@ Status NodeInfoAccessor::UnregisterSelf( client_impl_->GetGcsRpcClient().UnregisterNode( request, - [this, node_id, &unregister_done_callback](const Status &status, + [this, node_id, unregister_done_callback](const Status &status, const rpc::UnregisterNodeReply &reply) { if (status.ok()) { local_node_info_.set_state(GcsNodeInfo::DEAD); diff --git a/src/ray/gcs/gcs_client/accessor.h b/src/ray/gcs/gcs_client/accessor.h index 56d6585aefda3..094dab1f329bb 100644 --- a/src/ray/gcs/gcs_client/accessor.h +++ b/src/ray/gcs/gcs_client/accessor.h @@ -292,7 +292,7 @@ class NodeInfoAccessor { /// \param unregister_done_callback Callback that will be called when unregistration is /// done. \return Status virtual Status UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, - const std::function &unregister_done_callback); + std::function unregister_done_callback); /// Drain (remove the information of the node from the cluster) the local node from GCS /// synchronously. diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index fcc9974101809..e4c41959b3ff2 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -227,12 +227,21 @@ int main(int argc, char *argv[]) { auto shutted_down = std::make_shared>(false); + auto unregister_done_callback = + [&main_service, &raylet_socket_name, &raylet, &gcs_client]() { + raylet->Stop(); + gcs_client->Disconnect(); + ray::stats::Shutdown(); + main_service.stop(); + remove(raylet_socket_name.c_str()); + }; + // Shut down raylet gracefully. The pointer to main_service is // guaranteed to be valid since this function will run the event loop // instead of returning immediately. // We should stop the service and remove the local socket file. auto shutdown_raylet_gracefully_sync = - [&main_service, &raylet_socket_name, &raylet, &gcs_client, shutted_down]( + [&raylet, shutted_down, unregister_done_callback]( const ray::rpc::NodeDeathInfo &node_death_info) { // Make the shutdown handler idempotent since graceful shutdown can be triggered // by many places. @@ -247,15 +256,6 @@ int main(int argc, char *argv[]) { RAY_LOG(INFO) << "Shutting down..."; *shutted_down = true; - auto unregister_done_callback = - [&main_service, &raylet_socket_name, &raylet, &gcs_client]() { - raylet->Stop(); - gcs_client->Disconnect(); - ray::stats::Shutdown(); - main_service.stop(); - remove(raylet_socket_name.c_str()); - }; - raylet->UnregisterSelf(node_death_info, unregister_done_callback); }; @@ -457,5 +457,6 @@ int main(int argc, char *argv[]) { signals.async_wait(signal_handler); main_service.run(); + RAY_LOG(INFO) << "Raylet ends"; } #endif diff --git a/src/ray/raylet/raylet.cc b/src/ray/raylet/raylet.cc index 7638190a09fdc..0157c82c8eae8 100644 --- a/src/ray/raylet/raylet.cc +++ b/src/ray/raylet/raylet.cc @@ -119,7 +119,7 @@ void Raylet::Start() { } void Raylet::UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, - const std::function &unregister_done_callback) { + std::function unregister_done_callback) { RAY_CHECK_OK( gcs_client_->Nodes().UnregisterSelf(node_death_info, unregister_done_callback)); } diff --git a/src/ray/raylet/raylet.h b/src/ray/raylet/raylet.h index e8b590768df06..d77e2fc452802 100644 --- a/src/ray/raylet/raylet.h +++ b/src/ray/raylet/raylet.h @@ -69,7 +69,7 @@ class Raylet { /// Unregister this raylet from the GCS. void UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, - const std::function &unregister_done_callback); + std::function unregister_done_callback); /// Destroy the NodeServer. ~Raylet(); From 96f839937873fa52b68c9764d156077aeeb735f1 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Thu, 9 May 2024 13:20:49 -0700 Subject: [PATCH 13/40] fix Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 7 +++---- src/ray/raylet/main.cc | 14 +++++++------- src/ray/raylet/runtime_env_agent_client_test.cc | 9 +++++++++ 3 files changed, 19 insertions(+), 11 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 1f4fbbce2a2a8..09f0131972f8c 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -441,9 +441,8 @@ Status NodeInfoAccessor::RegisterSelf(const GcsNodeInfo &local_node_info, return Status::OK(); } -Status NodeInfoAccessor::UnregisterSelf( - const rpc::NodeDeathInfo &node_death_info, - std::function unregister_done_callback) { +Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, + std::function unregister_done_callback) { if (local_node_id_.IsNil()) { RAY_LOG(INFO) << "The node is already unregistered."; // This node is already unregistered. @@ -470,7 +469,7 @@ Status NodeInfoAccessor::UnregisterSelf( client_impl_->GetGcsRpcClient().UnregisterNode( request, [this, node_id, unregister_done_callback](const Status &status, - const rpc::UnregisterNodeReply &reply) { + const rpc::UnregisterNodeReply &reply) { if (status.ok()) { local_node_info_.set_state(GcsNodeInfo::DEAD); local_node_id_ = NodeID::Nil(); diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index e4c41959b3ff2..9eb0c83c20da9 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -228,13 +228,13 @@ int main(int argc, char *argv[]) { auto shutted_down = std::make_shared>(false); auto unregister_done_callback = - [&main_service, &raylet_socket_name, &raylet, &gcs_client]() { - raylet->Stop(); - gcs_client->Disconnect(); - ray::stats::Shutdown(); - main_service.stop(); - remove(raylet_socket_name.c_str()); - }; + [&main_service, &raylet_socket_name, &raylet, &gcs_client]() { + raylet->Stop(); + gcs_client->Disconnect(); + ray::stats::Shutdown(); + main_service.stop(); + remove(raylet_socket_name.c_str()); + }; // Shut down raylet gracefully. The pointer to main_service is // guaranteed to be valid since this function will run the event loop diff --git a/src/ray/raylet/runtime_env_agent_client_test.cc b/src/ray/raylet/runtime_env_agent_client_test.cc index dd044a2e9218d..f2e140a9231e6 100644 --- a/src/ray/raylet/runtime_env_agent_client_test.cc +++ b/src/ray/raylet/runtime_env_agent_client_test.cc @@ -186,6 +186,8 @@ delay_after(instrumented_io_context &ioc) { }; } +auto dummy_shutdown_raylet_gracefully = [](const rpc::NodeDeathInfo &) {}; + TEST(RuntimeEnvAgentClientTest, GetOrCreateRuntimeEnvOK) { int port = GetFreePort(); HttpServerThread http_server_thread( @@ -217,6 +219,7 @@ TEST(RuntimeEnvAgentClientTest, GetOrCreateRuntimeEnvOK) { "127.0.0.1", port, delay_after(ioc), + dummy_shutdown_raylet_gracefully, /*agent_register_timeout_ms=*/10000, /*agent_manager_retry_interval_ms=*/100); auto job_id = JobID::FromInt(123); @@ -277,6 +280,7 @@ TEST(RuntimeEnvAgentClientTest, GetOrCreateRuntimeEnvApplicationError) { "127.0.0.1", port, delay_after(ioc), + dummy_shutdown_raylet_gracefully, /*agent_register_timeout_ms=*/10000, /*agent_manager_retry_interval_ms=*/100); auto job_id = JobID::FromInt(123); @@ -342,6 +346,7 @@ TEST(RuntimeEnvAgentClientTest, GetOrCreateRuntimeEnvRetriesOnServerNotStarted) http_server_thread.start(); return execute_after(ioc, task, std::chrono::milliseconds(delay_ms)); }, + dummy_shutdown_raylet_gracefully, /*agent_register_timeout_ms=*/10000, /*agent_manager_retry_interval_ms=*/100); auto job_id = JobID::FromInt(123); @@ -398,6 +403,7 @@ TEST(RuntimeEnvAgentClientTest, DeleteRuntimeEnvIfPossibleOK) { "127.0.0.1", port, delay_after(ioc), + dummy_shutdown_raylet_gracefully, /*agent_register_timeout_ms=*/10000, /*agent_manager_retry_interval_ms=*/100); @@ -441,6 +447,7 @@ TEST(RuntimeEnvAgentClientTest, DeleteRuntimeEnvIfPossibleApplicationError) { "127.0.0.1", port, delay_after(ioc), + dummy_shutdown_raylet_gracefully, /*agent_register_timeout_ms=*/10000, /*agent_manager_retry_interval_ms=*/100); @@ -489,6 +496,7 @@ TEST(RuntimeEnvAgentClientTest, DeleteRuntimeEnvIfPossibleRetriesOnServerNotStar http_server_thread.start(); return execute_after(ioc, task, std::chrono::milliseconds(delay_ms)); }, + dummy_shutdown_raylet_gracefully, /*agent_register_timeout_ms=*/10000, /*agent_manager_retry_interval_ms=*/100); @@ -587,6 +595,7 @@ TEST(RuntimeEnvAgentClientTest, HoldsConcurrency) { "127.0.0.1", port, delay_after(ioc), + dummy_shutdown_raylet_gracefully, /*agent_register_timeout_ms=*/10000, /*agent_manager_retry_interval_ms=*/100); From 56ff6be6c820ec2231be38ce058ebde3829b2956 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Thu, 9 May 2024 18:18:11 -0700 Subject: [PATCH 14/40] fix Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_server/gcs_node_manager.cc | 8 ++++++-- src/ray/raylet/node_manager.cc | 5 +++-- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.cc b/src/ray/gcs/gcs_server/gcs_node_manager.cc index b8f3ed2e44f1e..22b8c7cf923d0 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_node_manager.cc @@ -125,9 +125,13 @@ void GcsNodeManager::HandleUnregisterNode(rpc::UnregisterNodeRequest request, node_info_delta->set_node_id(node->node_id()); node_info_delta->set_state(node->state()); node_info_delta->set_end_time_ms(node->end_time_ms()); + auto on_put_done = [&, node_info_delta](const Status &status) { + RAY_LOG(DEBUG) << "Publishing node info, node id = " << node_id; + RAY_CHECK_OK(gcs_publisher_->PublishNodeInfo(node_id, *node_info_delta, nullptr)); + RAY_LOG(DEBUG) << "Published node info, node id = " << node_id; + }; // Update node state to DEAD instead of deleting it. - RAY_CHECK_OK(gcs_table_storage_->NodeTable().Put(node_id, *node, nullptr)); - RAY_LOG(ERROR) << "Published node info, node id = " << node_id; + RAY_CHECK_OK(gcs_table_storage_->NodeTable().Put(node_id, *node, on_put_done)); // GCS_RPC_SEND_REPLY(send_reply_callback, reply, Status::OK()); auto status = Status::OK(); reply->mutable_status()->set_code((int)status.code()); diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index ca589e135287b..a1b1a7c7539aa 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1942,8 +1942,9 @@ void NodeManager::HandleDrainRaylet(rpc::DrainRayletRequest request, rpc::DrainRayletReply *reply, rpc::SendReplyCallback send_reply_callback) { RAY_LOG(INFO) << "Drain raylet RPC has received. Deadline is " - << request.deadline_timestamp_ms() - << ". Drain reason: " << request.reason_message(); + << request.deadline_timestamp_ms() << ". Drain reason: " + << rpc::autoscaler::DrainNodeReason_Name(request.reason()) + << ". Drain reason message: " << request.reason_message(); if (request.reason() == rpc::autoscaler::DrainNodeReason::DRAIN_NODE_REASON_IDLE_TERMINATION) { From 52d91cb68df3dd871b3af61d2918aa83fa153122 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Thu, 9 May 2024 22:49:04 -0700 Subject: [PATCH 15/40] fix Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_server/gcs_node_manager.cc | 5 ++--- src/ray/raylet/scheduling/local_resource_manager_test.cc | 5 +++++ 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.cc b/src/ray/gcs/gcs_server/gcs_node_manager.cc index 22b8c7cf923d0..8d43f1e706d03 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_node_manager.cc @@ -125,10 +125,9 @@ void GcsNodeManager::HandleUnregisterNode(rpc::UnregisterNodeRequest request, node_info_delta->set_node_id(node->node_id()); node_info_delta->set_state(node->state()); node_info_delta->set_end_time_ms(node->end_time_ms()); - auto on_put_done = [&, node_info_delta](const Status &status) { + auto on_put_done = [=](const Status &status) { RAY_LOG(DEBUG) << "Publishing node info, node id = " << node_id; RAY_CHECK_OK(gcs_publisher_->PublishNodeInfo(node_id, *node_info_delta, nullptr)); - RAY_LOG(DEBUG) << "Published node info, node id = " << node_id; }; // Update node state to DEAD instead of deleting it. RAY_CHECK_OK(gcs_table_storage_->NodeTable().Put(node_id, *node, on_put_done)); @@ -138,7 +137,7 @@ void GcsNodeManager::HandleUnregisterNode(rpc::UnregisterNodeRequest request, reply->mutable_status()->set_message(status.message()); send_reply_callback( status, - []() { RAY_LOG(DEBUG) << "HandleUnregisterNode reply successed"; }, + []() { RAY_LOG(DEBUG) << "HandleUnregisterNode reply succeeded"; }, []() { RAY_LOG(DEBUG) << "HandleUnregisterNode reply failed"; }); } diff --git a/src/ray/raylet/scheduling/local_resource_manager_test.cc b/src/ray/raylet/scheduling/local_resource_manager_test.cc index 0c2319904b0ad..c0125e8287c00 100644 --- a/src/ray/raylet/scheduling/local_resource_manager_test.cc +++ b/src/ray/raylet/scheduling/local_resource_manager_test.cc @@ -76,6 +76,7 @@ TEST_F(LocalResourceManagerTest, BasicGetResourceUsageMapTest) { {ResourceID(pg_index_1_resource), 2.0}}), nullptr, nullptr, + nullptr, nullptr); /// @@ -142,6 +143,7 @@ TEST_F(LocalResourceManagerTest, NodeDrainingTest) { CreateNodeResources({{ResourceID::CPU(), 8.0}}), nullptr, nullptr, + nullptr, nullptr); // Make the node non-idle. @@ -173,6 +175,7 @@ TEST_F(LocalResourceManagerTest, ObjectStoreMemoryDrainingTest) { /* get_used_object_store_memory */ [&used_object_store]() { return *used_object_store; }, nullptr, + nullptr, nullptr); // Make the node non-idle. @@ -207,6 +210,7 @@ TEST_F(LocalResourceManagerTest, IdleResourceTimeTest) { /* get_used_object_store_memory */ [&used_object_store]() { return *used_object_store; }, nullptr, + nullptr, nullptr); /// Test when the resource is all idle when initialized. @@ -352,6 +356,7 @@ TEST_F(LocalResourceManagerTest, CreateSyncMessageNegativeResourceAvailability) /* get_used_object_store_memory */ [&used_object_store]() { return *used_object_store; }, nullptr, + nullptr, nullptr); manager->SubtractResourceInstances( From dd3d2859f2090307d3b6ecabc2378f0c437ac348 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Fri, 10 May 2024 08:33:38 -0700 Subject: [PATCH 16/40] debug Signed-off-by: Rui Qiao --- .buildkite/core.rayci.yml | 269 +------------------------------------- 1 file changed, 1 insertion(+), 268 deletions(-) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index 85c025e8350cd..d645dbacde612 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -31,274 +31,6 @@ steps: depends_on: - corebuild - # tests - - label: ":ray: core: python tests" - tags: - - python - - dashboard - instance_type: large - parallelism: 4 - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... python/ray/autoscaler/v2/... core - --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" --parallelism-per-worker 3 - --except-tags debug_tests,asan_tests,post_wheel_build,ha_integration,mem_pressure,tmpfs,container,manual,use_all_core - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... python/ray/autoscaler/v2/... core - --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" - --only-tags use_all_core --skip-ray-installation - - - label: ":ray: core: redis tests" - tags: python - instance_type: large - parallelism: 4 - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... python/ray/autoscaler/v2/... core - --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" --parallelism-per-worker 3 - --test-env=TEST_EXTERNAL_REDIS=1 - --except-tags debug_tests,asan_tests,post_wheel_build,ha_integration,mem_pressure,tmpfs,container,manual,use_all_core - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... python/ray/autoscaler/v2/... core - --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" - --only-tags use_all_core --skip-ray-installation - - - label: ":ray: core: memory pressure tests" - tags: - - python - - oss - - skip-on-premerge - instance_type: medium - commands: - - cleanup() { ./ci/build/upload_build_info.sh; }; trap cleanup EXIT - - (cd dashboard/client && npm ci && npm run build) - - pip install -e python[client] - - bazel test --config=ci --jobs=1 $(./ci/run/bazel_export_options) - --test_tag_filters=mem_pressure -- //python/ray/tests/... - job_env: corebuild - - - label: ":ray: core: out of disk tests" - tags: - - python - - oss - instance_type: small - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core - --only-tags=tmpfs --tmp-filesystem=tmpfs - - - label: ":ray: core: out of disk redis tests" - tags: - - python - - oss - instance_type: small - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core - --test-env=TEST_EXTERNAL_REDIS=1 - --only-tags=tmpfs --tmp-filesystem=tmpfs - - - label: ":ray: core: workflow tests" - tags: - - python - - workflow - instance_type: medium - parallelism: 2 - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/workflow/... core - --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" - --except-tags use_all_core - --parallelism-per-worker 2 - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/workflow/... core - --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" - --skip-ray-installation - --only-tags use_all_core - - - label: ":ray: core: doc tests" - tags: - - python - - doc - instance_type: large - commands: - # doc tests - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/... //doc/... core - --only-tags doctest - --parallelism-per-worker 3 - # doc examples - - bazel run //ci/ray_ci:test_in_docker -- //doc/... core - --except-tags gpu,post_wheel_build,mem_pressure,doctest - --parallelism-per-worker 3 - --skip-ray-installation - - - label: ":ray: core: data tests" - tags: python - instance_type: medium - commands: - - bazel run //ci/ray_ci:test_in_docker -- - python/ray/util/dask/... python/ray/tests/modin/... core - --build-name data15build - --parallelism-per-worker 2 - depends_on: - - data15build - - forge - - - label: ":ray: core: dashboard tests" - tags: - - python - - dashboard - instance_type: medium - commands: - - bazel run //ci/ray_ci:test_in_docker -- python/ray/dashboard/... core - --parallelism-per-worker 3 - # ui tests - - docker run -i --rm --volume /tmp/artifacts:/artifact-mount --shm-size=2.5gb - "$${RAYCI_WORK_REPO}":"$${RAYCI_BUILD_ID}"-corebuild /bin/bash -iecuo pipefail - "./dashboard/tests/run_ui_tests.sh" - - - label: ":ray: core: debug test" - tags: python - instance_type: medium - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core - --build-type debug - --parallelism-per-worker 3 - --only-tags debug_tests - --except-tags kubernetes,manual - - - label: ":ray: core: asan tests" - tags: python - instance_type: medium - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core - --build-type asan - --parallelism-per-worker 2 - --only-tags asan_tests - --except-tags kubernetes,manual - - - label: ":ray: core: wheel tests" - tags: linux_wheels - instance_type: medium - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //doc/... core - --build-type wheel - --parallelism-per-worker 3 - --only-tags post_wheel_build - --test-env=RAY_CI_POST_WHEEL_TESTS=True - depends_on: - - manylinux - - corebuild - - forge - - - label: ":ray: core: minimal tests {{matrix}}" - tags: - - python - - dashboard - - oss - instance_type: medium - commands: - # validate minimal installation - - python ./ci/env/check_minimal_install.py - # core tests - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dashboard/... core - --parallelism-per-worker 3 - --build-name minbuild-core-py{{matrix}} - --test-env=RAY_MINIMAL=1 - --test-env=EXPECTED_PYTHON_VERSION={{matrix}} - --only-tags minimal - --except-tags basic_test,manual - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dashboard/... core - --parallelism-per-worker 3 - --build-name minbuild-core-py{{matrix}} - --test-env=RAY_MINIMAL=1 - --test-env=EXPECTED_PYTHON_VERSION={{matrix}} - --only-tags minimal - --except-tags no_basic_test,manual - --skip-ray-installation - # core redis tests - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dashboard/... core - --parallelism-per-worker 3 - --build-name minbuild-core-py{{matrix}} - --test-env=RAY_MINIMAL=1 - --test-env=TEST_EXTERNAL_REDIS=1 - --test-env=EXPECTED_PYTHON_VERSION={{matrix}} - --only-tags minimal - --except-tags no_basic_test,manual - --skip-ray-installation - # serve tests - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dashboard/... serve - --parallelism-per-worker 3 - --build-name minbuild-core-py{{matrix}} - --test-env=RAY_MINIMAL=1 - --only-tags minimal - --skip-ray-installation - depends_on: - - minbuild-core - matrix: - - "3.9" - - "3.10" - - "3.11" - - # cpp tests - - label: ":ray: core: cpp tests" - tags: core_cpp - instance_type: medium - commands: - - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type clang - --parallelism-per-worker 2 - - - label: ":ray: core: cpp asan tests" - tags: core_cpp - instance_type: medium - commands: - - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type asan-clang - --parallelism-per-worker 2 - - - label: ":ray: core: cpp ubsan tests" - tags: core_cpp - instance_type: large - commands: - - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type ubsan - --except-tags no_ubsan - --parallelism-per-worker 2 - - - label: ":ray: core: cpp tsan tests" - tags: core_cpp - instance_type: medium - commands: - - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type tsan-clang - --except-tags no_tsan - --parallelism-per-worker 2 - - - label: ":ray: core: flaky tests" - tags: - - python - - skip-on-premerge - instance_type: large - soft_fail: true - commands: - - bazel run //ci/ray_ci:test_in_docker -- //... core - --run-flaky-tests --build-type clang - - - label: ":ray: core: cpp worker tests" - tags: core_cpp - instance_type: medium - commands: - - if [[ "$${BUILDKITE_PIPELINE_ID}" == "0189942e-0876-4b8f-80a4-617f988ec59b" ]]; then - echo "build --remote_upload_local_results=false" >> ~/.bazelrc; - fi - - ci/ci.sh build - - ci/ci.sh test_cpp - depends_on: oss-ci-base_build - job_env: oss-ci-base_build - - - label: ":ray: core: HA integration tests" - tags: - - python - - docker - instance_type: medium - commands: - - bazel run //ci/ray_ci:build_in_docker -- docker --platform cpu --canonical-tag ha_integration - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core --only-tags ha_integration - depends_on: - - manylinux - - forge - - raycpubase - - corebuild - - label: ":ray: core: container tests" tags: - python @@ -310,6 +42,7 @@ steps: - docker build --progress=plain --build-arg BASE_IMAGE="rayproject/ray:test_container" -t rayproject/ray:runtime_env_container -f docker/runtime_env_container/Dockerfile . - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core --only-tags container + - sleep 18000 depends_on: - manylinux - forge From 3347b0afef43baed264e99d0ea517e648965ef64 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Fri, 10 May 2024 10:04:18 -0700 Subject: [PATCH 17/40] update Signed-off-by: Rui Qiao --- src/ray/raylet/node_manager.cc | 2 +- src/ray/raylet/node_manager.h | 4 ++-- src/ray/raylet/raylet.cc | 2 +- src/ray/raylet/raylet.h | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index a1b1a7c7539aa..60897320436ae 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -106,7 +106,7 @@ NodeManager::NodeManager( const NodeManagerConfig &config, const ObjectManagerConfig &object_manager_config, std::shared_ptr gcs_client, - std::function shutdown_raylet_gracefully) + std::function shutdown_raylet_gracefully) : self_node_id_(self_node_id), self_node_name_(self_node_name), io_service_(io_service), diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index 2de4d8d831d65..e0cdb10d389c7 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -136,7 +136,7 @@ class NodeManager : public rpc::NodeManagerServiceHandler, const NodeManagerConfig &config, const ObjectManagerConfig &object_manager_config, std::shared_ptr gcs_client, - std::function shutdown_raylet_gracefully); + std::function shutdown_raylet_gracefully); /// Process a new client connection. /// @@ -719,7 +719,7 @@ class NodeManager : public rpc::NodeManagerServiceHandler, /// A client connection to the GCS. std::shared_ptr gcs_client_; /// The function to shutdown raylet gracefully. - std::function shutdown_raylet_gracefully_; + std::function shutdown_raylet_gracefully_; /// A pool of workers. WorkerPool worker_pool_; /// The `ClientCallManager` object that is shared by all `NodeManagerClient`s diff --git a/src/ray/raylet/raylet.cc b/src/ray/raylet/raylet.cc index 0157c82c8eae8..f482ee9e83126 100644 --- a/src/ray/raylet/raylet.cc +++ b/src/ray/raylet/raylet.cc @@ -66,7 +66,7 @@ Raylet::Raylet(instrumented_io_context &main_service, std::shared_ptr gcs_client, int metrics_export_port, bool is_head_node, - std::function shutdown_raylet_gracefully) + std::function shutdown_raylet_gracefully) : self_node_id_(self_node_id), gcs_client_(gcs_client), node_manager_(main_service, diff --git a/src/ray/raylet/raylet.h b/src/ray/raylet/raylet.h index d77e2fc452802..442a3464f3266 100644 --- a/src/ray/raylet/raylet.h +++ b/src/ray/raylet/raylet.h @@ -59,7 +59,7 @@ class Raylet { std::shared_ptr gcs_client, int metrics_export_port, bool is_head_node, - std::function shutdown_raylet_gracefully); + std::function shutdown_raylet_gracefully); /// Start this raylet. void Start(); From 7acc5509edb27a362964c19a72817401f0fb4703 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Fri, 10 May 2024 12:04:32 -0700 Subject: [PATCH 18/40] debug Signed-off-by: Rui Qiao --- .buildkite/core.rayci.yml | 1 - .buildkite/serve.rayci.yml | 135 +------------------------------------ 2 files changed, 1 insertion(+), 135 deletions(-) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index d645dbacde612..4897f71ebef09 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -42,7 +42,6 @@ steps: - docker build --progress=plain --build-arg BASE_IMAGE="rayproject/ray:test_container" -t rayproject/ray:runtime_env_container -f docker/runtime_env_container/Dockerfile . - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core --only-tags container - - sleep 18000 depends_on: - manylinux - forge diff --git a/.buildkite/serve.rayci.yml b/.buildkite/serve.rayci.yml index c4d6183534136..f43a3bcd1fe39 100644 --- a/.buildkite/serve.rayci.yml +++ b/.buildkite/serve.rayci.yml @@ -29,103 +29,6 @@ steps: EXTRA_DEPENDENCY: "{{matrix}}" # tests - - label: ":ray-serve: serve: tests" - parallelism: 2 - tags: - - serve - - python - instance_type: large - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/serve/... //python/ray/tests/... serve - --except-tags post_wheel_build,gpu,worker-container,ha_integration - --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" --parallelism-per-worker 3 - --build-name servebuild --test-env=EXPECTED_PYTHON_VERSION=3.9 - depends_on: "servebuild" - - - label: ":ray-serve: serve: pydantic < 2.0 tests" - parallelism: 2 - tags: - - serve - - python - instance_type: large - soft_fail: true - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/serve/... //python/ray/tests/... serve - --except-tags post_wheel_build,gpu,worker-container,ha_integration - --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" --parallelism-per-worker 3 - --build-name servepydantic1build --test-env=EXPECTED_PYTHON_VERSION=3.9 --test-env=EXPECTED_PYDANTIC_VERSION=1.10.12 - depends_on: servepydantic1build - - - label: ":ray-serve: serve: python {{matrix.python}} tests ({{matrix.worker_id}})" - if: build.env("RAYCI_CONTINUOUS_BUILD") == "1" || build.pull_request.labels includes "continuous-build" - tags: - - serve - - python - instance_type: large - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/serve/... //python/ray/tests/... serve - --except-tags post_wheel_build,gpu,worker-container,ha_integration - --workers 2 --worker-id {{matrix.worker_id}} --parallelism-per-worker 3 - --python-version {{matrix.python}} - --test-env=EXPECTED_PYTHON_VERSION={{matrix.python}} - depends_on: servebuild-multipy - matrix: - setup: - python: ["3.11"] - worker_id: ["0", "1"] - - - label: ":ray-serve: serve: release tests" - tags: - - serve - - release_tests - - python - instance_type: medium - commands: - - bazel run //ci/ray_ci:test_in_docker -- //release/... serve --parallelism-per-worker 3 - depends_on: servebuild - - - label: ":ray-serve: serve: wheel tests" - tags: linux_wheels - instance_type: medium - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/serve/... //doc/... serve - --build-type wheel - --parallelism-per-worker 3 - --only-tags post_wheel_build - --test-env=RAY_CI_POST_WHEEL_TESTS=True - depends_on: - - manylinux - - servebuild - - forge - - - label: ":ray-serve: serve: doc tests" - tags: - - serve - - doc - instance_type: large - commands: - # doc tests - - bazel run //ci/ray_ci:test_in_docker -- python/ray/... //doc/... serve - --only-tags doctest - --parallelism-per-worker 3 - # doc examples - - bazel run //ci/ray_ci:test_in_docker -- //doc/... serve - --except-tags gpu,post_wheel_build,timeseries_libs,doctest - --parallelism-per-worker 3 - --skip-ray-installation - depends_on: servebuild - - - label: ":ray-serve: serve: default minimal" - tags: python - instance_type: small - commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/dashboard/... serve - --parallelism-per-worker 2 - --build-name minbuild-default-py3.9 - --test-env=RAY_DEFAULT=1 - --only-tags minimal - depends_on: minbuild-serve - - label: ":ray-serve: serve: serve minimal" tags: - serve @@ -148,42 +51,6 @@ steps: commands: - bazel run //ci/ray_ci:test_in_docker -- python/ray/dashboard/... serve --parallelism-per-worker 3 + - sleep 18000 depends_on: servebuild - - label: ":ray-serve: serve: HA integration tests" - tags: - - serve - - python - instance_type: medium - commands: - - bazel run //ci/ray_ci:build_in_docker -- docker --platform cpu --canonical-tag ha_integration - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/serve/tests/... serve - --only-tags ha_integration - depends_on: - - manylinux - - forge - - raycpubase - - servebuild - - - label: ":ray-serve: serve: doc gpu tests" - tags: - - serve - - doc - - gpu - instance_type: gpu - commands: - - bazel run //ci/ray_ci:test_in_docker -- //doc/... serve - --build-name docgpubuild - --only-tags gpu - depends_on: docgpubuild - - - label: ":ray-serve: serve: flaky tests" - tags: - - serve - - skip-on-premerge - - python - instance_type: medium - soft_fail: true - commands: - - bazel run //ci/ray_ci:test_in_docker -- //... serve --run-flaky-tests --parallelism-per-worker 3 - depends_on: servebuild From 6254ff1ed878375f8ae6e08427ec793d949ee0fe Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Fri, 10 May 2024 13:17:43 -0700 Subject: [PATCH 19/40] fix Signed-off-by: Rui Qiao --- src/ray/raylet/agent_manager.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/raylet/agent_manager.h b/src/ray/raylet/agent_manager.h index 9d5fea55965dd..d7c155f49ea23 100644 --- a/src/ray/raylet/agent_manager.h +++ b/src/ray/raylet/agent_manager.h @@ -62,7 +62,7 @@ class AgentManager { bool start_agent = true /* for test */) : options_(std::move(options)), delay_executor_(std::move(delay_executor)), - shutdown_raylet_gracefully_(std::move(shutdown_raylet_gracefully)), + shutdown_raylet_gracefully_(shutdown_raylet_gracefully), fate_shares_(options_.fate_shares) { if (options_.agent_name.empty()) { RAY_LOG(FATAL) << "AgentManager agent_name must not be empty."; From eb15a1adbc99296f94963c3c5886778e6a53e91b Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Fri, 10 May 2024 15:21:18 -0700 Subject: [PATCH 20/40] debug Signed-off-by: Rui Qiao --- .buildkite/serve.rayci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.buildkite/serve.rayci.yml b/.buildkite/serve.rayci.yml index f43a3bcd1fe39..38481477b1fc8 100644 --- a/.buildkite/serve.rayci.yml +++ b/.buildkite/serve.rayci.yml @@ -49,7 +49,7 @@ steps: - dashboard instance_type: medium commands: - - bazel run //ci/ray_ci:test_in_docker -- python/ray/dashboard/... serve + - bazel run //ci/ray_ci:test_in_docker -- python/ray/dashboard/... serve || true --parallelism-per-worker 3 - sleep 18000 depends_on: servebuild From 2b95e7e380ea6edd28f89b69092abdb8c4978d48 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Fri, 10 May 2024 15:23:17 -0700 Subject: [PATCH 21/40] fix Signed-off-by: Rui Qiao --- .buildkite/serve.rayci.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.buildkite/serve.rayci.yml b/.buildkite/serve.rayci.yml index 38481477b1fc8..e62cc47ed35a6 100644 --- a/.buildkite/serve.rayci.yml +++ b/.buildkite/serve.rayci.yml @@ -49,8 +49,8 @@ steps: - dashboard instance_type: medium commands: - - bazel run //ci/ray_ci:test_in_docker -- python/ray/dashboard/... serve || true - --parallelism-per-worker 3 + - bazel run //ci/ray_ci:test_in_docker -- python/ray/dashboard/... serve + --parallelism-per-worker 3 || true - sleep 18000 depends_on: servebuild From b562a90b73b5a3c01e0d0aa830a4c252067da762 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Fri, 10 May 2024 15:55:59 -0700 Subject: [PATCH 22/40] debug Signed-off-by: Rui Qiao --- .buildkite/serve.rayci.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.buildkite/serve.rayci.yml b/.buildkite/serve.rayci.yml index e62cc47ed35a6..c890c799712a1 100644 --- a/.buildkite/serve.rayci.yml +++ b/.buildkite/serve.rayci.yml @@ -49,8 +49,7 @@ steps: - dashboard instance_type: medium commands: - - bazel run //ci/ray_ci:test_in_docker -- python/ray/dashboard/... serve - --parallelism-per-worker 3 || true + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/dashboard:test_dashboard serve || true - sleep 18000 depends_on: servebuild From 223018925e2a75abc43c619b81f66a1e1b6c1fc8 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Fri, 10 May 2024 15:57:21 -0700 Subject: [PATCH 23/40] Revert "debug" This reverts commit dd3d2859f2090307d3b6ecabc2378f0c437ac348. Signed-off-by: Rui Qiao --- .buildkite/core.rayci.yml | 268 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 268 insertions(+) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index 4897f71ebef09..85c025e8350cd 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -31,6 +31,274 @@ steps: depends_on: - corebuild + # tests + - label: ":ray: core: python tests" + tags: + - python + - dashboard + instance_type: large + parallelism: 4 + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... python/ray/autoscaler/v2/... core + --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" --parallelism-per-worker 3 + --except-tags debug_tests,asan_tests,post_wheel_build,ha_integration,mem_pressure,tmpfs,container,manual,use_all_core + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... python/ray/autoscaler/v2/... core + --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" + --only-tags use_all_core --skip-ray-installation + + - label: ":ray: core: redis tests" + tags: python + instance_type: large + parallelism: 4 + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... python/ray/autoscaler/v2/... core + --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" --parallelism-per-worker 3 + --test-env=TEST_EXTERNAL_REDIS=1 + --except-tags debug_tests,asan_tests,post_wheel_build,ha_integration,mem_pressure,tmpfs,container,manual,use_all_core + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dag/... python/ray/autoscaler/v2/... core + --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" + --only-tags use_all_core --skip-ray-installation + + - label: ":ray: core: memory pressure tests" + tags: + - python + - oss + - skip-on-premerge + instance_type: medium + commands: + - cleanup() { ./ci/build/upload_build_info.sh; }; trap cleanup EXIT + - (cd dashboard/client && npm ci && npm run build) + - pip install -e python[client] + - bazel test --config=ci --jobs=1 $(./ci/run/bazel_export_options) + --test_tag_filters=mem_pressure -- //python/ray/tests/... + job_env: corebuild + + - label: ":ray: core: out of disk tests" + tags: + - python + - oss + instance_type: small + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core + --only-tags=tmpfs --tmp-filesystem=tmpfs + + - label: ":ray: core: out of disk redis tests" + tags: + - python + - oss + instance_type: small + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core + --test-env=TEST_EXTERNAL_REDIS=1 + --only-tags=tmpfs --tmp-filesystem=tmpfs + + - label: ":ray: core: workflow tests" + tags: + - python + - workflow + instance_type: medium + parallelism: 2 + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/workflow/... core + --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" + --except-tags use_all_core + --parallelism-per-worker 2 + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/workflow/... core + --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" + --skip-ray-installation + --only-tags use_all_core + + - label: ":ray: core: doc tests" + tags: + - python + - doc + instance_type: large + commands: + # doc tests + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/... //doc/... core + --only-tags doctest + --parallelism-per-worker 3 + # doc examples + - bazel run //ci/ray_ci:test_in_docker -- //doc/... core + --except-tags gpu,post_wheel_build,mem_pressure,doctest + --parallelism-per-worker 3 + --skip-ray-installation + + - label: ":ray: core: data tests" + tags: python + instance_type: medium + commands: + - bazel run //ci/ray_ci:test_in_docker -- + python/ray/util/dask/... python/ray/tests/modin/... core + --build-name data15build + --parallelism-per-worker 2 + depends_on: + - data15build + - forge + + - label: ":ray: core: dashboard tests" + tags: + - python + - dashboard + instance_type: medium + commands: + - bazel run //ci/ray_ci:test_in_docker -- python/ray/dashboard/... core + --parallelism-per-worker 3 + # ui tests + - docker run -i --rm --volume /tmp/artifacts:/artifact-mount --shm-size=2.5gb + "$${RAYCI_WORK_REPO}":"$${RAYCI_BUILD_ID}"-corebuild /bin/bash -iecuo pipefail + "./dashboard/tests/run_ui_tests.sh" + + - label: ":ray: core: debug test" + tags: python + instance_type: medium + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core + --build-type debug + --parallelism-per-worker 3 + --only-tags debug_tests + --except-tags kubernetes,manual + + - label: ":ray: core: asan tests" + tags: python + instance_type: medium + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core + --build-type asan + --parallelism-per-worker 2 + --only-tags asan_tests + --except-tags kubernetes,manual + + - label: ":ray: core: wheel tests" + tags: linux_wheels + instance_type: medium + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //doc/... core + --build-type wheel + --parallelism-per-worker 3 + --only-tags post_wheel_build + --test-env=RAY_CI_POST_WHEEL_TESTS=True + depends_on: + - manylinux + - corebuild + - forge + + - label: ":ray: core: minimal tests {{matrix}}" + tags: + - python + - dashboard + - oss + instance_type: medium + commands: + # validate minimal installation + - python ./ci/env/check_minimal_install.py + # core tests + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dashboard/... core + --parallelism-per-worker 3 + --build-name minbuild-core-py{{matrix}} + --test-env=RAY_MINIMAL=1 + --test-env=EXPECTED_PYTHON_VERSION={{matrix}} + --only-tags minimal + --except-tags basic_test,manual + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dashboard/... core + --parallelism-per-worker 3 + --build-name minbuild-core-py{{matrix}} + --test-env=RAY_MINIMAL=1 + --test-env=EXPECTED_PYTHON_VERSION={{matrix}} + --only-tags minimal + --except-tags no_basic_test,manual + --skip-ray-installation + # core redis tests + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dashboard/... core + --parallelism-per-worker 3 + --build-name minbuild-core-py{{matrix}} + --test-env=RAY_MINIMAL=1 + --test-env=TEST_EXTERNAL_REDIS=1 + --test-env=EXPECTED_PYTHON_VERSION={{matrix}} + --only-tags minimal + --except-tags no_basic_test,manual + --skip-ray-installation + # serve tests + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... //python/ray/dashboard/... serve + --parallelism-per-worker 3 + --build-name minbuild-core-py{{matrix}} + --test-env=RAY_MINIMAL=1 + --only-tags minimal + --skip-ray-installation + depends_on: + - minbuild-core + matrix: + - "3.9" + - "3.10" + - "3.11" + + # cpp tests + - label: ":ray: core: cpp tests" + tags: core_cpp + instance_type: medium + commands: + - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type clang + --parallelism-per-worker 2 + + - label: ":ray: core: cpp asan tests" + tags: core_cpp + instance_type: medium + commands: + - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type asan-clang + --parallelism-per-worker 2 + + - label: ":ray: core: cpp ubsan tests" + tags: core_cpp + instance_type: large + commands: + - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type ubsan + --except-tags no_ubsan + --parallelism-per-worker 2 + + - label: ":ray: core: cpp tsan tests" + tags: core_cpp + instance_type: medium + commands: + - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type tsan-clang + --except-tags no_tsan + --parallelism-per-worker 2 + + - label: ":ray: core: flaky tests" + tags: + - python + - skip-on-premerge + instance_type: large + soft_fail: true + commands: + - bazel run //ci/ray_ci:test_in_docker -- //... core + --run-flaky-tests --build-type clang + + - label: ":ray: core: cpp worker tests" + tags: core_cpp + instance_type: medium + commands: + - if [[ "$${BUILDKITE_PIPELINE_ID}" == "0189942e-0876-4b8f-80a4-617f988ec59b" ]]; then + echo "build --remote_upload_local_results=false" >> ~/.bazelrc; + fi + - ci/ci.sh build + - ci/ci.sh test_cpp + depends_on: oss-ci-base_build + job_env: oss-ci-base_build + + - label: ":ray: core: HA integration tests" + tags: + - python + - docker + instance_type: medium + commands: + - bazel run //ci/ray_ci:build_in_docker -- docker --platform cpu --canonical-tag ha_integration + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core --only-tags ha_integration + depends_on: + - manylinux + - forge + - raycpubase + - corebuild + - label: ":ray: core: container tests" tags: - python From 844419e7857980cdc6bb3d76bf147f629fea92d3 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Mon, 13 May 2024 13:00:38 -0700 Subject: [PATCH 24/40] Revert "debug" This reverts commit 7acc5509edb27a362964c19a72817401f0fb4703. Signed-off-by: Rui Qiao --- .buildkite/core.rayci.yml | 1 + .buildkite/serve.rayci.yml | 138 ++++++++++++++++++++++++++++++++++++- 2 files changed, 137 insertions(+), 2 deletions(-) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index 85c025e8350cd..1137f0e3060a1 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -310,6 +310,7 @@ steps: - docker build --progress=plain --build-arg BASE_IMAGE="rayproject/ray:test_container" -t rayproject/ray:runtime_env_container -f docker/runtime_env_container/Dockerfile . - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core --only-tags container + - sleep 18000 depends_on: - manylinux - forge diff --git a/.buildkite/serve.rayci.yml b/.buildkite/serve.rayci.yml index c890c799712a1..c4d6183534136 100644 --- a/.buildkite/serve.rayci.yml +++ b/.buildkite/serve.rayci.yml @@ -29,6 +29,103 @@ steps: EXTRA_DEPENDENCY: "{{matrix}}" # tests + - label: ":ray-serve: serve: tests" + parallelism: 2 + tags: + - serve + - python + instance_type: large + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/serve/... //python/ray/tests/... serve + --except-tags post_wheel_build,gpu,worker-container,ha_integration + --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" --parallelism-per-worker 3 + --build-name servebuild --test-env=EXPECTED_PYTHON_VERSION=3.9 + depends_on: "servebuild" + + - label: ":ray-serve: serve: pydantic < 2.0 tests" + parallelism: 2 + tags: + - serve + - python + instance_type: large + soft_fail: true + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/serve/... //python/ray/tests/... serve + --except-tags post_wheel_build,gpu,worker-container,ha_integration + --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" --parallelism-per-worker 3 + --build-name servepydantic1build --test-env=EXPECTED_PYTHON_VERSION=3.9 --test-env=EXPECTED_PYDANTIC_VERSION=1.10.12 + depends_on: servepydantic1build + + - label: ":ray-serve: serve: python {{matrix.python}} tests ({{matrix.worker_id}})" + if: build.env("RAYCI_CONTINUOUS_BUILD") == "1" || build.pull_request.labels includes "continuous-build" + tags: + - serve + - python + instance_type: large + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/serve/... //python/ray/tests/... serve + --except-tags post_wheel_build,gpu,worker-container,ha_integration + --workers 2 --worker-id {{matrix.worker_id}} --parallelism-per-worker 3 + --python-version {{matrix.python}} + --test-env=EXPECTED_PYTHON_VERSION={{matrix.python}} + depends_on: servebuild-multipy + matrix: + setup: + python: ["3.11"] + worker_id: ["0", "1"] + + - label: ":ray-serve: serve: release tests" + tags: + - serve + - release_tests + - python + instance_type: medium + commands: + - bazel run //ci/ray_ci:test_in_docker -- //release/... serve --parallelism-per-worker 3 + depends_on: servebuild + + - label: ":ray-serve: serve: wheel tests" + tags: linux_wheels + instance_type: medium + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/serve/... //doc/... serve + --build-type wheel + --parallelism-per-worker 3 + --only-tags post_wheel_build + --test-env=RAY_CI_POST_WHEEL_TESTS=True + depends_on: + - manylinux + - servebuild + - forge + + - label: ":ray-serve: serve: doc tests" + tags: + - serve + - doc + instance_type: large + commands: + # doc tests + - bazel run //ci/ray_ci:test_in_docker -- python/ray/... //doc/... serve + --only-tags doctest + --parallelism-per-worker 3 + # doc examples + - bazel run //ci/ray_ci:test_in_docker -- //doc/... serve + --except-tags gpu,post_wheel_build,timeseries_libs,doctest + --parallelism-per-worker 3 + --skip-ray-installation + depends_on: servebuild + + - label: ":ray-serve: serve: default minimal" + tags: python + instance_type: small + commands: + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/dashboard/... serve + --parallelism-per-worker 2 + --build-name minbuild-default-py3.9 + --test-env=RAY_DEFAULT=1 + --only-tags minimal + depends_on: minbuild-serve + - label: ":ray-serve: serve: serve minimal" tags: - serve @@ -49,7 +146,44 @@ steps: - dashboard instance_type: medium commands: - - bazel run //ci/ray_ci:test_in_docker -- //python/ray/dashboard:test_dashboard serve || true - - sleep 18000 + - bazel run //ci/ray_ci:test_in_docker -- python/ray/dashboard/... serve + --parallelism-per-worker 3 depends_on: servebuild + - label: ":ray-serve: serve: HA integration tests" + tags: + - serve + - python + instance_type: medium + commands: + - bazel run //ci/ray_ci:build_in_docker -- docker --platform cpu --canonical-tag ha_integration + - bazel run //ci/ray_ci:test_in_docker -- //python/ray/serve/tests/... serve + --only-tags ha_integration + depends_on: + - manylinux + - forge + - raycpubase + - servebuild + + - label: ":ray-serve: serve: doc gpu tests" + tags: + - serve + - doc + - gpu + instance_type: gpu + commands: + - bazel run //ci/ray_ci:test_in_docker -- //doc/... serve + --build-name docgpubuild + --only-tags gpu + depends_on: docgpubuild + + - label: ":ray-serve: serve: flaky tests" + tags: + - serve + - skip-on-premerge + - python + instance_type: medium + soft_fail: true + commands: + - bazel run //ci/ray_ci:test_in_docker -- //... serve --run-flaky-tests --parallelism-per-worker 3 + depends_on: servebuild From a8325572e559d71270f7305de92109205dfa7e1d Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Mon, 13 May 2024 13:05:48 -0700 Subject: [PATCH 25/40] fix Signed-off-by: Rui Qiao --- dashboard/tests/test_dashboard.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dashboard/tests/test_dashboard.py b/dashboard/tests/test_dashboard.py index 02d9a83626311..03149a30a1ca2 100644 --- a/dashboard/tests/test_dashboard.py +++ b/dashboard/tests/test_dashboard.py @@ -940,7 +940,11 @@ def test_dashboard_port_conflict(ray_start_with_dashboard): os.environ.get("RAY_MINIMAL") == "1", reason="This test is not supposed to work for minimal installation.", ) -def test_gcs_check_alive(fast_gcs_failure_detection, ray_start_with_dashboard): +def test_gcs_check_alive( + fast_gcs_failure_detection, ray_start_with_dashboard, call_ray_stop_only +): + # call_ray_stop_only is used to ensure a clean environment (especially + # killing dashboard agent in time) before the next test runs. assert wait_until_server_available(ray_start_with_dashboard["webui_url"]) is True all_processes = ray._private.worker._global_node.all_processes From fec13c26c9550666675d6de9c689e9ba95d4e1e0 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Mon, 13 May 2024 14:01:22 -0700 Subject: [PATCH 26/40] cleanup Signed-off-by: Rui Qiao --- .buildkite/core.rayci.yml | 1 - src/ray/gcs/gcs_client/accessor.cc | 18 ++-------- src/ray/gcs/gcs_client/accessor.h | 5 +-- src/ray/gcs/gcs_server/gcs_node_manager.cc | 14 ++------ src/ray/raylet/main.cc | 35 +++++++++---------- .../placement_group_resource_manager_test.cc | 2 +- src/ray/raylet/raylet.h | 4 +++ src/ray/raylet/runtime_env_agent_client.cc | 2 +- .../scheduling/cluster_task_manager_test.cc | 8 ++--- .../scheduling/local_resource_manager.cc | 7 ++-- .../scheduling/local_resource_manager_test.cc | 2 +- 11 files changed, 39 insertions(+), 59 deletions(-) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index 1137f0e3060a1..85c025e8350cd 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -310,7 +310,6 @@ steps: - docker build --progress=plain --build-arg BASE_IMAGE="rayproject/ray:test_container" -t rayproject/ray:runtime_env_container -f docker/runtime_env_container/Dockerfile . - bazel run //ci/ray_ci:test_in_docker -- //python/ray/tests/... core --only-tags container - - sleep 18000 depends_on: - manylinux - forge diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 09f0131972f8c..55bdbf9371064 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -445,27 +445,14 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf std::function unregister_done_callback) { if (local_node_id_.IsNil()) { RAY_LOG(INFO) << "The node is already unregistered."; - // This node is already unregistered. return Status::OK(); } auto node_id = NodeID::FromBinary(local_node_info_.node_id()); - RAY_LOG(INFO) << "Unregistering node info, node id = " << node_id; + RAY_LOG(INFO) << "Unregistering node, node id = " << node_id; + rpc::UnregisterNodeRequest request; request.set_node_id(local_node_info_.node_id()); request.mutable_node_death_info()->CopyFrom(node_death_info); - // Unregister the node synchronously because this method is only called in the - // raylet shutdown path: waiting is not an issue and actually desired. - - // rpc::UnregisterNodeReply reply; - // Status status = client_impl_->GetGcsRpcClient().SyncUnregisterNode(request, &reply); - // RAY_LOG(INFO) << "Finished unregistering node info, status = " << status - // << ", node id = " << node_id; - // if (status.ok()) { - // local_node_info_.set_state(GcsNodeInfo::DEAD); - // local_node_id_ = NodeID::Nil(); - // } - // return status; - client_impl_->GetGcsRpcClient().UnregisterNode( request, [this, node_id, unregister_done_callback](const Status &status, @@ -477,7 +464,6 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf RAY_LOG(INFO) << "Finished unregistering node info, status = " << status << ", node id = " << node_id; unregister_done_callback(); - RAY_LOG(INFO) << "Finished executing unregister_done_callback."; }); return Status::OK(); } diff --git a/src/ray/gcs/gcs_client/accessor.h b/src/ray/gcs/gcs_client/accessor.h index 094dab1f329bb..0d40438a8312b 100644 --- a/src/ray/gcs/gcs_client/accessor.h +++ b/src/ray/gcs/gcs_client/accessor.h @@ -288,9 +288,10 @@ class NodeInfoAccessor { /// Unregister local node to GCS synchronously /// - /// \param node_death_info The death information of the node to unregister from GCS. + /// \param node_death_info The death information regarding why to unregister from GCS. /// \param unregister_done_callback Callback that will be called when unregistration is - /// done. \return Status + /// done. + /// \return Status virtual Status UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, std::function unregister_done_callback); diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.cc b/src/ray/gcs/gcs_server/gcs_node_manager.cc index 8d43f1e706d03..94411b5f05d2b 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_node_manager.cc @@ -112,7 +112,7 @@ void GcsNodeManager::HandleUnregisterNode(rpc::UnregisterNodeRequest request, rpc::UnregisterNodeReply *reply, rpc::SendReplyCallback send_reply_callback) { NodeID node_id = NodeID::FromBinary(request.node_id()); - RAY_LOG(DEBUG) << "HandleUnregisterNode, node id = " << node_id; + RAY_LOG(DEBUG) << "HandleUnregisterNode() for node id = " << node_id; auto node = RemoveNode(node_id, /* is_intended = */ true); if (!node) { RAY_LOG(INFO) << "Node " << node_id << " is already removed"; @@ -126,19 +126,11 @@ void GcsNodeManager::HandleUnregisterNode(rpc::UnregisterNodeRequest request, node_info_delta->set_state(node->state()); node_info_delta->set_end_time_ms(node->end_time_ms()); auto on_put_done = [=](const Status &status) { - RAY_LOG(DEBUG) << "Publishing node info, node id = " << node_id; RAY_CHECK_OK(gcs_publisher_->PublishNodeInfo(node_id, *node_info_delta, nullptr)); }; - // Update node state to DEAD instead of deleting it. + RAY_CHECK_OK(gcs_table_storage_->NodeTable().Put(node_id, *node, on_put_done)); - // GCS_RPC_SEND_REPLY(send_reply_callback, reply, Status::OK()); - auto status = Status::OK(); - reply->mutable_status()->set_code((int)status.code()); - reply->mutable_status()->set_message(status.message()); - send_reply_callback( - status, - []() { RAY_LOG(DEBUG) << "HandleUnregisterNode reply succeeded"; }, - []() { RAY_LOG(DEBUG) << "HandleUnregisterNode reply failed"; }); + GCS_RPC_SEND_REPLY(send_reply_callback, reply, Status::OK()); } void GcsNodeManager::HandleDrainNode(rpc::DrainNodeRequest request, diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index 9eb0c83c20da9..b3a3a71606a3f 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -227,8 +227,9 @@ int main(int argc, char *argv[]) { auto shutted_down = std::make_shared>(false); - auto unregister_done_callback = + auto shutdown_raylet_after_unregistration = [&main_service, &raylet_socket_name, &raylet, &gcs_client]() { + // We should stop the service and remove the local socket file. raylet->Stop(); gcs_client->Disconnect(); ray::stats::Shutdown(); @@ -236,36 +237,33 @@ int main(int argc, char *argv[]) { remove(raylet_socket_name.c_str()); }; - // Shut down raylet gracefully. The pointer to main_service is - // guaranteed to be valid since this function will run the event loop - // instead of returning immediately. - // We should stop the service and remove the local socket file. - auto shutdown_raylet_gracefully_sync = - [&raylet, shutted_down, unregister_done_callback]( + // Shut down raylet gracefully, in a synchronous fashion. + // This is an internal method and should only be run on the main_service. + auto _shutdown_raylet_gracefully_sync = + [&raylet, shutted_down, shutdown_raylet_after_unregistration]( const ray::rpc::NodeDeathInfo &node_death_info) { - // Make the shutdown handler idempotent since graceful shutdown can be triggered + // Make the shutdown method idempotent since graceful shutdown can be triggered // by many places. if (*shutted_down) { RAY_LOG(INFO) << "Raylet shutdown already triggered, ignoring this request."; return; } - RAY_LOG(INFO) << "Raylet graceful shutdown triggered, " - << "reason = " + RAY_LOG(INFO) << "Raylet graceful shutdown triggered, reason = " << NodeDeathInfo_Reason_Name(node_death_info.reason()) << ", " << "reason message = " << node_death_info.reason_message(); RAY_LOG(INFO) << "Shutting down..."; *shutted_down = true; - raylet->UnregisterSelf(node_death_info, unregister_done_callback); + raylet->UnregisterSelf(node_death_info, shutdown_raylet_after_unregistration); }; - auto shutdown_raylet_gracefully = [&main_service, shutdown_raylet_gracefully_sync]( + auto shutdown_raylet_gracefully = [&main_service, _shutdown_raylet_gracefully_sync]( const ray::rpc::NodeDeathInfo &node_death_info) { main_service.post( - [shutdown_raylet_gracefully_sync, node_death_info]() { - shutdown_raylet_gracefully_sync(node_death_info); + [_shutdown_raylet_gracefully_sync, node_death_info]() { + _shutdown_raylet_gracefully_sync(node_death_info); }, - "shutdown_raylet_gracefully_sync"); + "_shutdown_raylet_gracefully_sync"); }; RAY_CHECK_OK(gcs_client->Nodes().AsyncGetInternalConfig( @@ -441,12 +439,12 @@ int main(int argc, char *argv[]) { raylet->Start(); })); - auto signal_handler = [shutdown_raylet_gracefully_sync]( + auto signal_handler = [_shutdown_raylet_gracefully_sync]( const boost::system::error_code &error, int signal_number) { ray::rpc::NodeDeathInfo node_death_info; node_death_info.set_reason(ray::rpc::NodeDeathInfo::EXPECTED_TERMINATION); - node_death_info.set_reason_message("SIGTERM received"); - shutdown_raylet_gracefully_sync(node_death_info); + node_death_info.set_reason_message("Received SIGTERM"); + _shutdown_raylet_gracefully_sync(node_death_info); }; boost::asio::signal_set signals(main_service); #ifdef _WIN32 @@ -457,6 +455,5 @@ int main(int argc, char *argv[]) { signals.async_wait(signal_handler); main_service.run(); - RAY_LOG(INFO) << "Raylet ends"; } #endif diff --git a/src/ray/raylet/placement_group_resource_manager_test.cc b/src/ray/raylet/placement_group_resource_manager_test.cc index a46694fe76a3a..607d82d9e25a5 100644 --- a/src/ray/raylet/placement_group_resource_manager_test.cc +++ b/src/ray/raylet/placement_group_resource_manager_test.cc @@ -184,7 +184,7 @@ TEST_F(NewPlacementGroupResourceManagerTest, TestNewPrepareBundleDuringDraining) ASSERT_TRUE(new_placement_group_resource_manager_->PrepareBundles(bundle1_specs)); // Drain the node, new bundle prepare will fail. - rpc::NodeDeathInfo node_death_info; // TODO: fill in proper values + rpc::NodeDeathInfo node_death_info; cluster_resource_scheduler_->GetLocalResourceManager().SetLocalNodeDraining( std::numeric_limits::max(), node_death_info); ASSERT_FALSE(new_placement_group_resource_manager_->PrepareBundles(bundle2_specs)); diff --git a/src/ray/raylet/raylet.h b/src/ray/raylet/raylet.h index 442a3464f3266..b0a92383e1827 100644 --- a/src/ray/raylet/raylet.h +++ b/src/ray/raylet/raylet.h @@ -68,6 +68,10 @@ class Raylet { void Stop(); /// Unregister this raylet from the GCS. + /// + /// \param node_death_info The death information regarding why to unregister self. + /// \param unregister_done_callback The callback to call when the unregistration is + /// done. void UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, std::function unregister_done_callback); diff --git a/src/ray/raylet/runtime_env_agent_client.cc b/src/ray/raylet/runtime_env_agent_client.cc index d0b400e35e11a..20715601e366e 100644 --- a/src/ray/raylet/runtime_env_agent_client.cc +++ b/src/ray/raylet/runtime_env_agent_client.cc @@ -289,7 +289,7 @@ class HttpRuntimeEnvAgentClient : public RuntimeEnvAgentClient { rpc::NodeDeathInfo node_death_info; node_death_info.set_reason( rpc::NodeDeathInfo_Reason::NodeDeathInfo_Reason_UNEXPECTED_TERMINATION); - node_death_info.set_reason_message("Runtime Env Agent timed out"); + node_death_info.set_reason_message("Raylet could not connect to Runtime Env Agent"); shutdown_raylet_gracefully_(node_death_info); // If the process is not terminated within 10 seconds, forcefully kill itself. delay_executor_([]() { QuickExit(); }, /*ms*/ 10000); diff --git a/src/ray/raylet/scheduling/cluster_task_manager_test.cc b/src/ray/raylet/scheduling/cluster_task_manager_test.cc index ef7ceccf130e4..2969071856a1c 100644 --- a/src/ray/raylet/scheduling/cluster_task_manager_test.cc +++ b/src/ray/raylet/scheduling/cluster_task_manager_test.cc @@ -743,7 +743,7 @@ TEST_F(ClusterTaskManagerTest, DrainingWhileResolving) { ASSERT_EQ(pool_.workers.size(), 1); // Drain the local node. - rpc::NodeDeathInfo node_death_info; // TODO: fill in values + rpc::NodeDeathInfo node_death_info; scheduler_->GetLocalResourceManager().SetLocalNodeDraining( std::numeric_limits::max(), node_death_info); @@ -1078,7 +1078,7 @@ TEST_F(ClusterTaskManagerTest, NotOKPopWorkerAfterDrainingTest) { AddNode(remote_node_id, 5); // Drain the local node. - rpc::NodeDeathInfo node_death_info; // TODO: fill in values + rpc::NodeDeathInfo node_death_info; scheduler_->GetLocalResourceManager().SetLocalNodeDraining( std::numeric_limits::max(), node_death_info); @@ -2622,7 +2622,7 @@ TEST_F(ClusterTaskManagerTest, PopWorkerBeforeDraining) { task_manager_.QueueAndScheduleTask(task, false, false, &reply, callback); // Drain the local node. - rpc::NodeDeathInfo node_death_info; // TODO: fill in values + rpc::NodeDeathInfo node_death_info; scheduler_->GetLocalResourceManager().SetLocalNodeDraining( std::numeric_limits::max(), node_death_info); @@ -2662,7 +2662,7 @@ TEST_F(ClusterTaskManagerTest, UnscheduleableWhileDraining) { AddNode(remote_node_id, 5); // Drain the local node. - rpc::NodeDeathInfo node_death_info; // TODO: fill in values + rpc::NodeDeathInfo node_death_info; scheduler_->GetLocalResourceManager().SetLocalNodeDraining( std::numeric_limits::max(), node_death_info); diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 301f16bbef2a2..14b77fb0c63ee 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -386,9 +386,10 @@ void LocalResourceManager::OnResourceOrStateChanged() { // The node is drained. if (shutdown_raylet_gracefully_ == nullptr) { - RAY_LOG(ERROR) << "shutdown_raylet_gracefully_ callback is not set, " - << "please check the LocalResourceManager constructor." - << "Using ShutdownRayletGracefully (sending SIGTERM) instead."; + RAY_LOG(ERROR) + << "shutdown_raylet_gracefully_ callback is not set, " + << "please check the LocalResourceManager constructor." + << "Using deprecated ShutdownRayletGracefully (sending SIGTERM) instead."; raylet::ShutdownRayletGracefully(); } else { shutdown_raylet_gracefully_(node_death_info_); diff --git a/src/ray/raylet/scheduling/local_resource_manager_test.cc b/src/ray/raylet/scheduling/local_resource_manager_test.cc index c0125e8287c00..5f453e4341457 100644 --- a/src/ray/raylet/scheduling/local_resource_manager_test.cc +++ b/src/ray/raylet/scheduling/local_resource_manager_test.cc @@ -182,7 +182,7 @@ TEST_F(LocalResourceManagerTest, ObjectStoreMemoryDrainingTest) { *used_object_store = 1; manager->UpdateAvailableObjectStoreMemResource(); - rpc::NodeDeathInfo node_death_info; // TODO: fill in values + rpc::NodeDeathInfo node_death_info; manager->SetLocalNodeDraining(std::numeric_limits::max(), node_death_info); ASSERT_TRUE(manager->IsLocalNodeDraining()); From 47a5810d5f58280d7366a76d3d6d2343d67dc171 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Mon, 13 May 2024 17:40:39 -0700 Subject: [PATCH 27/40] address comments Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.cc | 7 +++---- src/ray/gcs/gcs_client/accessor.h | 4 ++-- src/ray/gcs/gcs_server/gcs_node_manager.cc | 7 ++++++- src/ray/raylet/raylet.cc | 3 +-- 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 55bdbf9371064..b4a13918f1fe0 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -441,11 +441,11 @@ Status NodeInfoAccessor::RegisterSelf(const GcsNodeInfo &local_node_info, return Status::OK(); } -Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, - std::function unregister_done_callback) { +void NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, + std::function unregister_done_callback) { if (local_node_id_.IsNil()) { RAY_LOG(INFO) << "The node is already unregistered."; - return Status::OK(); + return; } auto node_id = NodeID::FromBinary(local_node_info_.node_id()); RAY_LOG(INFO) << "Unregistering node, node id = " << node_id; @@ -465,7 +465,6 @@ Status NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_inf << ", node id = " << node_id; unregister_done_callback(); }); - return Status::OK(); } Status NodeInfoAccessor::DrainSelf() { diff --git a/src/ray/gcs/gcs_client/accessor.h b/src/ray/gcs/gcs_client/accessor.h index 0d40438a8312b..c03c9d6ed9514 100644 --- a/src/ray/gcs/gcs_client/accessor.h +++ b/src/ray/gcs/gcs_client/accessor.h @@ -292,8 +292,8 @@ class NodeInfoAccessor { /// \param unregister_done_callback Callback that will be called when unregistration is /// done. /// \return Status - virtual Status UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, - std::function unregister_done_callback); + virtual void UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, + std::function unregister_done_callback); /// Drain (remove the information of the node from the cluster) the local node from GCS /// synchronously. diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.cc b/src/ray/gcs/gcs_server/gcs_node_manager.cc index 94411b5f05d2b..b0fde870002a6 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_node_manager.cc @@ -118,17 +118,22 @@ void GcsNodeManager::HandleUnregisterNode(rpc::UnregisterNodeRequest request, RAY_LOG(INFO) << "Node " << node_id << " is already removed"; return; } + node->set_state(rpc::GcsNodeInfo::DEAD); + node->mutable_death_info()->CopyFrom(request.node_death_info()); node->set_end_time_ms(current_sys_time_ms()); + AddDeadNodeToCache(node); + auto node_info_delta = std::make_shared(); node_info_delta->set_node_id(node->node_id()); + node_info_delta->mutable_death_info()->CopyFrom(request.node_death_info()); node_info_delta->set_state(node->state()); node_info_delta->set_end_time_ms(node->end_time_ms()); + auto on_put_done = [=](const Status &status) { RAY_CHECK_OK(gcs_publisher_->PublishNodeInfo(node_id, *node_info_delta, nullptr)); }; - RAY_CHECK_OK(gcs_table_storage_->NodeTable().Put(node_id, *node, on_put_done)); GCS_RPC_SEND_REPLY(send_reply_callback, reply, Status::OK()); } diff --git a/src/ray/raylet/raylet.cc b/src/ray/raylet/raylet.cc index f482ee9e83126..4b8a75c9edcce 100644 --- a/src/ray/raylet/raylet.cc +++ b/src/ray/raylet/raylet.cc @@ -120,8 +120,7 @@ void Raylet::Start() { void Raylet::UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, std::function unregister_done_callback) { - RAY_CHECK_OK( - gcs_client_->Nodes().UnregisterSelf(node_death_info, unregister_done_callback)); + gcs_client_->Nodes().UnregisterSelf(node_death_info, unregister_done_callback); } void Raylet::Stop() { From b9a369787dc16e75496afc6aef50731024baf477 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Mon, 13 May 2024 23:04:47 -0700 Subject: [PATCH 28/40] unit test Signed-off-by: Rui Qiao --- .../gcs/gcs_client/test/gcs_client_test.cc | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/src/ray/gcs/gcs_client/test/gcs_client_test.cc b/src/ray/gcs/gcs_client/test/gcs_client_test.cc index eaa7f251de895..d768c83e643d7 100644 --- a/src/ray/gcs/gcs_client/test/gcs_client_test.cc +++ b/src/ray/gcs/gcs_client/test/gcs_client_test.cc @@ -333,6 +333,11 @@ class GcsClientTest : public ::testing::TestWithParam { return WaitReady(promise.get_future(), timeout_ms_); } + void UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, + std::function unregister_done_callback) { + gcs_client_->Nodes().UnregisterSelf(node_death_info, unregister_done_callback); + } + std::vector GetNodeInfoList() { std::promise promise; std::vector nodes; @@ -569,6 +574,35 @@ TEST_P(GcsClientTest, TestNodeInfo) { ASSERT_TRUE(gcs_client_->Nodes().IsRemoved(node2_id)); } +TEST_P(GcsClientTest, TestUnregisterNode) { + // Create gcs node info. + auto gcs_node_info = Mocker::GenNodeInfo(); + NodeID node_id = NodeID::FromBinary(gcs_node_info->node_id()); + + // Register local node to GCS. + ASSERT_TRUE(RegisterSelf(*gcs_node_info)); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + EXPECT_EQ(gcs_client_->Nodes().GetSelfId(), node_id); + EXPECT_EQ(gcs_client_->Nodes().GetSelfInfo().node_id(), gcs_node_info->node_id()); + EXPECT_EQ(gcs_client_->Nodes().GetSelfInfo().state(), gcs_node_info->state()); + + // Unregister local node from GCS. + rpc::NodeDeathInfo node_death_info; + node_death_info.set_reason(rpc::NodeDeathInfo::EXPECTED_TERMINATION); + auto reason_message = "Testing unregister node from GCS."; + node_death_info.set_reason_message(reason_message); + + std::promise promise; + UnregisterSelf(node_death_info, [&promise]() { promise.set_value(true); }); + WaitReady(promise.get_future(), timeout_ms_); + + auto node_list = GetNodeInfoList(); + EXPECT_EQ(node_list.size(), 1); + EXPECT_EQ(node_list[0].state(), rpc::GcsNodeInfo::DEAD); + EXPECT_EQ(node_list[0].death_info().reason(), rpc::NodeDeathInfo::EXPECTED_TERMINATION); + EXPECT_EQ(node_list[0].death_info().reason_message(), reason_message); +} + TEST_P(GcsClientTest, TestGetAllAvailableResources) { // Register node. auto node_info = Mocker::GenNodeInfo(); From f31440e1733b629b60d2cc5dbc32411c88e0fa36 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Mon, 13 May 2024 23:15:18 -0700 Subject: [PATCH 29/40] address comments Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/accessor.h | 3 +-- src/ray/raylet/main.cc | 14 +++++++------- src/ray/raylet/runtime_env_agent_client.cc | 3 +-- src/ray/raylet/scheduling/local_resource_manager.h | 1 + 4 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/ray/gcs/gcs_client/accessor.h b/src/ray/gcs/gcs_client/accessor.h index c03c9d6ed9514..8c6ed9167c811 100644 --- a/src/ray/gcs/gcs_client/accessor.h +++ b/src/ray/gcs/gcs_client/accessor.h @@ -286,12 +286,11 @@ class NodeInfoAccessor { virtual Status RegisterSelf(const rpc::GcsNodeInfo &local_node_info, const StatusCallback &callback); - /// Unregister local node to GCS synchronously + /// Unregister local node to GCS asynchronously. /// /// \param node_death_info The death information regarding why to unregister from GCS. /// \param unregister_done_callback Callback that will be called when unregistration is /// done. - /// \return Status virtual void UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, std::function unregister_done_callback); diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index b3a3a71606a3f..e5d89d68fc1cf 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -239,7 +239,7 @@ int main(int argc, char *argv[]) { // Shut down raylet gracefully, in a synchronous fashion. // This is an internal method and should only be run on the main_service. - auto _shutdown_raylet_gracefully_sync = + auto shutdown_raylet_gracefully_internal = [&raylet, shutted_down, shutdown_raylet_after_unregistration]( const ray::rpc::NodeDeathInfo &node_death_info) { // Make the shutdown method idempotent since graceful shutdown can be triggered @@ -257,13 +257,13 @@ int main(int argc, char *argv[]) { raylet->UnregisterSelf(node_death_info, shutdown_raylet_after_unregistration); }; - auto shutdown_raylet_gracefully = [&main_service, _shutdown_raylet_gracefully_sync]( + auto shutdown_raylet_gracefully = [&main_service, shutdown_raylet_gracefully_internal]( const ray::rpc::NodeDeathInfo &node_death_info) { main_service.post( - [_shutdown_raylet_gracefully_sync, node_death_info]() { - _shutdown_raylet_gracefully_sync(node_death_info); + [shutdown_raylet_gracefully_internal, node_death_info]() { + shutdown_raylet_gracefully_internal(node_death_info); }, - "_shutdown_raylet_gracefully_sync"); + "shutdown_raylet_gracefully_internal"); }; RAY_CHECK_OK(gcs_client->Nodes().AsyncGetInternalConfig( @@ -439,12 +439,12 @@ int main(int argc, char *argv[]) { raylet->Start(); })); - auto signal_handler = [_shutdown_raylet_gracefully_sync]( + auto signal_handler = [shutdown_raylet_gracefully_internal]( const boost::system::error_code &error, int signal_number) { ray::rpc::NodeDeathInfo node_death_info; node_death_info.set_reason(ray::rpc::NodeDeathInfo::EXPECTED_TERMINATION); node_death_info.set_reason_message("Received SIGTERM"); - _shutdown_raylet_gracefully_sync(node_death_info); + shutdown_raylet_gracefully_internal(node_death_info); }; boost::asio::signal_set signals(main_service); #ifdef _WIN32 diff --git a/src/ray/raylet/runtime_env_agent_client.cc b/src/ray/raylet/runtime_env_agent_client.cc index 20715601e366e..999b82c5c1a91 100644 --- a/src/ray/raylet/runtime_env_agent_client.cc +++ b/src/ray/raylet/runtime_env_agent_client.cc @@ -287,8 +287,7 @@ class HttpRuntimeEnvAgentClient : public RuntimeEnvAgentClient { "https://docs.ray.io/en/master/ray-observability/user-guides/" "configure-logging.html#logging-directory-structure.\n"; rpc::NodeDeathInfo node_death_info; - node_death_info.set_reason( - rpc::NodeDeathInfo_Reason::NodeDeathInfo_Reason_UNEXPECTED_TERMINATION); + node_death_info.set_reason(rpc::NodeDeathInfo::UNEXPECTED_TERMINATION); node_death_info.set_reason_message("Raylet could not connect to Runtime Env Agent"); shutdown_raylet_gracefully_(node_death_info); // If the process is not terminated within 10 seconds, forcefully kill itself. diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index c70c90eb0e390..a238fffd8ec42 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -233,6 +233,7 @@ class LocalResourceManager : public syncer::ReporterInterface { // 0 if there is no deadline. int64_t local_node_draining_deadline_timestamp_ms_ = -1; + /// This is set when the node is being drained and indicates the reason for draining. rpc::NodeDeathInfo node_death_info_; FRIEND_TEST(ClusterResourceSchedulerTest, SchedulingUpdateTotalResourcesTest); From c6da857537e8b1246a3dfff3110bf569a4e5c4ea Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 14 May 2024 08:50:31 -0700 Subject: [PATCH 30/40] cleanup Signed-off-by: Rui Qiao --- src/mock/ray/gcs/gcs_client/accessor.h | 1 - src/ray/gcs/gcs_client/accessor.cc | 23 ------------------- src/ray/gcs/gcs_client/accessor.h | 8 ------- .../gcs/gcs_client/test/gcs_client_test.cc | 14 ++--------- .../gcs_server/test/gcs_server_test_util.h | 2 -- src/ray/raylet/agent_manager.cc | 4 ++-- src/ray/raylet/node_manager.cc | 2 +- 7 files changed, 5 insertions(+), 49 deletions(-) diff --git a/src/mock/ray/gcs/gcs_client/accessor.h b/src/mock/ray/gcs/gcs_client/accessor.h index d2eeb0a36f25f..96e2af50021b1 100644 --- a/src/mock/ray/gcs/gcs_client/accessor.h +++ b/src/mock/ray/gcs/gcs_client/accessor.h @@ -124,7 +124,6 @@ class MockNodeInfoAccessor : public NodeInfoAccessor { RegisterSelf, (const rpc::GcsNodeInfo &local_node_info, const StatusCallback &callback), (override)); - MOCK_METHOD(Status, DrainSelf, (), (override)); MOCK_METHOD(const NodeID &, GetSelfId, (), (const, override)); MOCK_METHOD(const rpc::GcsNodeInfo &, GetSelfInfo, (), (const, override)); MOCK_METHOD(Status, diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index b4a13918f1fe0..04ae5353da2dc 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -467,29 +467,6 @@ void NodeInfoAccessor::UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, }); } -Status NodeInfoAccessor::DrainSelf() { - if (local_node_id_.IsNil()) { - RAY_LOG(INFO) << "The node is already drained."; - // This node is already drained. - return Status::OK(); - } - NodeID node_id = NodeID::FromBinary(local_node_info_.node_id()); - RAY_LOG(INFO) << "Unregistering node info, node id = " << node_id; - rpc::DrainNodeRequest request; - auto draining_request = request.add_drain_node_data(); - draining_request->set_node_id(local_node_info_.node_id()); - client_impl_->GetGcsRpcClient().DrainNode( - request, [this, node_id](const Status &status, const rpc::DrainNodeReply &reply) { - if (status.ok()) { - local_node_info_.set_state(GcsNodeInfo::DEAD); - local_node_id_ = NodeID::Nil(); - } - RAY_LOG(INFO) << "Finished unregistering node info, status = " << status - << ", node id = " << node_id; - }); - return Status::OK(); -} - const NodeID &NodeInfoAccessor::GetSelfId() const { return local_node_id_; } const GcsNodeInfo &NodeInfoAccessor::GetSelfInfo() const { return local_node_info_; } diff --git a/src/ray/gcs/gcs_client/accessor.h b/src/ray/gcs/gcs_client/accessor.h index 8c6ed9167c811..f53806c0f9982 100644 --- a/src/ray/gcs/gcs_client/accessor.h +++ b/src/ray/gcs/gcs_client/accessor.h @@ -294,14 +294,6 @@ class NodeInfoAccessor { virtual void UnregisterSelf(const rpc::NodeDeathInfo &node_death_info, std::function unregister_done_callback); - /// Drain (remove the information of the node from the cluster) the local node from GCS - /// synchronously. - /// - /// Once the RPC is replied, it is guaranteed that GCS drains the information of the - /// local node, and all the nodes in the cluster will "eventually" be informed that the - /// node is drained. \return Status - virtual Status DrainSelf(); - /// Get id of local node which was registered by 'RegisterSelf'. /// /// \return NodeID diff --git a/src/ray/gcs/gcs_client/test/gcs_client_test.cc b/src/ray/gcs/gcs_client/test/gcs_client_test.cc index d768c83e643d7..f4cbeac9c6aac 100644 --- a/src/ray/gcs/gcs_client/test/gcs_client_test.cc +++ b/src/ray/gcs/gcs_client/test/gcs_client_test.cc @@ -321,11 +321,6 @@ class GcsClientTest : public ::testing::TestWithParam { return status.ok(); } - bool DrainSelf() { - Status status = gcs_client_->Nodes().DrainSelf(); - return status.ok(); - } - bool RegisterNode(const rpc::GcsNodeInfo &node_info) { std::promise promise; RAY_CHECK_OK(gcs_client_->Nodes().AsyncRegister( @@ -557,9 +552,6 @@ TEST_P(GcsClientTest, TestNodeInfo) { ASSERT_TRUE(gcs_client_->Nodes().Get(node1_id)); EXPECT_EQ(gcs_client_->Nodes().GetAll().size(), 2); - // Cancel registration of local node to GCS. - ASSERT_TRUE(DrainSelf()); - // Cancel registration of a node to GCS. ASSERT_TRUE(DrainNode(node2_id)); WaitForExpectedCount(unregister_count, 2); @@ -567,10 +559,8 @@ TEST_P(GcsClientTest, TestNodeInfo) { // Get information of all nodes from GCS. node_list = GetNodeInfoList(); EXPECT_EQ(node_list.size(), 2); - EXPECT_EQ(node_list[0].state(), - rpc::GcsNodeInfo_GcsNodeState::GcsNodeInfo_GcsNodeState_DEAD); - EXPECT_EQ(node_list[1].state(), - rpc::GcsNodeInfo_GcsNodeState::GcsNodeInfo_GcsNodeState_DEAD); + EXPECT_EQ(node_list[0].state(), rpc::GcsNodeInfo::ALIVE); + EXPECT_EQ(node_list[1].state(), rpc::GcsNodeInfo::DEAD); ASSERT_TRUE(gcs_client_->Nodes().IsRemoved(node2_id)); } diff --git a/src/ray/gcs/gcs_server/test/gcs_server_test_util.h b/src/ray/gcs/gcs_server/test/gcs_server_test_util.h index f58a33517a439..432ed655bf7fd 100644 --- a/src/ray/gcs/gcs_server/test/gcs_server_test_util.h +++ b/src/ray/gcs/gcs_server/test/gcs_server_test_util.h @@ -397,8 +397,6 @@ struct GcsServerMocker { return Status::NotImplemented(""); } - Status DrainSelf() override { return Status::NotImplemented(""); } - const NodeID &GetSelfId() const override { static NodeID node_id; return node_id; diff --git a/src/ray/raylet/agent_manager.cc b/src/ray/raylet/agent_manager.cc index 8710003f1dad0..b7038d53d8bc2 100644 --- a/src/ray/raylet/agent_manager.cc +++ b/src/ray/raylet/agent_manager.cc @@ -98,8 +98,8 @@ void AgentManager::StartAgent() { "- The agent is killed by the OS (e.g., out of memory)."; rpc::NodeDeathInfo node_death_info; node_death_info.set_reason(rpc::NodeDeathInfo::UNEXPECTED_TERMINATION); - node_death_info.set_reason_message( - "Ray agent failed and raylet fate-shares with it."); + node_death_info.set_reason_message(options_.agent_name + + " failed and raylet fate-shares with it."); shutdown_raylet_gracefully_(node_death_info); // If the process is not terminated within 10 seconds, forcefully kill raylet // itself. diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 74ddafc454c3e..7604f16181665 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2022,7 +2022,7 @@ void NodeManager::HandleShutdownRaylet(rpc::ShutdownRayletRequest request, // itself. rpc::NodeDeathInfo node_death_info; node_death_info.set_reason(rpc::NodeDeathInfo::EXPECTED_TERMINATION); - node_death_info.set_reason_message("ShutdownRaylet RPC has been received."); + node_death_info.set_reason_message("Terminated by autoscaler."); shutdown_raylet_gracefully_(node_death_info); }; is_shutdown_request_received_ = true; From 8ccfa099191cd2a0a48da190ebafa16a4c9aa29d Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 14 May 2024 08:55:40 -0700 Subject: [PATCH 31/40] rm raylet_util Signed-off-by: Rui Qiao --- BUILD.bazel | 1 - src/ray/raylet/agent_manager.cc | 1 - src/ray/raylet/node_manager.cc | 1 - src/ray/raylet/raylet_util.h | 37 ------------------- src/ray/raylet/runtime_env_agent_client.cc | 1 - .../scheduling/local_resource_manager.cc | 14 +------ 6 files changed, 2 insertions(+), 53 deletions(-) delete mode 100644 src/ray/raylet/raylet_util.h diff --git a/BUILD.bazel b/BUILD.bazel index 8474ab0fcdb4d..83573d0d8742a 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -615,7 +615,6 @@ ray_cc_library( ), hdrs = glob( [ - "src/ray/raylet/raylet_util.h", "src/ray/raylet/scheduling/**/*.h", "src/ray/core_worker/common.h", ], diff --git a/src/ray/raylet/agent_manager.cc b/src/ray/raylet/agent_manager.cc index b7038d53d8bc2..49f40ddcead40 100644 --- a/src/ray/raylet/agent_manager.cc +++ b/src/ray/raylet/agent_manager.cc @@ -17,7 +17,6 @@ #include #include "ray/common/ray_config.h" -#include "ray/raylet/raylet_util.h" #include "ray/util/event.h" #include "ray/util/event_label.h" #include "ray/util/logging.h" diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 7604f16181665..0f01440b9d113 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -33,7 +33,6 @@ #include "ray/common/task/task_common.h" #include "ray/gcs/pb_util.h" #include "ray/raylet/format/node_manager_generated.h" -#include "ray/raylet/raylet_util.h" #include "ray/raylet/scheduling/cluster_task_manager.h" #include "ray/raylet/worker_killing_policy.h" #include "ray/rpc/node_manager/node_manager_client.h" diff --git a/src/ray/raylet/raylet_util.h b/src/ray/raylet/raylet_util.h deleted file mode 100644 index 154cf6952c983..0000000000000 --- a/src/ray/raylet/raylet_util.h +++ /dev/null @@ -1,37 +0,0 @@ -// Copyright 2023 The Ray Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include - -namespace ray { -namespace raylet { - -inline void ShutdownRayletGracefully() { - // Implementation note: When raylet is shutdown by ray stop, the CLI sends a - // sigterm. Raylet knows how to gracefully shutdown when it receives a sigterm. Here, - // we raise a sigterm to itself so that it can re-use the same graceful shutdown code - // path. The sigterm is handled in the entry point (raylet/main.cc)'s signal handler. - - RAY_LOG(INFO) << "Sending SIGTERM to gracefully shutdown raylet"; - // raise return 0 if succeeds. If it fails to gracefully shutdown, it kills itself - // forcefully. - RAY_CHECK_EQ(std::raise(SIGTERM), 0) - << "There was a failure while sending a sigterm to itself. Raylet will not " - "gracefully shutdown."; -} - -} // namespace raylet -} // namespace ray diff --git a/src/ray/raylet/runtime_env_agent_client.cc b/src/ray/raylet/runtime_env_agent_client.cc index 999b82c5c1a91..49bbe7e78f2f2 100644 --- a/src/ray/raylet/runtime_env_agent_client.cc +++ b/src/ray/raylet/runtime_env_agent_client.cc @@ -27,7 +27,6 @@ #include "absl/strings/str_format.h" #include "ray/common/asio/instrumented_io_context.h" #include "ray/common/status.h" -#include "ray/raylet/raylet_util.h" #include "ray/util/logging.h" #include "src/ray/protobuf/runtime_env_agent.pb.h" diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 14b77fb0c63ee..38484be8e55aa 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -19,7 +19,6 @@ #include "ray/common/grpc_util.h" #include "ray/common/ray_config.h" -#include "ray/raylet/raylet_util.h" namespace ray { @@ -383,17 +382,8 @@ std::optional LocalResourceManager::CreateSyncMessage( void LocalResourceManager::OnResourceOrStateChanged() { if (IsLocalNodeDraining() && IsLocalNodeIdle()) { - // The node is drained. - - if (shutdown_raylet_gracefully_ == nullptr) { - RAY_LOG(ERROR) - << "shutdown_raylet_gracefully_ callback is not set, " - << "please check the LocalResourceManager constructor." - << "Using deprecated ShutdownRayletGracefully (sending SIGTERM) instead."; - raylet::ShutdownRayletGracefully(); - } else { - shutdown_raylet_gracefully_(node_death_info_); - } + // The node is drained, continue with shutdown + shutdown_raylet_gracefully_(node_death_info_); } ++version_; From 20aeff8bc34493542a25071290a6d7f9b220f0e4 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 14 May 2024 09:30:16 -0700 Subject: [PATCH 32/40] update GcsNodeManager::DrainNode Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_server/gcs_node_manager.cc | 67 ++++++++-------------- src/ray/protobuf/gcs_service.proto | 2 +- 2 files changed, 25 insertions(+), 44 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.cc b/src/ray/gcs/gcs_server/gcs_node_manager.cc index b0fde870002a6..689ea716ebcb9 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_node_manager.cc @@ -155,57 +155,38 @@ void GcsNodeManager::HandleDrainNode(rpc::DrainNodeRequest request, } void GcsNodeManager::DrainNode(const NodeID &node_id) { - RAY_LOG(INFO) << "Shutting down raylet, node id = " << node_id; - auto node = RemoveNode(node_id, /* is_intended = */ true); - if (!node) { - RAY_LOG(INFO) << "Node " << node_id << " is already removed"; + RAY_LOG(INFO) << "DrainNode() for node id = " << node_id; + auto maybe_node = GetAliveNode(node_id); + if (!maybe_node.has_value()) { + RAY_LOG(WARNING) << "Draining node " << node_id << " which is already removed"; return; } - - // Do the procedure to drain a node. - node->set_state(rpc::GcsNodeInfo::DEAD); - node->set_end_time_ms(current_sys_time_ms()); - AddDeadNodeToCache(node); - auto node_info_delta = std::make_shared(); - node_info_delta->set_node_id(node->node_id()); - node_info_delta->set_state(node->state()); - node_info_delta->set_end_time_ms(node->end_time_ms()); + auto node = maybe_node.value(); // Set the address. rpc::Address remote_address; remote_address.set_raylet_id(node->node_id()); remote_address.set_ip_address(node->node_manager_address()); remote_address.set_port(node->node_manager_port()); - auto on_put_done = [this, - remote_address = remote_address, - node_id, - node_info_delta = node_info_delta](const Status &status) { - auto raylet_client = raylet_client_pool_->GetOrConnectByAddress(remote_address); - RAY_CHECK(raylet_client); - // NOTE(sang): Drain API is not supposed to kill the raylet, but we are doing - // this until the proper "drain" behavior is implemented. Currently, before - // raylet is killed, it sends a drain request to GCS. That said, this can - // happen; - // - GCS updates the drain state and kills a raylet gracefully. - // - Raylet kills itself and send a drain request of itself to GCS. - // - Drain request will become a no-op in GCS. - // This behavior is redundant, but harmless. We'll keep this behavior until we - // implement the right drain behavior for the simplicity. Check - // https://github.com/ray-project/ray/pull/19350 for more details. - raylet_client->ShutdownRaylet( - node_id, - /*graceful*/ true, - [this, node_id, node_info_delta = node_info_delta]( - const Status &status, const rpc::ShutdownRayletReply &reply) { - RAY_LOG(INFO) << "Raylet " << node_id << " is drained. Status " << status - << ". The information will be published to the cluster."; - /// Once the raylet is shutdown, inform all nodes that the raylet is dead. - RAY_CHECK_OK( - gcs_publisher_->PublishNodeInfo(node_id, *node_info_delta, nullptr)); - }); - }; - // Update node state to DEAD instead of deleting it. - RAY_CHECK_OK(gcs_table_storage_->NodeTable().Put(node_id, *node, on_put_done)); + + auto raylet_client = raylet_client_pool_->GetOrConnectByAddress(remote_address); + RAY_CHECK(raylet_client); + // NOTE(sang): Drain API is not supposed to kill the raylet, but we are doing + // this until the proper "drain" behavior is implemented. Currently, before + // raylet is killed, it sends a drain request to GCS. That said, this can + // happen; + // - GCS updates the drain state and kills a raylet gracefully. + // - Raylet kills itself and send a drain request of itself to GCS. + // - Drain request will become a no-op in GCS. + // This behavior is redundant, but harmless. We'll keep this behavior until we + // implement the right drain behavior for the simplicity. Check + // https://github.com/ray-project/ray/pull/19350 for more details. + raylet_client->ShutdownRaylet( + node_id, + /*graceful*/ true, + [node_id](const Status &status, const rpc::ShutdownRayletReply &reply) { + RAY_LOG(INFO) << "Raylet " << node_id << " is drained. Status " << status; + }); } void GcsNodeManager::HandleGetAllNodeInfo(rpc::GetAllNodeInfoRequest request, diff --git a/src/ray/protobuf/gcs_service.proto b/src/ray/protobuf/gcs_service.proto index fd184a5f93879..7887f17944e6c 100644 --- a/src/ray/protobuf/gcs_service.proto +++ b/src/ray/protobuf/gcs_service.proto @@ -446,7 +446,7 @@ message DrainNodeRequest { } message DrainNodeStatus { - // The node id that is successfully drained from GCS. + // The node id that is acknowledged for draining by GCS. bytes node_id = 1; } From abc82621fbef130187bfd1497b169ddcb568a55d Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 14 May 2024 10:23:38 -0700 Subject: [PATCH 33/40] fix Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_client/test/gcs_client_test.cc | 6 ++++-- src/ray/gcs/gcs_server/gcs_node_manager.cc | 12 ++---------- 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/src/ray/gcs/gcs_client/test/gcs_client_test.cc b/src/ray/gcs/gcs_client/test/gcs_client_test.cc index f4cbeac9c6aac..299191cf434f9 100644 --- a/src/ray/gcs/gcs_client/test/gcs_client_test.cc +++ b/src/ray/gcs/gcs_client/test/gcs_client_test.cc @@ -552,15 +552,17 @@ TEST_P(GcsClientTest, TestNodeInfo) { ASSERT_TRUE(gcs_client_->Nodes().Get(node1_id)); EXPECT_EQ(gcs_client_->Nodes().GetAll().size(), 2); - // Cancel registration of a node to GCS. + // Cancel registration of both nodes to GCS. + ASSERT_TRUE(DrainNode(node1_id)); ASSERT_TRUE(DrainNode(node2_id)); WaitForExpectedCount(unregister_count, 2); // Get information of all nodes from GCS. node_list = GetNodeInfoList(); EXPECT_EQ(node_list.size(), 2); - EXPECT_EQ(node_list[0].state(), rpc::GcsNodeInfo::ALIVE); + EXPECT_EQ(node_list[0].state(), rpc::GcsNodeInfo::DEAD); EXPECT_EQ(node_list[1].state(), rpc::GcsNodeInfo::DEAD); + ASSERT_TRUE(gcs_client_->Nodes().IsRemoved(node1_id)); ASSERT_TRUE(gcs_client_->Nodes().IsRemoved(node2_id)); } diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.cc b/src/ray/gcs/gcs_server/gcs_node_manager.cc index 689ea716ebcb9..62115efc2a994 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_node_manager.cc @@ -158,7 +158,7 @@ void GcsNodeManager::DrainNode(const NodeID &node_id) { RAY_LOG(INFO) << "DrainNode() for node id = " << node_id; auto maybe_node = GetAliveNode(node_id); if (!maybe_node.has_value()) { - RAY_LOG(WARNING) << "Draining node " << node_id << " which is already removed"; + RAY_LOG(WARNING) << "Skip draining node " << node_id << " which is already removed"; return; } auto node = maybe_node.value(); @@ -172,15 +172,7 @@ void GcsNodeManager::DrainNode(const NodeID &node_id) { auto raylet_client = raylet_client_pool_->GetOrConnectByAddress(remote_address); RAY_CHECK(raylet_client); // NOTE(sang): Drain API is not supposed to kill the raylet, but we are doing - // this until the proper "drain" behavior is implemented. Currently, before - // raylet is killed, it sends a drain request to GCS. That said, this can - // happen; - // - GCS updates the drain state and kills a raylet gracefully. - // - Raylet kills itself and send a drain request of itself to GCS. - // - Drain request will become a no-op in GCS. - // This behavior is redundant, but harmless. We'll keep this behavior until we - // implement the right drain behavior for the simplicity. Check - // https://github.com/ray-project/ray/pull/19350 for more details. + // this until the proper "drain" behavior is implemented. raylet_client->ShutdownRaylet( node_id, /*graceful*/ true, From bbbbace2c5eb8736897ba115f9fb3bdd9a930ab4 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 14 May 2024 13:43:41 -0700 Subject: [PATCH 34/40] unit tests Signed-off-by: Rui Qiao --- dashboard/tests/test_dashboard.py | 16 +++++++++++++++- python/ray/includes/common.pxd | 5 +++++ python/ray/includes/global_state_accessor.pxi | 3 +++ python/ray/tests/test_draining.py | 14 +++++++++++++- python/ray/tests/test_node_death.py | 19 +++++++++++++++++++ python/ray/tests/test_runtime_env_agent.py | 16 +++++++++++++++- .../scheduling/local_resource_manager.cc | 2 +- .../scheduling/local_resource_manager_test.cc | 2 +- 8 files changed, 72 insertions(+), 5 deletions(-) create mode 100644 python/ray/tests/test_node_death.py diff --git a/dashboard/tests/test_dashboard.py b/dashboard/tests/test_dashboard.py index 03149a30a1ca2..bb5d7cdd8092c 100644 --- a/dashboard/tests/test_dashboard.py +++ b/dashboard/tests/test_dashboard.py @@ -35,6 +35,7 @@ wait_until_server_available, wait_until_succeeded_without_exception, ) +from ray.core.generated import gcs_pb2 import ray.scripts.scripts as scripts from ray.dashboard import dashboard from ray.dashboard.head import DashboardHead @@ -197,7 +198,7 @@ def test_raylet_and_agent_share_fate(shutdown_only): ray.shutdown() - ray.init() + ray_context = ray.init() all_processes = ray._private.worker._global_node.all_processes raylet_proc_info = all_processes[ray_constants.PROCESS_TYPE_RAYLET][0] raylet_proc = psutil.Process(raylet_proc_info.process.pid) @@ -212,6 +213,19 @@ def test_raylet_and_agent_share_fate(shutdown_only): agent_proc.wait() raylet_proc.wait(15) + worker_node_id = ray_context.address_info["node_id"] + worker_node_info = [ + node for node in ray.nodes() if node["NodeID"] == worker_node_id + ][0] + assert not worker_node_info["Alive"] + assert worker_node_info["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + "UNEXPECTED_TERMINATION" + ) + assert ( + "failed and raylet fate-shares with it." + in worker_node_info["DeathReasonMessage"] + ) + @pytest.mark.parametrize("parent_health_check_by_pipe", [True, False]) def test_agent_report_unexpected_raylet_death( diff --git a/python/ray/includes/common.pxd b/python/ray/includes/common.pxd index c43f78f3bad1d..02c96dcac7025 100644 --- a/python/ray/includes/common.pxd +++ b/python/ray/includes/common.pxd @@ -508,6 +508,10 @@ cdef extern from "src/ray/protobuf/gcs.pb.h" nogil: c_string ray_namespace() const const c_string &SerializeAsString() + cdef cppclass CNodeDeathInfo "ray::rpc::NodeDeathInfo": + int reason() const + c_string reason_message() const + cdef cppclass CGcsNodeInfo "ray::rpc::GcsNodeInfo": c_string node_id() const c_string node_name() const @@ -520,6 +524,7 @@ cdef extern from "src/ray/protobuf/gcs.pb.h" nogil: c_string raylet_socket_name() const int metrics_export_port() const int runtime_env_agent_port() const + CNodeDeathInfo death_info() const void ParseFromString(const c_string &serialized) cdef enum CGcsNodeState "ray::rpc::GcsNodeInfo_GcsNodeState": diff --git a/python/ray/includes/global_state_accessor.pxi b/python/ray/includes/global_state_accessor.pxi index daa99eb0b9775..6f38a8d1b9bbd 100644 --- a/python/ray/includes/global_state_accessor.pxi +++ b/python/ray/includes/global_state_accessor.pxi @@ -84,6 +84,9 @@ cdef class GlobalStateAccessor: "MetricsExportPort": c_node_info.metrics_export_port(), "NodeName": c_node_info.node_name().decode(), "RuntimeEnvAgentPort": c_node_info.runtime_env_agent_port(), + "DeathReason": c_node_info.death_info().reason(), + "DeathReasonMessage": + c_node_info.death_info().reason_message().decode(), } node_info["alive"] = node_info["Alive"] c_resources = PythonGetResourcesTotal(c_node_info) diff --git a/python/ray/tests/test_draining.py b/python/ray/tests/test_draining.py index b9a18d7ea2d7f..10e11606d2228 100644 --- a/python/ray/tests/test_draining.py +++ b/python/ray/tests/test_draining.py @@ -4,7 +4,7 @@ import ray import time from ray._raylet import GcsClient -from ray.core.generated import autoscaler_pb2 +from ray.core.generated import autoscaler_pb2, gcs_pb2 from ray._private.test_utils import wait_for_condition from ray.util.scheduling_strategies import ( NodeAffinitySchedulingStrategy, @@ -72,6 +72,12 @@ def drain_until_accept(): == {head_node_id} ) + worker_node = [node for node in ray.nodes() if node["NodeID"] == worker_node_id][0] + assert worker_node["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + "AUTOSCALER_DRAIN_IDLE" + ) + assert worker_node["DeathReasonMessage"] == "idle for long enough" + # Draining a dead node is always accepted. is_accepted, _ = gcs_client.drain_node( worker_node_id, @@ -138,6 +144,12 @@ def ping(self): == {head_node_id} ) + worker_node = [node for node in ray.nodes() if node["NodeID"] == worker_node_id][0] + assert worker_node["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + "AUTOSCALER_DRAIN_PREEMPTED" + ) + assert worker_node["DeathReasonMessage"] == "preemption" + def test_scheduling_placement_groups_during_draining(ray_start_cluster): """Test that the draining node is unschedulable for new pgs.""" diff --git a/python/ray/tests/test_node_death.py b/python/ray/tests/test_node_death.py new file mode 100644 index 0000000000000..153fd207d9760 --- /dev/null +++ b/python/ray/tests/test_node_death.py @@ -0,0 +1,19 @@ +import ray + +from ray.core.generated import gcs_pb2 + + +def test_node_death(ray_start_cluster): + cluster = ray_start_cluster + cluster.add_node(resources={"head": 1}) + ray.init(address=cluster.address) + worker_node = cluster.add_node(resources={"worker": 1}) + cluster.wait_for_nodes() + worker_node_id = worker_node.node_id + cluster.remove_node(worker_node) + + worker_node_info = [node for node in ray.nodes() if node["NodeID"] == worker_node_id][0] + assert worker_node_info["Alive"] == False + assert worker_node_info["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value("EXPECTED_TERMINATION") + assert worker_node_info["DeathReasonMessage"] == "Received SIGTERM" + diff --git a/python/ray/tests/test_runtime_env_agent.py b/python/ray/tests/test_runtime_env_agent.py index 2ba2daecfdf9c..4af9165754a33 100644 --- a/python/ray/tests/test_runtime_env_agent.py +++ b/python/ray/tests/test_runtime_env_agent.py @@ -13,6 +13,7 @@ init_error_pubsub, wait_for_condition, ) +from ray.core.generated import gcs_pb2 from ray.runtime_env import RuntimeEnv import psutil @@ -139,7 +140,7 @@ def test_raylet_and_agent_share_fate(shutdown_only): ray.shutdown() - ray.init() + ray_context = ray.init() all_processes = ray._private.worker._global_node.all_processes raylet_proc_info = all_processes[ray_constants.PROCESS_TYPE_RAYLET][0] raylet_proc = psutil.Process(raylet_proc_info.process.pid) @@ -154,6 +155,19 @@ def test_raylet_and_agent_share_fate(shutdown_only): agent_proc.wait() raylet_proc.wait(15) + worker_node_id = ray_context.address_info["node_id"] + worker_node_info = [ + node for node in ray.nodes() if node["NodeID"] == worker_node_id + ][0] + assert not worker_node_info["Alive"] + assert worker_node_info["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + "UNEXPECTED_TERMINATION" + ) + assert ( + "failed and raylet fate-shares with it." + in worker_node_info["DeathReasonMessage"] + ) + @pytest.mark.skipif(sys.platform == "win32", reason="no fate sharing for windows") def test_agent_report_unexpected_raylet_death(shutdown_only): diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 38484be8e55aa..dfb7bfb4918ba 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -382,7 +382,7 @@ std::optional LocalResourceManager::CreateSyncMessage( void LocalResourceManager::OnResourceOrStateChanged() { if (IsLocalNodeDraining() && IsLocalNodeIdle()) { - // The node is drained, continue with shutdown + RAY_LOG(INFO) << "The node is drained, exiting..."; shutdown_raylet_gracefully_(node_death_info_); } diff --git a/src/ray/raylet/scheduling/local_resource_manager_test.cc b/src/ray/raylet/scheduling/local_resource_manager_test.cc index 5f453e4341457..44bc0b2affcf0 100644 --- a/src/ray/raylet/scheduling/local_resource_manager_test.cc +++ b/src/ray/raylet/scheduling/local_resource_manager_test.cc @@ -155,7 +155,7 @@ TEST_F(LocalResourceManagerTest, NodeDrainingTest) { manager->AllocateLocalTaskResources(resource_request, task_allocation); } - rpc::NodeDeathInfo node_death_info; // TODO: fill in values + rpc::NodeDeathInfo node_death_info; manager->SetLocalNodeDraining(std::numeric_limits::max(), node_death_info); ASSERT_TRUE(manager->IsLocalNodeDraining()); From 0e8d2ec10c683fff6fb22ccb46cf4bb82383703b Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 14 May 2024 13:49:02 -0700 Subject: [PATCH 35/40] format Signed-off-by: Rui Qiao --- python/ray/tests/test_node_death.py | 11 +++++++---- src/ray/raylet/scheduling/local_resource_manager.cc | 2 +- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/python/ray/tests/test_node_death.py b/python/ray/tests/test_node_death.py index 153fd207d9760..230a919d0df95 100644 --- a/python/ray/tests/test_node_death.py +++ b/python/ray/tests/test_node_death.py @@ -12,8 +12,11 @@ def test_node_death(ray_start_cluster): worker_node_id = worker_node.node_id cluster.remove_node(worker_node) - worker_node_info = [node for node in ray.nodes() if node["NodeID"] == worker_node_id][0] - assert worker_node_info["Alive"] == False - assert worker_node_info["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value("EXPECTED_TERMINATION") + worker_node_info = [ + node for node in ray.nodes() if node["NodeID"] == worker_node_id + ][0] + assert not worker_node_info["Alive"] + assert worker_node_info["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + "EXPECTED_TERMINATION" + ) assert worker_node_info["DeathReasonMessage"] == "Received SIGTERM" - diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index dfb7bfb4918ba..e056f8c42d179 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -382,7 +382,7 @@ std::optional LocalResourceManager::CreateSyncMessage( void LocalResourceManager::OnResourceOrStateChanged() { if (IsLocalNodeDraining() && IsLocalNodeIdle()) { - RAY_LOG(INFO) << "The node is drained, exiting..."; + RAY_LOG(INFO) << "The node is drained, continue to shut down raylet..."; shutdown_raylet_gracefully_(node_death_info_); } From bd4746d67e327b37691a4ac17862bf6be9b51c10 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 14 May 2024 14:01:07 -0700 Subject: [PATCH 36/40] update Signed-off-by: Rui Qiao --- python/ray/tests/test_node_death.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tests/test_node_death.py b/python/ray/tests/test_node_death.py index 230a919d0df95..8b734f34b86c0 100644 --- a/python/ray/tests/test_node_death.py +++ b/python/ray/tests/test_node_death.py @@ -3,7 +3,7 @@ from ray.core.generated import gcs_pb2 -def test_node_death(ray_start_cluster): +def test_normal_termination(ray_start_cluster): cluster = ray_start_cluster cluster.add_node(resources={"head": 1}) ray.init(address=cluster.address) From f0858517202a2b94976fe73a45f39f801bc7af01 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 14 May 2024 15:28:31 -0700 Subject: [PATCH 37/40] lint Signed-off-by: Rui Qiao --- python/ray/tests/BUILD | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 82cdc9f0bcad4..9a560e06a3fce 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -299,6 +299,7 @@ py_test_module_list( "test_metrics_agent_2.py", "test_microbenchmarks.py", "test_mini.py", + "test_node_death.py", "test_numba.py", "test_raylet_output.py", "test_top_level_api.py", From ffbe09ee6698619a3770eaf0cce28b72483436b1 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 14 May 2024 16:50:57 -0700 Subject: [PATCH 38/40] fix Signed-off-by: Rui Qiao --- python/ray/tests/test_node_death.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/python/ray/tests/test_node_death.py b/python/ray/tests/test_node_death.py index 8b734f34b86c0..9a4faa80f25b3 100644 --- a/python/ray/tests/test_node_death.py +++ b/python/ray/tests/test_node_death.py @@ -1,3 +1,7 @@ +import os +import sys +import pytest + import ray from ray.core.generated import gcs_pb2 @@ -20,3 +24,10 @@ def test_normal_termination(ray_start_cluster): "EXPECTED_TERMINATION" ) assert worker_node_info["DeathReasonMessage"] == "Received SIGTERM" + + +if __name__ == "__main__": + if os.environ.get("PARALLEL_CI"): + sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) + else: + sys.exit(pytest.main(["-sv", __file__])) From 290a26f187ab012c827758940ed6af6c47c9041d Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 14 May 2024 16:55:38 -0700 Subject: [PATCH 39/40] cleanup Signed-off-by: Rui Qiao --- python/ray/tests/test_node_death.py | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/python/ray/tests/test_node_death.py b/python/ray/tests/test_node_death.py index 9a4faa80f25b3..20c61beedf973 100644 --- a/python/ray/tests/test_node_death.py +++ b/python/ray/tests/test_node_death.py @@ -1,4 +1,3 @@ -import os import sys import pytest @@ -27,7 +26,4 @@ def test_normal_termination(ray_start_cluster): if __name__ == "__main__": - if os.environ.get("PARALLEL_CI"): - sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) - else: - sys.exit(pytest.main(["-sv", __file__])) + sys.exit(pytest.main(["-sv", __file__])) From 42aa253765e2e572db3882a8d01d8431864a8f50 Mon Sep 17 00:00:00 2001 From: Rui Qiao Date: Tue, 14 May 2024 21:37:08 -0700 Subject: [PATCH 40/40] fix tests Signed-off-by: Rui Qiao --- .../gcs_server/test/gcs_server_rpc_test.cc | 28 +++++++++++-------- .../scheduling/local_resource_manager_test.cc | 4 +-- 2 files changed, 19 insertions(+), 13 deletions(-) diff --git a/src/ray/gcs/gcs_server/test/gcs_server_rpc_test.cc b/src/ray/gcs/gcs_server/test/gcs_server_rpc_test.cc index cf184422dcde4..136ef2a1017fc 100644 --- a/src/ray/gcs/gcs_server/test/gcs_server_rpc_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_server_rpc_test.cc @@ -118,13 +118,14 @@ class GcsServerTest : public ::testing::Test { return WaitReady(promise.get_future(), timeout_ms_); } - bool DrainNode(const rpc::DrainNodeRequest &request) { + bool UnregisterNode(const rpc::UnregisterNodeRequest &request) { std::promise promise; - client_->DrainNode( - request, [&promise](const Status &status, const rpc::DrainNodeReply &reply) { + client_->UnregisterNode( + request, [&promise](const Status &status, const rpc::UnregisterNodeReply &reply) { RAY_CHECK_OK(status); promise.set_value(true); }); + return WaitReady(promise.get_future(), timeout_ms_); } @@ -280,18 +281,23 @@ TEST_F(GcsServerTest, TestNodeInfo) { ASSERT_TRUE(RegisterNode(register_node_info_request)); std::vector node_info_list = GetAllNodeInfo(); ASSERT_TRUE(node_info_list.size() == 1); - ASSERT_TRUE(node_info_list[0].state() == - rpc::GcsNodeInfo_GcsNodeState::GcsNodeInfo_GcsNodeState_ALIVE); + ASSERT_TRUE(node_info_list[0].state() == rpc::GcsNodeInfo::ALIVE); // Unregister node info - rpc::DrainNodeRequest unregister_node_info_request; - auto draining_request = unregister_node_info_request.add_drain_node_data(); - draining_request->set_node_id(gcs_node_info->node_id()); - ASSERT_TRUE(DrainNode(unregister_node_info_request)); + rpc::UnregisterNodeRequest unregister_node_request; + unregister_node_request.set_node_id(gcs_node_info->node_id()); + rpc::NodeDeathInfo node_death_info; + node_death_info.set_reason(rpc::NodeDeathInfo::EXPECTED_TERMINATION); + std::string reason_message = "Terminate node for testing."; + node_death_info.set_reason_message(reason_message); + unregister_node_request.mutable_node_death_info()->CopyFrom(node_death_info); + ASSERT_TRUE(UnregisterNode(unregister_node_request)); node_info_list = GetAllNodeInfo(); ASSERT_TRUE(node_info_list.size() == 1); - ASSERT_TRUE(node_info_list[0].state() == - rpc::GcsNodeInfo_GcsNodeState::GcsNodeInfo_GcsNodeState_DEAD); + ASSERT_TRUE(node_info_list[0].state() == rpc::GcsNodeInfo::DEAD); + ASSERT_TRUE(node_info_list[0].death_info().reason() == + rpc::NodeDeathInfo::EXPECTED_TERMINATION); + ASSERT_TRUE(node_info_list[0].death_info().reason_message() == reason_message); } TEST_F(GcsServerTest, TestWorkerInfo) { diff --git a/src/ray/raylet/scheduling/local_resource_manager_test.cc b/src/ray/raylet/scheduling/local_resource_manager_test.cc index 44bc0b2affcf0..07d64a5331a63 100644 --- a/src/ray/raylet/scheduling/local_resource_manager_test.cc +++ b/src/ray/raylet/scheduling/local_resource_manager_test.cc @@ -143,7 +143,7 @@ TEST_F(LocalResourceManagerTest, NodeDrainingTest) { CreateNodeResources({{ResourceID::CPU(), 8.0}}), nullptr, nullptr, - nullptr, + [](const rpc::NodeDeathInfo &node_death_info) { _Exit(1); }, nullptr); // Make the node non-idle. @@ -175,7 +175,7 @@ TEST_F(LocalResourceManagerTest, ObjectStoreMemoryDrainingTest) { /* get_used_object_store_memory */ [&used_object_store]() { return *used_object_store; }, nullptr, - nullptr, + [](const rpc::NodeDeathInfo &node_death_info) { _Exit(1); }, nullptr); // Make the node non-idle.