From 4f598dae130df977ba0fa6e07507f4080f1c3d19 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 1 Apr 2022 20:12:22 +0000 Subject: [PATCH 01/77] up --- src/ray/common/ray_config_def.h | 3 ++ src/ray/protobuf/gcs.proto | 2 +- src/ray/raylet/node_manager.cc | 7 ++- src/ray/raylet/node_manager.h | 7 +++ .../scheduling/local_resource_manager.cc | 45 +++++++++++++++++++ .../scheduling/local_resource_manager.h | 9 +++- src/ray/rpc/grpc_server.cc | 4 ++ src/ray/rpc/grpc_server.h | 1 + 8 files changed, 73 insertions(+), 5 deletions(-) diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 6daaa70fe757..74db243e9e6f 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -330,6 +330,9 @@ RAY_CONFIG(int32_t, gcs_rpc_server_reconnect_timeout_s, 60) /// Minimum interval between reconnecting gcs rpc server when gcs server restarts. RAY_CONFIG(int32_t, minimum_gcs_reconnect_interval_milliseconds, 5000) +/// Feature flag to use the ray syncer for resource synchronization +RAY_CONFIG(bool, use_ray_syncer, false) + /// The interval at which the gcs client will check if the address of gcs service has /// changed. When the address changed, we will resubscribe again. RAY_CONFIG(uint64_t, gcs_service_address_check_interval_milliseconds, 1000) diff --git a/src/ray/protobuf/gcs.proto b/src/ray/protobuf/gcs.proto index 4f100465bafb..3e6859e7fb69 100644 --- a/src/ray/protobuf/gcs.proto +++ b/src/ray/protobuf/gcs.proto @@ -409,7 +409,7 @@ message ResourcesData { // Whether this node manager is requesting global GC. bool should_global_gc = 8; // IP address of the node. - string node_manager_address = 9; + // string node_manager_address = 9; // Whether this node has object pulls queued. This can happen if // the node has more pull requests than available object store // memory. This is a proxy for available object store memory. diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 9f17c07db482..29a7851c33e3 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -325,7 +325,9 @@ NodeManager::NodeManager(instrumented_io_context &io_service, global_gc_throttler_(RayConfig::instance().global_gc_min_interval_s() * 1e9), local_gc_interval_ns_(RayConfig::instance().local_gc_interval_s() * 1e9), record_metrics_period_ms_(config.record_metrics_period_ms), - next_resource_seq_no_(0) { + next_resource_seq_no_(0), + ray_syncer_(io_service_, self_node_id_.Binary()), + ray_syncer_service_(ray_syncer_) { RAY_LOG(INFO) << "Initializing NodeManager with ID " << self_node_id_; RAY_CHECK(RayConfig::instance().raylet_heartbeat_period_milliseconds() > 0); SchedulingResources local_resources(config.resource_config); @@ -401,6 +403,7 @@ NodeManager::NodeManager(instrumented_io_context &io_service, // Run the node manger rpc server. node_manager_server_.RegisterService(node_manager_service_); node_manager_server_.RegisterService(agent_manager_service_); + node_manager_server_.RegisterService(ray_syncer_service_); node_manager_server_.Run(); worker_pool_.SetNodeManagerPort(GetServerPort()); @@ -606,7 +609,7 @@ void NodeManager::FillNormalTaskResourceUsage(rpc::ResourcesData &resources_data void NodeManager::FillResourceReport(rpc::ResourcesData &resources_data) { resources_data.set_node_id(self_node_id_.Binary()); - resources_data.set_node_manager_address(initial_config_.node_manager_address); + // resources_data.set_node_manager_address(initial_config_.node_manager_address); // Update local cache from gcs remote cache, this is needed when gcs restart. // We should always keep the cache view consistent. cluster_resource_scheduler_->GetLocalResourceManager().ResetLastReportResourceUsage( diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index 54b3dcaa5dc8..7cf5de29f467 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -21,6 +21,7 @@ #include "ray/common/id.h" #include "ray/common/task/task.h" #include "ray/common/ray_object.h" +#include "ray/common/ray_syncer/ray_syncer.h" #include "ray/common/client_connection.h" #include "ray/common/task/task_common.h" #include "ray/common/task/scheduling_resources.h" @@ -763,6 +764,12 @@ class NodeManager : public rpc::NodeManagerServiceHandler { /// Whether or not if the node draining process has already received. bool is_node_drained_ = false; + + /// Ray syncer for synchronization + syncer::RaySyncer ray_syncer_; + + /// RaySyncerService for gRPC + syncer::RaySyncerService ray_syncer_service_; }; } // namespace raylet diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 4cba96c988d0..b42f3645e97b 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -397,4 +397,49 @@ bool LocalResourceManager::ResourcesExist(scheduling::ResourceID resource_id) co return local_resources_.total.Has(resource_id); } +std::optional LocalResourceManager::Snapshot(int64_t version_after, syncer::RayComponentId component_id) const { + if (version_ <= version_after) { + return std::nullopt; + } + + const_cast(this)->UpdateAvailableObjectStoreMemResource(); + + rpc::ResourcesData resources_data; + NodeResources resources = ToNodeResources(local_resources_); + + for (auto entry : resources.total.ToMap()) { + auto resource_id = entry.first; + auto label = ResourceID(resource_id).Binary(); + auto total = entry.second; + auto available = resources.available.Get(resource_id); + + // Note: available may be negative, but only report positive to GCS. + if (available > 0) { + resources_data.set_resources_available_changed(true); + (*resources_data.mutable_resources_available())[label] = available.Double(); + } + (*resources_data.mutable_resources_total())[label] = total.Double(); + } + + if (get_pull_manager_at_capacity_ != nullptr) { + resources.object_pulls_queued = get_pull_manager_at_capacity_(); + resources_data.set_object_pulls_queued(resources.object_pulls_queued); + resources_data.set_resources_available_changed(true); + } + + resources_data.set_resources_available_changed(true); + resources_data.set_node_id(local_node_id_.Binary()); + + // Generate RaySyncMessage + ray::rpc::syncer::RaySyncMessage msg; + msg.set_component_id(ray::rpc::syncer::RayComponentId::RESOURCE_MANAGER); + msg.set_version(version_); + std::string serialized_msg; + RAY_CHECK(resources_data.SerializeToString(&serialized_msg)); + msg.set_sync_message(std::move(serialized_msg)); + msg.set_node_id(local_node_id_.Binary()); + msg.set_version(version_); + return std::make_optional(std::move(msg)); +} + } // namespace ray diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index ae6dc71b8cc5..bf4fc4876456 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -23,6 +23,7 @@ #include "absl/container/flat_hash_map.h" #include "absl/container/flat_hash_set.h" #include "ray/common/task/scheduling_resources.h" +#include "ray/common/ray_syncer/ray_syncer.h" #include "ray/gcs/gcs_client/accessor.h" #include "ray/gcs/gcs_client/gcs_client.h" #include "ray/raylet/scheduling/cluster_resource_data.h" @@ -37,7 +38,7 @@ namespace ray { /// it also supports creating a new resource or delete an existing resource. /// Whenever the resouce changes, it notifies the subscriber of the change. /// This class is not thread safe. -class LocalResourceManager { +class LocalResourceManager : public syncer::ReporterInterface { public: LocalResourceManager( scheduling::NodeID local_node_id, @@ -145,6 +146,7 @@ class LocalResourceManager { /// \return true, if exist. otherwise, false. bool ResourcesExist(scheduling::ResourceID resource_id) const; + std::optional Snapshot(int64_t version_after, syncer::RayComponentId component_id) const override; private: /// Notify the subscriber that the local resouces has changed. void OnResourceChanged(); @@ -237,7 +239,7 @@ class LocalResourceManager { /// Resources of local node. NodeResourceInstances local_resources_; /// Cached resources, used to compare with newest one in light heartbeat mode. - std::unique_ptr last_report_resources_; + mutable std::unique_ptr last_report_resources_; /// Function to get used object store memory. std::function get_used_object_store_memory_; /// Function to get whether the pull manager is at capacity. @@ -251,6 +253,9 @@ class LocalResourceManager { // Specify custom resources that consists of unit-size instances. std::unordered_set custom_unit_instance_resources_{}; + // Version of this resource. It will incr by one whenever when the state changed. + int64_t version_ = 0; + FRIEND_TEST(ClusterResourceSchedulerTest, SchedulingUpdateTotalResourcesTest); FRIEND_TEST(ClusterResourceSchedulerTest, AvailableResourceInstancesOpsTest); FRIEND_TEST(ClusterResourceSchedulerTest, TaskResourceInstancesTest); diff --git a/src/ray/rpc/grpc_server.cc b/src/ray/rpc/grpc_server.cc index 820612f98ddb..f3839f5052c2 100644 --- a/src/ray/rpc/grpc_server.cc +++ b/src/ray/rpc/grpc_server.cc @@ -127,6 +127,10 @@ void GrpcServer::Run() { is_closed_ = false; } +void GrpcServer::RegisterService(grpc::Service &service) { + services_.emplace_back(service); +} + void GrpcServer::RegisterService(GrpcService &service) { services_.emplace_back(service.GetGrpcService()); diff --git a/src/ray/rpc/grpc_server.h b/src/ray/rpc/grpc_server.h index c03d87cdae31..b8dfdb064d1a 100644 --- a/src/ray/rpc/grpc_server.h +++ b/src/ray/rpc/grpc_server.h @@ -107,6 +107,7 @@ class GrpcServer { /// /// \param[in] service A `GrpcService` to register to this server. void RegisterService(GrpcService &service); + void RegisterService(grpc::Service &service); protected: /// This function runs in a background thread. It keeps polling events from the From e908415270ad4c934c682c2c3b867f39fd76f1ec Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 1 Apr 2022 20:47:47 +0000 Subject: [PATCH 02/77] up --- src/ray/raylet/node_manager.cc | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 29a7851c33e3..c74548d4ae9d 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -403,7 +403,9 @@ NodeManager::NodeManager(instrumented_io_context &io_service, // Run the node manger rpc server. node_manager_server_.RegisterService(node_manager_service_); node_manager_server_.RegisterService(agent_manager_service_); - node_manager_server_.RegisterService(ray_syncer_service_); + if(RayConfig::instance().use_ray_syncer()) { + node_manager_server_.RegisterService(ray_syncer_service_); + } node_manager_server_.Run(); worker_pool_.SetNodeManagerPort(GetServerPort()); From a0388c2e43073dc418675051bc768e20f89e14b1 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 1 Apr 2022 20:47:58 +0000 Subject: [PATCH 03/77] format --- src/ray/raylet/node_manager.cc | 2 +- src/ray/raylet/scheduling/local_resource_manager.cc | 5 +++-- src/ray/raylet/scheduling/local_resource_manager.h | 6 ++++-- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index c74548d4ae9d..6e098de69cea 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -403,7 +403,7 @@ NodeManager::NodeManager(instrumented_io_context &io_service, // Run the node manger rpc server. node_manager_server_.RegisterService(node_manager_service_); node_manager_server_.RegisterService(agent_manager_service_); - if(RayConfig::instance().use_ray_syncer()) { + if (RayConfig::instance().use_ray_syncer()) { node_manager_server_.RegisterService(ray_syncer_service_); } node_manager_server_.Run(); diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index b42f3645e97b..e8622a264b26 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -397,12 +397,13 @@ bool LocalResourceManager::ResourcesExist(scheduling::ResourceID resource_id) co return local_resources_.total.Has(resource_id); } -std::optional LocalResourceManager::Snapshot(int64_t version_after, syncer::RayComponentId component_id) const { +std::optional LocalResourceManager::Snapshot( + int64_t version_after, syncer::RayComponentId component_id) const { if (version_ <= version_after) { return std::nullopt; } - const_cast(this)->UpdateAvailableObjectStoreMemResource(); + const_cast(this)->UpdateAvailableObjectStoreMemResource(); rpc::ResourcesData resources_data; NodeResources resources = ToNodeResources(local_resources_); diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index bf4fc4876456..32c52fc7ceee 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -22,8 +22,8 @@ #include "absl/container/flat_hash_map.h" #include "absl/container/flat_hash_set.h" -#include "ray/common/task/scheduling_resources.h" #include "ray/common/ray_syncer/ray_syncer.h" +#include "ray/common/task/scheduling_resources.h" #include "ray/gcs/gcs_client/accessor.h" #include "ray/gcs/gcs_client/gcs_client.h" #include "ray/raylet/scheduling/cluster_resource_data.h" @@ -146,7 +146,9 @@ class LocalResourceManager : public syncer::ReporterInterface { /// \return true, if exist. otherwise, false. bool ResourcesExist(scheduling::ResourceID resource_id) const; - std::optional Snapshot(int64_t version_after, syncer::RayComponentId component_id) const override; + std::optional Snapshot( + int64_t version_after, syncer::RayComponentId component_id) const override; + private: /// Notify the subscriber that the local resouces has changed. void OnResourceChanged(); From d7de78a3b73c9d2953bfc78185049142e199da06 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 2 Apr 2022 02:02:23 +0000 Subject: [PATCH 04/77] integrate with raylet --- src/ray/raylet/node_manager.cc | 50 +++++++++++++++++++ src/ray/raylet/node_manager.h | 9 +++- .../scheduling/local_resource_manager.cc | 47 +---------------- .../scheduling/local_resource_manager.h | 8 ++- 4 files changed, 62 insertions(+), 52 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 6e098de69cea..6f2a0ce17109 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2564,6 +2564,56 @@ void NodeManager::RecordMetrics() { object_directory_->RecordMetrics(duration_ms); } +void NodeManager::Update(std::shared_ptr message) { + rpc::ResourcesData data; + data.ParseFromString(message->sync_message()); + NodeID node_id = NodeID::FromBinary(data.node_id()); + + UpdateResourceUsage(node_id, data); +} + +std::optional NodeManager::Snapshot( + int64_t after_version, syncer::RayComponentId component_id) const { + if (component_id == syncer::RayComponentId::SCHEDULER) { + static uint64_t version = 0; + syncer::RaySyncMessage msg; + rpc::ResourcesData resource_data; + cluster_task_manager_->FillResourceUsage(resource_data); + resource_data.set_node_id(self_node_id_.Binary()); + // resource_data.set_node_manager_address(initial_config_.node_manager_address); + + msg.set_version(++version); + msg.set_node_id(self_node_id_.Binary()); + msg.set_component_id(syncer::RayComponentId::SCHEDULER); + std::string serialized_msg; + RAY_CHECK(resource_data.SerializeToString(&serialized_msg)); + msg.set_sync_message(std::move(serialized_msg)); + return std::make_optional(std::move(msg)); + } else if (component_id == syncer::RayComponentId::RESOURCE_MANAGER) { + auto &local = cluster_resource_scheduler_->GetLocalResourceManager(); + + if (local.Version() <= after_version) { + return std::nullopt; + } + + syncer::RaySyncMessage msg; + rpc::ResourcesData resource_data; + local.FillResourceUsage(resource_data); + resource_data.set_node_id(self_node_id_.Binary()); + // resource_data.set_node_manager_address(initial_config_.node_manager_address); + + msg.set_node_id(self_node_id_.Binary()); + msg.set_version(local.Version()); + msg.set_component_id(syncer::RayComponentId::RESOURCE_MANAGER); + std::string serialized_msg; + RAY_CHECK(resource_data.SerializeToString(&serialized_msg)); + msg.set_sync_message(std::move(serialized_msg)); + return std::make_optional(std::move(msg)); + } else { + return std::nullopt; + } +} + void NodeManager::PublishInfeasibleTaskError(const RayTask &task) const { bool suppress_warning = false; diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index 7cf5de29f467..2a650919a9f1 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -139,7 +139,9 @@ class HeartbeatSender { uint64_t last_heartbeat_at_ms_; }; -class NodeManager : public rpc::NodeManagerServiceHandler { +class NodeManager : public rpc::NodeManagerServiceHandler, + public syncer::ReporterInterface, + public syncer::ReceiverInterface { public: /// Create a node manager. /// @@ -188,6 +190,11 @@ class NodeManager : public rpc::NodeManagerServiceHandler { /// Get the port of the node manager rpc server. int GetServerPort() const { return node_manager_server_.GetPort(); } + void Update(std::shared_ptr message) override; + + std::optional Snapshot( + int64_t after_version, syncer::RayComponentId component_id) const override; + int GetObjectManagerPort() const { return object_manager_.GetServerPort(); } LocalObjectManager &GetLocalObjectManager() { return local_object_manager_; } diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index e8622a264b26..85e6fa95f9aa 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -380,6 +380,7 @@ ray::gcs::NodeResourceInfoAccessor::ResourceMap LocalResourceManager::GetResourc } void LocalResourceManager::OnResourceChanged() { + ++version_; if (resource_change_subscriber_ == nullptr) { return; } @@ -397,50 +398,4 @@ bool LocalResourceManager::ResourcesExist(scheduling::ResourceID resource_id) co return local_resources_.total.Has(resource_id); } -std::optional LocalResourceManager::Snapshot( - int64_t version_after, syncer::RayComponentId component_id) const { - if (version_ <= version_after) { - return std::nullopt; - } - - const_cast(this)->UpdateAvailableObjectStoreMemResource(); - - rpc::ResourcesData resources_data; - NodeResources resources = ToNodeResources(local_resources_); - - for (auto entry : resources.total.ToMap()) { - auto resource_id = entry.first; - auto label = ResourceID(resource_id).Binary(); - auto total = entry.second; - auto available = resources.available.Get(resource_id); - - // Note: available may be negative, but only report positive to GCS. - if (available > 0) { - resources_data.set_resources_available_changed(true); - (*resources_data.mutable_resources_available())[label] = available.Double(); - } - (*resources_data.mutable_resources_total())[label] = total.Double(); - } - - if (get_pull_manager_at_capacity_ != nullptr) { - resources.object_pulls_queued = get_pull_manager_at_capacity_(); - resources_data.set_object_pulls_queued(resources.object_pulls_queued); - resources_data.set_resources_available_changed(true); - } - - resources_data.set_resources_available_changed(true); - resources_data.set_node_id(local_node_id_.Binary()); - - // Generate RaySyncMessage - ray::rpc::syncer::RaySyncMessage msg; - msg.set_component_id(ray::rpc::syncer::RayComponentId::RESOURCE_MANAGER); - msg.set_version(version_); - std::string serialized_msg; - RAY_CHECK(resources_data.SerializeToString(&serialized_msg)); - msg.set_sync_message(std::move(serialized_msg)); - msg.set_node_id(local_node_id_.Binary()); - msg.set_version(version_); - return std::make_optional(std::move(msg)); -} - } // namespace ray diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index 32c52fc7ceee..50daefc3ffcd 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -22,7 +22,6 @@ #include "absl/container/flat_hash_map.h" #include "absl/container/flat_hash_set.h" -#include "ray/common/ray_syncer/ray_syncer.h" #include "ray/common/task/scheduling_resources.h" #include "ray/gcs/gcs_client/accessor.h" #include "ray/gcs/gcs_client/gcs_client.h" @@ -38,7 +37,7 @@ namespace ray { /// it also supports creating a new resource or delete an existing resource. /// Whenever the resouce changes, it notifies the subscriber of the change. /// This class is not thread safe. -class LocalResourceManager : public syncer::ReporterInterface { +class LocalResourceManager { public: LocalResourceManager( scheduling::NodeID local_node_id, @@ -108,6 +107,8 @@ class LocalResourceManager : public syncer::ReporterInterface { void ReleaseWorkerResources(std::shared_ptr task_allocation); + int64_t Version() const { return version_; } + /// Populate the relevant parts of the heartbeat table. This is intended for /// sending resource usage of raylet to gcs. In particular, this should fill in /// resources_available and resources_total. @@ -146,9 +147,6 @@ class LocalResourceManager : public syncer::ReporterInterface { /// \return true, if exist. otherwise, false. bool ResourcesExist(scheduling::ResourceID resource_id) const; - std::optional Snapshot( - int64_t version_after, syncer::RayComponentId component_id) const override; - private: /// Notify the subscriber that the local resouces has changed. void OnResourceChanged(); From f9187ce942089f23ee48cd094742e6a1099943d1 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 4 Apr 2022 03:21:19 +0000 Subject: [PATCH 05/77] format --- src/ray/gcs/gcs_server/gcs_server.cc | 50 +++++++++++++------ src/ray/gcs/gcs_server/gcs_server.h | 12 ++++- src/ray/raylet_client/raylet_client.cc | 4 ++ src/ray/raylet_client/raylet_client.h | 4 ++ src/ray/rpc/grpc_client.h | 12 +++-- .../rpc/node_manager/node_manager_client.h | 4 ++ 6 files changed, 65 insertions(+), 21 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 20acafd4eceb..69d29f8f023a 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -200,8 +200,12 @@ void GcsServer::Stop() { // won't handle heartbeat calls anymore, some nodes will be marked as dead during this // time, causing many nodes die after GCS's failure. gcs_heartbeat_manager_->Stop(); - - ray_syncer_->Stop(); + if (RayConfig::instance().use_ray_syncer()) { + ray_syncer_io_context_.stop(); + ray_syncer_thread_->join(); + } else { + gcs_ray_syncer_->Stop(); + } // Shutdown the rpc server rpc_server_.Shutdown(); @@ -366,7 +370,7 @@ void GcsServer::InitGcsPlacementGroupManager(const GcsInitData &gcs_init_data) { *gcs_resource_manager_, *cluster_resource_scheduler_, raylet_client_pool_, - *ray_syncer_); + *gcs_ray_syncer_); gcs_placement_group_manager_ = std::make_shared( main_service_, @@ -417,15 +421,25 @@ void GcsServer::StoreGcsServerAddressInRedis() { } void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { - /* - The current synchronization flow is: - raylet -> syncer::poller --> syncer::update -> gcs_resource_manager - gcs_placement_scheduler --/ - */ - ray_syncer_ = std::make_unique( - main_service_, raylet_client_pool_, *gcs_resource_manager_); - ray_syncer_->Initialize(gcs_init_data); - ray_syncer_->Start(); + if (RayConfig::instance().use_ray_syncer()) { + ray_syncer_node_id_ = NodeID::FromRandom(); + ray_syncer_ = std::make_unique(ray_syncer_io_context_, + ray_syncer_node_id_.Binary()); + ray_syncer_thread_ = std::make_unique([this]() { + boost::asio::io_service::work work(ray_syncer_io_context_); + ray_syncer_io_context_.run(); + }); + } else { + /* + The current synchronization flow is: + raylet -> syncer::poller --> syncer::update -> gcs_resource_manager + gcs_placement_scheduler --/ + */ + gcs_ray_syncer_ = std::make_unique( + main_service_, raylet_client_pool_, *gcs_resource_manager_); + gcs_ray_syncer_->Initialize(gcs_init_data); + gcs_ray_syncer_->Start(); + } } void GcsServer::InitStatsHandler() { @@ -515,7 +529,10 @@ void GcsServer::InstallEventListeners() { gcs_placement_group_manager_->OnNodeAdd(NodeID::FromBinary(node->node_id())); gcs_actor_manager_->SchedulePendingActors(); gcs_heartbeat_manager_->AddNode(NodeID::FromBinary(node->node_id())); - ray_syncer_->AddNode(*node); + if (RayConfig::instance().use_ray_syncer()) { + } else { + gcs_ray_syncer_->AddNode(*node); + } }); gcs_node_manager_->AddNodeRemovedListener( [this](std::shared_ptr node) { @@ -527,7 +544,10 @@ void GcsServer::InstallEventListeners() { gcs_placement_group_manager_->OnNodeDead(node_id); gcs_actor_manager_->OnNodeDead(node_id, node_ip_address); raylet_client_pool_->Disconnect(NodeID::FromBinary(node->node_id())); - ray_syncer_->RemoveNode(*node); + if (RayConfig::instance().use_ray_syncer()) { + } else { + gcs_ray_syncer_->RemoveNode(*node); + } }); // Install worker event listener. @@ -595,7 +615,7 @@ std::string GcsServer::GetDebugState() const { << gcs_publisher_->DebugString() << "\n\n" << runtime_env_manager_->DebugString() << "\n\n"; - stream << ray_syncer_->DebugString(); + stream << gcs_ray_syncer_->DebugString(); return stream.str(); } diff --git a/src/ray/gcs/gcs_server/gcs_server.h b/src/ray/gcs/gcs_server/gcs_server.h index 9249b6cbad12..6a34563ed066 100644 --- a/src/ray/gcs/gcs_server/gcs_server.h +++ b/src/ray/gcs/gcs_server/gcs_server.h @@ -15,6 +15,7 @@ #pragma once #include "ray/common/asio/instrumented_io_context.h" +#include "ray/common/ray_syncer/ray_syncer.h" #include "ray/common/runtime_env_manager.h" #include "ray/gcs/gcs_server/gcs_function_manager.h" #include "ray/gcs/gcs_server/gcs_heartbeat_manager.h" @@ -207,8 +208,15 @@ class GcsServer { /// Stats handler and service. std::unique_ptr stats_handler_; std::unique_ptr stats_service_; - // Synchronization service for ray. - std::unique_ptr ray_syncer_; + /// Synchronization service for ray. + std::unique_ptr gcs_ray_syncer_; + + /// Ray Syncer realted fields. + std::unique_ptr ray_syncer_; + std::unique_ptr ray_syncer_thread_; + instrumented_io_context ray_syncer_io_context_; + NodeID ray_syncer_node_id_; + /// The gcs worker manager. std::unique_ptr gcs_worker_manager_; /// Worker info service. diff --git a/src/ray/raylet_client/raylet_client.cc b/src/ray/raylet_client/raylet_client.cc index b075c629f931..238da3795260 100644 --- a/src/ray/raylet_client/raylet_client.cc +++ b/src/ray/raylet_client/raylet_client.cc @@ -345,6 +345,10 @@ void raylet::RayletClient::RequestObjectSpillage( grpc_client_->RequestObjectSpillage(request, callback); } +std::shared_ptr raylet::RayletClient::GetChannel() const { + return grpc_client_->Channel(); +} + void raylet::RayletClient::ReportWorkerBacklog( const WorkerID &worker_id, const std::vector &backlog_reports) { diff --git a/src/ray/raylet_client/raylet_client.h b/src/ray/raylet_client/raylet_client.h index f286f3861195..0b2120962384 100644 --- a/src/ray/raylet_client/raylet_client.h +++ b/src/ray/raylet_client/raylet_client.h @@ -189,6 +189,8 @@ class RayletClientInterface : public PinObjectsInterface, const NodeID &node_id, bool graceful, const rpc::ClientCallback &callback) = 0; + + virtual std::shared_ptr GetChannel() const = 0; }; namespace raylet { @@ -382,6 +384,8 @@ class RayletClient : public RayletClientInterface { const ObjectID &object_id, const rpc::ClientCallback &callback); + std::shared_ptr GetChannel() const override; + /// Implements WorkerLeaseInterface. void RequestWorkerLease( const rpc::TaskSpec &resource_spec, diff --git a/src/ray/rpc/grpc_client.h b/src/ray/rpc/grpc_client.h index 087d30fcbcae..94d1a86f0412 100644 --- a/src/ray/rpc/grpc_client.h +++ b/src/ray/rpc/grpc_client.h @@ -60,9 +60,9 @@ class GrpcClient { argument.SetMaxSendMessageSize(::RayConfig::instance().max_grpc_message_size()); argument.SetMaxReceiveMessageSize(::RayConfig::instance().max_grpc_message_size()); - std::shared_ptr channel = BuildChannel(argument, address, port); + channel_ = BuildChannel(argument, address, port); - stub_ = GrpcService::NewStub(channel); + stub_ = GrpcService::NewStub(channel_); } GrpcClient(const std::string &address, @@ -79,9 +79,9 @@ class GrpcClient { argument.SetMaxSendMessageSize(::RayConfig::instance().max_grpc_message_size()); argument.SetMaxReceiveMessageSize(::RayConfig::instance().max_grpc_message_size()); - std::shared_ptr channel = BuildChannel(argument, address, port); + channel_ = BuildChannel(argument, address, port); - stub_ = GrpcService::NewStub(channel); + stub_ = GrpcService::NewStub(channel_); } /// Create a new `ClientCall` and send request. @@ -115,12 +115,16 @@ class GrpcClient { RAY_CHECK(call != nullptr); } + std::shared_ptr Channel() const { return channel_; } + private: ClientCallManager &client_call_manager_; /// The gRPC-generated stub. std::unique_ptr stub_; /// Whether to use TLS. bool use_tls_; + /// The channel of the stub. + std::shared_ptr channel_; std::shared_ptr BuildChannel(const grpc::ChannelArguments &argument, const std::string &address, diff --git a/src/ray/rpc/node_manager/node_manager_client.h b/src/ray/rpc/node_manager/node_manager_client.h index e22f274765c1..41425fe6e3b5 100644 --- a/src/ray/rpc/node_manager/node_manager_client.h +++ b/src/ray/rpc/node_manager/node_manager_client.h @@ -53,6 +53,8 @@ class NodeManagerClient { GetNodeStats(request, callback); } + std::shared_ptr Channel() const { return grpc_client_->Channel(); } + private: /// The RPC client. std::unique_ptr> grpc_client_; @@ -75,6 +77,8 @@ class NodeManagerWorkerClient return std::shared_ptr(instance); } + std::shared_ptr Channel() const { return grpc_client_->Channel(); } + /// Update cluster resource usage. VOID_RPC_CLIENT_METHOD(NodeManagerService, UpdateResourceUsage, From 34be999c7519c29f52d277e8860d5bab233b421a Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 4 Apr 2022 03:31:12 +0000 Subject: [PATCH 06/77] finish add and remove --- src/ray/gcs/gcs_server/gcs_server.cc | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 69d29f8f023a..45a5ba5c055f 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -525,11 +525,19 @@ void GcsServer::InstallEventListeners() { gcs_node_manager_->AddNodeAddedListener([this](std::shared_ptr node) { // Because a new node has been added, we need to try to schedule the pending // placement groups and the pending actors. + auto node_id = NodeID::FromBinary(node->node_id()); gcs_resource_manager_->OnNodeAdd(*node); - gcs_placement_group_manager_->OnNodeAdd(NodeID::FromBinary(node->node_id())); + gcs_placement_group_manager_->OnNodeAdd(node_id); gcs_actor_manager_->SchedulePendingActors(); - gcs_heartbeat_manager_->AddNode(NodeID::FromBinary(node->node_id())); + gcs_heartbeat_manager_->AddNode(node_id); if (RayConfig::instance().use_ray_syncer()) { + rpc::Address address; + address.set_raylet_id(node->node_id()); + address.set_ip_address(node->node_manager_address()); + address.set_port(node->node_manager_port()); + + auto raylet_client = raylet_client_pool_->GetOrConnectByAddress(address); + ray_syncer_->Connect(raylet_client->GetChannel()); } else { gcs_ray_syncer_->AddNode(*node); } @@ -545,6 +553,7 @@ void GcsServer::InstallEventListeners() { gcs_actor_manager_->OnNodeDead(node_id, node_ip_address); raylet_client_pool_->Disconnect(NodeID::FromBinary(node->node_id())); if (RayConfig::instance().use_ray_syncer()) { + ray_syncer_->Disconnect(node_id.Binary()); } else { gcs_ray_syncer_->RemoveNode(*node); } From 3e887748b09fcf252962dd2c651b6d16965de327 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 4 Apr 2022 03:54:55 +0000 Subject: [PATCH 07/77] integrate with gcs --- src/ray/common/ray_config_def.h | 2 ++ .../gcs/gcs_server/gcs_resource_manager.cc | 16 ++++++++++++++++ src/ray/gcs/gcs_server/gcs_resource_manager.h | 7 ++++++- src/ray/gcs/gcs_server/gcs_server.cc | 4 ++++ src/ray/raylet/node_manager.cc | 19 ++++++++++++++++--- 5 files changed, 44 insertions(+), 4 deletions(-) diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 74db243e9e6f..8506efbf4660 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -60,6 +60,8 @@ RAY_CONFIG(uint64_t, num_heartbeats_warning, 5) /// The duration between reporting resources sent by the raylets. RAY_CONFIG(uint64_t, raylet_report_resources_period_milliseconds, 100) +RAY_CONFIG(uint64_t, raylet_report_loads_period_milliseconds, 1000) + /// For a raylet, if the last resource report was sent more than this many /// report periods ago, then a warning will be logged that the report /// handler is drifting. diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index 8cfb7ee4e9f8..5e860e85c407 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -26,6 +26,22 @@ GcsResourceManager::GcsResourceManager( : gcs_table_storage_(gcs_table_storage), cluster_resource_manager_(cluster_resource_manager) {} +void GcsResourceManager::Update(std::shared_ptr message) { + // Make sure thread safety. + main_io_service_.post([this, message]() { + rpc::ResourcesData resources; + resources.ParseFromString(message->sync_message()); + resources.set_node_id(message->node_id()); + auto node_id = NodeID::FromBinary(message->node_id()); + if (message->component_id() == syncing::RayComponentId::RESOURCE_MANAGER) { + cluster_resource_manager_.UpdateNodeAvailableResourcesIfExist( + scheduling::NodeID(message->node_id()), resources); + } else if (message->component_id() == syncing::RayComponentId::SCHEDULER) { + UpdateNodeResourceUsage(node_id, resources); + } + }); +} + void GcsResourceManager::HandleGetResources(const rpc::GetResourcesRequest &request, rpc::GetResourcesReply *reply, rpc::SendReplyCallback send_reply_callback) { diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.h b/src/ray/gcs/gcs_server/gcs_resource_manager.h index cc76563da77f..d14d6736be81 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.h +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.h @@ -16,6 +16,7 @@ #include "absl/container/flat_hash_map.h" #include "absl/container/flat_hash_set.h" #include "ray/common/id.h" +#include "ray/common/ray_syncer/ray_syncer.h" #include "ray/gcs/gcs_server/gcs_init_data.h" #include "ray/gcs/gcs_server/gcs_table_storage.h" #include "ray/raylet/scheduling/cluster_resource_data.h" @@ -44,7 +45,8 @@ namespace gcs { /// It is responsible for handing node resource related rpc requests and it is used for /// actor and placement group scheduling. It obtains the available resources of nodes /// through heartbeat reporting. Non-thread safe. -class GcsResourceManager : public rpc::NodeResourceInfoHandler { +class GcsResourceManager : public rpc::NodeResourceInfoHandler, + syncer::ReceiverInterface { public: /// Create a GcsResourceManager. /// @@ -54,6 +56,9 @@ class GcsResourceManager : public rpc::NodeResourceInfoHandler { virtual ~GcsResourceManager() {} + /// Handle the resource update. + void Update(std::shared_ptr message) override; + /// Handle get resource rpc request. void HandleGetResources(const rpc::GetResourcesRequest &request, rpc::GetResourcesReply *reply, diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 45a5ba5c055f..b9c8877c5157 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -425,6 +425,10 @@ void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { ray_syncer_node_id_ = NodeID::FromRandom(); ray_syncer_ = std::make_unique(ray_syncer_io_context_, ray_syncer_node_id_.Binary()); + syncer_->Register( + syncing::RayComponentId::RESOURCE_MANAGER, nullptr, gcs_resource_manager_.get()); + syncer_->Register( + syncing::RayComponentId::SCHEDULER, nullptr, gcs_resource_manager_.get()); ray_syncer_thread_ = std::make_unique([this]() { boost::asio::io_service::work work(ray_syncer_io_context_); ray_syncer_io_context_.run(); diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 6f2a0ce17109..ce82b7ff2e11 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -543,6 +543,19 @@ ray::Status NodeManager::RegisterGcs() { "NodeManager.deadline_timer.print_event_loop_stats"); } + if (RayConfig::instance().use_ray_syncer()) { + syncer_->Register( + syncer::RayComponentId::RESOURCE_MANAGER, + this, + this, + false, + RayConfig::instance().raylet_report_resources_period_milliseconds()); + syncer_->Register(syncer::RayComponentId::SCHEDULER, + this, + nullptr, + true, + RayConfig::instance().raylet_report_loads_period_milliseconds()); + } return ray::Status::OK(); } @@ -611,7 +624,7 @@ void NodeManager::FillNormalTaskResourceUsage(rpc::ResourcesData &resources_data void NodeManager::FillResourceReport(rpc::ResourcesData &resources_data) { resources_data.set_node_id(self_node_id_.Binary()); - // resources_data.set_node_manager_address(initial_config_.node_manager_address); + resources_data.set_node_manager_address(initial_config_.node_manager_address); // Update local cache from gcs remote cache, this is needed when gcs restart. // We should always keep the cache view consistent. cluster_resource_scheduler_->GetLocalResourceManager().ResetLastReportResourceUsage( @@ -2580,7 +2593,7 @@ std::optional NodeManager::Snapshot( rpc::ResourcesData resource_data; cluster_task_manager_->FillResourceUsage(resource_data); resource_data.set_node_id(self_node_id_.Binary()); - // resource_data.set_node_manager_address(initial_config_.node_manager_address); + resource_data.set_node_manager_address(initial_config_.node_manager_address); msg.set_version(++version); msg.set_node_id(self_node_id_.Binary()); @@ -2600,7 +2613,7 @@ std::optional NodeManager::Snapshot( rpc::ResourcesData resource_data; local.FillResourceUsage(resource_data); resource_data.set_node_id(self_node_id_.Binary()); - // resource_data.set_node_manager_address(initial_config_.node_manager_address); + resource_data.set_node_manager_address(initial_config_.node_manager_address); msg.set_node_id(self_node_id_.Binary()); msg.set_version(local.Version()); From c45efd43882d0962b967492c232c4c3ccfdfee47 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 4 Apr 2022 04:39:51 +0000 Subject: [PATCH 08/77] fix it --- .../gcs/gcs_server/gcs_resource_manager.cc | 30 +++++++++++-------- src/ray/gcs/gcs_server/gcs_resource_manager.h | 8 +++-- src/ray/gcs/gcs_server/gcs_server.cc | 22 ++++++++++---- 3 files changed, 40 insertions(+), 20 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index 5e860e85c407..a4a220970038 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -21,25 +21,29 @@ namespace ray { namespace gcs { GcsResourceManager::GcsResourceManager( + instrumented_io_context &io_context, std::shared_ptr gcs_table_storage, ClusterResourceManager &cluster_resource_manager) - : gcs_table_storage_(gcs_table_storage), + : io_context_(io_context), + gcs_table_storage_(gcs_table_storage), cluster_resource_manager_(cluster_resource_manager) {} void GcsResourceManager::Update(std::shared_ptr message) { // Make sure thread safety. - main_io_service_.post([this, message]() { - rpc::ResourcesData resources; - resources.ParseFromString(message->sync_message()); - resources.set_node_id(message->node_id()); - auto node_id = NodeID::FromBinary(message->node_id()); - if (message->component_id() == syncing::RayComponentId::RESOURCE_MANAGER) { - cluster_resource_manager_.UpdateNodeAvailableResourcesIfExist( - scheduling::NodeID(message->node_id()), resources); - } else if (message->component_id() == syncing::RayComponentId::SCHEDULER) { - UpdateNodeResourceUsage(node_id, resources); - } - }); + io_context_.post( + [this, message]() { + rpc::ResourcesData resources; + resources.ParseFromString(message->sync_message()); + resources.set_node_id(message->node_id()); + auto node_id = NodeID::FromBinary(message->node_id()); + if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { + cluster_resource_manager_.UpdateNodeAvailableResourcesIfExist( + scheduling::NodeID(message->node_id()), resources); + } else if (message->component_id() == syncer::RayComponentId::SCHEDULER) { + UpdateNodeResourceUsage(node_id, resources); + } + }, + "GcsResourceManager::Update"); } void GcsResourceManager::HandleGetResources(const rpc::GetResourcesRequest &request, diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.h b/src/ray/gcs/gcs_server/gcs_resource_manager.h index d14d6736be81..40819968ec6d 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.h +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.h @@ -46,12 +46,13 @@ namespace gcs { /// actor and placement group scheduling. It obtains the available resources of nodes /// through heartbeat reporting. Non-thread safe. class GcsResourceManager : public rpc::NodeResourceInfoHandler, - syncer::ReceiverInterface { + public syncer::ReceiverInterface { public: /// Create a GcsResourceManager. /// /// \param gcs_table_storage GCS table external storage accessor. - explicit GcsResourceManager(std::shared_ptr gcs_table_storage, + explicit GcsResourceManager(instrumented_io_context &io_context, + std::shared_ptr gcs_table_storage, ClusterResourceManager &cluster_resource_manager); virtual ~GcsResourceManager() {} @@ -138,6 +139,9 @@ class GcsResourceManager : public rpc::NodeResourceInfoHandler, const std::shared_ptr placement_group_load); private: + /// io context + instrumented_io_context &io_context_; + /// Newest resource usage of all nodes. absl::flat_hash_map node_resource_usages_; diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index b9c8877c5157..9a72b236fdac 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -250,7 +250,9 @@ void GcsServer::InitGcsHeartbeatManager(const GcsInitData &gcs_init_data) { void GcsServer::InitGcsResourceManager(const GcsInitData &gcs_init_data) { RAY_CHECK(gcs_table_storage_ && cluster_resource_scheduler_); gcs_resource_manager_ = std::make_shared( - gcs_table_storage_, cluster_resource_scheduler_->GetClusterResourceManager()); + main_service_, + gcs_table_storage_, + cluster_resource_scheduler_->GetClusterResourceManager()); // Initialize by gcs tables data. gcs_resource_manager_->Initialize(gcs_init_data); @@ -425,14 +427,24 @@ void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { ray_syncer_node_id_ = NodeID::FromRandom(); ray_syncer_ = std::make_unique(ray_syncer_io_context_, ray_syncer_node_id_.Binary()); - syncer_->Register( - syncing::RayComponentId::RESOURCE_MANAGER, nullptr, gcs_resource_manager_.get()); - syncer_->Register( - syncing::RayComponentId::SCHEDULER, nullptr, gcs_resource_manager_.get()); + ray_syncer_->Register( + syncer::RayComponentId::RESOURCE_MANAGER, nullptr, gcs_resource_manager_.get()); + ray_syncer_->Register( + syncer::RayComponentId::SCHEDULER, nullptr, gcs_resource_manager_.get()); ray_syncer_thread_ = std::make_unique([this]() { boost::asio::io_service::work work(ray_syncer_io_context_); ray_syncer_io_context_.run(); }); + + for (const auto &pair : gcs_init_data.Nodes()) { + rpc::Address address; + address.set_raylet_id(pair.second.node_id()); + address.set_ip_address(pair.second.node_manager_address()); + address.set_port(pair.second.node_manager_port()); + + auto raylet_client = raylet_client_pool_->GetOrConnectByAddress(address); + ray_syncer_->Connect(raylet_client->GetChannel()); + } } else { /* The current synchronization flow is: From dad1885187aafa98c39de2d4f1aedb915b6ba927 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 4 Apr 2022 21:52:11 +0000 Subject: [PATCH 09/77] make raylet compile --- src/ray/protobuf/gcs.proto | 2 +- src/ray/raylet/node_manager.cc | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/ray/protobuf/gcs.proto b/src/ray/protobuf/gcs.proto index 3e6859e7fb69..4f100465bafb 100644 --- a/src/ray/protobuf/gcs.proto +++ b/src/ray/protobuf/gcs.proto @@ -409,7 +409,7 @@ message ResourcesData { // Whether this node manager is requesting global GC. bool should_global_gc = 8; // IP address of the node. - // string node_manager_address = 9; + string node_manager_address = 9; // Whether this node has object pulls queued. This can happen if // the node has more pull requests than available object store // memory. This is a proxy for available object store memory. diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index ce82b7ff2e11..be168285efad 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -544,17 +544,17 @@ ray::Status NodeManager::RegisterGcs() { } if (RayConfig::instance().use_ray_syncer()) { - syncer_->Register( + ray_syncer_.Register( syncer::RayComponentId::RESOURCE_MANAGER, this, this, false, RayConfig::instance().raylet_report_resources_period_milliseconds()); - syncer_->Register(syncer::RayComponentId::SCHEDULER, - this, - nullptr, - true, - RayConfig::instance().raylet_report_loads_period_milliseconds()); + ray_syncer_.Register(syncer::RayComponentId::SCHEDULER, + this, + nullptr, + true, + RayConfig::instance().raylet_report_loads_period_milliseconds()); } return ray::Status::OK(); } From 1e812bd32e2ac66c8065f0ad4a2382f066f59f9e Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Tue, 5 Apr 2022 02:39:55 +0000 Subject: [PATCH 10/77] enable for testing --- src/ray/common/ray_config_def.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 8506efbf4660..16e7a3663910 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -333,7 +333,7 @@ RAY_CONFIG(int32_t, gcs_rpc_server_reconnect_timeout_s, 60) RAY_CONFIG(int32_t, minimum_gcs_reconnect_interval_milliseconds, 5000) /// Feature flag to use the ray syncer for resource synchronization -RAY_CONFIG(bool, use_ray_syncer, false) +RAY_CONFIG(bool, use_ray_syncer, true) /// The interval at which the gcs client will check if the address of gcs service has /// changed. When the address changed, we will resubscribe again. From c2e715bed9b07196c45ff70a11cfde01b3b12e1f Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Tue, 5 Apr 2022 18:26:58 +0000 Subject: [PATCH 11/77] fix one issue --- src/ray/gcs/gcs_server/gcs_server.cc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 9a72b236fdac..cf716416af74 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -639,8 +639,9 @@ std::string GcsServer::GetDebugState() const { << gcs_placement_group_manager_->DebugString() << "\n\n" << gcs_publisher_->DebugString() << "\n\n" << runtime_env_manager_->DebugString() << "\n\n"; - - stream << gcs_ray_syncer_->DebugString(); + if(gcs_ray_syncer_) { + stream << gcs_ray_syncer_->DebugString(); + } return stream.str(); } From e61e63f85463bbc02aa084490f5581144c1a86cc Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Wed, 6 Apr 2022 00:17:05 +0000 Subject: [PATCH 12/77] fix --- python/ray/tests/test_basic.py | 30 +++++++++---------- .../gcs/gcs_server/gcs_resource_manager.cc | 4 +++ src/ray/raylet/node_manager.cc | 2 +- .../scheduling/local_resource_manager.cc | 19 ++++++------ .../scheduling/local_resource_manager.h | 3 +- 5 files changed, 32 insertions(+), 26 deletions(-) diff --git a/python/ray/tests/test_basic.py b/python/ray/tests/test_basic.py index 202781056382..1ee47f3c9dbc 100644 --- a/python/ray/tests/test_basic.py +++ b/python/ray/tests/test_basic.py @@ -234,25 +234,25 @@ class A2: x = 1 -def test_user_setup_function(): - script = """ -import ray -ray.init() -@ray.remote -def get_pkg_dir(): - return ray._private.runtime_env.VAR +# def test_user_setup_function(): +# script = """ +# import ray +# ray.init() +# @ray.remote +# def get_pkg_dir(): +# return ray._private.runtime_env.VAR -print("remote", ray.get(get_pkg_dir.remote())) -print("local", ray._private.runtime_env.VAR) +# print("remote", ray.get(get_pkg_dir.remote())) +# print("local", ray._private.runtime_env.VAR) -""" +# """ - env = {"RAY_USER_SETUP_FUNCTION": "ray._private.test_utils.set_setup_func"} - out = run_string_as_driver(script, dict(os.environ, **env)) - (remote_out, local_out) = out.strip().splitlines()[-2:] - assert remote_out == "remote hello world" - assert local_out == "local hello world" +# env = {"RAY_USER_SETUP_FUNCTION": "ray._private.test_utils.set_setup_func"} +# out = run_string_as_driver(script, dict(os.environ, **env)) +# (remote_out, local_out) = out.strip().splitlines()[-2:] +# assert remote_out == "remote hello world" +# assert local_out == "local hello world" # https://github.com/ray-project/ray/issues/17842 diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index a4a220970038..a91641395ba2 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -16,6 +16,7 @@ #include "ray/common/ray_config.h" #include "ray/stats/metric_defs.h" +#include namespace ray { namespace gcs { @@ -35,6 +36,9 @@ void GcsResourceManager::Update(std::shared_ptr me rpc::ResourcesData resources; resources.ParseFromString(message->sync_message()); resources.set_node_id(message->node_id()); + // std::string json_str; + // google::protobuf::util::MessageToJsonString(resources, &json_str); + // RAY_LOG(ERROR) << "Received a message: (" << message->component_id() << ")\t" << json_str; auto node_id = NodeID::FromBinary(message->node_id()); if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { cluster_resource_manager_.UpdateNodeAvailableResourcesIfExist( diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index d9588cb1d22d..26823fd51c99 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2612,7 +2612,7 @@ std::optional NodeManager::Snapshot( syncer::RaySyncMessage msg; rpc::ResourcesData resource_data; - local.FillResourceUsage(resource_data); + local.FillResourceUsage(resource_data, true); resource_data.set_node_id(self_node_id_.Binary()); resource_data.set_node_manager_address(initial_config_.node_manager_address); diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 85e6fa95f9aa..10b8928e9f0a 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -312,13 +312,14 @@ void LocalResourceManager::UpdateAvailableObjectStoreMemResource() { OnResourceChanged(); } -void LocalResourceManager::FillResourceUsage(rpc::ResourcesData &resources_data) { +void LocalResourceManager::FillResourceUsage(rpc::ResourcesData &resources_data, + bool for_ray_syncer) { UpdateAvailableObjectStoreMemResource(); NodeResources resources = ToNodeResources(local_resources_); // Initialize if last report resources is empty. - if (!last_report_resources_) { + if (!for_ray_syncer && !last_report_resources_) { NodeResources node_resources = ResourceMapToNodeResources({{}}, {{}}); last_report_resources_.reset(new NodeResources(node_resources)); } @@ -327,32 +328,32 @@ void LocalResourceManager::FillResourceUsage(rpc::ResourcesData &resources_data) auto label = ResourceID(resource_id).Binary(); auto total = entry.second; auto available = resources.available.Get(resource_id); - auto last_total = last_report_resources_->total.Get(resource_id); - auto last_available = last_report_resources_->available.Get(resource_id); + auto last_total = for_ray_syncer ? total : last_report_resources_->total.Get(resource_id); + auto last_available = for_ray_syncer ? available : last_report_resources_->available.Get(resource_id); // Note: available may be negative, but only report positive to GCS. - if (available != last_available && available > 0) { + if (for_ray_syncer || (available != last_available && available > 0)) { resources_data.set_resources_available_changed(true); (*resources_data.mutable_resources_available())[label] = available.Double(); } - if (total != last_total) { + if (for_ray_syncer || (total != last_total)) { (*resources_data.mutable_resources_total())[label] = total.Double(); } } if (get_pull_manager_at_capacity_ != nullptr) { resources.object_pulls_queued = get_pull_manager_at_capacity_(); - if (last_report_resources_->object_pulls_queued != resources.object_pulls_queued) { + if (for_ray_syncer || (last_report_resources_->object_pulls_queued != resources.object_pulls_queued)) { resources_data.set_object_pulls_queued(resources.object_pulls_queued); resources_data.set_resources_available_changed(true); } } - if (resources != *last_report_resources_.get()) { + if (!for_ray_syncer && resources != *last_report_resources_.get()) { last_report_resources_.reset(new NodeResources(resources)); } - if (!RayConfig::instance().enable_light_weight_resource_report()) { + if (for_ray_syncer || !RayConfig::instance().enable_light_weight_resource_report()) { resources_data.set_resources_available_changed(true); } } diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index 50daefc3ffcd..00a573604d37 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -115,7 +115,8 @@ class LocalResourceManager { /// /// \param Output parameter. `resources_available` and `resources_total` are the only /// fields used. - void FillResourceUsage(rpc::ResourcesData &resources_data); + /// \param for_ray_syncer. The resource report is for ray syncer. + void FillResourceUsage(rpc::ResourcesData &resources_data, bool for_ray_syncer = false); /// Populate a UpdateResourcesRequest. This is inteneded to update the /// resource totals on a node when a custom resource is created or deleted From abc4579bacdb4a39e2e6fa5622346792aaa68452 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Wed, 6 Apr 2022 02:42:22 +0000 Subject: [PATCH 13/77] format --- src/ray/gcs/gcs_server/gcs_resource_manager.cc | 6 ++++-- src/ray/gcs/gcs_server/gcs_server.cc | 2 +- src/ray/raylet/scheduling/local_resource_manager.cc | 9 ++++++--- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index a91641395ba2..b3be6329e870 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -14,9 +14,10 @@ #include "ray/gcs/gcs_server/gcs_resource_manager.h" +#include + #include "ray/common/ray_config.h" #include "ray/stats/metric_defs.h" -#include namespace ray { namespace gcs { @@ -38,7 +39,8 @@ void GcsResourceManager::Update(std::shared_ptr me resources.set_node_id(message->node_id()); // std::string json_str; // google::protobuf::util::MessageToJsonString(resources, &json_str); - // RAY_LOG(ERROR) << "Received a message: (" << message->component_id() << ")\t" << json_str; + // RAY_LOG(ERROR) << "Received a message: (" << message->component_id() << ")\t" + // << json_str; auto node_id = NodeID::FromBinary(message->node_id()); if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { cluster_resource_manager_.UpdateNodeAvailableResourcesIfExist( diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index cf716416af74..89b31cf2e248 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -639,7 +639,7 @@ std::string GcsServer::GetDebugState() const { << gcs_placement_group_manager_->DebugString() << "\n\n" << gcs_publisher_->DebugString() << "\n\n" << runtime_env_manager_->DebugString() << "\n\n"; - if(gcs_ray_syncer_) { + if (gcs_ray_syncer_) { stream << gcs_ray_syncer_->DebugString(); } return stream.str(); diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 10b8928e9f0a..9e49f6e097e5 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -328,8 +328,10 @@ void LocalResourceManager::FillResourceUsage(rpc::ResourcesData &resources_data, auto label = ResourceID(resource_id).Binary(); auto total = entry.second; auto available = resources.available.Get(resource_id); - auto last_total = for_ray_syncer ? total : last_report_resources_->total.Get(resource_id); - auto last_available = for_ray_syncer ? available : last_report_resources_->available.Get(resource_id); + auto last_total = + for_ray_syncer ? total : last_report_resources_->total.Get(resource_id); + auto last_available = + for_ray_syncer ? available : last_report_resources_->available.Get(resource_id); // Note: available may be negative, but only report positive to GCS. if (for_ray_syncer || (available != last_available && available > 0)) { @@ -343,7 +345,8 @@ void LocalResourceManager::FillResourceUsage(rpc::ResourcesData &resources_data, if (get_pull_manager_at_capacity_ != nullptr) { resources.object_pulls_queued = get_pull_manager_at_capacity_(); - if (for_ray_syncer || (last_report_resources_->object_pulls_queued != resources.object_pulls_queued)) { + if (for_ray_syncer || + (last_report_resources_->object_pulls_queued != resources.object_pulls_queued)) { resources_data.set_object_pulls_queued(resources.object_pulls_queued); resources_data.set_resources_available_changed(true); } From c124f2fae09c410ab7a6fd14206641456a4fcf28 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 7 Apr 2022 05:57:01 +0000 Subject: [PATCH 14/77] fix test_basic_3.py --- src/ray/common/test/ray_syncer_test.cc | 22 ++++++++++++---------- src/ray/raylet/node_manager.cc | 26 ++++++++++++++++++++++++++ src/ray/raylet/node_manager.h | 8 ++++---- 3 files changed, 42 insertions(+), 14 deletions(-) diff --git a/src/ray/common/test/ray_syncer_test.cc b/src/ray/common/test/ray_syncer_test.cc index 46f0064d13d2..2320140662b2 100644 --- a/src/ray/common/test/ray_syncer_test.cc +++ b/src/ray/common/test/ray_syncer_test.cc @@ -42,6 +42,8 @@ using ::testing::WithArg; namespace ray { namespace syncer { +constexpr size_t kTestComponents = 2; + RaySyncMessage MakeMessage(RayComponentId cid, int64_t version, const NodeID &id) { auto msg = RaySyncMessage(); msg.set_version(version); @@ -96,10 +98,10 @@ class RaySyncerTest : public ::testing::Test { thread_->join(); } - std::array local_versions_; - std::array, kComponentArraySize> reporters_ = { + std::array local_versions_; + std::array, kTestComponents> reporters_ = { nullptr}; - std::array, kComponentArraySize> receivers_ = { + std::array, kTestComponents> receivers_ = { nullptr}; instrumented_io_context io_context_; @@ -310,8 +312,8 @@ struct SyncerServerTest { } } - std::array, kComponentArraySize> _v; - const std::array, kComponentArraySize> &GetReceivedVersions( + std::array, kTestComponents> _v; + const std::array, kTestComponents> &GetReceivedVersions( const std::string &node_id) { auto iter = received_versions.find(node_id); if (iter == received_versions.end()) { @@ -328,21 +330,21 @@ struct SyncerServerTest { std::unique_ptr thread; instrumented_io_context io_context; std::string server_port; - std::array, kComponentArraySize> local_versions; - std::array, kComponentArraySize> reporters = { + std::array, kTestComponents> local_versions; + std::array, kTestComponents> reporters = { nullptr}; int64_t snapshot_taken = 0; - std::unordered_map, kComponentArraySize>> + std::unordered_map, kTestComponents>> received_versions; std::unordered_map> message_consumed; - std::array, kComponentArraySize> receivers = { + std::array, kTestComponents> receivers = { nullptr}; }; // Useful for debugging // std::ostream &operator<<(std::ostream &os, const SyncerServerTest &server) { -// auto dump_array = [&os](const std::array &v, +// auto dump_array = [&os](const std::array &v, // std::string label, // int indent) mutable -> std::ostream & { // os << std::string('\t', indent); diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 26823fd51c99..ac119f494823 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2592,6 +2592,32 @@ std::optional NodeManager::Snapshot( static uint64_t version = 0; syncer::RaySyncMessage msg; rpc::ResourcesData resource_data; + + // If plasma store is under high pressure, we should try to schedule a global gc. + bool plasma_high_pressure = + object_manager_.GetUsedMemoryPercentage() > high_plasma_storage_usage_; + if (plasma_high_pressure && global_gc_throttler_.AbleToRun()) { + const_cast(this)->TriggerGlobalGC(); + } + + // Set the global gc bit on the outgoing heartbeat message. + bool triggered_by_global_gc = false; + if (should_global_gc_) { + resource_data.set_should_global_gc(true); + triggered_by_global_gc = true; + should_global_gc_ = false; + global_gc_throttler_.RunNow(); + } + + // Trigger local GC if needed. This throttles the frequency of local GC calls + // to at most once per heartbeat interval. + if ((should_local_gc_ || + (absl::GetCurrentTimeNanos() - local_gc_run_time_ns_ > local_gc_interval_ns_)) && + local_gc_throttler_.AbleToRun()) { + const_cast(this)->DoLocalGC(triggered_by_global_gc); + should_local_gc_ = false; + } + cluster_task_manager_->FillResourceUsage(resource_data); resource_data.set_node_id(self_node_id_.Binary()); resource_data.set_node_manager_address(initial_config_.node_manager_address); diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index 2a650919a9f1..e8c332774e70 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -702,11 +702,11 @@ class NodeManager : public rpc::NodeManagerServiceHandler, /// Whether to trigger global GC in the next resource usage report. This will broadcast /// a global GC message to all raylets except for this one. - bool should_global_gc_ = false; + mutable bool should_global_gc_ = false; /// Whether to trigger local GC in the next resource usage report. This will trigger gc /// on all local workers of this raylet. - bool should_local_gc_ = false; + mutable bool should_local_gc_ = false; /// When plasma storage usage is high, we'll run gc to reduce it. double high_plasma_storage_usage_ = 1.0; @@ -715,10 +715,10 @@ class NodeManager : public rpc::NodeManagerServiceHandler, uint64_t local_gc_run_time_ns_; /// Throttler for local gc - Throttler local_gc_throttler_; + mutable Throttler local_gc_throttler_; /// Throttler for global gc - Throttler global_gc_throttler_; + mutable Throttler global_gc_throttler_; /// Seconds to initialize a local gc const uint64_t local_gc_interval_ns_; From f2034b92835a953edb66fe8b52810cf18ffa10b5 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 7 Apr 2022 05:57:50 +0000 Subject: [PATCH 15/77] format --- src/ray/raylet/node_manager.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index ac119f494823..ee5e8f928434 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2597,7 +2597,7 @@ std::optional NodeManager::Snapshot( bool plasma_high_pressure = object_manager_.GetUsedMemoryPercentage() > high_plasma_storage_usage_; if (plasma_high_pressure && global_gc_throttler_.AbleToRun()) { - const_cast(this)->TriggerGlobalGC(); + const_cast(this)->TriggerGlobalGC(); } // Set the global gc bit on the outgoing heartbeat message. @@ -2614,7 +2614,7 @@ std::optional NodeManager::Snapshot( if ((should_local_gc_ || (absl::GetCurrentTimeNanos() - local_gc_run_time_ns_ > local_gc_interval_ns_)) && local_gc_throttler_.AbleToRun()) { - const_cast(this)->DoLocalGC(triggered_by_global_gc); + const_cast(this)->DoLocalGC(triggered_by_global_gc); should_local_gc_ = false; } From df968a727f88e43eb9df2f85374841c3a4c0dcb2 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 7 Apr 2022 21:15:38 +0000 Subject: [PATCH 16/77] up --- .bazelrc | 2 ++ src/ray/gcs/gcs_server/gcs_resource_manager.cc | 18 ++++++++++-------- src/ray/gcs/gcs_server/ray_syncer.h | 4 ++++ src/ray/raylet/node_manager.cc | 3 +++ 4 files changed, 19 insertions(+), 8 deletions(-) diff --git a/.bazelrc b/.bazelrc index d103098fc384..7e4f62020574 100644 --- a/.bazelrc +++ b/.bazelrc @@ -9,6 +9,8 @@ build:windows --action_env=PATH build --compilation_mode=opt # Using C++ 17 on all platforms. build:linux --cxxopt="-std=c++17" +build:linux --cxxopt="-g" +build:linux --cxxopt="-ggdb" build:macos --cxxopt="-std=c++17" build:clang-cl --cxxopt="-std=c++17" build:msvc-cl --cxxopt="/std:c++17" diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index b3be6329e870..dc3648166326 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -37,17 +37,19 @@ void GcsResourceManager::Update(std::shared_ptr me rpc::ResourcesData resources; resources.ParseFromString(message->sync_message()); resources.set_node_id(message->node_id()); - // std::string json_str; - // google::protobuf::util::MessageToJsonString(resources, &json_str); - // RAY_LOG(ERROR) << "Received a message: (" << message->component_id() << ")\t" - // << json_str; + std::string json_str; + google::protobuf::util::MessageToJsonString(resources, &json_str); + RAY_LOG(ERROR) << "Received a message: (" << message->component_id() << ")\t" << json_str; auto node_id = NodeID::FromBinary(message->node_id()); if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { - cluster_resource_manager_.UpdateNodeAvailableResourcesIfExist( - scheduling::NodeID(message->node_id()), resources); - } else if (message->component_id() == syncer::RayComponentId::SCHEDULER) { - UpdateNodeResourceUsage(node_id, resources); + if (RayConfig::instance().gcs_actor_scheduling_enabled()) { + UpdateNodeNormalTaskResources(NodeID::FromBinary(message->node_id()), resources); + } else { + cluster_resource_manager_.UpdateNodeAvailableResourcesIfExist( + scheduling::NodeID(message->node_id()), resources); + } } + UpdateNodeResourceUsage(node_id, resources); }, "GcsResourceManager::Update"); } diff --git a/src/ray/gcs/gcs_server/ray_syncer.h b/src/ray/gcs/gcs_server/ray_syncer.h index 431679ba6d26..23024ae6eab9 100644 --- a/src/ray/gcs/gcs_server/ray_syncer.h +++ b/src/ray/gcs/gcs_server/ray_syncer.h @@ -111,6 +111,10 @@ class RaySyncer { static_assert(std::is_same_v || std::is_same_v, "unknown type"); + std::string json_str; + google::protobuf::util::MessageToJsonString(update, &json_str); + RAY_LOG(ERROR) << "GcsRaySyncer: Received a message: " << json_str; + if constexpr (std::is_same_v) { resources_buffer_proto_.add_batch()->mutable_change()->Swap(&update); } else if constexpr (std::is_same_v) { diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index ee5e8f928434..3cbf2ee7c833 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2619,6 +2619,9 @@ std::optional NodeManager::Snapshot( } cluster_task_manager_->FillResourceUsage(resource_data); + if (RayConfig::instance().gcs_actor_scheduling_enabled()) { + const_cast(this)->FillNormalTaskResourceUsage(resource_data); + } resource_data.set_node_id(self_node_id_.Binary()); resource_data.set_node_manager_address(initial_config_.node_manager_address); From 4219b1d5f9313439a81f21497cae8a5691a9908f Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 7 Apr 2022 21:15:47 +0000 Subject: [PATCH 17/77] up --- src/ray/gcs/gcs_server/gcs_resource_manager.cc | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index dc3648166326..19e47a20575a 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -39,11 +39,13 @@ void GcsResourceManager::Update(std::shared_ptr me resources.set_node_id(message->node_id()); std::string json_str; google::protobuf::util::MessageToJsonString(resources, &json_str); - RAY_LOG(ERROR) << "Received a message: (" << message->component_id() << ")\t" << json_str; + RAY_LOG(ERROR) << "Received a message: (" << message->component_id() << ")\t" + << json_str; auto node_id = NodeID::FromBinary(message->node_id()); if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { if (RayConfig::instance().gcs_actor_scheduling_enabled()) { - UpdateNodeNormalTaskResources(NodeID::FromBinary(message->node_id()), resources); + UpdateNodeNormalTaskResources(NodeID::FromBinary(message->node_id()), + resources); } else { cluster_resource_manager_.UpdateNodeAvailableResourcesIfExist( scheduling::NodeID(message->node_id()), resources); From 42bd6d07f0776a0503cc24f0d08c2429c5e936ec Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 7 Apr 2022 21:54:57 +0000 Subject: [PATCH 18/77] fix pg mini test --- .../gcs_placement_group_scheduler.cc | 28 +++++++++++-------- .../gcs_placement_group_scheduler.h | 4 +-- .../gcs/gcs_server/gcs_resource_manager.cc | 8 +++--- src/ray/gcs/gcs_server/gcs_server.cc | 2 +- 4 files changed, 24 insertions(+), 18 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc index 07eeecb0c490..f9f260de091f 100644 --- a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc +++ b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc @@ -27,7 +27,7 @@ GcsPlacementGroupScheduler::GcsPlacementGroupScheduler( GcsResourceManager &gcs_resource_manager, ClusterResourceScheduler &cluster_resource_scheduler, std::shared_ptr raylet_client_pool, - gcs_syncer::RaySyncer &ray_syncer) + gcs_syncer::RaySyncer *ray_syncer) : return_timer_(io_context), gcs_table_storage_(std::move(gcs_table_storage)), gcs_node_manager_(gcs_node_manager), @@ -237,14 +237,18 @@ void GcsPlacementGroupScheduler::CancelResourceReserve( RAY_LOG(DEBUG) << "Finished cancelling the resource reserved for bundle: " << bundle_spec->DebugString() << " at node " << node_id; std::vector resource_names; - rpc::NodeResourceChange node_resource_change; auto &resources = bundle_spec->GetFormattedResources(); for (const auto &iter : resources) { resource_names.push_back(iter.first); - node_resource_change.add_deleted_resources(iter.first); } - node_resource_change.set_node_id(node_id.Binary()); - ray_syncer_.Update(std::move(node_resource_change)); + if(ray_syncer_ != nullptr) { + rpc::NodeResourceChange node_resource_change; + for (const auto &iter : resources) { + node_resource_change.add_deleted_resources(iter.first); + } + node_resource_change.set_node_id(node_id.Binary()); + ray_syncer_->Update(std::move(node_resource_change)); + } gcs_resource_manager_.DeleteResources(node_id, std::move(resource_names)); }); } @@ -298,12 +302,14 @@ void GcsPlacementGroupScheduler::CommitAllBundles( auto &resources = bundle->GetFormattedResources(); gcs_resource_manager_.UpdateResources(node_id, resources); - // Push the message to syncer so that it can be broadcasted to all other nodes - rpc::NodeResourceChange node_resource_change; - node_resource_change.set_node_id(node_id.Binary()); - node_resource_change.mutable_updated_resources()->insert(resources.begin(), - resources.end()); - ray_syncer_.Update(std::move(node_resource_change)); + if(ray_syncer_ != nullptr) { + // Push the message to syncer so that it can be broadcasted to all other nodes + rpc::NodeResourceChange node_resource_change; + node_resource_change.set_node_id(node_id.Binary()); + node_resource_change.mutable_updated_resources()->insert(resources.begin(), + resources.end()); + ray_syncer_->Update(std::move(node_resource_change)); + } } if (lease_status_tracker->AllCommitRequestReturned()) { OnAllBundleCommitRequestReturned( diff --git a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h index 010018ec2ff0..274f8f10ce2b 100644 --- a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h +++ b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h @@ -338,7 +338,7 @@ class GcsPlacementGroupScheduler : public GcsPlacementGroupSchedulerInterface { GcsResourceManager &gcs_resource_manager, ClusterResourceScheduler &cluster_resource_scheduler, std::shared_ptr raylet_client_pool, - gcs_syncer::RaySyncer &ray_syncer); + gcs_syncer::RaySyncer *ray_syncer); virtual ~GcsPlacementGroupScheduler() = default; @@ -514,7 +514,7 @@ class GcsPlacementGroupScheduler : public GcsPlacementGroupSchedulerInterface { /// The syncer of resource. This is used to report placement group updates. /// TODO (iycheng): Remove this one from pg once we finish the refactor - gcs_syncer::RaySyncer &ray_syncer_; + gcs_syncer::RaySyncer *ray_syncer_; }; } // namespace gcs diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index 19e47a20575a..2905ebfc732f 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -37,10 +37,10 @@ void GcsResourceManager::Update(std::shared_ptr me rpc::ResourcesData resources; resources.ParseFromString(message->sync_message()); resources.set_node_id(message->node_id()); - std::string json_str; - google::protobuf::util::MessageToJsonString(resources, &json_str); - RAY_LOG(ERROR) << "Received a message: (" << message->component_id() << ")\t" - << json_str; + // std::string json_str; + // google::protobuf::util::MessageToJsonString(resources, &json_str); + // RAY_LOG(ERROR) << "Received a message: (" << message->component_id() << ")\t" + // << json_str; auto node_id = NodeID::FromBinary(message->node_id()); if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { if (RayConfig::instance().gcs_actor_scheduling_enabled()) { diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 89b31cf2e248..625f00e68962 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -372,7 +372,7 @@ void GcsServer::InitGcsPlacementGroupManager(const GcsInitData &gcs_init_data) { *gcs_resource_manager_, *cluster_resource_scheduler_, raylet_client_pool_, - *gcs_ray_syncer_); + gcs_ray_syncer_.get()); gcs_placement_group_manager_ = std::make_shared( main_service_, From dd25eb6e48b98d781cb9c714c933f4fe2c06a7aa Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 7 Apr 2022 21:57:34 +0000 Subject: [PATCH 19/77] format --- src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc index f9f260de091f..8f844eb1a1b8 100644 --- a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc +++ b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc @@ -241,7 +241,7 @@ void GcsPlacementGroupScheduler::CancelResourceReserve( for (const auto &iter : resources) { resource_names.push_back(iter.first); } - if(ray_syncer_ != nullptr) { + if (ray_syncer_ != nullptr) { rpc::NodeResourceChange node_resource_change; for (const auto &iter : resources) { node_resource_change.add_deleted_resources(iter.first); @@ -302,7 +302,7 @@ void GcsPlacementGroupScheduler::CommitAllBundles( auto &resources = bundle->GetFormattedResources(); gcs_resource_manager_.UpdateResources(node_id, resources); - if(ray_syncer_ != nullptr) { + if (ray_syncer_ != nullptr) { // Push the message to syncer so that it can be broadcasted to all other nodes rpc::NodeResourceChange node_resource_change; node_resource_change.set_node_id(node_id.Binary()); From 3be93619f865480fbc1d9b0d31952efc8abe6b8b Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 7 Apr 2022 22:25:04 +0000 Subject: [PATCH 20/77] add pg updates in raylet --- src/ray/common/ray_syncer/ray_syncer.cc | 14 +++++++++----- src/ray/common/ray_syncer/ray_syncer.h | 5 +++++ src/ray/raylet/node_manager.cc | 6 ++++++ 3 files changed, 20 insertions(+), 5 deletions(-) diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index b9ba3ba8405e..24530142d08b 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -340,11 +340,7 @@ bool RaySyncer::Register(RayComponentId component_id, RAY_CHECK(pull_from_reporter_interval_ms > 0); timer_.RunFnPeriodically( [this, component_id]() { - auto snapshot = node_state_->GetSnapshot(component_id); - if (snapshot) { - RAY_CHECK(snapshot->node_id() == GetLocalNodeID()); - BroadcastMessage(std::make_shared(std::move(*snapshot))); - } + BroadcastMessage(component_id); }, pull_from_reporter_interval_ms); } @@ -374,6 +370,14 @@ void RaySyncer::BroadcastMessage(std::shared_ptr message) } } +void RaySyncer::BroadcastMessage(RayComponentId cid) { + auto snapshot = node_state_->GetSnapshot(cid); + if (snapshot) { + RAY_CHECK(snapshot->node_id() == GetLocalNodeID()); + BroadcastMessage(std::make_shared(std::move(*snapshot))); + } +} + grpc::ServerUnaryReactor *RaySyncerService::StartSync( grpc::CallbackServerContext *context, const StartSyncRequest *request, diff --git a/src/ray/common/ray_syncer/ray_syncer.h b/src/ray/common/ray_syncer/ray_syncer.h index c1029767cfde..b7b9acb9a615 100644 --- a/src/ray/common/ray_syncer/ray_syncer.h +++ b/src/ray/common/ray_syncer/ray_syncer.h @@ -142,6 +142,11 @@ class RaySyncer { /// \param message The message to be broadcasted. void BroadcastMessage(std::shared_ptr message); + /// Function to force a broadcasting of messages come from a component_id + /// + /// \param component_id The component which need to send the broadcasting messages + void BroadcastMessage(ray::rpc::syncer::RayComponentId component_id); + /// Get the current node id. const std::string &GetLocalNodeID() const { return local_node_id_; } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 3cbf2ee7c833..7af76bd57ab7 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1733,6 +1733,9 @@ void NodeManager::HandleCommitBundleResources( RAY_LOG(DEBUG) << "Request to commit resources for bundles: " << GetDebugStringForBundles(bundle_specs); placement_group_resource_manager_->CommitBundles(bundle_specs); + if (RayConfig::instance().use_ray_syncer()) { + ray_syncer_->BroadcastMessage(syncer::RayComponentId::RESOURCE_MANAGER); + } send_reply_callback(Status::OK(), nullptr, nullptr); cluster_task_manager_->ScheduleAndDispatchTasks(); @@ -1770,6 +1773,9 @@ void NodeManager::HandleCancelResourceReserve( // Return bundle resources. placement_group_resource_manager_->ReturnBundle(bundle_spec); + if (RayConfig::instance().use_ray_syncer()) { + ray_syncer_->BroadcastMessage(syncer::RayComponentId::RESOURCE_MANAGER); + } cluster_task_manager_->ScheduleAndDispatchTasks(); send_reply_callback(Status::OK(), nullptr, nullptr); } From 478987aedb0d890cffc8dec90f9402b4fc2f860b Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 7 Apr 2022 22:38:34 +0000 Subject: [PATCH 21/77] format and fix some cpp test --- src/mock/ray/raylet_client/raylet_client.h | 1 + src/ray/common/ray_syncer/ray_syncer.cc | 7 ++----- src/ray/gcs/gcs_server/test/gcs_server_test_util.h | 2 ++ 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/mock/ray/raylet_client/raylet_client.h b/src/mock/ray/raylet_client/raylet_client.h index 1de4d76b841a..0ef9fb7b4bf5 100644 --- a/src/mock/ray/raylet_client/raylet_client.h +++ b/src/mock/ray/raylet_client/raylet_client.h @@ -128,6 +128,7 @@ class MockRayletClientInterface : public RayletClientInterface { WaitForDirectActorCallArgs, (const std::vector &references, int64_t tag), (override)); + MOCK_METHOD(std::shared_ptr, GetChannel, (), (const)); MOCK_METHOD(void, ReportWorkerBacklog, (const WorkerID &worker_id, diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index 24530142d08b..ec374b091768 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -338,11 +338,8 @@ bool RaySyncer::Register(RayComponentId component_id, // Set job to pull from reporter periodically if (reporter != nullptr) { RAY_CHECK(pull_from_reporter_interval_ms > 0); - timer_.RunFnPeriodically( - [this, component_id]() { - BroadcastMessage(component_id); - }, - pull_from_reporter_interval_ms); + timer_.RunFnPeriodically([this, component_id]() { BroadcastMessage(component_id); }, + pull_from_reporter_interval_ms); } RAY_LOG(DEBUG) << "Registered components: " 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 2606e31481b2..0a746a7e4aea 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 @@ -73,6 +73,8 @@ struct GcsServerMocker { return Status::OK(); } + std::shared_ptr GetChannel() const override { return nullptr; } + void ReportWorkerBacklog( const WorkerID &worker_id, const std::vector &backlog_reports) override {} From 8db538a8a10e1bf725bafe08a87b2a3d2231017d Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 8 Apr 2022 00:58:40 +0000 Subject: [PATCH 22/77] fix raylet --- .../gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc | 2 +- src/ray/raylet/node_manager.cc | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc b/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc index 0cb5332540e3..c338a33b9b24 100644 --- a/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc @@ -63,7 +63,7 @@ class GcsPlacementGroupSchedulerTest : public ::testing::Test { *gcs_resource_manager_, *cluster_resource_scheduler_, raylet_client_pool_, - *ray_syncer_); + ray_syncer_.get()); } void TearDown() override { diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 7af76bd57ab7..788020f78d3c 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1734,7 +1734,7 @@ void NodeManager::HandleCommitBundleResources( << GetDebugStringForBundles(bundle_specs); placement_group_resource_manager_->CommitBundles(bundle_specs); if (RayConfig::instance().use_ray_syncer()) { - ray_syncer_->BroadcastMessage(syncer::RayComponentId::RESOURCE_MANAGER); + ray_syncer_.BroadcastMessage(syncer::RayComponentId::RESOURCE_MANAGER); } send_reply_callback(Status::OK(), nullptr, nullptr); @@ -1774,7 +1774,7 @@ void NodeManager::HandleCancelResourceReserve( // Return bundle resources. placement_group_resource_manager_->ReturnBundle(bundle_spec); if (RayConfig::instance().use_ray_syncer()) { - ray_syncer_->BroadcastMessage(syncer::RayComponentId::RESOURCE_MANAGER); + ray_syncer_.BroadcastMessage(syncer::RayComponentId::RESOURCE_MANAGER); } cluster_task_manager_->ScheduleAndDispatchTasks(); send_reply_callback(Status::OK(), nullptr, nullptr); From fd4b89a412f473301fcd59524e0d6157d37f63eb Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 00:15:37 +0000 Subject: [PATCH 23/77] obj memory and warning issue --- python/ray/tests/test_actor_resources.py | 1 - src/ray/gcs/gcs_server/ray_syncer.h | 6 +++--- src/ray/raylet/node_manager.cc | 1 + .../raylet/scheduling/local_resource_manager.cc | 14 +++++++++----- src/ray/raylet/scheduling/local_resource_manager.h | 5 ++++- 5 files changed, 17 insertions(+), 10 deletions(-) diff --git a/python/ray/tests/test_actor_resources.py b/python/ray/tests/test_actor_resources.py index f4fe402a25a9..d33c2513bc27 100644 --- a/python/ray/tests/test_actor_resources.py +++ b/python/ray/tests/test_actor_resources.py @@ -397,7 +397,6 @@ def locations_to_intervals_for_many_tasks(): check_intervals_non_overlapping(locations_to_intervals[locations]) # Make sure that the actor's GPU was not used. assert actor_location not in locations_to_intervals - # Create more actors to fill up all the GPUs. more_actors = [Actor1.remote() for _ in range(num_nodes * num_gpus_per_raylet - 1)] # Wait for the actors to finish being created. diff --git a/src/ray/gcs/gcs_server/ray_syncer.h b/src/ray/gcs/gcs_server/ray_syncer.h index 23024ae6eab9..e827b5716b4a 100644 --- a/src/ray/gcs/gcs_server/ray_syncer.h +++ b/src/ray/gcs/gcs_server/ray_syncer.h @@ -111,9 +111,9 @@ class RaySyncer { static_assert(std::is_same_v || std::is_same_v, "unknown type"); - std::string json_str; - google::protobuf::util::MessageToJsonString(update, &json_str); - RAY_LOG(ERROR) << "GcsRaySyncer: Received a message: " << json_str; + // std::string json_str; + // google::protobuf::util::MessageToJsonString(update, &json_str); + // RAY_LOG(ERROR) << "GcsRaySyncer: Received a message: " << json_str; if constexpr (std::is_same_v) { resources_buffer_proto_.add_batch()->mutable_change()->Swap(&update); diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 788020f78d3c..e6961dcb5316 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2630,6 +2630,7 @@ std::optional NodeManager::Snapshot( } resource_data.set_node_id(self_node_id_.Binary()); resource_data.set_node_manager_address(initial_config_.node_manager_address); + resource_data.set_cluster_full_of_actors_detected(resource_deadlock_warned_ >= 1); msg.set_version(++version); msg.set_node_id(self_node_id_.Binary()); diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 9e49f6e097e5..0752b31e4a3a 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -306,15 +306,19 @@ void LocalResourceManager::UpdateAvailableObjectStoreMemResource() { RAY_CHECK_EQ(total_instances.size(), 1u); const double used = get_used_object_store_memory_(); const double total = total_instances[0].Double(); - local_resources_.available.Set(ResourceID::ObjectStoreMemory(), - {FixedPoint(total >= used ? total - used : 0.0)}); - - OnResourceChanged(); + auto new_available = std::vector{FixedPoint(total >= used ? total - used : 0.0)}; + if(new_available != local_resources_.available.Get(ResourceID::ObjectStoreMemory())) { + local_resources_.available.Set(ResourceID::ObjectStoreMemory(), + std::move(new_available)); + OnResourceChanged(); + } } void LocalResourceManager::FillResourceUsage(rpc::ResourcesData &resources_data, bool for_ray_syncer) { - UpdateAvailableObjectStoreMemResource(); + if(!for_ray_syncer) { + UpdateAvailableObjectStoreMemResource(); + } NodeResources resources = ToNodeResources(local_resources_); diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index 00a573604d37..14ff14403184 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -107,7 +107,10 @@ class LocalResourceManager { void ReleaseWorkerResources(std::shared_ptr task_allocation); - int64_t Version() const { return version_; } + int64_t Version() { + UpdateAvailableObjectStoreMemResource(); + return version_; + } /// Populate the relevant parts of the heartbeat table. This is intended for /// sending resource usage of raylet to gcs. In particular, this should fill in From 5903e2a64c0ea905c8234d2c0aafe880a70e1e58 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 00:16:04 +0000 Subject: [PATCH 24/77] format --- src/ray/raylet/scheduling/local_resource_manager.cc | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 0752b31e4a3a..4dd5a4208d9b 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -306,8 +306,9 @@ void LocalResourceManager::UpdateAvailableObjectStoreMemResource() { RAY_CHECK_EQ(total_instances.size(), 1u); const double used = get_used_object_store_memory_(); const double total = total_instances[0].Double(); - auto new_available = std::vector{FixedPoint(total >= used ? total - used : 0.0)}; - if(new_available != local_resources_.available.Get(ResourceID::ObjectStoreMemory())) { + auto new_available = + std::vector{FixedPoint(total >= used ? total - used : 0.0)}; + if (new_available != local_resources_.available.Get(ResourceID::ObjectStoreMemory())) { local_resources_.available.Set(ResourceID::ObjectStoreMemory(), std::move(new_available)); OnResourceChanged(); @@ -316,7 +317,7 @@ void LocalResourceManager::UpdateAvailableObjectStoreMemResource() { void LocalResourceManager::FillResourceUsage(rpc::ResourcesData &resources_data, bool for_ray_syncer) { - if(!for_ray_syncer) { + if (!for_ray_syncer) { UpdateAvailableObjectStoreMemResource(); } From 617f2966f1796faec1503d7e51a8420d96266763 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 00:32:28 +0000 Subject: [PATCH 25/77] fix gcs reconnect --- src/ray/common/ray_syncer/ray_syncer.cc | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index ec374b091768..e3429b657c22 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -320,8 +320,14 @@ void RaySyncer::Connect(std::unique_ptr connection) { } void RaySyncer::Disconnect(const std::string &node_id) { - io_context_.post([this, node_id]() { sync_connections_.erase(node_id); }, - "RaySyncerDisconnect"); + io_context_.post([this, node_id]() { + auto iter = sync_connections_.find(node_id); + if(iter != sync_connections_.end()) { + upward_connections_.erase(iter->second.get()); + sync_connections_.erase(iter); + } + }, + "RaySyncerDisconnect"); } bool RaySyncer::Register(RayComponentId component_id, @@ -381,7 +387,12 @@ grpc::ServerUnaryReactor *RaySyncerService::StartSync( StartSyncResponse *response) { auto *reactor = context->DefaultReactor(); // Make sure server only have one client - RAY_CHECK(remote_node_id_.empty()); + if(!remote_node_id_.empty()) { + RAY_LOG(WARNING) << "Get a new sync request from " + << NodeID::FromBinary(request->node_id()) << ". " + << "Now disconnect from " << NodeID::FromBinary(remote_node_id_); + syncer_.Disconnect(remote_node_id_); + } remote_node_id_ = request->node_id(); RAY_LOG(DEBUG) << "Get connect from: " << NodeID::FromBinary(remote_node_id_); syncer_.GetIOContext().post( From fa612f14dd6e92f2ba7ea92a732c4499f90b9688 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 00:34:35 +0000 Subject: [PATCH 26/77] format --- src/ray/common/ray_syncer/ray_syncer.cc | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index e3429b657c22..6f0839b4c6dd 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -320,14 +320,15 @@ void RaySyncer::Connect(std::unique_ptr connection) { } void RaySyncer::Disconnect(const std::string &node_id) { - io_context_.post([this, node_id]() { - auto iter = sync_connections_.find(node_id); - if(iter != sync_connections_.end()) { - upward_connections_.erase(iter->second.get()); - sync_connections_.erase(iter); - } - }, - "RaySyncerDisconnect"); + io_context_.post( + [this, node_id]() { + auto iter = sync_connections_.find(node_id); + if (iter != sync_connections_.end()) { + upward_connections_.erase(iter->second.get()); + sync_connections_.erase(iter); + } + }, + "RaySyncerDisconnect"); } bool RaySyncer::Register(RayComponentId component_id, @@ -387,7 +388,7 @@ grpc::ServerUnaryReactor *RaySyncerService::StartSync( StartSyncResponse *response) { auto *reactor = context->DefaultReactor(); // Make sure server only have one client - if(!remote_node_id_.empty()) { + if (!remote_node_id_.empty()) { RAY_LOG(WARNING) << "Get a new sync request from " << NodeID::FromBinary(request->node_id()) << ". " << "Now disconnect from " << NodeID::FromBinary(remote_node_id_); From 172fa7efabd0dd20659e4b6ce97741b7222738ad Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 01:00:49 +0000 Subject: [PATCH 27/77] fix cpp issues --- .../gcs/gcs_server/test/gcs_based_actor_scheduler_test.cc | 4 +++- .../gcs_server/test/gcs_placement_group_manager_mock_test.cc | 4 ++-- .../gcs_server/test/gcs_placement_group_scheduler_test.cc | 4 +++- src/ray/gcs/gcs_server/test/gcs_resource_manager_test.cc | 5 +++-- 4 files changed, 11 insertions(+), 6 deletions(-) diff --git a/src/ray/gcs/gcs_server/test/gcs_based_actor_scheduler_test.cc b/src/ray/gcs/gcs_server/test/gcs_based_actor_scheduler_test.cc index bb9bbf6a7d04..5a2740f059d4 100644 --- a/src/ray/gcs/gcs_server/test/gcs_based_actor_scheduler_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_based_actor_scheduler_test.cc @@ -46,7 +46,9 @@ class GcsBasedActorSchedulerTest : public ::testing::Test { std::make_shared(store_client_); cluster_resource_scheduler_ = std::make_shared(); gcs_resource_manager_ = std::make_shared( - gcs_table_storage_, cluster_resource_scheduler_->GetClusterResourceManager()); + io_service_, + gcs_table_storage_, + cluster_resource_scheduler_->GetClusterResourceManager()); gcs_actor_scheduler_ = std::make_shared( io_service_, diff --git a/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_mock_test.cc b/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_mock_test.cc index 2e08d45306c4..266566ed30cf 100644 --- a/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_mock_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_mock_test.cc @@ -37,8 +37,8 @@ class GcsPlacementGroupManagerMockTest : public Test { gcs_table_storage_ = std::make_shared(store_client_); gcs_placement_group_scheduler_ = std::make_shared(); - resource_manager_ = - std::make_shared(nullptr, cluster_resource_manager_); + resource_manager_ = std::make_shared( + io_context_, nullptr, cluster_resource_manager_); gcs_placement_group_manager_ = std::make_unique(io_context_, diff --git a/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc b/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc index c338a33b9b24..d7d0bf1ca36c 100644 --- a/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc @@ -48,7 +48,9 @@ class GcsPlacementGroupSchedulerTest : public ::testing::Test { std::make_unique()); cluster_resource_scheduler_ = std::make_shared(); gcs_resource_manager_ = std::make_shared( - gcs_table_storage_, cluster_resource_scheduler_->GetClusterResourceManager()); + io_service_, + gcs_table_storage_, + cluster_resource_scheduler_->GetClusterResourceManager()); ray_syncer_ = std::make_shared( io_service_, nullptr, *gcs_resource_manager_); store_client_ = std::make_shared(io_service_); diff --git a/src/ray/gcs/gcs_server/test/gcs_resource_manager_test.cc b/src/ray/gcs/gcs_server/test/gcs_resource_manager_test.cc index b63fdfe46e01..82b2cb2d91a5 100644 --- a/src/ray/gcs/gcs_server/test/gcs_resource_manager_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_resource_manager_test.cc @@ -28,10 +28,11 @@ using ::testing::_; class GcsResourceManagerTest : public ::testing::Test { public: GcsResourceManagerTest() { - gcs_resource_manager_ = - std::make_shared(nullptr, cluster_resource_manager_); + gcs_resource_manager_ = std::make_shared( + io_service_, nullptr, cluster_resource_manager_); } + instrumented_io_context io_service_; ClusterResourceManager cluster_resource_manager_; std::shared_ptr gcs_resource_manager_; }; From 57c3ecca7ea9f24dd7ade81de3c92d166ee63def Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 04:58:44 +0000 Subject: [PATCH 28/77] move gc to command channel --- src/ray/common/ray_config_def.h | 3 ++ src/ray/protobuf/ray_syncer.proto | 1 + src/ray/raylet/node_manager.cc | 70 ++++++++++++++++++++----------- 3 files changed, 49 insertions(+), 25 deletions(-) diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 4c3906c7befa..1ecd3f9bbd55 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -62,6 +62,9 @@ RAY_CONFIG(uint64_t, num_heartbeats_warning, 5) RAY_CONFIG(uint64_t, raylet_report_resources_period_milliseconds, 100) RAY_CONFIG(uint64_t, raylet_report_loads_period_milliseconds, 1000) +/// The duration between raylet check memory pressure and send gc request +RAY_CONFIG(uint64_t, raylet_check_gc_period_milliseconds, 100) + /// For a raylet, if the last resource report was sent more than this many /// report periods ago, then a warning will be logged that the report /// handler is drifting. diff --git a/src/ray/protobuf/ray_syncer.proto b/src/ray/protobuf/ray_syncer.proto index 33c60dc96b1b..fd0e79581de9 100644 --- a/src/ray/protobuf/ray_syncer.proto +++ b/src/ray/protobuf/ray_syncer.proto @@ -19,6 +19,7 @@ package ray.rpc.syncer; enum RayComponentId { RESOURCE_MANAGER = 0; SCHEDULER = 1; + COMMANDS = 2; } message RaySyncMessage { diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index e6961dcb5316..d7bfb5796eff 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -556,6 +556,50 @@ ray::Status NodeManager::RegisterGcs() { nullptr, true, RayConfig::instance().raylet_report_loads_period_milliseconds()); + ray_syncer_.Register(syncer::RayComponentId::COMMANDS, nullptr, this, false, 0); + periodical_runner_.RunFnPeriodically( + [this] { + // If plasma store is under high pressure, we should try to schedule a global + // gc. + bool plasma_high_pressure = + object_manager_.GetUsedMemoryPercentage() > high_plasma_storage_usage_; + if (plasma_high_pressure && global_gc_throttler_.AbleToRun()) { + TriggerGlobalGC(); + } + + // Set the global gc bit on the outgoing heartbeat message. + bool triggered_by_global_gc = false; + if (should_global_gc_) { + triggered_by_global_gc = true; + should_global_gc_ = false; + global_gc_throttler_.RunNow(); + } + + // Trigger local GC if needed. This throttles the frequency of local GC calls + // to at most once per heartbeat interval. + if ((should_local_gc_ || (absl::GetCurrentTimeNanos() - local_gc_run_time_ns_ > + local_gc_interval_ns_)) && + local_gc_throttler_.AbleToRun()) { + DoLocalGC(triggered_by_global_gc); + should_local_gc_ = false; + } + static int64_t version = 0; + + if (triggered_by_global_gc) { + rpc::ResourcesData resources_data; + resources_data.set_should_global_gc(true); + syncer::RaySyncMessage msg; + msg.set_version(++version); + msg.set_node_id(self_node_id_.Binary()); + msg.set_component_id(syncer::RayComponentId::COMMAND); + std::string serialized_msg; + RAY_CHECK(resource_data.SerializeToString(&serialized_msg)); + msg.set_sync_message(std::move(serialized_msg)); + ray_syncer_->BroadcastMessage(std::make_shared(std::move(msg)); + } + }, + RayConfig::instance().raylet_check_gc_period_milliseconds(), + "NodeManager.CheckGC"); } return ray::Status::OK(); } @@ -2599,35 +2643,11 @@ std::optional NodeManager::Snapshot( syncer::RaySyncMessage msg; rpc::ResourcesData resource_data; - // If plasma store is under high pressure, we should try to schedule a global gc. - bool plasma_high_pressure = - object_manager_.GetUsedMemoryPercentage() > high_plasma_storage_usage_; - if (plasma_high_pressure && global_gc_throttler_.AbleToRun()) { - const_cast(this)->TriggerGlobalGC(); - } - - // Set the global gc bit on the outgoing heartbeat message. - bool triggered_by_global_gc = false; - if (should_global_gc_) { - resource_data.set_should_global_gc(true); - triggered_by_global_gc = true; - should_global_gc_ = false; - global_gc_throttler_.RunNow(); - } - - // Trigger local GC if needed. This throttles the frequency of local GC calls - // to at most once per heartbeat interval. - if ((should_local_gc_ || - (absl::GetCurrentTimeNanos() - local_gc_run_time_ns_ > local_gc_interval_ns_)) && - local_gc_throttler_.AbleToRun()) { - const_cast(this)->DoLocalGC(triggered_by_global_gc); - should_local_gc_ = false; - } - cluster_task_manager_->FillResourceUsage(resource_data); if (RayConfig::instance().gcs_actor_scheduling_enabled()) { const_cast(this)->FillNormalTaskResourceUsage(resource_data); } + resource_data.set_node_id(self_node_id_.Binary()); resource_data.set_node_manager_address(initial_config_.node_manager_address); resource_data.set_cluster_full_of_actors_detected(resource_deadlock_warned_ >= 1); From 6f3c6bd7998d2a90e4765449b13dbca2ac472b93 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 05:25:38 +0000 Subject: [PATCH 29/77] fix gc --- src/ray/raylet/node_manager.cc | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index d7bfb5796eff..0899d0e76d96 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -591,11 +591,12 @@ ray::Status NodeManager::RegisterGcs() { syncer::RaySyncMessage msg; msg.set_version(++version); msg.set_node_id(self_node_id_.Binary()); - msg.set_component_id(syncer::RayComponentId::COMMAND); + msg.set_component_id(syncer::RayComponentId::COMMANDS); std::string serialized_msg; - RAY_CHECK(resource_data.SerializeToString(&serialized_msg)); + RAY_CHECK(resources_data.SerializeToString(&serialized_msg)); msg.set_sync_message(std::move(serialized_msg)); - ray_syncer_->BroadcastMessage(std::make_shared(std::move(msg)); + ray_syncer_.BroadcastMessage( + std::make_shared(std::move(msg))); } }, RayConfig::instance().raylet_check_gc_period_milliseconds(), @@ -2629,11 +2630,18 @@ void NodeManager::RecordMetrics() { } void NodeManager::Update(std::shared_ptr message) { - rpc::ResourcesData data; - data.ParseFromString(message->sync_message()); - NodeID node_id = NodeID::FromBinary(data.node_id()); - - UpdateResourceUsage(node_id, data); + if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { + rpc::ResourcesData data; + data.ParseFromString(message->sync_message()); + NodeID node_id = NodeID::FromBinary(data.node_id()); + UpdateResourceUsage(node_id, data); + } else if (message->component_id() == syncer::RayComponentId::COMMANDS) { + rpc::ResourcesData data; + data.ParseFromString(message->sync_message()); + if (data.should_global_gc()) { + should_local_gc_ = true; + } + } } std::optional NodeManager::Snapshot( From 94a4fadae00db1e20df20bcab06998e0efcaef30 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 05:42:28 +0000 Subject: [PATCH 30/77] format --- .../gcs/gcs_server/test/gcs_placement_group_manager_test.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_test.cc b/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_test.cc index 7d8c87152647..533f7652de63 100644 --- a/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_test.cc @@ -84,8 +84,8 @@ class GcsPlacementGroupManagerTest : public ::testing::Test { gcs_publisher_ = std::make_shared(std::make_unique()); gcs_table_storage_ = std::make_shared(io_service_); - gcs_resource_manager_ = - std::make_shared(nullptr, cluster_resource_manager_); + gcs_resource_manager_ = std::make_shared( + io_service_, nullptr, cluster_resource_manager_); gcs_placement_group_manager_.reset(new gcs::GcsPlacementGroupManager( io_service_, mock_placement_group_scheduler_, From 2b813f4bcc96a1c76352bdfd97e661a265e4a2ae Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 05:46:23 +0000 Subject: [PATCH 31/77] revert one line --- python/ray/tests/test_actor_resources.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/tests/test_actor_resources.py b/python/ray/tests/test_actor_resources.py index d33c2513bc27..f4fe402a25a9 100644 --- a/python/ray/tests/test_actor_resources.py +++ b/python/ray/tests/test_actor_resources.py @@ -397,6 +397,7 @@ def locations_to_intervals_for_many_tasks(): check_intervals_non_overlapping(locations_to_intervals[locations]) # Make sure that the actor's GPU was not used. assert actor_location not in locations_to_intervals + # Create more actors to fill up all the GPUs. more_actors = [Actor1.remote() for _ in range(num_nodes * num_gpus_per_raylet - 1)] # Wait for the actors to finish being created. From 2b5b1fffe7abf1d8315eff3a32efeb0e49786bee Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 05:55:58 +0000 Subject: [PATCH 32/77] update --- src/ray/raylet/node_manager.cc | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 0899d0e76d96..67c513b719fa 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -19,6 +19,7 @@ #include #include +#include "absl/time/clock.h" #include "boost/filesystem.hpp" #include "boost/system/error_code.hpp" #include "ray/common/asio/asio_util.h" @@ -583,13 +584,12 @@ ray::Status NodeManager::RegisterGcs() { DoLocalGC(triggered_by_global_gc); should_local_gc_ = false; } - static int64_t version = 0; if (triggered_by_global_gc) { rpc::ResourcesData resources_data; resources_data.set_should_global_gc(true); syncer::RaySyncMessage msg; - msg.set_version(++version); + msg.set_version(absl::GetCurrentTimeNanos()); msg.set_node_id(self_node_id_.Binary()); msg.set_component_id(syncer::RayComponentId::COMMANDS); std::string serialized_msg; @@ -2647,7 +2647,6 @@ void NodeManager::Update(std::shared_ptr message) std::optional NodeManager::Snapshot( int64_t after_version, syncer::RayComponentId component_id) const { if (component_id == syncer::RayComponentId::SCHEDULER) { - static uint64_t version = 0; syncer::RaySyncMessage msg; rpc::ResourcesData resource_data; @@ -2660,7 +2659,7 @@ std::optional NodeManager::Snapshot( resource_data.set_node_manager_address(initial_config_.node_manager_address); resource_data.set_cluster_full_of_actors_detected(resource_deadlock_warned_ >= 1); - msg.set_version(++version); + msg.set_version(absl::GetCurrentTimeNanos()); msg.set_node_id(self_node_id_.Binary()); msg.set_component_id(syncer::RayComponentId::SCHEDULER); std::string serialized_msg; From 7a14e80e039deab51da1776b9882066fdde1b396 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 06:04:04 +0000 Subject: [PATCH 33/77] fix --- src/ray/common/ray_syncer/ray_syncer.cc | 30 ++++++++++++++----------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index 6f0839b4c6dd..e0ccb560af5d 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -358,20 +358,24 @@ bool RaySyncer::Register(RayComponentId component_id, } void RaySyncer::BroadcastMessage(std::shared_ptr message) { - // The message is stale. Just skip this one. - if (!node_state_->ConsumeMessage(message)) { - return; - } + io_context_.dispatch( + [this, message] { + // The message is stale. Just skip this one. + if (!node_state_->ConsumeMessage(message)) { + return; + } - if (upward_only_[message->component_id()]) { - for (auto &connection : upward_connections_) { - connection->PushToSendingQueue(message); - } - } else { - for (auto &connection : sync_connections_) { - connection.second->PushToSendingQueue(message); - } - } + if (upward_only_[message->component_id()]) { + for (auto &connection : upward_connections_) { + connection->PushToSendingQueue(message); + } + } else { + for (auto &connection : sync_connections_) { + connection.second->PushToSendingQueue(message); + } + } + }, + "RaySyncer.BroadcastMessage"); } void RaySyncer::BroadcastMessage(RayComponentId cid) { From e8daa87c413c63283e60a2831ce679e8df8185c3 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 06:32:20 +0000 Subject: [PATCH 34/77] add test case for ray syncer --- src/ray/common/ray_syncer/ray_syncer.h | 1 + src/ray/common/test/ray_syncer_test.cc | 32 ++++++++++++++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/src/ray/common/ray_syncer/ray_syncer.h b/src/ray/common/ray_syncer/ray_syncer.h index b7b9acb9a615..48217093de41 100644 --- a/src/ray/common/ray_syncer/ray_syncer.h +++ b/src/ray/common/ray_syncer/ray_syncer.h @@ -195,6 +195,7 @@ class RaySyncer { /// Test purpose friend struct SyncerServerTest; FRIEND_TEST(SyncerTest, Broadcast); + FRIEND_TEST(SyncerTest, Reconnect); FRIEND_TEST(SyncerTest, Test1To1); FRIEND_TEST(SyncerTest, Test1ToN); FRIEND_TEST(SyncerTest, TestMToN); diff --git a/src/ray/common/test/ray_syncer_test.cc b/src/ray/common/test/ray_syncer_test.cc index 2320140662b2..119844284622 100644 --- a/src/ray/common/test/ray_syncer_test.cc +++ b/src/ray/common/test/ray_syncer_test.cc @@ -499,6 +499,38 @@ TEST(SyncerTest, Test1To1) { ASSERT_LE(s2.GetNumConsumedMessages(s1.syncer->GetLocalNodeID()), max_sends + 3); } +TEST(SyncerTest, Reconnect) { + // This test covers the broadcast feature of ray syncer. + auto s1 = SyncerServerTest("19990", false); + auto s2 = SyncerServerTest("19991", true); + auto s3 = SyncerServerTest("19992", true); + + s1.syncer->Connect(MakeChannel("19992")); + + // Make sure the setup is correct + ASSERT_TRUE(s1.WaitUntil( + [&s1]() { + return s1.syncer->sync_connections_.size() == 1 && s1.snapshot_taken == 1; + }, + 5)); + + ASSERT_TRUE(s1.WaitUntil( + [&s3]() { + return s3.syncer->sync_connections_.size() == 1 && s3.snapshot_taken == 2; + }, + 5)); + s2.syncer->Connect(MakeChannel("19992")); + + ASSERT_TRUE(s1.WaitUntil( + [&s2]() { + return s2.syncer->sync_connections_.size() == 1 && s2.snapshot_taken == 2; + }, + 5)); + ASSERT_EQ(1, s3.syncer->upward_connections_.size()); + ASSERT_EQ(s2.syncer->GetLocalNodeID(), (*s3.syncer->upward_connections_.begin())->GetRemoteNodeID()); +} + + TEST(SyncerTest, Broadcast) { // This test covers the broadcast feature of ray syncer. auto s1 = SyncerServerTest("19990", false); From fb49bd8a982b5e1990ec4eaa8605d2b622bf746b Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 06:32:29 +0000 Subject: [PATCH 35/77] format --- src/ray/common/test/ray_syncer_test.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/ray/common/test/ray_syncer_test.cc b/src/ray/common/test/ray_syncer_test.cc index 119844284622..b1f4ef8a680f 100644 --- a/src/ray/common/test/ray_syncer_test.cc +++ b/src/ray/common/test/ray_syncer_test.cc @@ -527,10 +527,10 @@ TEST(SyncerTest, Reconnect) { }, 5)); ASSERT_EQ(1, s3.syncer->upward_connections_.size()); - ASSERT_EQ(s2.syncer->GetLocalNodeID(), (*s3.syncer->upward_connections_.begin())->GetRemoteNodeID()); + ASSERT_EQ(s2.syncer->GetLocalNodeID(), + (*s3.syncer->upward_connections_.begin())->GetRemoteNodeID()); } - TEST(SyncerTest, Broadcast) { // This test covers the broadcast feature of ray syncer. auto s1 = SyncerServerTest("19990", false); From cbffd67d1177b9f2faac39ed703506ad6d308dad Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Sat, 9 Apr 2022 06:32:52 +0000 Subject: [PATCH 36/77] up --- .bazelrc | 2 -- 1 file changed, 2 deletions(-) diff --git a/.bazelrc b/.bazelrc index 7e4f62020574..d103098fc384 100644 --- a/.bazelrc +++ b/.bazelrc @@ -9,8 +9,6 @@ build:windows --action_env=PATH build --compilation_mode=opt # Using C++ 17 on all platforms. build:linux --cxxopt="-std=c++17" -build:linux --cxxopt="-g" -build:linux --cxxopt="-ggdb" build:macos --cxxopt="-std=c++17" build:clang-cl --cxxopt="-std=c++17" build:msvc-cl --cxxopt="/std:c++17" From e73688e8e4bb6ffad65089f40f095b95af03943e Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 11 Apr 2022 01:57:53 +0000 Subject: [PATCH 37/77] add test --- .buildkite/pipeline.yml | 59 ++++++++++--------- python/ray/includes/ray_config.pxd | 2 + python/ray/includes/ray_config.pxi | 4 ++ python/ray/tests/test_metrics_agent.py | 7 ++- .../many_nodes_tests/app_config.yaml | 4 +- src/ray/common/ray_config_def.h | 2 +- 6 files changed, 45 insertions(+), 33 deletions(-) diff --git a/.buildkite/pipeline.yml b/.buildkite/pipeline.yml index c3d4c25b17cf..43316cab13ed 100644 --- a/.buildkite/pipeline.yml +++ b/.buildkite/pipeline.yml @@ -301,33 +301,6 @@ --test_tag_filters=release_unit release/... -- label: ":python: (Small & Client)" - conditions: ["RAY_CI_PYTHON_AFFECTED"] - commands: - - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT - - bazel test --config=ci $(./scripts/bazel_export_options) - --test_tag_filters=client_tests,small_size_python_tests - -- python/ray/tests/... -- label: ":python: (Large)" - conditions: ["RAY_CI_PYTHON_AFFECTED"] - parallelism: 3 - commands: - - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT - - . ./ci/travis/ci.sh test_large -- label: ":python: (Medium A-J)" - conditions: ["RAY_CI_PYTHON_AFFECTED"] - commands: - - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT - - bazel test --config=ci $(./scripts/bazel_export_options) - --test_tag_filters=-kubernetes,medium_size_python_tests_a_to_j - python/ray/tests/... -- label: ":python: (Medium K-Z)" - conditions: ["RAY_CI_PYTHON_AFFECTED"] - commands: - - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT - - bazel test --config=ci $(./scripts/bazel_export_options) - --test_tag_filters=-kubernetes,medium_size_python_tests_k_to_z - python/ray/tests/... - label: ":python: Debug Test" conditions: ["RAY_CI_PYTHON_AFFECTED"] commands: @@ -353,6 +326,38 @@ --test_env=CONDA_DEFAULT_ENV python/ray/tests/... +- label: ":construction: :python: (Small & Client)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT + - bazel test --config=ci $(./scripts/bazel_export_options) + --test_tag_filters=client_tests,small_size_python_tests + --test_env=RAY_use_ray_syncer=true + -- python/ray/tests/... +- label: ":construction: :python: (Large)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + parallelism: 3 + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT + - . ./ci/travis/ci.sh test_large +- label: ":construction: :python: (Medium A-J)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT + - bazel test --config=ci $(./scripts/bazel_export_options) + --test_tag_filters=-kubernetes,medium_size_python_tests_a_to_j + --test_env=RAY_use_ray_syncer=true + python/ray/tests/... +- label: ":construction: :python: (Medium K-Z)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT + - bazel test --config=ci $(./scripts/bazel_export_options) + --test_tag_filters=-kubernetes,medium_size_python_tests_k_to_z + --test_env=RAY_use_ray_syncer=true + python/ray/tests/... + + # https://github.com/ray-project/ray/issues/22460 #- label: ":python: (Privileged test)" #conditions: ["RAY_CI_PYTHON_AFFECTED"] diff --git a/python/ray/includes/ray_config.pxd b/python/ray/includes/ray_config.pxd index 32838a64242a..e9ac6d2ccd2b 100644 --- a/python/ray/includes/ray_config.pxd +++ b/python/ray/includes/ray_config.pxd @@ -66,3 +66,5 @@ cdef extern from "ray/common/ray_config.h" nogil: c_bool gcs_grpc_based_pubsub() const c_bool start_python_importer_thread() const + + c_bool use_ray_syncer() const diff --git a/python/ray/includes/ray_config.pxi b/python/ray/includes/ray_config.pxi index bf048fc03096..c65bfbc291c3 100644 --- a/python/ray/includes/ray_config.pxi +++ b/python/ray/includes/ray_config.pxi @@ -107,3 +107,7 @@ cdef class Config: @staticmethod def start_python_importer_thread(): return RayConfig.instance().start_python_importer_thread() + + @staticmethod + def use_ray_syncer(): + return RayConfig.instance().use_ray_syncer() diff --git a/python/ray/tests/test_metrics_agent.py b/python/ray/tests/test_metrics_agent.py index f0a50b903413..ec0fd65e3ab3 100644 --- a/python/ray/tests/test_metrics_agent.py +++ b/python/ray/tests/test_metrics_agent.py @@ -44,7 +44,6 @@ "ray_internal_num_spilled_tasks", # "ray_unintentional_worker_failures_total", # "ray_node_failure_total", - "ray_outbound_heartbeat_size_kb_sum", "ray_operation_count", "ray_operation_run_time_ms", "ray_operation_queue_time_ms", @@ -75,6 +74,9 @@ "ray_gcs_actors_count", ] +if not ray._raylet.Config.use_ray_syncer(): + _METRICS.append("ray_outbound_heartbeat_size_kb_sum") + # This list of metrics should be kept in sync with # ray/python/ray/autoscaler/_private/prom_metrics.py _AUTOSCALER_METRICS = [ @@ -136,7 +138,7 @@ def f(): # Generate some metrics for the placement group. pg = ray.util.placement_group(bundles=[{"CPU": 1}]) ray.get(pg.ready()) - print(ray.util.placement_group_table()) + ray.util.placement_group_table() ray.util.remove_placement_group(pg) @ray.remote @@ -153,7 +155,6 @@ async def ping(self): obj_refs = [f.remote(), a.ping.remote()] # Infeasible task b = f.options(resources={"a": 1}) - print(b) node_info_list = ray.nodes() prom_addresses = [] diff --git a/release/nightly_tests/many_nodes_tests/app_config.yaml b/release/nightly_tests/many_nodes_tests/app_config.yaml index a9622664bc60..4933776355c1 100644 --- a/release/nightly_tests/many_nodes_tests/app_config.yaml +++ b/release/nightly_tests/many_nodes_tests/app_config.yaml @@ -1,5 +1,5 @@ -base_image: "anyscale/ray:nightly-py37" -env_vars: {"RAY_gcs_server_rpc_server_thread_num": "8"} +base_image: "anyscale/ray:nightly-py38" +env_vars: {"RAY_gcs_server_rpc_server_thread_num": "8", "RAY_raylet_report_resources_period_milliseconds": "50"} debian_packages: [] python: diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 1ecd3f9bbd55..50dd1a6ace99 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -336,7 +336,7 @@ RAY_CONFIG(int32_t, gcs_rpc_server_reconnect_timeout_s, 60) RAY_CONFIG(int32_t, minimum_gcs_reconnect_interval_milliseconds, 5000) /// Feature flag to use the ray syncer for resource synchronization -RAY_CONFIG(bool, use_ray_syncer, true) +RAY_CONFIG(bool, use_ray_syncer, false) /// The interval at which the gcs client will check if the address of gcs service has /// changed. When the address changed, we will resubscribe again. From 25aae70fb881868205f89654e01669d8c1928de6 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 11 Apr 2022 02:01:09 +0000 Subject: [PATCH 38/77] format --- python/ray/tests/test_metrics_agent.py | 3 ++- release/nightly_tests/many_nodes_tests/app_config.yaml | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/python/ray/tests/test_metrics_agent.py b/python/ray/tests/test_metrics_agent.py index ec0fd65e3ab3..925f758cfa43 100644 --- a/python/ray/tests/test_metrics_agent.py +++ b/python/ray/tests/test_metrics_agent.py @@ -138,7 +138,7 @@ def f(): # Generate some metrics for the placement group. pg = ray.util.placement_group(bundles=[{"CPU": 1}]) ray.get(pg.ready()) - ray.util.placement_group_table() + print(ray.util.placement_group_table()) ray.util.remove_placement_group(pg) @ray.remote @@ -155,6 +155,7 @@ async def ping(self): obj_refs = [f.remote(), a.ping.remote()] # Infeasible task b = f.options(resources={"a": 1}) + print(b) node_info_list = ray.nodes() prom_addresses = [] diff --git a/release/nightly_tests/many_nodes_tests/app_config.yaml b/release/nightly_tests/many_nodes_tests/app_config.yaml index 4933776355c1..a9622664bc60 100644 --- a/release/nightly_tests/many_nodes_tests/app_config.yaml +++ b/release/nightly_tests/many_nodes_tests/app_config.yaml @@ -1,5 +1,5 @@ -base_image: "anyscale/ray:nightly-py38" -env_vars: {"RAY_gcs_server_rpc_server_thread_num": "8", "RAY_raylet_report_resources_period_milliseconds": "50"} +base_image: "anyscale/ray:nightly-py37" +env_vars: {"RAY_gcs_server_rpc_server_thread_num": "8"} debian_packages: [] python: From 8fe8e3e2a8b0badd659aa4f02857f59d43cf6f5f Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 11 Apr 2022 02:12:52 +0000 Subject: [PATCH 39/77] up --- .buildkite/pipeline.yml | 28 +++++++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/.buildkite/pipeline.yml b/.buildkite/pipeline.yml index 43316cab13ed..425184807e47 100644 --- a/.buildkite/pipeline.yml +++ b/.buildkite/pipeline.yml @@ -300,7 +300,33 @@ --build_tests_only --test_tag_filters=release_unit release/... - +- label: ":python: (Small & Client)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT + - bazel test --config=ci $(./scripts/bazel_export_options) + --test_tag_filters=client_tests,small_size_python_tests + -- python/ray/tests/... +- label: ":python: (Large)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + parallelism: 3 + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT + - . ./ci/travis/ci.sh test_large +- label: ":python: (Medium A-J)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT + - bazel test --config=ci $(./scripts/bazel_export_options) + --test_tag_filters=-kubernetes,medium_size_python_tests_a_to_j + python/ray/tests/... +- label: ":python: (Medium K-Z)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT + - bazel test --config=ci $(./scripts/bazel_export_options) + --test_tag_filters=-kubernetes,medium_size_python_tests_k_to_z + python/ray/tests/... - label: ":python: Debug Test" conditions: ["RAY_CI_PYTHON_AFFECTED"] commands: From 1503e101127d46def1e2aa44f5a3a393dae82e3d Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 11 Apr 2022 02:14:23 +0000 Subject: [PATCH 40/77] update --- src/ray/gcs/gcs_server/gcs_resource_manager.cc | 6 ------ src/ray/gcs/gcs_server/ray_syncer.h | 4 ---- 2 files changed, 10 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index 2905ebfc732f..9499983d9efd 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -14,8 +14,6 @@ #include "ray/gcs/gcs_server/gcs_resource_manager.h" -#include - #include "ray/common/ray_config.h" #include "ray/stats/metric_defs.h" @@ -37,10 +35,6 @@ void GcsResourceManager::Update(std::shared_ptr me rpc::ResourcesData resources; resources.ParseFromString(message->sync_message()); resources.set_node_id(message->node_id()); - // std::string json_str; - // google::protobuf::util::MessageToJsonString(resources, &json_str); - // RAY_LOG(ERROR) << "Received a message: (" << message->component_id() << ")\t" - // << json_str; auto node_id = NodeID::FromBinary(message->node_id()); if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { if (RayConfig::instance().gcs_actor_scheduling_enabled()) { diff --git a/src/ray/gcs/gcs_server/ray_syncer.h b/src/ray/gcs/gcs_server/ray_syncer.h index e827b5716b4a..431679ba6d26 100644 --- a/src/ray/gcs/gcs_server/ray_syncer.h +++ b/src/ray/gcs/gcs_server/ray_syncer.h @@ -111,10 +111,6 @@ class RaySyncer { static_assert(std::is_same_v || std::is_same_v, "unknown type"); - // std::string json_str; - // google::protobuf::util::MessageToJsonString(update, &json_str); - // RAY_LOG(ERROR) << "GcsRaySyncer: Received a message: " << json_str; - if constexpr (std::is_same_v) { resources_buffer_proto_.add_batch()->mutable_change()->Swap(&update); } else if constexpr (std::is_same_v) { From a0e2d6851994634b2f09310b84b4df9d759aa38f Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 11 Apr 2022 02:17:32 +0000 Subject: [PATCH 41/77] comments --- src/ray/raylet/node_manager.cc | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 67c513b719fa..7b1b242418fe 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -546,6 +546,7 @@ ray::Status NodeManager::RegisterGcs() { } if (RayConfig::instance().use_ray_syncer()) { + // Register resource manager and scheduler ray_syncer_.Register( syncer::RayComponentId::RESOURCE_MANAGER, this, @@ -557,6 +558,8 @@ ray::Status NodeManager::RegisterGcs() { nullptr, true, RayConfig::instance().raylet_report_loads_period_milliseconds()); + // Register a commands channel. + // It's only used for GC right now. ray_syncer_.Register(syncer::RayComponentId::COMMANDS, nullptr, this, false, 0); periodical_runner_.RunFnPeriodically( [this] { @@ -2646,6 +2649,11 @@ void NodeManager::Update(std::shared_ptr message) std::optional NodeManager::Snapshot( int64_t after_version, syncer::RayComponentId component_id) const { + // Right now snapshot is put in NodeManager which in long-term, them should + // be put into each component directly. + // The current blocker of doing this is some fields in NodeManager + // is being used. + // TODO(iycheng): Move the logic into the components directly. if (component_id == syncer::RayComponentId::SCHEDULER) { syncer::RaySyncMessage msg; rpc::ResourcesData resource_data; From 8f421c0e744c3f8a9fa5a680f74b16c8cece3baa Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 14 Apr 2022 22:34:27 +0000 Subject: [PATCH 42/77] fix some --- .buildkite/pipeline.yml | 6 +++--- src/ray/gcs/gcs_server/gcs_server.cc | 16 +++++++++------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/.buildkite/pipeline.yml b/.buildkite/pipeline.yml index 425184807e47..f83d6e8d63a8 100644 --- a/.buildkite/pipeline.yml +++ b/.buildkite/pipeline.yml @@ -352,7 +352,7 @@ --test_env=CONDA_DEFAULT_ENV python/ray/tests/... -- label: ":construction: :python: (Small & Client)" +- label: ":construction: :python: (syncer) (Small & Client)" conditions: ["RAY_CI_PYTHON_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT @@ -360,7 +360,7 @@ --test_tag_filters=client_tests,small_size_python_tests --test_env=RAY_use_ray_syncer=true -- python/ray/tests/... -- label: ":construction: :python: (Large)" +- label: ":construction: :python: (syncer) (Large)" conditions: ["RAY_CI_PYTHON_AFFECTED"] parallelism: 3 commands: @@ -374,7 +374,7 @@ --test_tag_filters=-kubernetes,medium_size_python_tests_a_to_j --test_env=RAY_use_ray_syncer=true python/ray/tests/... -- label: ":construction: :python: (Medium K-Z)" +- label: ":construction: :python: (syncer) (Medium K-Z)" conditions: ["RAY_CI_PYTHON_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 625f00e68962..c4272d0b4a57 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -437,13 +437,15 @@ void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { }); for (const auto &pair : gcs_init_data.Nodes()) { - rpc::Address address; - address.set_raylet_id(pair.second.node_id()); - address.set_ip_address(pair.second.node_manager_address()); - address.set_port(pair.second.node_manager_port()); - - auto raylet_client = raylet_client_pool_->GetOrConnectByAddress(address); - ray_syncer_->Connect(raylet_client->GetChannel()); + if(pair.second.state() == rpc::GcsNodeInfo_GcsNodeState::GcsNodeInfo_GcsNodeState_ALIVE) { + rpc::Address address; + address.set_raylet_id(pair.second.node_id()); + address.set_ip_address(pair.second.node_manager_address()); + address.set_port(pair.second.node_manager_port()); + + auto raylet_client = raylet_client_pool_->GetOrConnectByAddress(address); + ray_syncer_->Connect(raylet_client->GetChannel()); + } } } else { /* From 63da67400565f891ca33448eb17bb3fe84fd38fe Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 14 Apr 2022 22:35:28 +0000 Subject: [PATCH 43/77] format --- src/ray/gcs/gcs_server/gcs_server.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index c4272d0b4a57..511c370052b8 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -437,7 +437,8 @@ void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { }); for (const auto &pair : gcs_init_data.Nodes()) { - if(pair.second.state() == rpc::GcsNodeInfo_GcsNodeState::GcsNodeInfo_GcsNodeState_ALIVE) { + if (pair.second.state() == + rpc::GcsNodeInfo_GcsNodeState::GcsNodeInfo_GcsNodeState_ALIVE) { rpc::Address address; address.set_raylet_id(pair.second.node_id()); address.set_ip_address(pair.second.node_manager_address()); From 08c91b86f593aab4bb147f17b6c592fb798d4e04 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 21 Apr 2022 04:31:36 +0000 Subject: [PATCH 44/77] fix compiling and some comments --- src/ray/common/ray_syncer/ray_syncer.cc | 10 +--------- src/ray/common/ray_syncer/ray_syncer.h | 5 ----- src/ray/gcs/gcs_server/gcs_resource_manager.cc | 2 +- src/ray/gcs/gcs_server/gcs_resource_manager.h | 2 +- src/ray/raylet/node_manager.cc | 4 ++-- src/ray/raylet/node_manager.h | 4 ++-- 6 files changed, 7 insertions(+), 20 deletions(-) diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index 16512ae88fbf..2e14302139d2 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -370,7 +370,7 @@ void RaySyncer::BroadcastMessage(std::shared_ptr message) io_context_.dispatch( [this, message] { // The message is stale. Just skip this one. - if (!node_state_->ConsumeMessage(message)) { + if (!node_state_->ConsumeSyncMessage(message)) { return; } if (upward_only_[message->component_id()]) { @@ -386,14 +386,6 @@ void RaySyncer::BroadcastMessage(std::shared_ptr message) "RaySyncer.BroadcastMessage"); } -void RaySyncer::BroadcastMessage(RayComponentId cid) { - auto snapshot = node_state_->GetSnapshot(cid); - if (snapshot) { - RAY_CHECK(snapshot->node_id() == GetLocalNodeID()); - BroadcastMessage(std::make_shared(std::move(*snapshot))); - } -} - grpc::ServerUnaryReactor *RaySyncerService::StartSync( grpc::CallbackServerContext *context, const StartSyncRequest *request, diff --git a/src/ray/common/ray_syncer/ray_syncer.h b/src/ray/common/ray_syncer/ray_syncer.h index 87acc4473ad0..c12290d6bd98 100644 --- a/src/ray/common/ray_syncer/ray_syncer.h +++ b/src/ray/common/ray_syncer/ray_syncer.h @@ -144,11 +144,6 @@ class RaySyncer { /// \param message The message to be broadcasted. void BroadcastMessage(std::shared_ptr message); - /// Function to force a broadcasting of messages come from a component_id - /// - /// \param component_id The component which need to send the broadcasting messages - void BroadcastMessage(ray::rpc::syncer::RayComponentId component_id); - /// Get the current node id. const std::string &GetLocalNodeID() const { return local_node_id_; } diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index 9499983d9efd..fddb241fb5ac 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -28,7 +28,7 @@ GcsResourceManager::GcsResourceManager( gcs_table_storage_(gcs_table_storage), cluster_resource_manager_(cluster_resource_manager) {} -void GcsResourceManager::Update(std::shared_ptr message) { +void GcsResourceManager::ConsumeSyncMessage(std::shared_ptr message) { // Make sure thread safety. io_context_.post( [this, message]() { diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.h b/src/ray/gcs/gcs_server/gcs_resource_manager.h index 40819968ec6d..0e590751ba2b 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.h +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.h @@ -58,7 +58,7 @@ class GcsResourceManager : public rpc::NodeResourceInfoHandler, virtual ~GcsResourceManager() {} /// Handle the resource update. - void Update(std::shared_ptr message) override; + void ConsumeSyncMessage(std::shared_ptr message) override; /// Handle get resource rpc request. void HandleGetResources(const rpc::GetResourcesRequest &request, diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 8aafb6dc6ce2..041d0ef9cde8 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2637,7 +2637,7 @@ void NodeManager::RecordMetrics() { object_directory_->RecordMetrics(duration_ms); } -void NodeManager::Update(std::shared_ptr message) { +void NodeManager::ConsumeSyncMessage(std::shared_ptr message) { if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { rpc::ResourcesData data; data.ParseFromString(message->sync_message()); @@ -2652,7 +2652,7 @@ void NodeManager::Update(std::shared_ptr message) } } -std::optional NodeManager::Snapshot( +std::optional NodeManager::CreateSyncMessage( int64_t after_version, syncer::RayComponentId component_id) const { // Right now snapshot is put in NodeManager which in long-term, them should // be put into each component directly. diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index a9a7ad1cfba7..be222b02cedf 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -191,9 +191,9 @@ class NodeManager : public rpc::NodeManagerServiceHandler, /// Get the port of the node manager rpc server. int GetServerPort() const { return node_manager_server_.GetPort(); } - void Update(std::shared_ptr message) override; + void ConsumeSyncMessage(std::shared_ptr message) override; - std::optional Snapshot( + std::optional CreateSyncMessage( int64_t after_version, syncer::RayComponentId component_id) const override; int GetObjectManagerPort() const { return object_manager_.GetServerPort(); } From f07cf339fb75fd57f80915c183e7756b91d546e3 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 21 Apr 2022 04:43:15 +0000 Subject: [PATCH 45/77] fix compiling --- src/ray/raylet/node_manager.cc | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 041d0ef9cde8..4f8da6357202 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1785,7 +1785,9 @@ void NodeManager::HandleCommitBundleResources( << GetDebugStringForBundles(bundle_specs); placement_group_resource_manager_->CommitBundles(bundle_specs); if (RayConfig::instance().use_ray_syncer()) { - ray_syncer_.BroadcastMessage(syncer::RayComponentId::RESOURCE_MANAGER); + auto sync_message = CreateSyncMessage(0, syncer::RayComponentId::RESOURCE_MANAGER); + RAY_CHECK(sync_message); + ray_syncer_.BroadcastMessage(std::make_shared(std::move(*sync_message))); } send_reply_callback(Status::OK(), nullptr, nullptr); @@ -1825,7 +1827,9 @@ void NodeManager::HandleCancelResourceReserve( // Return bundle resources. placement_group_resource_manager_->ReturnBundle(bundle_spec); if (RayConfig::instance().use_ray_syncer()) { - ray_syncer_.BroadcastMessage(syncer::RayComponentId::RESOURCE_MANAGER); + auto sync_message = CreateSyncMessage(0, syncer::RayComponentId::RESOURCE_MANAGER); + RAY_CHECK(sync_message); + ray_syncer_.BroadcastMessage(std::make_shared(std::move(*sync_message))); } cluster_task_manager_->ScheduleAndDispatchTasks(); send_reply_callback(Status::OK(), nullptr, nullptr); From 91f2f93c10acae351438345c94c63f9677760a34 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 21 Apr 2022 04:47:25 +0000 Subject: [PATCH 46/77] format --- src/ray/gcs/gcs_server/gcs_resource_manager.cc | 3 ++- src/ray/raylet/node_manager.cc | 9 ++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index fddb241fb5ac..a14f52940224 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -28,7 +28,8 @@ GcsResourceManager::GcsResourceManager( gcs_table_storage_(gcs_table_storage), cluster_resource_manager_(cluster_resource_manager) {} -void GcsResourceManager::ConsumeSyncMessage(std::shared_ptr message) { +void GcsResourceManager::ConsumeSyncMessage( + std::shared_ptr message) { // Make sure thread safety. io_context_.post( [this, message]() { diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 4f8da6357202..b662501e3c67 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1787,7 +1787,8 @@ void NodeManager::HandleCommitBundleResources( if (RayConfig::instance().use_ray_syncer()) { auto sync_message = CreateSyncMessage(0, syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); - ray_syncer_.BroadcastMessage(std::make_shared(std::move(*sync_message))); + ray_syncer_.BroadcastMessage( + std::make_shared(std::move(*sync_message))); } send_reply_callback(Status::OK(), nullptr, nullptr); @@ -1829,7 +1830,8 @@ void NodeManager::HandleCancelResourceReserve( if (RayConfig::instance().use_ray_syncer()) { auto sync_message = CreateSyncMessage(0, syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); - ray_syncer_.BroadcastMessage(std::make_shared(std::move(*sync_message))); + ray_syncer_.BroadcastMessage( + std::make_shared(std::move(*sync_message))); } cluster_task_manager_->ScheduleAndDispatchTasks(); send_reply_callback(Status::OK(), nullptr, nullptr); @@ -2641,7 +2643,8 @@ void NodeManager::RecordMetrics() { object_directory_->RecordMetrics(duration_ms); } -void NodeManager::ConsumeSyncMessage(std::shared_ptr message) { +void NodeManager::ConsumeSyncMessage( + std::shared_ptr message) { if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { rpc::ResourcesData data; data.ParseFromString(message->sync_message()); From 1984ea3b3705af1d74577ec6a2f402fd611b9aea Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 21 Apr 2022 04:53:04 +0000 Subject: [PATCH 47/77] update --- src/ray/gcs/gcs_server/gcs_resource_manager.cc | 9 +++++++-- src/ray/gcs/gcs_server/gcs_server.cc | 4 ++-- src/ray/gcs/gcs_server/gcs_server.h | 7 ++++++- 3 files changed, 15 insertions(+), 5 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index a14f52940224..d7595d97b561 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -30,8 +30,13 @@ GcsResourceManager::GcsResourceManager( void GcsResourceManager::ConsumeSyncMessage( std::shared_ptr message) { - // Make sure thread safety. - io_context_.post( + // ConsumeSyncMessage is called by ray_syncer which might not run + // in a dedicated thread for performance. + // GcsResourceManager is a module always run in the main thread, so we just + // delegate the work to the main thread for thread safety. + // Ideally, all public api in GcsResourceManager need to be put into this + // io context for thread safety. + io_context_.dispatch( [this, message]() { rpc::ResourcesData resources; resources.ParseFromString(message->sync_message()); diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 511c370052b8..383bca895a24 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -424,9 +424,9 @@ void GcsServer::StoreGcsServerAddressInRedis() { void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { if (RayConfig::instance().use_ray_syncer()) { - ray_syncer_node_id_ = NodeID::FromRandom(); + gcs_node_id_ = NodeID::FromRandom(); ray_syncer_ = std::make_unique(ray_syncer_io_context_, - ray_syncer_node_id_.Binary()); + gcs_node_id_.Binary()); ray_syncer_->Register( syncer::RayComponentId::RESOURCE_MANAGER, nullptr, gcs_resource_manager_.get()); ray_syncer_->Register( diff --git a/src/ray/gcs/gcs_server/gcs_server.h b/src/ray/gcs/gcs_server/gcs_server.h index 6a34563ed066..3d7cec671939 100644 --- a/src/ray/gcs/gcs_server/gcs_server.h +++ b/src/ray/gcs/gcs_server/gcs_server.h @@ -208,14 +208,19 @@ class GcsServer { /// Stats handler and service. std::unique_ptr stats_handler_; std::unique_ptr stats_service_; + /// Synchronization service for ray. + /// TODO(iycheng): Deprecate this gcs_ray_syncer_ one once we roll out + /// to ray_syncer_. std::unique_ptr gcs_ray_syncer_; /// Ray Syncer realted fields. std::unique_ptr ray_syncer_; std::unique_ptr ray_syncer_thread_; instrumented_io_context ray_syncer_io_context_; - NodeID ray_syncer_node_id_; + + /// The node id of GCS. + NodeID gcs_node_id_; /// The gcs worker manager. std::unique_ptr gcs_worker_manager_; From 575009a3e2feb737ba0ac2b0c7859b9c63bb7bf6 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 21 Apr 2022 18:05:49 +0000 Subject: [PATCH 48/77] update --- .buildkite/pipeline.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.buildkite/pipeline.yml b/.buildkite/pipeline.yml index dd916b5edcf6..905c7b8ce659 100644 --- a/.buildkite/pipeline.yml +++ b/.buildkite/pipeline.yml @@ -367,7 +367,7 @@ commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT - . ./ci/travis/ci.sh test_large -- label: ":construction: :python: (Medium A-J)" +- label: ":construction: :python: (syncer) (Medium A-J)" conditions: ["RAY_CI_PYTHON_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT From 78e24329cced09078c575d84f2540e14c48ab0bf Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 21 Apr 2022 20:11:39 +0000 Subject: [PATCH 49/77] update --- src/ray/common/ray_syncer/ray_syncer.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index 2e14302139d2..2e77e2cfccc3 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -314,7 +314,7 @@ void RaySyncer::Connect(std::unique_ptr connection) { if (upward_only_[message->component_id()] && !is_upward_conn) { continue; } - RAY_CHECK(conn.PushToSendingQueue(message)); + conn.PushToSendingQueue(message); } } }, From 001c413129465ef65a593338636cf6dce45936c3 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 21 Apr 2022 20:49:35 +0000 Subject: [PATCH 50/77] format --- src/ray/raylet/node_manager.cc | 53 ++++++-------- .../scheduling/local_resource_manager.cc | 73 +++++++++++++++---- .../scheduling/local_resource_manager.h | 9 ++- 3 files changed, 85 insertions(+), 50 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index b662501e3c67..192d3f7cc13b 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -551,19 +551,29 @@ ray::Status NodeManager::RegisterGcs() { if (RayConfig::instance().use_ray_syncer()) { // Register resource manager and scheduler ray_syncer_.Register( - syncer::RayComponentId::RESOURCE_MANAGER, - this, - this, - false, - RayConfig::instance().raylet_report_resources_period_milliseconds()); - ray_syncer_.Register(syncer::RayComponentId::SCHEDULER, - this, - nullptr, - true, - RayConfig::instance().raylet_report_loads_period_milliseconds()); + /* component_id */ syncer::RayComponentId::RESOURCE_MANAGER, + /* reporter */ &cluster_resource_scheduler_->GetLocalResourceManager(), + /* receiver */ this, + /* upward_only */ false, + /* pull_from_reporter_interval_ms */ + RayConfig::instance() + .raylet_report_resources_period_milliseconds()); + ray_syncer_.Register( + /* component_id */ syncer::RayComponentId::SCHEDULER, + /* reporter */ this, + /* receiver */ nullptr, + /* upward_only */ true, + /* pull_from_reporter_interval_ms */ + RayConfig::instance() + .raylet_report_loads_period_milliseconds()); // Register a commands channel. // It's only used for GC right now. - ray_syncer_.Register(syncer::RayComponentId::COMMANDS, nullptr, this, false, 0); + ray_syncer_.Register( + /* component_id */ syncer::RayComponentId::COMMANDS, + /* reporter */ nullptr, + /* receiver */ this, + /* upward_only */ false, + /* pull_from_reporter_interval_ms */ 0); periodical_runner_.RunFnPeriodically( [this] { // If plasma store is under high pressure, we should try to schedule a global @@ -2686,27 +2696,8 @@ std::optional NodeManager::CreateSyncMessage( RAY_CHECK(resource_data.SerializeToString(&serialized_msg)); msg.set_sync_message(std::move(serialized_msg)); return std::make_optional(std::move(msg)); - } else if (component_id == syncer::RayComponentId::RESOURCE_MANAGER) { - auto &local = cluster_resource_scheduler_->GetLocalResourceManager(); - - if (local.Version() <= after_version) { - return std::nullopt; - } - - syncer::RaySyncMessage msg; - rpc::ResourcesData resource_data; - local.FillResourceUsage(resource_data, true); - resource_data.set_node_id(self_node_id_.Binary()); - resource_data.set_node_manager_address(initial_config_.node_manager_address); - - msg.set_node_id(self_node_id_.Binary()); - msg.set_version(local.Version()); - msg.set_component_id(syncer::RayComponentId::RESOURCE_MANAGER); - std::string serialized_msg; - RAY_CHECK(resource_data.SerializeToString(&serialized_msg)); - msg.set_sync_message(std::move(serialized_msg)); - return std::make_optional(std::move(msg)); } else { + RAY_CHECK(false) << "Invalid component id: " << component_id; return std::nullopt; } } diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 4dd5a4208d9b..884edd816288 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -315,53 +315,48 @@ void LocalResourceManager::UpdateAvailableObjectStoreMemResource() { } } -void LocalResourceManager::FillResourceUsage(rpc::ResourcesData &resources_data, - bool for_ray_syncer) { - if (!for_ray_syncer) { - UpdateAvailableObjectStoreMemResource(); - } +void LocalResourceManager::FillResourceUsage(rpc::ResourcesData &resources_data) { + UpdateAvailableObjectStoreMemResource(); NodeResources resources = ToNodeResources(local_resources_); // Initialize if last report resources is empty. - if (!for_ray_syncer && !last_report_resources_) { + if (!last_report_resources_) { NodeResources node_resources = ResourceMapToNodeResources({{}}, {{}}); last_report_resources_.reset(new NodeResources(node_resources)); } + for (auto entry : resources.total.ToMap()) { auto resource_id = entry.first; auto label = ResourceID(resource_id).Binary(); auto total = entry.second; auto available = resources.available.Get(resource_id); - auto last_total = - for_ray_syncer ? total : last_report_resources_->total.Get(resource_id); - auto last_available = - for_ray_syncer ? available : last_report_resources_->available.Get(resource_id); + auto last_total = last_report_resources_->total.Get(resource_id); + auto last_available = last_report_resources_->available.Get(resource_id); // Note: available may be negative, but only report positive to GCS. - if (for_ray_syncer || (available != last_available && available > 0)) { + if (available != last_available && available > 0) { resources_data.set_resources_available_changed(true); (*resources_data.mutable_resources_available())[label] = available.Double(); } - if (for_ray_syncer || (total != last_total)) { + if (total != last_total) { (*resources_data.mutable_resources_total())[label] = total.Double(); } } if (get_pull_manager_at_capacity_ != nullptr) { resources.object_pulls_queued = get_pull_manager_at_capacity_(); - if (for_ray_syncer || - (last_report_resources_->object_pulls_queued != resources.object_pulls_queued)) { + if (last_report_resources_->object_pulls_queued != resources.object_pulls_queued) { resources_data.set_object_pulls_queued(resources.object_pulls_queued); resources_data.set_resources_available_changed(true); } } - if (!for_ray_syncer && resources != *last_report_resources_.get()) { + if (resources != *last_report_resources_.get()) { last_report_resources_.reset(new NodeResources(resources)); } - if (for_ray_syncer || !RayConfig::instance().enable_light_weight_resource_report()) { + if (!RayConfig::instance().enable_light_weight_resource_report()) { resources_data.set_resources_available_changed(true); } } @@ -370,6 +365,52 @@ double LocalResourceManager::GetLocalAvailableCpus() const { return local_resources_.available.Sum(ResourceID::CPU()).Double(); } +std::optional LocalResourceManager::CreateSyncMessage( + int64_t after_version, syncer::RayComponentId component_id) const { + RAY_CHECK(component_id == syncer::RayComponentId::RESOURCE_MANAGER); + // We check the memory inside version, so version is not a const function. + // Ideally, we need to move the memory check somewhere else. + // TODO(iycheng): Make version as a const function. + auto curr_version = const_cast(this)->Version(); + if (curr_version <= after_version) { + return std::nullopt; + } + + syncer::RaySyncMessage msg; + rpc::ResourcesData resources_data; + + resources_data.set_node_id(local_node_id_.Binary()); + + NodeResources resources = ToNodeResources(local_resources_); + + for (auto entry : resources.total.ToMap()) { + auto resource_id = entry.first; + auto label = ResourceID(resource_id).Binary(); + auto total = entry.second; + auto available = resources.available.Get(resource_id); + + resources_data.set_resources_available_changed(true); + (*resources_data.mutable_resources_available())[label] = available.Double(); + (*resources_data.mutable_resources_total())[label] = total.Double(); + } + + if (get_pull_manager_at_capacity_ != nullptr) { + resources.object_pulls_queued = get_pull_manager_at_capacity_(); + resources_data.set_object_pulls_queued(resources.object_pulls_queued); + resources_data.set_resources_available_changed(true); + } + + resources_data.set_resources_available_changed(true); + + msg.set_node_id(local_node_id_.Binary()); + msg.set_version(curr_version); + msg.set_component_id(component_id); + std::string serialized_msg; + RAY_CHECK(resources_data.SerializeToString(&serialized_msg)); + msg.set_sync_message(std::move(serialized_msg)); + return std::make_optional(std::move(msg)); +} + ray::gcs::NodeResourceInfoAccessor::ResourceMap LocalResourceManager::GetResourceTotals( const absl::flat_hash_map &resource_map_filter) const { ray::gcs::NodeResourceInfoAccessor::ResourceMap map; diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index 14ff14403184..e31c53d8d162 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -22,6 +22,7 @@ #include "absl/container/flat_hash_map.h" #include "absl/container/flat_hash_set.h" +#include "ray/common/ray_syncer/ray_syncer.h" #include "ray/common/task/scheduling_resources.h" #include "ray/gcs/gcs_client/accessor.h" #include "ray/gcs/gcs_client/gcs_client.h" @@ -37,7 +38,7 @@ namespace ray { /// it also supports creating a new resource or delete an existing resource. /// Whenever the resouce changes, it notifies the subscriber of the change. /// This class is not thread safe. -class LocalResourceManager { +class LocalResourceManager : public syncer::ReporterInterface { public: LocalResourceManager( scheduling::NodeID local_node_id, @@ -118,8 +119,7 @@ class LocalResourceManager { /// /// \param Output parameter. `resources_available` and `resources_total` are the only /// fields used. - /// \param for_ray_syncer. The resource report is for ray syncer. - void FillResourceUsage(rpc::ResourcesData &resources_data, bool for_ray_syncer = false); + void FillResourceUsage(rpc::ResourcesData &resources_data); /// Populate a UpdateResourcesRequest. This is inteneded to update the /// resource totals on a node when a custom resource is created or deleted @@ -151,6 +151,9 @@ class LocalResourceManager { /// \return true, if exist. otherwise, false. bool ResourcesExist(scheduling::ResourceID resource_id) const; + std::optional CreateSyncMessage( + int64_t after_version, syncer::RayComponentId component_id) const override; + private: /// Notify the subscriber that the local resouces has changed. void OnResourceChanged(); From ff520abd7eecf74235930a34a4d6697b9b27cb8a Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 21 Apr 2022 20:49:44 +0000 Subject: [PATCH 51/77] format --- src/ray/raylet/node_manager.cc | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 192d3f7cc13b..928fbac08003 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -556,16 +556,14 @@ ray::Status NodeManager::RegisterGcs() { /* receiver */ this, /* upward_only */ false, /* pull_from_reporter_interval_ms */ - RayConfig::instance() - .raylet_report_resources_period_milliseconds()); + RayConfig::instance().raylet_report_resources_period_milliseconds()); ray_syncer_.Register( /* component_id */ syncer::RayComponentId::SCHEDULER, /* reporter */ this, /* receiver */ nullptr, /* upward_only */ true, /* pull_from_reporter_interval_ms */ - RayConfig::instance() - .raylet_report_loads_period_milliseconds()); + RayConfig::instance().raylet_report_loads_period_milliseconds()); // Register a commands channel. // It's only used for GC right now. ray_syncer_.Register( From d0352fec4dfcbbf682b4345f526046c67475bfaf Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 21 Apr 2022 21:35:57 +0000 Subject: [PATCH 52/77] format --- src/ray/raylet/node_manager.cc | 74 +++++++++++++--------------------- src/ray/raylet/node_manager.h | 2 + 2 files changed, 31 insertions(+), 45 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 928fbac08003..377f26d71d0f 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -576,28 +576,7 @@ ray::Status NodeManager::RegisterGcs() { [this] { // If plasma store is under high pressure, we should try to schedule a global // gc. - bool plasma_high_pressure = - object_manager_.GetUsedMemoryPercentage() > high_plasma_storage_usage_; - if (plasma_high_pressure && global_gc_throttler_.AbleToRun()) { - TriggerGlobalGC(); - } - - // Set the global gc bit on the outgoing heartbeat message. - bool triggered_by_global_gc = false; - if (should_global_gc_) { - triggered_by_global_gc = true; - should_global_gc_ = false; - global_gc_throttler_.RunNow(); - } - - // Trigger local GC if needed. This throttles the frequency of local GC calls - // to at most once per heartbeat interval. - if ((should_local_gc_ || (absl::GetCurrentTimeNanos() - local_gc_run_time_ns_ > - local_gc_interval_ns_)) && - local_gc_throttler_.AbleToRun()) { - DoLocalGC(triggered_by_global_gc); - should_local_gc_ = false; - } + auto triggered_by_global_gc = TryLocalGC(); if (triggered_by_global_gc) { rpc::ResourcesData resources_data; @@ -696,30 +675,8 @@ void NodeManager::FillResourceReport(rpc::ResourcesData &resources_data) { if (RayConfig::instance().gcs_actor_scheduling_enabled()) { FillNormalTaskResourceUsage(resources_data); } - // If plasma store is under high pressure, we should try to schedule a global gc. - bool plasma_high_pressure = - object_manager_.GetUsedMemoryPercentage() > high_plasma_storage_usage_; - if (plasma_high_pressure && global_gc_throttler_.AbleToRun()) { - TriggerGlobalGC(); - } - - // Set the global gc bit on the outgoing heartbeat message. - bool triggered_by_global_gc = false; - if (should_global_gc_) { - resources_data.set_should_global_gc(true); - triggered_by_global_gc = true; - should_global_gc_ = false; - global_gc_throttler_.RunNow(); - } - // Trigger local GC if needed. This throttles the frequency of local GC calls - // to at most once per heartbeat interval. - if ((should_local_gc_ || - (absl::GetCurrentTimeNanos() - local_gc_run_time_ns_ > local_gc_interval_ns_)) && - local_gc_throttler_.AbleToRun()) { - DoLocalGC(triggered_by_global_gc); - should_local_gc_ = false; - } + resources_data.set_should_global_gc(TryLocalGC()); } void NodeManager::DoLocalGC(bool triggered_by_global_gc) { @@ -2622,6 +2579,33 @@ void NodeManager::HandleGlobalGC(const rpc::GlobalGCRequest &request, TriggerGlobalGC(); } +bool NodeManager::TryLocalGC() { + // If plasma store is under high pressure, we should try to schedule a global gc. + bool plasma_high_pressure = + object_manager_.GetUsedMemoryPercentage() > high_plasma_storage_usage_; + if (plasma_high_pressure && global_gc_throttler_.AbleToRun()) { + TriggerGlobalGC(); + } + + // Set the global gc bit on the outgoing heartbeat message. + bool triggered_by_global_gc = false; + if (should_global_gc_) { + triggered_by_global_gc = true; + should_global_gc_ = false; + global_gc_throttler_.RunNow(); + } + + // Trigger local GC if needed. This throttles the frequency of local GC calls + // to at most once per heartbeat interval. + if ((should_local_gc_ || + (absl::GetCurrentTimeNanos() - local_gc_run_time_ns_ > local_gc_interval_ns_)) && + local_gc_throttler_.AbleToRun()) { + DoLocalGC(triggered_by_global_gc); + should_local_gc_ = false; + } + return triggered_by_global_gc; +} + void NodeManager::TriggerGlobalGC() { should_global_gc_ = true; // We won't see our own request, so trigger local GC in the next heartbeat. diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index be222b02cedf..39fcb5725148 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -624,6 +624,8 @@ class NodeManager : public rpc::NodeManagerServiceHandler, rpc::WorkerExitType disconnect_type = rpc::WorkerExitType::SYSTEM_ERROR_EXIT, const rpc::RayException *creation_task_exception = nullptr); + bool TryLocalGC(); + /// ID of this node. NodeID self_node_id_; /// The user-given identifier or name of this node. From 4ea66a7918cd7e9ae2e7b74da10748b13fdfaaae Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 21 Apr 2022 22:55:15 +0000 Subject: [PATCH 53/77] fix large test --- .buildkite/pipeline.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.buildkite/pipeline.yml b/.buildkite/pipeline.yml index 905c7b8ce659..4bd7f7b0b79f 100644 --- a/.buildkite/pipeline.yml +++ b/.buildkite/pipeline.yml @@ -365,8 +365,8 @@ conditions: ["RAY_CI_PYTHON_AFFECTED"] parallelism: 3 commands: - - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/travis/upload_build_info.sh; fi }; trap cleanup EXIT - - . ./ci/travis/ci.sh test_large + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - RAY_use_ray_syncer=true . ./ci/ci.sh test_large - label: ":construction: :python: (syncer) (Medium A-J)" conditions: ["RAY_CI_PYTHON_AFFECTED"] commands: From 2e03a4808524541092363b8d8d79a51ae5081376 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 22 Apr 2022 01:52:08 +0000 Subject: [PATCH 54/77] update --- src/ray/raylet/node_manager.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 377f26d71d0f..79d9c5403299 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1750,7 +1750,7 @@ void NodeManager::HandleCommitBundleResources( << GetDebugStringForBundles(bundle_specs); placement_group_resource_manager_->CommitBundles(bundle_specs); if (RayConfig::instance().use_ray_syncer()) { - auto sync_message = CreateSyncMessage(0, syncer::RayComponentId::RESOURCE_MANAGER); + auto sync_message = cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage(, syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); ray_syncer_.BroadcastMessage( std::make_shared(std::move(*sync_message))); @@ -1793,7 +1793,7 @@ void NodeManager::HandleCancelResourceReserve( // Return bundle resources. placement_group_resource_manager_->ReturnBundle(bundle_spec); if (RayConfig::instance().use_ray_syncer()) { - auto sync_message = CreateSyncMessage(0, syncer::RayComponentId::RESOURCE_MANAGER); + auto sync_message = cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage(, syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); ray_syncer_.BroadcastMessage( std::make_shared(std::move(*sync_message))); From fb79b6cca588625a0be732f60f17888a9cf79a66 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 22 Apr 2022 01:52:59 +0000 Subject: [PATCH 55/77] format --- src/ray/raylet/node_manager.cc | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 79d9c5403299..7b4452008a18 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1750,7 +1750,9 @@ void NodeManager::HandleCommitBundleResources( << GetDebugStringForBundles(bundle_specs); placement_group_resource_manager_->CommitBundles(bundle_specs); if (RayConfig::instance().use_ray_syncer()) { - auto sync_message = cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage(, syncer::RayComponentId::RESOURCE_MANAGER); + auto sync_message = + cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage( + , syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); ray_syncer_.BroadcastMessage( std::make_shared(std::move(*sync_message))); @@ -1793,7 +1795,9 @@ void NodeManager::HandleCancelResourceReserve( // Return bundle resources. placement_group_resource_manager_->ReturnBundle(bundle_spec); if (RayConfig::instance().use_ray_syncer()) { - auto sync_message = cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage(, syncer::RayComponentId::RESOURCE_MANAGER); + auto sync_message = + cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage( + , syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); ray_syncer_.BroadcastMessage( std::make_shared(std::move(*sync_message))); From 351c4bd7c5055ef42755f307bf6c189fc40df771 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 22 Apr 2022 01:54:08 +0000 Subject: [PATCH 56/77] format --- src/ray/raylet/node_manager.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 7b4452008a18..f0d042dc41c8 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1752,7 +1752,7 @@ void NodeManager::HandleCommitBundleResources( if (RayConfig::instance().use_ray_syncer()) { auto sync_message = cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage( - , syncer::RayComponentId::RESOURCE_MANAGER); + 0, syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); ray_syncer_.BroadcastMessage( std::make_shared(std::move(*sync_message))); @@ -1797,7 +1797,7 @@ void NodeManager::HandleCancelResourceReserve( if (RayConfig::instance().use_ray_syncer()) { auto sync_message = cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage( - , syncer::RayComponentId::RESOURCE_MANAGER); + 0, syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); ray_syncer_.BroadcastMessage( std::make_shared(std::move(*sync_message))); From 16ccbba4009547160a8cc35e53bfad929ec3e721 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 22 Apr 2022 04:08:26 +0000 Subject: [PATCH 57/77] fix comment --- src/ray/raylet/node_manager.cc | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index f0d042dc41c8..b7e5200dade8 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1752,7 +1752,8 @@ void NodeManager::HandleCommitBundleResources( if (RayConfig::instance().use_ray_syncer()) { auto sync_message = cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage( - 0, syncer::RayComponentId::RESOURCE_MANAGER); + 0, // Use version = 0 to for a snapshot + syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); ray_syncer_.BroadcastMessage( std::make_shared(std::move(*sync_message))); @@ -1797,7 +1798,8 @@ void NodeManager::HandleCancelResourceReserve( if (RayConfig::instance().use_ray_syncer()) { auto sync_message = cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage( - 0, syncer::RayComponentId::RESOURCE_MANAGER); + 0, // Use version = 0 to get a snapshot + syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); ray_syncer_.BroadcastMessage( std::make_shared(std::move(*sync_message))); From 7107aa12889cb2cdde528cdd87c696ce64408a5b Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 22 Apr 2022 04:17:51 +0000 Subject: [PATCH 58/77] format --- src/ray/gcs/gcs_server/gcs_resource_manager.cc | 11 +++-------- src/ray/raylet/node_manager.cc | 4 ++-- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index d7595d97b561..45eb224f414e 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -43,15 +43,10 @@ void GcsResourceManager::ConsumeSyncMessage( resources.set_node_id(message->node_id()); auto node_id = NodeID::FromBinary(message->node_id()); if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { - if (RayConfig::instance().gcs_actor_scheduling_enabled()) { - UpdateNodeNormalTaskResources(NodeID::FromBinary(message->node_id()), - resources); - } else { - cluster_resource_manager_.UpdateNodeAvailableResourcesIfExist( - scheduling::NodeID(message->node_id()), resources); - } + UpdateFromResourceReport(resources); + } else { + UpdateNodeResourceUsage(node_id, resources); } - UpdateNodeResourceUsage(node_id, resources); }, "GcsResourceManager::Update"); } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index b7e5200dade8..bd6df36ff60f 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1752,7 +1752,7 @@ void NodeManager::HandleCommitBundleResources( if (RayConfig::instance().use_ray_syncer()) { auto sync_message = cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage( - 0, // Use version = 0 to for a snapshot + 0, // Use version = 0 to for a snapshot syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); ray_syncer_.BroadcastMessage( @@ -1798,7 +1798,7 @@ void NodeManager::HandleCancelResourceReserve( if (RayConfig::instance().use_ray_syncer()) { auto sync_message = cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage( - 0, // Use version = 0 to get a snapshot + 0, // Use version = 0 to get a snapshot syncer::RayComponentId::RESOURCE_MANAGER); RAY_CHECK(sync_message); ray_syncer_.BroadcastMessage( From 63991bee5e98344415398d942322d9289fe250a3 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 28 Apr 2022 00:15:46 +0000 Subject: [PATCH 59/77] up --- .bazelrc | 2 + src/ray/common/ray_config_def.h | 1 - src/ray/common/ray_syncer/ray_syncer.cc | 29 +--- src/ray/common/ray_syncer/ray_syncer.h | 31 ++--- src/ray/common/test/ray_syncer_test.cc | 167 +++++------------------- src/ray/protobuf/ray_syncer.proto | 3 +- src/ray/raylet/node_manager.cc | 13 +- 7 files changed, 52 insertions(+), 194 deletions(-) diff --git a/.bazelrc b/.bazelrc index e1c37ce4c539..fe84ba3d6e61 100644 --- a/.bazelrc +++ b/.bazelrc @@ -8,6 +8,8 @@ build:windows --action_env=PATH # For --compilation_mode=dbg, consider enabling checks in the standard library as well (below). build --compilation_mode=opt # Using C++ 17 on all platforms. +build:linux --cxxopt="-g" +build:linux --cxxopt="-ggdb" build:linux --cxxopt="-std=c++17" build:macos --cxxopt="-std=c++17" build:clang-cl --cxxopt="-std=c++17" diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index f14fe0eb0bbd..a3005b57e18e 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -64,7 +64,6 @@ RAY_CONFIG(uint64_t, num_heartbeats_warning, 5) /// The duration between reporting resources sent by the raylets. RAY_CONFIG(uint64_t, raylet_report_resources_period_milliseconds, 100) -RAY_CONFIG(uint64_t, raylet_report_loads_period_milliseconds, 1000) /// The duration between raylet check memory pressure and send gc request RAY_CONFIG(uint64_t, raylet_check_gc_period_milliseconds, 100) diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index 2e77e2cfccc3..2ab322fee80c 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -249,7 +249,6 @@ RaySyncer::RaySyncer(instrumented_io_context &io_context, node_state_(std::make_unique()), timer_(io_context) { stopped_ = std::make_shared(false); - upward_only_.fill(false); timer_.RunFnPeriodically( [this]() { for (auto &[_, sync_connection] : sync_connections_) { @@ -300,20 +299,12 @@ void RaySyncer::Connect(std::unique_ptr connection) { RAY_CHECK(connection != nullptr); RAY_CHECK(sync_connections_[connection->GetRemoteNodeID()] == nullptr); auto &conn = *connection; - bool is_upward_conn = false; - if (dynamic_cast(connection) == nullptr) { - upward_connections_.insert(connection); - is_upward_conn = true; - } sync_connections_[connection->GetRemoteNodeID()].reset(connection); for (const auto &[_, messages] : node_state_->GetClusterView()) { for (auto &message : messages) { if (!message) { continue; } - if (upward_only_[message->component_id()] && !is_upward_conn) { - continue; - } conn.PushToSendingQueue(message); } } @@ -326,7 +317,6 @@ void RaySyncer::Disconnect(const std::string &node_id) { [this, node_id]() { auto iter = sync_connections_.find(node_id); if (iter != sync_connections_.end()) { - upward_connections_.erase(iter->second.get()); sync_connections_.erase(iter); } }, @@ -336,17 +326,13 @@ void RaySyncer::Disconnect(const std::string &node_id) { bool RaySyncer::Register(RayComponentId component_id, const ReporterInterface *reporter, ReceiverInterface *receiver, - bool upward_only, int64_t pull_from_reporter_interval_ms) { if (!node_state_->SetComponent(component_id, reporter, receiver)) { return false; } - upward_only_[component_id] = upward_only; - // Set job to pull from reporter periodically - if (reporter != nullptr) { - RAY_CHECK(pull_from_reporter_interval_ms > 0); + if (reporter != nullptr && pull_from_reporter_interval_ms > 0) { timer_.RunFnPeriodically( [this, component_id]() { auto msg = node_state_->CreateSyncMessage(component_id); @@ -361,8 +347,7 @@ bool RaySyncer::Register(RayComponentId component_id, RAY_LOG(DEBUG) << "Registered components: " << "component_id:" << component_id << ", reporter:" << reporter << ", receiver:" << receiver - << ", pull_from_reporter_interval_ms:" << pull_from_reporter_interval_ms - << ", upward_only:" << upward_only_[component_id]; + << ", pull_from_reporter_interval_ms:" << pull_from_reporter_interval_ms; return true; } @@ -373,14 +358,8 @@ void RaySyncer::BroadcastMessage(std::shared_ptr message) if (!node_state_->ConsumeSyncMessage(message)) { return; } - if (upward_only_[message->component_id()]) { - for (auto &connection : upward_connections_) { - connection->PushToSendingQueue(message); - } - } else { - for (auto &connection : sync_connections_) { - connection.second->PushToSendingQueue(message); - } + for (auto &connection : sync_connections_) { + connection.second->PushToSendingQueue(message); } }, "RaySyncer.BroadcastMessage"); diff --git a/src/ray/common/ray_syncer/ray_syncer.h b/src/ray/common/ray_syncer/ray_syncer.h index c12290d6bd98..b46efefd24eb 100644 --- a/src/ray/common/ray_syncer/ray_syncer.h +++ b/src/ray/common/ray_syncer/ray_syncer.h @@ -113,15 +113,6 @@ class RaySyncer { /// Register the components to the syncer module. Syncer will make sure eventually /// it'll have a global view of the cluster. /// - /// Right now there are two types of components. One type of components will - /// try to broadcast the messages to make sure eventually the cluster will reach - /// an agreement (upward_only=false). The other type of components will only - /// send the message to upward (upward_only=true). Right now, upward is defined - /// to be the place which received the connection. In Ray, one type of this message - /// is resource load which only GCS needs. - /// TODO (iycheng): 1) Revisit this and come with a better solution; or 2) implement - /// resource loads in another way to avoid this feature; or 3) broadcast resource - /// loads so the scheduler can also use this. /// /// \param component_id The component to sync. /// \param reporter The local component to be broadcasted. @@ -134,19 +125,12 @@ class RaySyncer { bool Register(RayComponentId component_id, const ReporterInterface *reporter, ReceiverInterface *receiver, - bool upward_only = false, int64_t pull_from_reporter_interval_ms = 100); - /// Function to broadcast the messages to other nodes. - /// A message will be sent to a node if that node doesn't have this message. - /// The message can be generated by local reporter or received by the other node. - /// - /// \param message The message to be broadcasted. - void BroadcastMessage(std::shared_ptr message); - /// Get the current node id. const std::string &GetLocalNodeID() const { return local_node_id_; } + private: /// Get the io_context used by RaySyncer. instrumented_io_context &GetIOContext() { return io_context_; } @@ -164,7 +148,13 @@ class RaySyncer { return iter->second.get(); } - private: + /// Function to broadcast the messages to other nodes. + /// A message will be sent to a node if that node doesn't have this message. + /// The message can be generated by local reporter or received by the other node. + /// + /// \param message The message to be broadcasted. + void BroadcastMessage(std::shared_ptr message); + /// io_context for this thread instrumented_io_context &io_context_; @@ -180,15 +170,12 @@ class RaySyncer { /// The local node state std::unique_ptr node_state_; - /// Each component will define a flag to indicate whether the message should be sent - /// to ClientSyncConnection only. - std::array upward_only_; - /// Timer is used to do broadcasting. ray::PeriodicalRunner timer_; std::shared_ptr stopped_; + friend class RaySyncerService; /// Test purpose friend struct SyncerServerTest; FRIEND_TEST(SyncerTest, Broadcast); diff --git a/src/ray/common/test/ray_syncer_test.cc b/src/ray/common/test/ray_syncer_test.cc index c2b10a701b81..e0fd1bf49bb6 100644 --- a/src/ray/common/test/ray_syncer_test.cc +++ b/src/ray/common/test/ray_syncer_test.cc @@ -42,7 +42,7 @@ using ::testing::WithArg; namespace ray { namespace syncer { -constexpr size_t kTestComponents = 2; +constexpr size_t kTestComponents = 1; RaySyncMessage MakeMessage(RayComponentId cid, int64_t version, const NodeID &id) { auto msg = RaySyncMessage(); @@ -117,7 +117,6 @@ TEST_F(RaySyncerTest, NodeStateCreateSyncMessage) { node_status->SetComponent(RayComponentId::RESOURCE_MANAGER, nullptr, nullptr); ASSERT_EQ(std::nullopt, node_status->CreateSyncMessage(RayComponentId::RESOURCE_MANAGER)); - ASSERT_EQ(std::nullopt, node_status->CreateSyncMessage(RayComponentId::SCHEDULER)); auto reporter = std::make_unique(); ASSERT_TRUE(node_status->SetComponent(RayComponentId::RESOURCE_MANAGER, @@ -125,7 +124,6 @@ TEST_F(RaySyncerTest, NodeStateCreateSyncMessage) { nullptr)); // Take a snapshot - ASSERT_EQ(std::nullopt, node_status->CreateSyncMessage(RayComponentId::SCHEDULER)); auto msg = node_status->CreateSyncMessage(RayComponentId::RESOURCE_MANAGER); ASSERT_EQ(LocalVersion(RayComponentId::RESOURCE_MANAGER), msg->version()); // Revert one version back. @@ -183,9 +181,8 @@ TEST_F(RaySyncerTest, NodeSyncConnection) { } struct SyncerServerTest { - SyncerServerTest(std::string port, bool has_scheduler_reporter = true) { + SyncerServerTest(std::string port) { this->server_port = port; - bool has_scheduler_receiver = !has_scheduler_reporter; // Setup io context auto node_id = NodeID::FromRandom(); for (auto &v : local_versions) { @@ -215,14 +212,9 @@ struct SyncerServerTest { message->version(); message_consumed[message->node_id()]++; }; - - if (has_scheduler_receiver || - static_cast(cid) != RayComponentId::SCHEDULER) { - receivers[cid] = std::make_unique(); - EXPECT_CALL(*receivers[cid], ConsumeSyncMessage(_)) - .WillRepeatedly(WithArg<0>(Invoke(snapshot_received))); - } - + receivers[cid] = std::make_unique(); + EXPECT_CALL(*receivers[cid], ConsumeSyncMessage(_)) + .WillRepeatedly(WithArg<0>(Invoke(snapshot_received))); auto &reporter = reporters[cid]; auto take_snapshot = [this, cid](int64_t version_after) mutable -> std::optional { @@ -237,16 +229,11 @@ struct SyncerServerTest { return std::make_optional(std::move(msg)); } }; - if (has_scheduler_reporter || - static_cast(cid) != RayComponentId::SCHEDULER) { - reporter = std::make_unique(); - EXPECT_CALL(*reporter, CreateSyncMessage(_, Eq(cid))) - .WillRepeatedly(WithArg<0>(Invoke(take_snapshot))); - } - syncer->Register(static_cast(cid), - reporter.get(), - receivers[cid].get(), - static_cast(cid) == RayComponentId::SCHEDULER); + reporter = std::make_unique(); + EXPECT_CALL(*reporter, CreateSyncMessage(_, Eq(cid))) + .WillRepeatedly(WithArg<0>(Invoke(take_snapshot))); + syncer->Register( + static_cast(cid), reporter.get(), receivers[cid].get()); } thread = std::make_unique([this] { boost::asio::io_context::work work(io_context); @@ -385,20 +372,11 @@ using TClusterView = absl::flat_hash_map< std::array, kComponentArraySize>>; TEST(SyncerTest, Test1To1) { - // s1: reporter: RayComponentId::RESOURCE_MANAGER - // s1: receiver: RayComponentId::SCHEDULER, RayComponentId::RESOURCE_MANAGER - auto s1 = SyncerServerTest("19990", false); + auto s1 = SyncerServerTest("19990"); - // s2: reporter: RayComponentId::RESOURCE_MANAGER, RayComponentId::SCHEDULER - // s2: receiver: RayComponentId::RESOURCE_MANAGER - auto s2 = SyncerServerTest("19991", true); + auto s2 = SyncerServerTest("19991"); // Make sure the setup is correct - ASSERT_NE(nullptr, s1.receivers[RayComponentId::SCHEDULER]); - ASSERT_EQ(nullptr, s2.receivers[RayComponentId::SCHEDULER]); - ASSERT_EQ(nullptr, s1.reporters[RayComponentId::SCHEDULER]); - ASSERT_NE(nullptr, s2.reporters[RayComponentId::SCHEDULER]); - ASSERT_NE(nullptr, s1.receivers[RayComponentId::RESOURCE_MANAGER]); ASSERT_NE(nullptr, s2.receivers[RayComponentId::RESOURCE_MANAGER]); ASSERT_NE(nullptr, s1.reporters[RayComponentId::RESOURCE_MANAGER]); @@ -408,10 +386,10 @@ TEST(SyncerTest, Test1To1) { s1.syncer->Connect(channel_to_s2); - // Make sure s2 adds s1n + // Make sure s2 adds s1 ASSERT_TRUE(s2.WaitUntil( [&s2]() { - return s2.syncer->sync_connections_.size() == 1 && s2.snapshot_taken == 2; + return s2.syncer->sync_connections_.size() == 1 && s2.snapshot_taken == 1; }, 5)); @@ -432,19 +410,19 @@ TEST(SyncerTest, Test1To1) { // s2 will send 2 messages to s1 because it has two reporters. ASSERT_TRUE(s1.WaitUntil( [&s1, node_id = s2.syncer->GetLocalNodeID()]() { - return s1.GetNumConsumedMessages(node_id) == 2; + return s1.GetNumConsumedMessages(node_id) == 1; }, 5)); // s2 local module version advance s2.local_versions[0] = 1; - ASSERT_TRUE(s2.WaitUntil([&s2]() { return s2.snapshot_taken == 3; }, 2)); + ASSERT_TRUE(s2.WaitUntil([&s2]() { return s2.snapshot_taken == 2; }, 2)); // Make sure s2 send the new message to s1. ASSERT_TRUE(s1.WaitUntil( [&s1, node_id = s2.syncer->GetLocalNodeID()]() { return s1.GetReceivedVersions(node_id)[RayComponentId::RESOURCE_MANAGER] == 1 && - s1.GetNumConsumedMessages(node_id) == 3; + s1.GetNumConsumedMessages(node_id) == 2; }, 5)); @@ -452,7 +430,7 @@ TEST(SyncerTest, Test1To1) { s2.local_versions[0] = 0; std::this_thread::sleep_for(1s); - ASSERT_TRUE(s1.GetNumConsumedMessages(s2.syncer->GetLocalNodeID()) == 3); + ASSERT_TRUE(s1.GetNumConsumedMessages(s2.syncer->GetLocalNodeID()) == 2); ASSERT_TRUE(s2.GetNumConsumedMessages(s1.syncer->GetLocalNodeID()) == 1); // Change it back s2.local_versions[0] = 1; @@ -509,9 +487,9 @@ TEST(SyncerTest, Reconnect) { // s2 -> s3 // And we need to ensure s3 is connecting to s2 - auto s1 = SyncerServerTest("19990", false); - auto s2 = SyncerServerTest("19991", true); - auto s3 = SyncerServerTest("19992", true); + auto s1 = SyncerServerTest("19990"); + auto s2 = SyncerServerTest("19991"); + auto s3 = SyncerServerTest("19992"); s1.syncer->Connect(MakeChannel("19992")); @@ -524,26 +502,23 @@ TEST(SyncerTest, Reconnect) { ASSERT_TRUE(s1.WaitUntil( [&s3]() { - return s3.syncer->sync_connections_.size() == 1 && s3.snapshot_taken == 2; + return s3.syncer->sync_connections_.size() == 1 && s3.snapshot_taken == 1; }, 5)); s2.syncer->Connect(MakeChannel("19992")); ASSERT_TRUE(s1.WaitUntil( [&s2]() { - return s2.syncer->sync_connections_.size() == 1 && s2.snapshot_taken == 2; + return s2.syncer->sync_connections_.size() == 1 && s2.snapshot_taken == 1; }, 5)); - ASSERT_EQ(1, s3.syncer->upward_connections_.size()); - ASSERT_EQ(s2.syncer->GetLocalNodeID(), - (*s3.syncer->upward_connections_.begin())->GetRemoteNodeID()); } TEST(SyncerTest, Broadcast) { // This test covers the broadcast feature of ray syncer. - auto s1 = SyncerServerTest("19990", false); - auto s2 = SyncerServerTest("19991", true); - auto s3 = SyncerServerTest("19992", true); + auto s1 = SyncerServerTest("19990"); + auto s2 = SyncerServerTest("19991"); + auto s3 = SyncerServerTest("19992"); // We need to make sure s1 is sending data to s3 for s2 s1.syncer->Connect(MakeChannel("19991")); s1.syncer->Connect(MakeChannel("19992")); @@ -557,32 +532,28 @@ TEST(SyncerTest, Broadcast) { ASSERT_TRUE(s1.WaitUntil( [&s2]() { - return s2.syncer->sync_connections_.size() == 1 && s2.snapshot_taken == 2; + return s2.syncer->sync_connections_.size() == 1 && s2.snapshot_taken == 1; }, 5)); ASSERT_TRUE(s1.WaitUntil( [&s3]() { - return s3.syncer->sync_connections_.size() == 1 && s3.snapshot_taken == 2; + return s3.syncer->sync_connections_.size() == 1 && s3.snapshot_taken == 1; }, 5)); // Change the resource in s2 and make sure s1 && s3 are correct s2.local_versions[0] = 1; - s2.local_versions[1] = 1; ASSERT_TRUE(s1.WaitUntil( [&s1, node_id = s2.syncer->GetLocalNodeID()]() mutable { - return s1.received_versions[node_id][0] == 1 && - s1.received_versions[node_id][1] == 1; + return s1.received_versions[node_id][0] == 1; }, 5)); ASSERT_TRUE(s1.WaitUntil( [&s3, node_id = s2.syncer->GetLocalNodeID()]() mutable { - return s3.received_versions[node_id][0] == 1 && - // Make sure SCHEDULE information is not sent to s3 - s3.received_versions[node_id][1] == 0; + return s3.received_versions[node_id][0] == 1; }, 5)); } @@ -608,8 +579,7 @@ bool CompareViews(const std::vector> &servers, const auto &vv = iter->second; if (!google::protobuf::util::MessageDifferencer::Equals(*v[0], *vv[0])) { - RAY_LOG(ERROR) << i << ": FAIL RESOURCE: " << v[0] << ", " << vv[0] << ", " - << v[1] << ", " << vv[1]; + RAY_LOG(ERROR) << i << ": FAIL RESOURCE: " << v[0] << ", " << vv[0]; std::string dbg_message; google::protobuf::util::MessageToJsonString(*v[0], &dbg_message); RAY_LOG(ERROR) << "server[0] >> " @@ -625,75 +595,6 @@ bool CompareViews(const std::vector> &servers, } } - std::map node_id_to_idx; - for (size_t i = 0; i < servers.size(); ++i) { - node_id_to_idx[servers[i]->syncer->GetLocalNodeID()] = i; - } - // Check whether j is reachable from i - auto reachable = [&g](size_t i, size_t j) { - if (i == j) { - return true; - } - std::deque q; - q.push_back(i); - while (!q.empty()) { - auto f = q.front(); - q.pop_front(); - for (auto m : g[f]) { - if (m == j) { - return true; - } - q.push_back(m); - } - } - return false; - }; - // Check scheduler which is aggregating only - for (size_t i = 0; i < servers.size(); ++i) { - const auto &view = views[i]; - // view: node_id -> msg - for (auto [node_id, msgs] : view) { - if (node_id_to_idx[node_id] == i) { - continue; - } - auto msg = msgs[1]; - auto is_reachable = reachable(i, node_id_to_idx[node_id]); - if (msg == nullptr) { - if (is_reachable) { - RAY_LOG(ERROR) << i << " is null, but it can reach " << node_id_to_idx[node_id]; - return false; - } - } else { - if (!is_reachable) { - RAY_LOG(ERROR) << i << " is not null, but it can't reachable " - << node_id_to_idx[node_id]; - return false; - } - auto iter = views[node_id_to_idx[node_id]].find(node_id); - if (iter == views[node_id_to_idx[node_id]].end()) { - return false; - } - auto msg2 = iter->second[1]; - if (msg2 == nullptr) { - return false; - } - if (!google::protobuf::util::MessageDifferencer::Equals(*msg, *msg2)) { - std::string dbg_message; - google::protobuf::util::MessageToJsonString(*msg, &dbg_message); - RAY_LOG(ERROR) << "server[" << i << "] >> " - << NodeID::FromBinary(servers[i]->syncer->GetLocalNodeID()) - << ": " << dbg_message; - dbg_message.clear(); - google::protobuf::util::MessageToJsonString(*msg2, &dbg_message); - RAY_LOG(ERROR) << "server[" << node_id_to_idx[node_id] << "] << " - << NodeID::FromBinary(servers[node_id_to_idx[node_id]] - ->syncer->GetLocalNodeID()) - << ": " << dbg_message; - return false; - } - } - } - } return true; } @@ -763,8 +664,7 @@ TEST(SyncerTest, Test1ToN) { size_t base_port = 18990; std::vector> servers; for (int i = 0; i < 20; ++i) { - servers.push_back( - std::make_unique(std::to_string(i + base_port), i != 0)); + servers.push_back(std::make_unique(std::to_string(i + base_port))); } std::vector> g(servers.size()); for (size_t i = 1; i < servers.size(); ++i) { @@ -788,8 +688,7 @@ TEST(SyncerTest, TestMToN) { size_t base_port = 18990; std::vector> servers; for (int i = 0; i < 20; ++i) { - servers.push_back( - std::make_unique(std::to_string(i + base_port), i != 0)); + servers.push_back(std::make_unique(std::to_string(i + base_port))); } std::vector> g(servers.size()); // Try to construct a tree based structure diff --git a/src/ray/protobuf/ray_syncer.proto b/src/ray/protobuf/ray_syncer.proto index fd0e79581de9..7b277704bea9 100644 --- a/src/ray/protobuf/ray_syncer.proto +++ b/src/ray/protobuf/ray_syncer.proto @@ -18,8 +18,7 @@ package ray.rpc.syncer; enum RayComponentId { RESOURCE_MANAGER = 0; - SCHEDULER = 1; - COMMANDS = 2; + COMMANDS = 1; } message RaySyncMessage { diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 8dd15f7d87fd..acc682a1015e 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -553,23 +553,16 @@ ray::Status NodeManager::RegisterGcs() { /* component_id */ syncer::RayComponentId::RESOURCE_MANAGER, /* reporter */ &cluster_resource_scheduler_->GetLocalResourceManager(), /* receiver */ this, - /* upward_only */ false, /* pull_from_reporter_interval_ms */ RayConfig::instance().raylet_report_resources_period_milliseconds()); - ray_syncer_.Register( - /* component_id */ syncer::RayComponentId::SCHEDULER, - /* reporter */ this, - /* receiver */ nullptr, - /* upward_only */ true, - /* pull_from_reporter_interval_ms */ - RayConfig::instance().raylet_report_loads_period_milliseconds()); + // Register a commands channel. // It's only used for GC right now. ray_syncer_.Register( /* component_id */ syncer::RayComponentId::COMMANDS, - /* reporter */ nullptr, + /* reporter */ this, + /* receiver */ this, - /* upward_only */ false, /* pull_from_reporter_interval_ms */ 0); periodical_runner_.RunFnPeriodically( [this] { From 86fb73da6cda867e8c9f68d4f233c518f5997ef3 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 28 Apr 2022 00:23:43 +0000 Subject: [PATCH 60/77] fix ray syncer test --- src/ray/common/test/ray_syncer_test.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/ray/common/test/ray_syncer_test.cc b/src/ray/common/test/ray_syncer_test.cc index e0fd1bf49bb6..896f8153f135 100644 --- a/src/ray/common/test/ray_syncer_test.cc +++ b/src/ray/common/test/ray_syncer_test.cc @@ -439,7 +439,7 @@ TEST(SyncerTest, Test1To1) { std::random_device rd; std::mt19937 gen(rd()); std::uniform_int_distribution<> rand_sleep(0, 10000); - std::uniform_int_distribution<> choose_component(0, 1); + std::uniform_int_distribution<> choose_component(0, kTestComponents - 1); size_t s1_updated = 0; size_t s2_updated = 0; @@ -629,7 +629,7 @@ bool TestCorrectness(std::function get_cluster_ std::random_device rd; std::mt19937 gen(rd()); std::uniform_int_distribution<> rand_sleep(0, 1000000); - std::uniform_int_distribution<> choose_component(0, 1); + std::uniform_int_distribution<> choose_component(0, kTestComponents - 1); std::uniform_int_distribution<> choose_server(0, servers.size() - 1); for (size_t i = 0; i < 1000000; ++i) { From 0471f120762b2ccf2a8fddfb839ea930a937875a Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 28 Apr 2022 00:33:52 +0000 Subject: [PATCH 61/77] update --- .bazelrc | 2 - src/ray/common/ray_syncer/ray_syncer.cc | 18 +++--- src/ray/common/ray_syncer/ray_syncer.h | 13 ++++- src/ray/raylet/node_manager.cc | 78 ++++++------------------- 4 files changed, 40 insertions(+), 71 deletions(-) diff --git a/.bazelrc b/.bazelrc index fe84ba3d6e61..e1c37ce4c539 100644 --- a/.bazelrc +++ b/.bazelrc @@ -8,8 +8,6 @@ build:windows --action_env=PATH # For --compilation_mode=dbg, consider enabling checks in the standard library as well (below). build --compilation_mode=opt # Using C++ 17 on all platforms. -build:linux --cxxopt="-g" -build:linux --cxxopt="-ggdb" build:linux --cxxopt="-std=c++17" build:macos --cxxopt="-std=c++17" build:clang-cl --cxxopt="-std=c++17" diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index 2ab322fee80c..3bb47098f0e2 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -334,13 +334,7 @@ bool RaySyncer::Register(RayComponentId component_id, // Set job to pull from reporter periodically if (reporter != nullptr && pull_from_reporter_interval_ms > 0) { timer_.RunFnPeriodically( - [this, component_id]() { - auto msg = node_state_->CreateSyncMessage(component_id); - if (msg) { - RAY_CHECK(msg->node_id() == GetLocalNodeID()); - BroadcastMessage(std::make_shared(std::move(*msg))); - } - }, + [this, component_id]() { OnDemandBroadcasting(component_id); }, pull_from_reporter_interval_ms); } @@ -351,6 +345,16 @@ bool RaySyncer::Register(RayComponentId component_id, return true; } +bool RaySyncer::OnDemandBroadcasting(RayComponentId component_id) { + auto msg = node_state_->CreateSyncMessage(component_id); + if (msg) { + RAY_CHECK(msg->node_id() == GetLocalNodeID()); + BroadcastMessage(std::make_shared(std::move(*msg))); + return true; + } + return false; +} + void RaySyncer::BroadcastMessage(std::shared_ptr message) { io_context_.dispatch( [this, message] { diff --git a/src/ray/common/ray_syncer/ray_syncer.h b/src/ray/common/ray_syncer/ray_syncer.h index b46efefd24eb..7725b518c734 100644 --- a/src/ray/common/ray_syncer/ray_syncer.h +++ b/src/ray/common/ray_syncer/ray_syncer.h @@ -118,9 +118,8 @@ class RaySyncer { /// \param reporter The local component to be broadcasted. /// \param receiver The consumer of the sync message sent by the other nodes in the /// cluster. - /// \param upward_only Only send the message to the upward of this node. - /// component. - /// \param pull_from_reporter_interval_ms The frequence to pull a message + /// \param pull_from_reporter_interval_ms The frequence to pull a message. 0 means + /// never pull a message in syncer. /// from reporter and push it to sending queue. bool Register(RayComponentId component_id, const ReporterInterface *reporter, @@ -130,6 +129,14 @@ class RaySyncer { /// Get the current node id. const std::string &GetLocalNodeID() const { return local_node_id_; } + /// Request trigger a broadcasting for a specific component immediately instead of + /// waiting for ray syncer to poll the message. + /// + /// \param component_id The component to check. + /// \return true if a message is generated. If the component doesn't have a new + /// version of message, false will be returned. + bool OnDemandBroadcasting(RayComponentId component_id); + private: /// Get the io_context used by RaySyncer. instrumented_io_context &GetIOContext() { return io_context_; } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index acc682a1015e..14e0c1823691 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -561,28 +561,15 @@ ray::Status NodeManager::RegisterGcs() { ray_syncer_.Register( /* component_id */ syncer::RayComponentId::COMMANDS, /* reporter */ this, - /* receiver */ this, /* pull_from_reporter_interval_ms */ 0); + periodical_runner_.RunFnPeriodically( [this] { + auto triggered_by_global_gc = TryLocalGC(); // If plasma store is under high pressure, we should try to schedule a global // gc. - auto triggered_by_global_gc = TryLocalGC(); - - if (triggered_by_global_gc) { - rpc::ResourcesData resources_data; - resources_data.set_should_global_gc(true); - syncer::RaySyncMessage msg; - msg.set_version(absl::GetCurrentTimeNanos()); - msg.set_node_id(self_node_id_.Binary()); - msg.set_component_id(syncer::RayComponentId::COMMANDS); - std::string serialized_msg; - RAY_CHECK(resources_data.SerializeToString(&serialized_msg)); - msg.set_sync_message(std::move(serialized_msg)); - ray_syncer_.BroadcastMessage( - std::make_shared(std::move(msg))); - } + ray_syncer_.OnDemandBroadcasting(syncer::RayComponentId::COMMANDS); }, RayConfig::instance().raylet_check_gc_period_milliseconds(), "NodeManager.CheckGC"); @@ -1824,13 +1811,8 @@ void NodeManager::HandleCommitBundleResources( << GetDebugStringForBundles(bundle_specs); placement_group_resource_manager_->CommitBundles(bundle_specs); if (RayConfig::instance().use_ray_syncer()) { - auto sync_message = - cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage( - 0, // Use version = 0 to for a snapshot - syncer::RayComponentId::RESOURCE_MANAGER); - RAY_CHECK(sync_message); - ray_syncer_.BroadcastMessage( - std::make_shared(std::move(*sync_message))); + // To reduce the lag, we trigger a broadcasting immediately. + RAY_CHECK(ray_syncer_.OnDemandBroadcasting(syncer::RayComponentId::RESOURCE_MANAGER)); } send_reply_callback(Status::OK(), nullptr, nullptr); @@ -1870,13 +1852,8 @@ void NodeManager::HandleCancelResourceReserve( // Return bundle resources. placement_group_resource_manager_->ReturnBundle(bundle_spec); if (RayConfig::instance().use_ray_syncer()) { - auto sync_message = - cluster_resource_scheduler_->GetLocalResourceManager().CreateSyncMessage( - 0, // Use version = 0 to get a snapshot - syncer::RayComponentId::RESOURCE_MANAGER); - RAY_CHECK(sync_message); - ray_syncer_.BroadcastMessage( - std::make_shared(std::move(*sync_message))); + // To reduce the lag, we trigger a broadcasting immediately. + RAY_CHECK(ray_syncer_.OnDemandBroadcasting(syncer::RayComponentId::RESOURCE_MANAGER)); } cluster_task_manager_->ScheduleAndDispatchTasks(); send_reply_callback(Status::OK(), nullptr, nullptr); @@ -2733,35 +2710,18 @@ void NodeManager::ConsumeSyncMessage( std::optional NodeManager::CreateSyncMessage( int64_t after_version, syncer::RayComponentId component_id) const { - // Right now snapshot is put in NodeManager which in long-term, them should - // be put into each component directly. - // The current blocker of doing this is some fields in NodeManager - // is being used. - // TODO(iycheng): Move the logic into the components directly. - if (component_id == syncer::RayComponentId::SCHEDULER) { - syncer::RaySyncMessage msg; - rpc::ResourcesData resource_data; - - cluster_task_manager_->FillResourceUsage(resource_data); - if (RayConfig::instance().gcs_actor_scheduling_enabled()) { - const_cast(this)->FillNormalTaskResourceUsage(resource_data); - } - - resource_data.set_node_id(self_node_id_.Binary()); - resource_data.set_node_manager_address(initial_config_.node_manager_address); - resource_data.set_cluster_full_of_actors_detected(resource_deadlock_warned_ >= 1); - - msg.set_version(absl::GetCurrentTimeNanos()); - msg.set_node_id(self_node_id_.Binary()); - msg.set_component_id(syncer::RayComponentId::SCHEDULER); - std::string serialized_msg; - RAY_CHECK(resource_data.SerializeToString(&serialized_msg)); - msg.set_sync_message(std::move(serialized_msg)); - return std::make_optional(std::move(msg)); - } else { - RAY_CHECK(false) << "Invalid component id: " << component_id; - return std::nullopt; - } + RAY_CHECK(component_id == syncer::RayComponentId::COMMANDS); + + rpc::ResourcesData resources_data; + resources_data.set_should_global_gc(true); + syncer::RaySyncMessage msg; + msg.set_version(absl::GetCurrentTimeNanos()); + msg.set_node_id(self_node_id_.Binary()); + msg.set_component_id(syncer::RayComponentId::COMMANDS); + std::string serialized_msg; + RAY_CHECK(resources_data.SerializeToString(&serialized_msg)); + msg.set_sync_message(std::move(serialized_msg)); + return std::make_optional(std::move(msg)); } void NodeManager::PublishInfeasibleTaskError(const RayTask &task) const { From 234a7d68abd0874e4f1373b771324edc068f2186 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 28 Apr 2022 00:47:37 +0000 Subject: [PATCH 62/77] update --- src/ray/gcs/gcs_server/gcs_server.cc | 2 -- src/ray/raylet/node_manager.cc | 4 +++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index a6d7bb92774c..2be21a58b0ae 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -463,8 +463,6 @@ void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { gcs_node_id_.Binary()); ray_syncer_->Register( syncer::RayComponentId::RESOURCE_MANAGER, nullptr, gcs_resource_manager_.get()); - ray_syncer_->Register( - syncer::RayComponentId::SCHEDULER, nullptr, gcs_resource_manager_.get()); ray_syncer_thread_ = std::make_unique([this]() { boost::asio::io_service::work work(ray_syncer_io_context_); ray_syncer_io_context_.run(); diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 14e0c1823691..b4dbb3f47126 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -569,7 +569,9 @@ ray::Status NodeManager::RegisterGcs() { auto triggered_by_global_gc = TryLocalGC(); // If plasma store is under high pressure, we should try to schedule a global // gc. - ray_syncer_.OnDemandBroadcasting(syncer::RayComponentId::COMMANDS); + if (triggered_by_global_gc) { + ray_syncer_.OnDemandBroadcasting(syncer::RayComponentId::COMMANDS); + } }, RayConfig::instance().raylet_check_gc_period_milliseconds(), "NodeManager.CheckGC"); From 5701f83b0bb628575211f1c70c92153d45f25cf5 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Thu, 28 Apr 2022 20:29:08 +0000 Subject: [PATCH 63/77] up --- src/ray/gcs/gcs_server/gcs_resource_manager.cc | 7 ++----- src/ray/gcs/gcs_server/gcs_resource_manager.h | 3 ++- src/ray/raylet/node_manager.h | 8 ++++---- src/ray/raylet/scheduling/local_resource_manager.h | 2 +- 4 files changed, 9 insertions(+), 11 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index 1c46f9c2cc03..5b1435e61f92 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -42,11 +42,8 @@ void GcsResourceManager::ConsumeSyncMessage( resources.ParseFromString(message->sync_message()); resources.set_node_id(message->node_id()); auto node_id = NodeID::FromBinary(message->node_id()); - if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { - UpdateFromResourceReport(resources); - } else { - UpdateNodeResourceUsage(node_id, resources); - } + RAY_CHECK(message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER); + UpdateFromResourceReport(resources); }, "GcsResourceManager::Update"); } diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.h b/src/ray/gcs/gcs_server/gcs_resource_manager.h index 306e2058ec94..0d4c9ddf2cbb 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.h +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.h @@ -144,7 +144,8 @@ class GcsResourceManager : public rpc::NodeResourceInfoHandler, void UpdateResourceLoads(const rpc::ResourcesData &data); private: - /// io context + /// io context. This is to ensure thread safety. Ideally, all public + /// funciton needs to post job to this io_context. instrumented_io_context &io_context_; /// Newest resource usage of all nodes. diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index 33e957ed58a8..f9ca2696318d 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -738,11 +738,11 @@ class NodeManager : public rpc::NodeManagerServiceHandler, /// Whether to trigger global GC in the next resource usage report. This will broadcast /// a global GC message to all raylets except for this one. - mutable bool should_global_gc_ = false; + bool should_global_gc_ = false; /// Whether to trigger local GC in the next resource usage report. This will trigger gc /// on all local workers of this raylet. - mutable bool should_local_gc_ = false; + bool should_local_gc_ = false; /// When plasma storage usage is high, we'll run gc to reduce it. double high_plasma_storage_usage_ = 1.0; @@ -751,10 +751,10 @@ class NodeManager : public rpc::NodeManagerServiceHandler, uint64_t local_gc_run_time_ns_; /// Throttler for local gc - mutable Throttler local_gc_throttler_; + Throttler local_gc_throttler_; /// Throttler for global gc - mutable Throttler global_gc_throttler_; + Throttler global_gc_throttler_; /// Seconds to initialize a local gc const uint64_t local_gc_interval_ns_; diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index 9f012d681745..0e1f804cb123 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -246,7 +246,7 @@ class LocalResourceManager : public syncer::ReporterInterface { /// Resources of local node. NodeResourceInstances local_resources_; /// Cached resources, used to compare with newest one in light heartbeat mode. - mutable std::unique_ptr last_report_resources_; + std::unique_ptr last_report_resources_; /// Function to get used object store memory. std::function get_used_object_store_memory_; /// Function to get whether the pull manager is at capacity. From d33298637708b1efa751d4d959a0880950973703 Mon Sep 17 00:00:00 2001 From: Yi Cheng <74173148+iycheng@users.noreply.github.com> Date: Thu, 28 Apr 2022 23:26:03 -0700 Subject: [PATCH 64/77] fix --- python/ray/serve/controller.py | 2 +- python/ray/serve/http_proxy.py | 2 +- python/ray/serve/logging_utils.py | 10 +-- python/ray/serve/replica.py | 2 +- python/ray/state.py | 10 +-- src/mock/ray/common/ray_syncer/ray_syncer.h | 2 +- src/ray/common/ray_syncer/ray_syncer-inl.h | 6 +- src/ray/common/ray_syncer/ray_syncer.cc | 38 ++++++------ src/ray/common/ray_syncer/ray_syncer.h | 16 ++--- src/ray/common/test/ray_syncer_test.cc | 62 +++++++++---------- .../common/test/syncer_service_e2e_test.cc | 6 +- src/ray/core_worker/profiling.cc | 4 +- src/ray/gcs/gcs_client/accessor.cc | 2 +- .../gcs/gcs_server/gcs_resource_manager.cc | 3 +- src/ray/gcs/gcs_server/gcs_server.cc | 2 +- src/ray/gcs/gcs_server/stats_handler_impl.cc | 2 +- .../gcs_server/test/gcs_server_rpc_test.cc | 2 +- src/ray/gcs/test/gcs_test_util.h | 2 +- src/ray/protobuf/gcs.proto | 2 +- src/ray/protobuf/ray_syncer.proto | 6 +- src/ray/raylet/node_manager.cc | 20 +++--- src/ray/raylet/node_manager.h | 2 +- .../scheduling/local_resource_manager.cc | 6 +- .../scheduling/local_resource_manager.h | 2 +- 24 files changed, 105 insertions(+), 106 deletions(-) diff --git a/python/ray/serve/controller.py b/python/ray/serve/controller.py index 530072e3d8a4..a4bfd6657b52 100644 --- a/python/ray/serve/controller.py +++ b/python/ray/serve/controller.py @@ -77,7 +77,7 @@ async def __init__( _override_controller_namespace: Optional[str] = None, ): configure_component_logger( - component_name="controller", component_id=str(os.getpid()) + component_name="controller", message_type=str(os.getpid()) ) # Used to read/write checkpoints. diff --git a/python/ray/serve/http_proxy.py b/python/ray/serve/http_proxy.py index 171dd131c08d..b699fab20bed 100644 --- a/python/ray/serve/http_proxy.py +++ b/python/ray/serve/http_proxy.py @@ -337,7 +337,7 @@ def __init__( http_middlewares: Optional[List["starlette.middleware.Middleware"]] = None, ): # noqa: F821 configure_component_logger( - component_name="http_proxy", component_id=node_id_to_ip_addr(node_id) + component_name="http_proxy", message_type=node_id_to_ip_addr(node_id) ) if http_middlewares is None: diff --git a/python/ray/serve/logging_utils.py b/python/ray/serve/logging_utils.py index 268e1efa65cc..23f773f91569 100644 --- a/python/ray/serve/logging_utils.py +++ b/python/ray/serve/logging_utils.py @@ -5,8 +5,8 @@ import ray from ray.serve.constants import DEBUG_LOG_ENV_VAR, SERVE_LOGGER_NAME -COMPONENT_LOG_FMT = "%(levelname)s %(asctime)s {component_name} {component_id} %(filename)s:%(lineno)d - %(message)s" # noqa:E501 -LOG_FILE_FMT = "{component_name}_{component_id}.log" +COMPONENT_LOG_FMT = "%(levelname)s %(asctime)s {component_name} {message_type} %(filename)s:%(lineno)d - %(message)s" # noqa:E501 +LOG_FILE_FMT = "{component_name}_{message_type}.log" def access_log_msg(*, method: str, route: str, status: str, latency_ms: float): @@ -17,7 +17,7 @@ def access_log_msg(*, method: str, route: str, status: str, latency_ms: float): def configure_component_logger( *, component_name: str, - component_id: str, + message_type: str, component_type: Optional[str] = None, log_level: int = logging.INFO, log_to_stream: bool = True, @@ -38,7 +38,7 @@ def configure_component_logger( formatter = logging.Formatter( COMPONENT_LOG_FMT.format( - component_name=component_name, component_id=component_id + component_name=component_name, message_type=message_type ) ) if log_to_stream: @@ -52,7 +52,7 @@ def configure_component_logger( if component_type is not None: component_name = f"{component_type}_{component_name}" log_file_name = LOG_FILE_FMT.format( - component_name=component_name, component_id=component_id + component_name=component_name, message_type=message_type ) file_handler = logging.FileHandler(os.path.join(logs_dir, log_file_name)) file_handler.setFormatter(formatter) diff --git a/python/ray/serve/replica.py b/python/ray/serve/replica.py index f6b0a554669a..45783d881524 100644 --- a/python/ray/serve/replica.py +++ b/python/ray/serve/replica.py @@ -75,7 +75,7 @@ async def __init__( configure_component_logger( component_type="deployment", component_name=deployment_name, - component_id=replica_tag, + message_type=replica_tag, ) if import_path is not None: diff --git a/python/ray/state.py b/python/ray/state.py index 9f4223345f2f..f068aeda6170 100644 --- a/python/ray/state.py +++ b/python/ray/state.py @@ -247,7 +247,7 @@ def profile_table(self): profile = gcs_utils.ProfileTableData.FromString(profile_table[i]) component_type = profile.component_type - component_id = binary_to_hex(profile.component_id) + message_type = binary_to_hex(profile.message_type) node_ip_address = profile.node_ip_address for event in profile.profile_events: @@ -257,7 +257,7 @@ def profile_table(self): extra_data = {} profile_event = { "event_type": event.event_type, - "component_id": component_id, + "message_type": message_type, "node_ip_address": node_ip_address, "component_type": component_type, "start_time": event.start_time, @@ -265,7 +265,7 @@ def profile_table(self): "extra_data": extra_data, } - result[component_id].append(profile_event) + result[message_type].append(profile_event) return dict(result) @@ -459,7 +459,7 @@ def chrome_tracing_dump(self, filename=None): profile_table = self.profile_table() all_events = [] - for component_id_hex, component_events in profile_table.items(): + for message_type_hex, component_events in profile_table.items(): # Only consider workers and drivers. component_type = component_events[0]["component_type"] if component_type not in ["worker", "driver"]: @@ -475,7 +475,7 @@ def chrome_tracing_dump(self, filename=None): # appears in. "pid": event["node_ip_address"], # The identifier for the row that the event appears in. - "tid": event["component_type"] + ":" + event["component_id"], + "tid": event["component_type"] + ":" + event["message_type"], # The start time in microseconds. "ts": self._seconds_to_microseconds(event["start_time"]), # The duration in microseconds. diff --git a/src/mock/ray/common/ray_syncer/ray_syncer.h b/src/mock/ray/common/ray_syncer/ray_syncer.h index f18c9d4d3d1b..0f768dab64c3 100644 --- a/src/mock/ray/common/ray_syncer/ray_syncer.h +++ b/src/mock/ray/common/ray_syncer/ray_syncer.h @@ -19,7 +19,7 @@ class MockReporterInterface : public ReporterInterface { public: MOCK_METHOD(std::optional, CreateSyncMessage, - (int64_t current_version, RayComponentId component_id), + (int64_t current_version, MessageType message_type), (const, override)); }; diff --git a/src/ray/common/ray_syncer/ray_syncer-inl.h b/src/ray/common/ray_syncer/ray_syncer-inl.h index b7b2ed6cf1dc..c6a9571dd035 100644 --- a/src/ray/common/ray_syncer/ray_syncer-inl.h +++ b/src/ray/common/ray_syncer/ray_syncer-inl.h @@ -36,7 +36,7 @@ class NodeState { /// received messages are consumed. /// /// \return true if set successfully. - bool SetComponent(RayComponentId cid, + bool SetComponent(MessageType cid, const ReporterInterface *reporter, ReceiverInterface *receiver); @@ -45,7 +45,7 @@ class NodeState { /// \param cid The component id to take the snapshot. /// /// \return If a snapshot is taken, return the message, otherwise std::nullopt. - std::optional CreateSyncMessage(RayComponentId cid); + std::optional CreateSyncMessage(MessageType cid); /// Consume a message. Receiver will consume this message if it doesn't have /// this message. @@ -127,7 +127,7 @@ class NodeSyncConnection { std::function)> message_processor_; /// Buffering all the updates. Sending will be done in an async way. - absl::flat_hash_map, + absl::flat_hash_map, std::shared_ptr> sending_buffer_; diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index 3bb47098f0e2..d61aa2c50603 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -22,22 +22,22 @@ namespace syncer { NodeState::NodeState() { sync_message_versions_taken_.fill(-1); } -bool NodeState::SetComponent(RayComponentId cid, +bool NodeState::SetComponent(MessageType cid, const ReporterInterface *reporter, ReceiverInterface *receiver) { - if (cid < static_cast(kComponentArraySize) && + if (cid < static_cast(kComponentArraySize) && reporters_[cid] == nullptr && receivers_[cid] == nullptr) { reporters_[cid] = reporter; receivers_[cid] = receiver; return true; } else { - RAY_LOG(FATAL) << "Fail to set components, component_id:" << cid + RAY_LOG(FATAL) << "Fail to set components, message_type:" << cid << ", reporter:" << reporter << ", receiver:" << receiver; return false; } } -std::optional NodeState::CreateSyncMessage(RayComponentId cid) { +std::optional NodeState::CreateSyncMessage(MessageType cid) { if (reporters_[cid] == nullptr) { return std::nullopt; } @@ -53,7 +53,7 @@ std::optional NodeState::CreateSyncMessage(RayComponentId cid) { } bool NodeState::ConsumeSyncMessage(std::shared_ptr message) { - auto ¤t = cluster_view_[message->node_id()][message->component_id()]; + auto ¤t = cluster_view_[message->node_id()][message->message_type()]; RAY_LOG(DEBUG) << "ConsumeSyncMessage: " << (current ? current->version() : -1) << " message_version: " << message->version() @@ -64,7 +64,7 @@ bool NodeState::ConsumeSyncMessage(std::shared_ptr message } current = message; - auto receiver = receivers_[message->component_id()]; + auto receiver = receivers_[message->message_type()]; if (receiver != nullptr) { receiver->ConsumeSyncMessage(message); } @@ -83,11 +83,11 @@ void NodeSyncConnection::ReceiveUpdate(RaySyncMessages messages) { for (auto &message : *messages.mutable_sync_messages()) { auto &node_versions = GetNodeComponentVersions(message.node_id()); RAY_LOG(DEBUG) << "Receive update: " - << " component_id=" << message.component_id() + << " message_type=" << message.message_type() << ", message_version=" << message.version() - << ", local_message_version=" << node_versions[message.component_id()]; - if (node_versions[message.component_id()] < message.version()) { - node_versions[message.component_id()] = message.version(); + << ", local_message_version=" << node_versions[message.message_type()]; + if (node_versions[message.message_type()] < message.version()) { + node_versions[message.message_type()] = message.version(); message_processor_(std::make_shared(std::move(message))); } } @@ -104,9 +104,9 @@ bool NodeSyncConnection::PushToSendingQueue( } auto &node_versions = GetNodeComponentVersions(message->node_id()); - if (node_versions[message->component_id()] < message->version()) { - node_versions[message->component_id()] = message->version(); - sending_buffer_[std::make_pair(message->node_id(), message->component_id())] = + if (node_versions[message->message_type()] < message->version()) { + node_versions[message->message_type()] = message->version(); + sending_buffer_[std::make_pair(message->node_id(), message->message_type())] = message; return true; } @@ -323,30 +323,30 @@ void RaySyncer::Disconnect(const std::string &node_id) { "RaySyncerDisconnect"); } -bool RaySyncer::Register(RayComponentId component_id, +bool RaySyncer::Register(MessageType message_type, const ReporterInterface *reporter, ReceiverInterface *receiver, int64_t pull_from_reporter_interval_ms) { - if (!node_state_->SetComponent(component_id, reporter, receiver)) { + if (!node_state_->SetComponent(message_type, reporter, receiver)) { return false; } // Set job to pull from reporter periodically if (reporter != nullptr && pull_from_reporter_interval_ms > 0) { timer_.RunFnPeriodically( - [this, component_id]() { OnDemandBroadcasting(component_id); }, + [this, message_type]() { OnDemandBroadcasting(message_type); }, pull_from_reporter_interval_ms); } RAY_LOG(DEBUG) << "Registered components: " - << "component_id:" << component_id << ", reporter:" << reporter + << "message_type:" << message_type << ", reporter:" << reporter << ", receiver:" << receiver << ", pull_from_reporter_interval_ms:" << pull_from_reporter_interval_ms; return true; } -bool RaySyncer::OnDemandBroadcasting(RayComponentId component_id) { - auto msg = node_state_->CreateSyncMessage(component_id); +bool RaySyncer::OnDemandBroadcasting(MessageType message_type) { + auto msg = node_state_->CreateSyncMessage(message_type); if (msg) { RAY_CHECK(msg->node_id() == GetLocalNodeID()); BroadcastMessage(std::make_shared(std::move(*msg))); diff --git a/src/ray/common/ray_syncer/ray_syncer.h b/src/ray/common/ray_syncer/ray_syncer.h index 7725b518c734..78fc34d95b55 100644 --- a/src/ray/common/ray_syncer/ray_syncer.h +++ b/src/ray/common/ray_syncer/ray_syncer.h @@ -29,14 +29,14 @@ namespace syncer { using ray::rpc::syncer::DummyRequest; using ray::rpc::syncer::DummyResponse; -using ray::rpc::syncer::RayComponentId; +using ray::rpc::syncer::MessageType; using ray::rpc::syncer::RaySyncMessage; using ray::rpc::syncer::RaySyncMessages; using ray::rpc::syncer::StartSyncRequest; using ray::rpc::syncer::StartSyncResponse; static constexpr size_t kComponentArraySize = - static_cast(ray::rpc::syncer::RayComponentId_ARRAYSIZE); + static_cast(ray::rpc::syncer::MessageType_ARRAYSIZE); /// The interface for a reporter. Reporter is defined to be a local module which would /// like to let the other nodes know its state. For example, local cluster resource @@ -49,13 +49,13 @@ struct ReporterInterface { /// /// \param version_after Request message with version after `version_after`. If the /// reporter doesn't have the qualified one, just return std::nullopt - /// \param component_id The component id asked for. + /// \param message_type The component id asked for. /// /// \return std::nullopt if the reporter doesn't have such component or the current /// snapshot of the component is not newer the asked one. Otherwise, return the /// actual message. virtual std::optional CreateSyncMessage( - int64_t version_after, RayComponentId component_id) const = 0; + int64_t version_after, MessageType message_type) const = 0; virtual ~ReporterInterface() {} }; @@ -114,14 +114,14 @@ class RaySyncer { /// it'll have a global view of the cluster. /// /// - /// \param component_id The component to sync. + /// \param message_type The component to sync. /// \param reporter The local component to be broadcasted. /// \param receiver The consumer of the sync message sent by the other nodes in the /// cluster. /// \param pull_from_reporter_interval_ms The frequence to pull a message. 0 means /// never pull a message in syncer. /// from reporter and push it to sending queue. - bool Register(RayComponentId component_id, + bool Register(MessageType message_type, const ReporterInterface *reporter, ReceiverInterface *receiver, int64_t pull_from_reporter_interval_ms = 100); @@ -132,10 +132,10 @@ class RaySyncer { /// Request trigger a broadcasting for a specific component immediately instead of /// waiting for ray syncer to poll the message. /// - /// \param component_id The component to check. + /// \param message_type The component to check. /// \return true if a message is generated. If the component doesn't have a new /// version of message, false will be returned. - bool OnDemandBroadcasting(RayComponentId component_id); + bool OnDemandBroadcasting(MessageType message_type); private: /// Get the io_context used by RaySyncer. diff --git a/src/ray/common/test/ray_syncer_test.cc b/src/ray/common/test/ray_syncer_test.cc index 896f8153f135..5160b744c4ca 100644 --- a/src/ray/common/test/ray_syncer_test.cc +++ b/src/ray/common/test/ray_syncer_test.cc @@ -44,10 +44,10 @@ namespace syncer { constexpr size_t kTestComponents = 1; -RaySyncMessage MakeMessage(RayComponentId cid, int64_t version, const NodeID &id) { +RaySyncMessage MakeMessage(MessageType cid, int64_t version, const NodeID &id) { auto msg = RaySyncMessage(); msg.set_version(version); - msg.set_component_id(cid); + msg.set_message_type(cid); msg.set_node_id(id.Binary()); return msg; } @@ -66,7 +66,7 @@ class RaySyncerTest : public ::testing::Test { return std::nullopt; } else { auto msg = RaySyncMessage(); - msg.set_component_id(static_cast(cid)); + msg.set_message_type(static_cast(cid)); msg.set_version(++local_versions_[cid]); return std::make_optional(std::move(msg)); } @@ -82,15 +82,15 @@ class RaySyncerTest : public ::testing::Test { syncer_ = std::make_unique(io_context_, local_id_.Binary()); } - MockReporterInterface *GetReporter(RayComponentId cid) { + MockReporterInterface *GetReporter(MessageType cid) { return reporters_[static_cast(cid)].get(); } - MockReceiverInterface *GetReceiver(RayComponentId cid) { + MockReceiverInterface *GetReceiver(MessageType cid) { return receivers_[static_cast(cid)].get(); } - int64_t &LocalVersion(RayComponentId cid) { + int64_t &LocalVersion(MessageType cid) { return local_versions_[static_cast(cid)]; } @@ -114,32 +114,32 @@ class RaySyncerTest : public ::testing::Test { TEST_F(RaySyncerTest, NodeStateCreateSyncMessage) { auto node_status = std::make_unique(); - node_status->SetComponent(RayComponentId::RESOURCE_MANAGER, nullptr, nullptr); + node_status->SetComponent(MessageType::RESOURCE_MANAGER, nullptr, nullptr); ASSERT_EQ(std::nullopt, - node_status->CreateSyncMessage(RayComponentId::RESOURCE_MANAGER)); + node_status->CreateSyncMessage(MessageType::RESOURCE_MANAGER)); auto reporter = std::make_unique(); - ASSERT_TRUE(node_status->SetComponent(RayComponentId::RESOURCE_MANAGER, - GetReporter(RayComponentId::RESOURCE_MANAGER), + ASSERT_TRUE(node_status->SetComponent(MessageType::RESOURCE_MANAGER, + GetReporter(MessageType::RESOURCE_MANAGER), nullptr)); // Take a snapshot - auto msg = node_status->CreateSyncMessage(RayComponentId::RESOURCE_MANAGER); - ASSERT_EQ(LocalVersion(RayComponentId::RESOURCE_MANAGER), msg->version()); + auto msg = node_status->CreateSyncMessage(MessageType::RESOURCE_MANAGER); + ASSERT_EQ(LocalVersion(MessageType::RESOURCE_MANAGER), msg->version()); // Revert one version back. - LocalVersion(RayComponentId::RESOURCE_MANAGER) -= 1; - msg = node_status->CreateSyncMessage(RayComponentId::RESOURCE_MANAGER); + LocalVersion(MessageType::RESOURCE_MANAGER) -= 1; + msg = node_status->CreateSyncMessage(MessageType::RESOURCE_MANAGER); ASSERT_EQ(std::nullopt, msg); } TEST_F(RaySyncerTest, NodeStateConsume) { auto node_status = std::make_unique(); - node_status->SetComponent(RayComponentId::RESOURCE_MANAGER, + node_status->SetComponent(MessageType::RESOURCE_MANAGER, nullptr, - GetReceiver(RayComponentId::RESOURCE_MANAGER)); + GetReceiver(MessageType::RESOURCE_MANAGER)); auto from_node_id = NodeID::FromRandom(); // The first time receiver the message - auto msg = MakeMessage(RayComponentId::RESOURCE_MANAGER, 0, from_node_id); + auto msg = MakeMessage(MessageType::RESOURCE_MANAGER, 0, from_node_id); ASSERT_TRUE(node_status->ConsumeSyncMessage(std::make_shared(msg))); ASSERT_FALSE(node_status->ConsumeSyncMessage(std::make_shared(msg))); @@ -156,7 +156,7 @@ TEST_F(RaySyncerTest, NodeSyncConnection) { node_id.Binary(), [](std::shared_ptr) {}); auto from_node_id = NodeID::FromRandom(); - auto msg = MakeMessage(RayComponentId::RESOURCE_MANAGER, 0, from_node_id); + auto msg = MakeMessage(MessageType::RESOURCE_MANAGER, 0, from_node_id); // First push will succeed and the second one will be deduplicated. ASSERT_TRUE(sync_connection.PushToSendingQueue(std::make_shared(msg))); @@ -166,7 +166,7 @@ TEST_F(RaySyncerTest, NodeSyncConnection) { ASSERT_EQ(1, sync_connection.node_versions_.size()); ASSERT_EQ(0, sync_connection - .node_versions_[from_node_id.Binary()][RayComponentId::RESOURCE_MANAGER]); + .node_versions_[from_node_id.Binary()][MessageType::RESOURCE_MANAGER]); msg.set_version(2); ASSERT_TRUE(sync_connection.PushToSendingQueue(std::make_shared(msg))); @@ -177,7 +177,7 @@ TEST_F(RaySyncerTest, NodeSyncConnection) { ASSERT_EQ(2, sync_connection.sending_buffer_.begin()->second->version()); ASSERT_EQ(2, sync_connection - .node_versions_[from_node_id.Binary()][RayComponentId::RESOURCE_MANAGER]); + .node_versions_[from_node_id.Binary()][MessageType::RESOURCE_MANAGER]); } struct SyncerServerTest { @@ -208,7 +208,7 @@ struct SyncerServerTest { iter = received_versions.find(message->node_id()); } - received_versions[message->node_id()][message->component_id()] = + received_versions[message->node_id()][message->message_type()] = message->version(); message_consumed[message->node_id()]++; }; @@ -222,7 +222,7 @@ struct SyncerServerTest { return std::nullopt; } else { auto msg = RaySyncMessage(); - msg.set_component_id(static_cast(cid)); + msg.set_message_type(static_cast(cid)); msg.set_version(local_versions[cid]); msg.set_node_id(syncer->GetLocalNodeID()); snapshot_taken++; @@ -233,7 +233,7 @@ struct SyncerServerTest { EXPECT_CALL(*reporter, CreateSyncMessage(_, Eq(cid))) .WillRepeatedly(WithArg<0>(Invoke(take_snapshot))); syncer->Register( - static_cast(cid), reporter.get(), receivers[cid].get()); + static_cast(cid), reporter.get(), receivers[cid].get()); } thread = std::make_unique([this] { boost::asio::io_context::work work(io_context); @@ -377,10 +377,10 @@ TEST(SyncerTest, Test1To1) { auto s2 = SyncerServerTest("19991"); // Make sure the setup is correct - ASSERT_NE(nullptr, s1.receivers[RayComponentId::RESOURCE_MANAGER]); - ASSERT_NE(nullptr, s2.receivers[RayComponentId::RESOURCE_MANAGER]); - ASSERT_NE(nullptr, s1.reporters[RayComponentId::RESOURCE_MANAGER]); - ASSERT_NE(nullptr, s2.reporters[RayComponentId::RESOURCE_MANAGER]); + ASSERT_NE(nullptr, s1.receivers[MessageType::RESOURCE_MANAGER]); + ASSERT_NE(nullptr, s2.receivers[MessageType::RESOURCE_MANAGER]); + ASSERT_NE(nullptr, s1.reporters[MessageType::RESOURCE_MANAGER]); + ASSERT_NE(nullptr, s2.reporters[MessageType::RESOURCE_MANAGER]); auto channel_to_s2 = MakeChannel("19991"); @@ -421,7 +421,7 @@ TEST(SyncerTest, Test1To1) { // Make sure s2 send the new message to s1. ASSERT_TRUE(s1.WaitUntil( [&s1, node_id = s2.syncer->GetLocalNodeID()]() { - return s1.GetReceivedVersions(node_id)[RayComponentId::RESOURCE_MANAGER] == 1 && + return s1.GetReceivedVersions(node_id)[MessageType::RESOURCE_MANAGER] == 1 && s1.GetNumConsumedMessages(node_id) == 2; }, 5)); @@ -634,11 +634,11 @@ bool TestCorrectness(std::function get_cluster_ for (size_t i = 0; i < 1000000; ++i) { auto server_idx = choose_server(gen); - auto component_id = choose_component(gen); + auto message_type = choose_component(gen); if (server_idx == 0) { - component_id = 0; + message_type = 0; } - servers[server_idx]->local_versions[component_id]++; + servers[server_idx]->local_versions[message_type]++; // expect to sleep for 100 times for the whole loop. if (rand_sleep(gen) < 100) { std::this_thread::sleep_for(100ms); diff --git a/src/ray/common/test/syncer_service_e2e_test.cc b/src/ray/common/test/syncer_service_e2e_test.cc index 7673b21d792e..fa629c8b4572 100644 --- a/src/ray/common/test/syncer_service_e2e_test.cc +++ b/src/ray/common/test/syncer_service_e2e_test.cc @@ -49,12 +49,12 @@ class LocalNode : public ReporterInterface { } std::optional CreateSyncMessage(int64_t current_version, - RayComponentId) const override { + MessageType) const override { if (current_version > version_) { return std::nullopt; } ray::rpc::syncer::RaySyncMessage msg; - msg.set_component_id(ray::rpc::syncer::RayComponentId::RESOURCE_MANAGER); + msg.set_message_type(ray::rpc::syncer::MessageType::RESOURCE_MANAGER); msg.set_version(version_); msg.set_sync_message( std::string(reinterpret_cast(&state_), sizeof(state_))); @@ -103,7 +103,7 @@ int main(int argc, char *argv[]) { std::unique_ptr service; std::unique_ptr server; std::shared_ptr channel; - syncer.Register(ray::rpc::syncer::RayComponentId::RESOURCE_MANAGER, + syncer.Register(ray::rpc::syncer::MessageType::RESOURCE_MANAGER, local_node.get(), remote_node.get()); if (server_port != ".") { diff --git a/src/ray/core_worker/profiling.cc b/src/ray/core_worker/profiling.cc index 1d9b4d306939..4aa56a192b0f 100644 --- a/src/ray/core_worker/profiling.cc +++ b/src/ray/core_worker/profiling.cc @@ -37,7 +37,7 @@ Profiler::Profiler(WorkerContext &worker_context, rpc_profile_data_(new rpc::ProfileTableData()), gcs_client_(gcs_client) { rpc_profile_data_->set_component_type(WorkerTypeString(worker_context.GetWorkerType())); - rpc_profile_data_->set_component_id(worker_context.GetWorkerID().Binary()); + rpc_profile_data_->set_message_type(worker_context.GetWorkerID().Binary()); rpc_profile_data_->set_node_ip_address(node_ip_address); periodical_runner_.RunFnPeriodically( [this] { FlushEvents(); }, @@ -56,7 +56,7 @@ void Profiler::FlushEvents() { absl::MutexLock lock(&mutex_); if (rpc_profile_data_->profile_events_size() != 0) { cur_profile_data->set_component_type(rpc_profile_data_->component_type()); - cur_profile_data->set_component_id(rpc_profile_data_->component_id()); + cur_profile_data->set_message_type(rpc_profile_data_->message_type()); cur_profile_data->set_node_ip_address(rpc_profile_data_->node_ip_address()); rpc_profile_data_.swap(cur_profile_data); } diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 0346e7951f2e..34a9bc85e4e5 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -780,7 +780,7 @@ StatsInfoAccessor::StatsInfoAccessor(GcsClient *client_impl) Status StatsInfoAccessor::AsyncAddProfileData( const std::shared_ptr &data_ptr, const StatusCallback &callback) { - NodeID node_id = NodeID::FromBinary(data_ptr->component_id()); + NodeID node_id = NodeID::FromBinary(data_ptr->message_type()); RAY_LOG(DEBUG) << "Adding profile data, component type = " << data_ptr->component_type() << ", node id = " << node_id; rpc::AddProfileDataRequest request; diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index 5b1435e61f92..813a3e3a7472 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -41,8 +41,7 @@ void GcsResourceManager::ConsumeSyncMessage( rpc::ResourcesData resources; resources.ParseFromString(message->sync_message()); resources.set_node_id(message->node_id()); - auto node_id = NodeID::FromBinary(message->node_id()); - RAY_CHECK(message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER); + RAY_CHECK(message->message_type() == syncer::MessageType::RESOURCE_MANAGER); UpdateFromResourceReport(resources); }, "GcsResourceManager::Update"); diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index fc2f8f6a4998..173ee4dad127 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -463,7 +463,7 @@ void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { ray_syncer_ = std::make_unique(ray_syncer_io_context_, gcs_node_id_.Binary()); ray_syncer_->Register( - syncer::RayComponentId::RESOURCE_MANAGER, nullptr, gcs_resource_manager_.get()); + syncer::MessageType::RESOURCE_MANAGER, nullptr, gcs_resource_manager_.get()); ray_syncer_thread_ = std::make_unique([this]() { boost::asio::io_service::work work(ray_syncer_io_context_); ray_syncer_io_context_.run(); diff --git a/src/ray/gcs/gcs_server/stats_handler_impl.cc b/src/ray/gcs/gcs_server/stats_handler_impl.cc index 2d9425140630..628299b92f54 100644 --- a/src/ray/gcs/gcs_server/stats_handler_impl.cc +++ b/src/ray/gcs/gcs_server/stats_handler_impl.cc @@ -22,7 +22,7 @@ namespace rpc { void DefaultStatsHandler::HandleAddProfileData(const AddProfileDataRequest &request, AddProfileDataReply *reply, SendReplyCallback send_reply_callback) { - NodeID node_id = NodeID::FromBinary(request.profile_data().component_id()); + NodeID node_id = NodeID::FromBinary(request.profile_data().message_type()); RAY_LOG(DEBUG) << "Adding profile data, component type = " << request.profile_data().component_type() << ", node id = " << node_id; auto profile_table_data = std::make_shared(); 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 fe7863baec23..7f04673918b5 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 @@ -378,7 +378,7 @@ TEST_F(GcsServerTest, TestHeartbeatWithNoRegistering) { TEST_F(GcsServerTest, TestStats) { rpc::ProfileTableData profile_table_data; - profile_table_data.set_component_id(NodeID::FromRandom().Binary()); + profile_table_data.set_message_type(NodeID::FromRandom().Binary()); rpc::AddProfileDataRequest add_profile_data_request; add_profile_data_request.mutable_profile_data()->CopyFrom(profile_table_data); ASSERT_TRUE(AddProfileData(add_profile_data_request)); diff --git a/src/ray/gcs/test/gcs_test_util.h b/src/ray/gcs/test/gcs_test_util.h index e9cca3866307..d9a6db039090 100644 --- a/src/ray/gcs/test/gcs_test_util.h +++ b/src/ray/gcs/test/gcs_test_util.h @@ -222,7 +222,7 @@ struct Mocker { static std::shared_ptr GenProfileTableData( const NodeID &node_id) { auto profile_table_data = std::make_shared(); - profile_table_data->set_component_id(node_id.Binary()); + profile_table_data->set_message_type(node_id.Binary()); return profile_table_data; } diff --git a/src/ray/protobuf/gcs.proto b/src/ray/protobuf/gcs.proto index d77ade189939..67013a2a40e3 100644 --- a/src/ray/protobuf/gcs.proto +++ b/src/ray/protobuf/gcs.proto @@ -187,7 +187,7 @@ message ProfileTableData { // object_manager, or node_manager. string component_type = 1; // An identifier for the component that generated the event. - bytes component_id = 2; + bytes message_type = 2; // An identifier for the node that generated the event. string node_ip_address = 3; // This is a batch of profiling events. We batch these together for diff --git a/src/ray/protobuf/ray_syncer.proto b/src/ray/protobuf/ray_syncer.proto index 7b277704bea9..e9386278b11d 100644 --- a/src/ray/protobuf/ray_syncer.proto +++ b/src/ray/protobuf/ray_syncer.proto @@ -16,7 +16,7 @@ syntax = "proto3"; option cc_enable_arenas = true; package ray.rpc.syncer; -enum RayComponentId { +enum MessageType { RESOURCE_MANAGER = 0; COMMANDS = 1; } @@ -24,8 +24,8 @@ enum RayComponentId { message RaySyncMessage { // The version of the message. -1 means the version is not set. int64 version = 1; - // The component this message is for. - RayComponentId component_id = 2; + // The type of this message. + MessageType message_type = 2; // The actual payload. bytes sync_message = 3; // The node id which initially sent this message. diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index b4dbb3f47126..b203d65dac28 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -550,7 +550,7 @@ ray::Status NodeManager::RegisterGcs() { if (RayConfig::instance().use_ray_syncer()) { // Register resource manager and scheduler ray_syncer_.Register( - /* component_id */ syncer::RayComponentId::RESOURCE_MANAGER, + /* message_type */ syncer::MessageType::RESOURCE_MANAGER, /* reporter */ &cluster_resource_scheduler_->GetLocalResourceManager(), /* receiver */ this, /* pull_from_reporter_interval_ms */ @@ -559,7 +559,7 @@ ray::Status NodeManager::RegisterGcs() { // Register a commands channel. // It's only used for GC right now. ray_syncer_.Register( - /* component_id */ syncer::RayComponentId::COMMANDS, + /* message_type */ syncer::MessageType::COMMANDS, /* reporter */ this, /* receiver */ this, /* pull_from_reporter_interval_ms */ 0); @@ -570,7 +570,7 @@ ray::Status NodeManager::RegisterGcs() { // If plasma store is under high pressure, we should try to schedule a global // gc. if (triggered_by_global_gc) { - ray_syncer_.OnDemandBroadcasting(syncer::RayComponentId::COMMANDS); + ray_syncer_.OnDemandBroadcasting(syncer::MessageType::COMMANDS); } }, RayConfig::instance().raylet_check_gc_period_milliseconds(), @@ -1814,7 +1814,7 @@ void NodeManager::HandleCommitBundleResources( placement_group_resource_manager_->CommitBundles(bundle_specs); if (RayConfig::instance().use_ray_syncer()) { // To reduce the lag, we trigger a broadcasting immediately. - RAY_CHECK(ray_syncer_.OnDemandBroadcasting(syncer::RayComponentId::RESOURCE_MANAGER)); + RAY_CHECK(ray_syncer_.OnDemandBroadcasting(syncer::MessageType::RESOURCE_MANAGER)); } send_reply_callback(Status::OK(), nullptr, nullptr); @@ -1855,7 +1855,7 @@ void NodeManager::HandleCancelResourceReserve( placement_group_resource_manager_->ReturnBundle(bundle_spec); if (RayConfig::instance().use_ray_syncer()) { // To reduce the lag, we trigger a broadcasting immediately. - RAY_CHECK(ray_syncer_.OnDemandBroadcasting(syncer::RayComponentId::RESOURCE_MANAGER)); + RAY_CHECK(ray_syncer_.OnDemandBroadcasting(syncer::MessageType::RESOURCE_MANAGER)); } cluster_task_manager_->ScheduleAndDispatchTasks(); send_reply_callback(Status::OK(), nullptr, nullptr); @@ -2696,12 +2696,12 @@ void NodeManager::RecordMetrics() { void NodeManager::ConsumeSyncMessage( std::shared_ptr message) { - if (message->component_id() == syncer::RayComponentId::RESOURCE_MANAGER) { + if (message->message_type() == syncer::MessageType::RESOURCE_MANAGER) { rpc::ResourcesData data; data.ParseFromString(message->sync_message()); NodeID node_id = NodeID::FromBinary(data.node_id()); UpdateResourceUsage(node_id, data); - } else if (message->component_id() == syncer::RayComponentId::COMMANDS) { + } else if (message->message_type() == syncer::MessageType::COMMANDS) { rpc::ResourcesData data; data.ParseFromString(message->sync_message()); if (data.should_global_gc()) { @@ -2711,15 +2711,15 @@ void NodeManager::ConsumeSyncMessage( } std::optional NodeManager::CreateSyncMessage( - int64_t after_version, syncer::RayComponentId component_id) const { - RAY_CHECK(component_id == syncer::RayComponentId::COMMANDS); + int64_t after_version, syncer::MessageType message_type) const { + RAY_CHECK(message_type == syncer::MessageType::COMMANDS); rpc::ResourcesData resources_data; resources_data.set_should_global_gc(true); syncer::RaySyncMessage msg; msg.set_version(absl::GetCurrentTimeNanos()); msg.set_node_id(self_node_id_.Binary()); - msg.set_component_id(syncer::RayComponentId::COMMANDS); + msg.set_message_type(syncer::MessageType::COMMANDS); std::string serialized_msg; RAY_CHECK(resources_data.SerializeToString(&serialized_msg)); msg.set_sync_message(std::move(serialized_msg)); diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index f9ca2696318d..abccaa968d6c 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -194,7 +194,7 @@ class NodeManager : public rpc::NodeManagerServiceHandler, void ConsumeSyncMessage(std::shared_ptr message) override; std::optional CreateSyncMessage( - int64_t after_version, syncer::RayComponentId component_id) const override; + int64_t after_version, syncer::MessageType message_type) const override; int GetObjectManagerPort() const { return object_manager_.GetServerPort(); } diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 878e267400c4..6eb1ff425896 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -366,8 +366,8 @@ double LocalResourceManager::GetLocalAvailableCpus() const { } std::optional LocalResourceManager::CreateSyncMessage( - int64_t after_version, syncer::RayComponentId component_id) const { - RAY_CHECK(component_id == syncer::RayComponentId::RESOURCE_MANAGER); + int64_t after_version, syncer::MessageType message_type) const { + RAY_CHECK(message_type == syncer::MessageType::RESOURCE_MANAGER); // We check the memory inside version, so version is not a const function. // Ideally, we need to move the memory check somewhere else. // TODO(iycheng): Make version as a const function. @@ -404,7 +404,7 @@ std::optional LocalResourceManager::CreateSyncMessage( msg.set_node_id(local_node_id_.Binary()); msg.set_version(curr_version); - msg.set_component_id(component_id); + msg.set_message_type(message_type); std::string serialized_msg; RAY_CHECK(resources_data.SerializeToString(&serialized_msg)); msg.set_sync_message(std::move(serialized_msg)); diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index 0e1f804cb123..745a851184d6 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -152,7 +152,7 @@ class LocalResourceManager : public syncer::ReporterInterface { bool ResourcesExist(scheduling::ResourceID resource_id) const; std::optional CreateSyncMessage( - int64_t after_version, syncer::RayComponentId component_id) const override; + int64_t after_version, syncer::MessageType message_type) const override; private: /// Notify the subscriber that the local resouces has changed. From f8361246051d3043204d0ae49f578a4ebcd9cb19 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 29 Apr 2022 06:26:28 +0000 Subject: [PATCH 65/77] format --- src/ray/common/ray_syncer/ray_syncer.cc | 4 ++-- src/ray/common/test/ray_syncer_test.cc | 8 +++----- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index d61aa2c50603..eef4b53c8254 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -25,8 +25,8 @@ NodeState::NodeState() { sync_message_versions_taken_.fill(-1); } bool NodeState::SetComponent(MessageType cid, const ReporterInterface *reporter, ReceiverInterface *receiver) { - if (cid < static_cast(kComponentArraySize) && - reporters_[cid] == nullptr && receivers_[cid] == nullptr) { + if (cid < static_cast(kComponentArraySize) && reporters_[cid] == nullptr && + receivers_[cid] == nullptr) { reporters_[cid] = reporter; receivers_[cid] = receiver; return true; diff --git a/src/ray/common/test/ray_syncer_test.cc b/src/ray/common/test/ray_syncer_test.cc index 5160b744c4ca..870054717b42 100644 --- a/src/ray/common/test/ray_syncer_test.cc +++ b/src/ray/common/test/ray_syncer_test.cc @@ -115,8 +115,7 @@ class RaySyncerTest : public ::testing::Test { TEST_F(RaySyncerTest, NodeStateCreateSyncMessage) { auto node_status = std::make_unique(); node_status->SetComponent(MessageType::RESOURCE_MANAGER, nullptr, nullptr); - ASSERT_EQ(std::nullopt, - node_status->CreateSyncMessage(MessageType::RESOURCE_MANAGER)); + ASSERT_EQ(std::nullopt, node_status->CreateSyncMessage(MessageType::RESOURCE_MANAGER)); auto reporter = std::make_unique(); ASSERT_TRUE(node_status->SetComponent(MessageType::RESOURCE_MANAGER, @@ -134,9 +133,8 @@ TEST_F(RaySyncerTest, NodeStateCreateSyncMessage) { TEST_F(RaySyncerTest, NodeStateConsume) { auto node_status = std::make_unique(); - node_status->SetComponent(MessageType::RESOURCE_MANAGER, - nullptr, - GetReceiver(MessageType::RESOURCE_MANAGER)); + node_status->SetComponent( + MessageType::RESOURCE_MANAGER, nullptr, GetReceiver(MessageType::RESOURCE_MANAGER)); auto from_node_id = NodeID::FromRandom(); // The first time receiver the message auto msg = MakeMessage(MessageType::RESOURCE_MANAGER, 0, from_node_id); From c13a22c7888809a32c7d3f71ef809e222ff0ff2b Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 29 Apr 2022 06:28:33 +0000 Subject: [PATCH 66/77] revert serve --- python/ray/serve/controller.py | 2 +- python/ray/serve/http_proxy.py | 2 +- python/ray/serve/logging_utils.py | 10 +++++----- python/ray/serve/replica.py | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/python/ray/serve/controller.py b/python/ray/serve/controller.py index a4bfd6657b52..530072e3d8a4 100644 --- a/python/ray/serve/controller.py +++ b/python/ray/serve/controller.py @@ -77,7 +77,7 @@ async def __init__( _override_controller_namespace: Optional[str] = None, ): configure_component_logger( - component_name="controller", message_type=str(os.getpid()) + component_name="controller", component_id=str(os.getpid()) ) # Used to read/write checkpoints. diff --git a/python/ray/serve/http_proxy.py b/python/ray/serve/http_proxy.py index b699fab20bed..171dd131c08d 100644 --- a/python/ray/serve/http_proxy.py +++ b/python/ray/serve/http_proxy.py @@ -337,7 +337,7 @@ def __init__( http_middlewares: Optional[List["starlette.middleware.Middleware"]] = None, ): # noqa: F821 configure_component_logger( - component_name="http_proxy", message_type=node_id_to_ip_addr(node_id) + component_name="http_proxy", component_id=node_id_to_ip_addr(node_id) ) if http_middlewares is None: diff --git a/python/ray/serve/logging_utils.py b/python/ray/serve/logging_utils.py index 23f773f91569..268e1efa65cc 100644 --- a/python/ray/serve/logging_utils.py +++ b/python/ray/serve/logging_utils.py @@ -5,8 +5,8 @@ import ray from ray.serve.constants import DEBUG_LOG_ENV_VAR, SERVE_LOGGER_NAME -COMPONENT_LOG_FMT = "%(levelname)s %(asctime)s {component_name} {message_type} %(filename)s:%(lineno)d - %(message)s" # noqa:E501 -LOG_FILE_FMT = "{component_name}_{message_type}.log" +COMPONENT_LOG_FMT = "%(levelname)s %(asctime)s {component_name} {component_id} %(filename)s:%(lineno)d - %(message)s" # noqa:E501 +LOG_FILE_FMT = "{component_name}_{component_id}.log" def access_log_msg(*, method: str, route: str, status: str, latency_ms: float): @@ -17,7 +17,7 @@ def access_log_msg(*, method: str, route: str, status: str, latency_ms: float): def configure_component_logger( *, component_name: str, - message_type: str, + component_id: str, component_type: Optional[str] = None, log_level: int = logging.INFO, log_to_stream: bool = True, @@ -38,7 +38,7 @@ def configure_component_logger( formatter = logging.Formatter( COMPONENT_LOG_FMT.format( - component_name=component_name, message_type=message_type + component_name=component_name, component_id=component_id ) ) if log_to_stream: @@ -52,7 +52,7 @@ def configure_component_logger( if component_type is not None: component_name = f"{component_type}_{component_name}" log_file_name = LOG_FILE_FMT.format( - component_name=component_name, message_type=message_type + component_name=component_name, component_id=component_id ) file_handler = logging.FileHandler(os.path.join(logs_dir, log_file_name)) file_handler.setFormatter(formatter) diff --git a/python/ray/serve/replica.py b/python/ray/serve/replica.py index 45783d881524..f6b0a554669a 100644 --- a/python/ray/serve/replica.py +++ b/python/ray/serve/replica.py @@ -75,7 +75,7 @@ async def __init__( configure_component_logger( component_type="deployment", component_name=deployment_name, - message_type=replica_tag, + component_id=replica_tag, ) if import_path is not None: From b66b907ae444ef8e3998bf7daa6399bc92036052 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Fri, 29 Apr 2022 06:29:23 +0000 Subject: [PATCH 67/77] revert state --- python/ray/state.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/python/ray/state.py b/python/ray/state.py index f068aeda6170..9f4223345f2f 100644 --- a/python/ray/state.py +++ b/python/ray/state.py @@ -247,7 +247,7 @@ def profile_table(self): profile = gcs_utils.ProfileTableData.FromString(profile_table[i]) component_type = profile.component_type - message_type = binary_to_hex(profile.message_type) + component_id = binary_to_hex(profile.component_id) node_ip_address = profile.node_ip_address for event in profile.profile_events: @@ -257,7 +257,7 @@ def profile_table(self): extra_data = {} profile_event = { "event_type": event.event_type, - "message_type": message_type, + "component_id": component_id, "node_ip_address": node_ip_address, "component_type": component_type, "start_time": event.start_time, @@ -265,7 +265,7 @@ def profile_table(self): "extra_data": extra_data, } - result[message_type].append(profile_event) + result[component_id].append(profile_event) return dict(result) @@ -459,7 +459,7 @@ def chrome_tracing_dump(self, filename=None): profile_table = self.profile_table() all_events = [] - for message_type_hex, component_events in profile_table.items(): + for component_id_hex, component_events in profile_table.items(): # Only consider workers and drivers. component_type = component_events[0]["component_type"] if component_type not in ["worker", "driver"]: @@ -475,7 +475,7 @@ def chrome_tracing_dump(self, filename=None): # appears in. "pid": event["node_ip_address"], # The identifier for the row that the event appears in. - "tid": event["component_type"] + ":" + event["message_type"], + "tid": event["component_type"] + ":" + event["component_id"], # The start time in microseconds. "ts": self._seconds_to_microseconds(event["start_time"]), # The duration in microseconds. From a11c98e3ae169740d8a4e1a89b6e18b6a29e3575 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 9 May 2022 20:52:18 +0000 Subject: [PATCH 68/77] up --- src/ray/gcs/gcs_server/gcs_resource_manager.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.h b/src/ray/gcs/gcs_server/gcs_resource_manager.h index 7bc60004fb2b..3b5eda1ddee2 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.h +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.h @@ -56,7 +56,6 @@ class GcsResourceManager : public rpc::NodeResourceInfoHandler, ClusterResourceManager &cluster_resource_manager, scheduling::NodeID local_node_id_ = scheduling::NodeID::Nil()); - virtual ~GcsResourceManager() {} /// Handle the resource update. From 4e6e81eb64120bd1eeed1c2272a91a306675cb04 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 9 May 2022 21:20:09 +0000 Subject: [PATCH 69/77] merge and fix --- src/ray/common/ray_syncer/ray_syncer-inl.h | 8 +++---- src/ray/common/ray_syncer/ray_syncer.cc | 24 +++++++++---------- .../gcs/gcs_server/gcs_resource_manager.cc | 1 - src/ray/gcs/gcs_server/gcs_resource_manager.h | 9 +++---- src/ray/gcs/gcs_server/gcs_server.cc | 3 +-- src/ray/rpc/grpc_client.h | 13 +++++----- 6 files changed, 29 insertions(+), 29 deletions(-) diff --git a/src/ray/common/ray_syncer/ray_syncer-inl.h b/src/ray/common/ray_syncer/ray_syncer-inl.h index c6a9571dd035..17948622f37f 100644 --- a/src/ray/common/ray_syncer/ray_syncer-inl.h +++ b/src/ray/common/ray_syncer/ray_syncer-inl.h @@ -25,7 +25,7 @@ class NodeState { /// Set the local component. /// - /// \param cid The component id. + /// \param message_type The type of the message for this component. /// \param reporter The reporter is defined to be the local module which wants to /// broadcast its internal status to the whole clsuter. When it's null, it means there /// is no reporter in the local node for this component. This is the place there @@ -36,16 +36,16 @@ class NodeState { /// received messages are consumed. /// /// \return true if set successfully. - bool SetComponent(MessageType cid, + bool SetComponent(MessageType message_type, const ReporterInterface *reporter, ReceiverInterface *receiver); /// Get the snapshot of a component for a newer version. /// - /// \param cid The component id to take the snapshot. + /// \param message_type The component to take the snapshot. /// /// \return If a snapshot is taken, return the message, otherwise std::nullopt. - std::optional CreateSyncMessage(MessageType cid); + std::optional CreateSyncMessage(MessageType message_type); /// Consume a message. Receiver will consume this message if it doesn't have /// this message. diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index eef4b53c8254..3972efd58114 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -22,30 +22,30 @@ namespace syncer { NodeState::NodeState() { sync_message_versions_taken_.fill(-1); } -bool NodeState::SetComponent(MessageType cid, +bool NodeState::SetComponent(MessageType message_type, const ReporterInterface *reporter, ReceiverInterface *receiver) { - if (cid < static_cast(kComponentArraySize) && reporters_[cid] == nullptr && - receivers_[cid] == nullptr) { - reporters_[cid] = reporter; - receivers_[cid] = receiver; + if (message_type < static_cast(kComponentArraySize) && + reporters_[message_type] == nullptr && receivers_[message_type] == nullptr) { + reporters_[message_type] = reporter; + receivers_[message_type] = receiver; return true; } else { - RAY_LOG(FATAL) << "Fail to set components, message_type:" << cid + RAY_LOG(FATAL) << "Fail to set components, message_type:" << message_type << ", reporter:" << reporter << ", receiver:" << receiver; return false; } } -std::optional NodeState::CreateSyncMessage(MessageType cid) { - if (reporters_[cid] == nullptr) { +std::optional NodeState::CreateSyncMessage(MessageType message_type) { + if (reporters_[message_type] == nullptr) { return std::nullopt; } - auto message = - reporters_[cid]->CreateSyncMessage(sync_message_versions_taken_[cid], cid); + auto message = reporters_[message_type]->CreateSyncMessage( + sync_message_versions_taken_[message_type], message_type); if (message != std::nullopt) { - sync_message_versions_taken_[cid] = message->version(); - RAY_LOG(DEBUG) << "Sync message taken: cid:" << cid + sync_message_versions_taken_[message_type] = message->version(); + RAY_LOG(DEBUG) << "Sync message taken: message_type:" << message_type << ", version:" << message->version() << ", node:" << NodeID::FromBinary(message->node_id()); } diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index a08e12341bed..bc1f7e2a3f31 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -30,7 +30,6 @@ GcsResourceManager::GcsResourceManager( cluster_resource_manager_(cluster_resource_manager), local_node_id_(local_node_id) {} - void GcsResourceManager::ConsumeSyncMessage( std::shared_ptr message) { // ConsumeSyncMessage is called by ray_syncer which might not run diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.h b/src/ray/gcs/gcs_server/gcs_resource_manager.h index 3b5eda1ddee2..db191cff359e 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.h +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.h @@ -51,10 +51,11 @@ class GcsResourceManager : public rpc::NodeResourceInfoHandler, /// Create a GcsResourceManager. /// /// \param gcs_table_storage GCS table external storage accessor. - explicit GcsResourceManager(instrumented_io_context &io_context, - std::shared_ptr gcs_table_storage, - ClusterResourceManager &cluster_resource_manager, - scheduling::NodeID local_node_id_ = scheduling::NodeID::Nil()); + explicit GcsResourceManager( + instrumented_io_context &io_context, + std::shared_ptr gcs_table_storage, + ClusterResourceManager &cluster_resource_manager, + scheduling::NodeID local_node_id_ = scheduling::NodeID::Nil()); virtual ~GcsResourceManager() {} diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index eb92acc2dbb4..3a2e396bc317 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -257,8 +257,7 @@ void GcsServer::InitGcsResourceManager(const GcsInitData &gcs_init_data) { main_service_, gcs_table_storage_, cluster_resource_scheduler_->GetClusterResourceManager()); - scheduling::NodeID(local_node_id_.Binary())); - + scheduling::NodeID(local_node_id_.Binary()); // Initialize by gcs tables data. gcs_resource_manager_->Initialize(gcs_init_data); diff --git a/src/ray/rpc/grpc_client.h b/src/ray/rpc/grpc_client.h index 22c8c8f693df..d71cc69e494a 100644 --- a/src/ray/rpc/grpc_client.h +++ b/src/ray/rpc/grpc_client.h @@ -89,7 +89,8 @@ class GrpcClient { ClientCallManager &call_manager, bool use_tls = false) : client_call_manager_(call_manager), use_tls_(use_tls) { - stub_ = GrpcService::NewStub(std::move(channel)); + channel_ = std::move(channel); + stub_ = GrpcService::NewStub(channel_); } GrpcClient(const std::string &address, @@ -98,9 +99,7 @@ class GrpcClient { bool use_tls = false) : client_call_manager_(call_manager), use_tls_(use_tls) { std::shared_ptr channel = BuildChannel(address, port); - - channel_ = BuildChannel(argument, address, port); - + channel_ = BuildChannel(address, port); stub_ = GrpcService::NewStub(channel_); } @@ -118,8 +117,8 @@ class GrpcClient { argument.SetMaxSendMessageSize(::RayConfig::instance().max_grpc_message_size()); argument.SetMaxReceiveMessageSize(::RayConfig::instance().max_grpc_message_size()); - channel_ = BuildChannel(argument, address, port); - stub_ = GrpcService::NewStub(std::move(channel)); + channel_ = BuildChannel(address, port, argument); + stub_ = GrpcService::NewStub(channel_); } /// Create a new `ClientCall` and send request. @@ -161,6 +160,8 @@ class GrpcClient { std::unique_ptr stub_; /// Whether to use TLS. bool use_tls_; + /// The channel of the stub. + std::shared_ptr channel_; }; } // namespace rpc From bf4223936da2a91b6b003275dda8440f737a2f98 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 9 May 2022 21:38:30 +0000 Subject: [PATCH 70/77] fix comment --- src/ray/common/ray_syncer/ray_syncer.h | 2 +- src/ray/raylet/scheduling/local_resource_manager.cc | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/ray/common/ray_syncer/ray_syncer.h b/src/ray/common/ray_syncer/ray_syncer.h index 78fc34d95b55..acc57541ad9e 100644 --- a/src/ray/common/ray_syncer/ray_syncer.h +++ b/src/ray/common/ray_syncer/ray_syncer.h @@ -49,7 +49,7 @@ struct ReporterInterface { /// /// \param version_after Request message with version after `version_after`. If the /// reporter doesn't have the qualified one, just return std::nullopt - /// \param message_type The component id asked for. + /// \param message_type The message type asked for. /// /// \return std::nullopt if the reporter doesn't have such component or the current /// snapshot of the component is not newer the asked one. Otherwise, return the diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 6eb1ff425896..de8e6df4dfea 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -371,8 +371,9 @@ std::optional LocalResourceManager::CreateSyncMessage( // We check the memory inside version, so version is not a const function. // Ideally, we need to move the memory check somewhere else. // TODO(iycheng): Make version as a const function. - auto curr_version = const_cast(this)->Version(); - if (curr_version <= after_version) { + const_cast(this)->UpdateAvailableObjectStoreMemResource(); + + if (version_ <= after_version) { return std::nullopt; } From e185a34b2e27c4e9420dcdda7dbb6fbd2e945eaa Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 9 May 2022 21:45:15 +0000 Subject: [PATCH 71/77] fix comment --- src/ray/raylet/scheduling/local_resource_manager.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index de8e6df4dfea..75537ec7e4de 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -404,7 +404,7 @@ std::optional LocalResourceManager::CreateSyncMessage( resources_data.set_resources_available_changed(true); msg.set_node_id(local_node_id_.Binary()); - msg.set_version(curr_version); + msg.set_version(version_); msg.set_message_type(message_type); std::string serialized_msg; RAY_CHECK(resources_data.SerializeToString(&serialized_msg)); From 263fcecf5e8f691781dc56d734f53e6d9f624403 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 9 May 2022 21:46:27 +0000 Subject: [PATCH 72/77] fix comment --- src/ray/raylet/scheduling/local_resource_manager.h | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index 745a851184d6..f857a87313b9 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -108,11 +108,6 @@ class LocalResourceManager : public syncer::ReporterInterface { void ReleaseWorkerResources(std::shared_ptr task_allocation); - int64_t Version() { - UpdateAvailableObjectStoreMemResource(); - return version_; - } - /// Populate the relevant parts of the heartbeat table. This is intended for /// sending resource usage of raylet to gcs. In particular, this should fill in /// resources_available and resources_total. @@ -260,7 +255,7 @@ class LocalResourceManager : public syncer::ReporterInterface { // Specify custom resources that consists of unit-size instances. std::unordered_set custom_unit_instance_resources_{}; - // Version of this resource. It will incr by one whenever when the state changed. + // Version of this resource. It will incr by one whenever the state changed. int64_t version_ = 0; FRIEND_TEST(ClusterResourceSchedulerTest, SchedulingUpdateTotalResourcesTest); From d18834aabbbd29a35db0c61c706d951687ff60c5 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Mon, 9 May 2022 22:58:19 +0000 Subject: [PATCH 73/77] fix comments --- src/ray/common/ray_syncer/ray_syncer.h | 2 +- src/ray/common/test/ray_syncer_test.cc | 45 +++++++++---------- .../common/test/syncer_service_e2e_test.cc | 7 ++- .../gcs/gcs_server/gcs_resource_manager.cc | 2 +- src/ray/gcs/gcs_server/gcs_server.cc | 2 +- src/ray/protobuf/ray_syncer.proto | 2 +- src/ray/raylet/node_manager.cc | 8 ++-- .../scheduling/local_resource_manager.cc | 2 +- 8 files changed, 34 insertions(+), 36 deletions(-) diff --git a/src/ray/common/ray_syncer/ray_syncer.h b/src/ray/common/ray_syncer/ray_syncer.h index acc57541ad9e..42880200c802 100644 --- a/src/ray/common/ray_syncer/ray_syncer.h +++ b/src/ray/common/ray_syncer/ray_syncer.h @@ -114,7 +114,7 @@ class RaySyncer { /// it'll have a global view of the cluster. /// /// - /// \param message_type The component to sync. + /// \param message_type The message type of the component. /// \param reporter The local component to be broadcasted. /// \param receiver The consumer of the sync message sent by the other nodes in the /// cluster. diff --git a/src/ray/common/test/ray_syncer_test.cc b/src/ray/common/test/ray_syncer_test.cc index 870054717b42..d38e2fcfb1aa 100644 --- a/src/ray/common/test/ray_syncer_test.cc +++ b/src/ray/common/test/ray_syncer_test.cc @@ -114,30 +114,29 @@ class RaySyncerTest : public ::testing::Test { TEST_F(RaySyncerTest, NodeStateCreateSyncMessage) { auto node_status = std::make_unique(); - node_status->SetComponent(MessageType::RESOURCE_MANAGER, nullptr, nullptr); - ASSERT_EQ(std::nullopt, node_status->CreateSyncMessage(MessageType::RESOURCE_MANAGER)); + node_status->SetComponent(MessageType::RESOURCE_VIEW, nullptr, nullptr); + ASSERT_EQ(std::nullopt, node_status->CreateSyncMessage(MessageType::RESOURCE_VIEW)); auto reporter = std::make_unique(); - ASSERT_TRUE(node_status->SetComponent(MessageType::RESOURCE_MANAGER, - GetReporter(MessageType::RESOURCE_MANAGER), - nullptr)); + ASSERT_TRUE(node_status->SetComponent( + MessageType::RESOURCE_VIEW, GetReporter(MessageType::RESOURCE_VIEW), nullptr)); // Take a snapshot - auto msg = node_status->CreateSyncMessage(MessageType::RESOURCE_MANAGER); - ASSERT_EQ(LocalVersion(MessageType::RESOURCE_MANAGER), msg->version()); + auto msg = node_status->CreateSyncMessage(MessageType::RESOURCE_VIEW); + ASSERT_EQ(LocalVersion(MessageType::RESOURCE_VIEW), msg->version()); // Revert one version back. - LocalVersion(MessageType::RESOURCE_MANAGER) -= 1; - msg = node_status->CreateSyncMessage(MessageType::RESOURCE_MANAGER); + LocalVersion(MessageType::RESOURCE_VIEW) -= 1; + msg = node_status->CreateSyncMessage(MessageType::RESOURCE_VIEW); ASSERT_EQ(std::nullopt, msg); } TEST_F(RaySyncerTest, NodeStateConsume) { auto node_status = std::make_unique(); node_status->SetComponent( - MessageType::RESOURCE_MANAGER, nullptr, GetReceiver(MessageType::RESOURCE_MANAGER)); + MessageType::RESOURCE_VIEW, nullptr, GetReceiver(MessageType::RESOURCE_VIEW)); auto from_node_id = NodeID::FromRandom(); // The first time receiver the message - auto msg = MakeMessage(MessageType::RESOURCE_MANAGER, 0, from_node_id); + auto msg = MakeMessage(MessageType::RESOURCE_VIEW, 0, from_node_id); ASSERT_TRUE(node_status->ConsumeSyncMessage(std::make_shared(msg))); ASSERT_FALSE(node_status->ConsumeSyncMessage(std::make_shared(msg))); @@ -154,7 +153,7 @@ TEST_F(RaySyncerTest, NodeSyncConnection) { node_id.Binary(), [](std::shared_ptr) {}); auto from_node_id = NodeID::FromRandom(); - auto msg = MakeMessage(MessageType::RESOURCE_MANAGER, 0, from_node_id); + auto msg = MakeMessage(MessageType::RESOURCE_VIEW, 0, from_node_id); // First push will succeed and the second one will be deduplicated. ASSERT_TRUE(sync_connection.PushToSendingQueue(std::make_shared(msg))); @@ -162,9 +161,9 @@ TEST_F(RaySyncerTest, NodeSyncConnection) { ASSERT_EQ(1, sync_connection.sending_buffer_.size()); ASSERT_EQ(0, sync_connection.sending_buffer_.begin()->second->version()); ASSERT_EQ(1, sync_connection.node_versions_.size()); - ASSERT_EQ(0, - sync_connection - .node_versions_[from_node_id.Binary()][MessageType::RESOURCE_MANAGER]); + ASSERT_EQ( + 0, + sync_connection.node_versions_[from_node_id.Binary()][MessageType::RESOURCE_VIEW]); msg.set_version(2); ASSERT_TRUE(sync_connection.PushToSendingQueue(std::make_shared(msg))); @@ -173,9 +172,9 @@ TEST_F(RaySyncerTest, NodeSyncConnection) { ASSERT_EQ(1, sync_connection.sending_buffer_.size()); ASSERT_EQ(1, sync_connection.node_versions_.size()); ASSERT_EQ(2, sync_connection.sending_buffer_.begin()->second->version()); - ASSERT_EQ(2, - sync_connection - .node_versions_[from_node_id.Binary()][MessageType::RESOURCE_MANAGER]); + ASSERT_EQ( + 2, + sync_connection.node_versions_[from_node_id.Binary()][MessageType::RESOURCE_VIEW]); } struct SyncerServerTest { @@ -375,10 +374,10 @@ TEST(SyncerTest, Test1To1) { auto s2 = SyncerServerTest("19991"); // Make sure the setup is correct - ASSERT_NE(nullptr, s1.receivers[MessageType::RESOURCE_MANAGER]); - ASSERT_NE(nullptr, s2.receivers[MessageType::RESOURCE_MANAGER]); - ASSERT_NE(nullptr, s1.reporters[MessageType::RESOURCE_MANAGER]); - ASSERT_NE(nullptr, s2.reporters[MessageType::RESOURCE_MANAGER]); + ASSERT_NE(nullptr, s1.receivers[MessageType::RESOURCE_VIEW]); + ASSERT_NE(nullptr, s2.receivers[MessageType::RESOURCE_VIEW]); + ASSERT_NE(nullptr, s1.reporters[MessageType::RESOURCE_VIEW]); + ASSERT_NE(nullptr, s2.reporters[MessageType::RESOURCE_VIEW]); auto channel_to_s2 = MakeChannel("19991"); @@ -419,7 +418,7 @@ TEST(SyncerTest, Test1To1) { // Make sure s2 send the new message to s1. ASSERT_TRUE(s1.WaitUntil( [&s1, node_id = s2.syncer->GetLocalNodeID()]() { - return s1.GetReceivedVersions(node_id)[MessageType::RESOURCE_MANAGER] == 1 && + return s1.GetReceivedVersions(node_id)[MessageType::RESOURCE_VIEW] == 1 && s1.GetNumConsumedMessages(node_id) == 2; }, 5)); diff --git a/src/ray/common/test/syncer_service_e2e_test.cc b/src/ray/common/test/syncer_service_e2e_test.cc index fa629c8b4572..08936d2f2ee9 100644 --- a/src/ray/common/test/syncer_service_e2e_test.cc +++ b/src/ray/common/test/syncer_service_e2e_test.cc @@ -54,7 +54,7 @@ class LocalNode : public ReporterInterface { return std::nullopt; } ray::rpc::syncer::RaySyncMessage msg; - msg.set_message_type(ray::rpc::syncer::MessageType::RESOURCE_MANAGER); + msg.set_message_type(ray::rpc::syncer::MessageType::RESOURCE_VIEW); msg.set_version(version_); msg.set_sync_message( std::string(reinterpret_cast(&state_), sizeof(state_))); @@ -103,9 +103,8 @@ int main(int argc, char *argv[]) { std::unique_ptr service; std::unique_ptr server; std::shared_ptr channel; - syncer.Register(ray::rpc::syncer::MessageType::RESOURCE_MANAGER, - local_node.get(), - remote_node.get()); + syncer.Register( + ray::rpc::syncer::MessageType::RESOURCE_VIEW, local_node.get(), remote_node.get()); if (server_port != ".") { RAY_LOG(INFO) << "Start server on port " << server_port; auto server_address = "0.0.0.0:" + server_port; diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index bc1f7e2a3f31..6bf9a628b936 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -43,7 +43,7 @@ void GcsResourceManager::ConsumeSyncMessage( rpc::ResourcesData resources; resources.ParseFromString(message->sync_message()); resources.set_node_id(message->node_id()); - RAY_CHECK(message->message_type() == syncer::MessageType::RESOURCE_MANAGER); + RAY_CHECK(message->message_type() == syncer::MessageType::RESOURCE_VIEW); UpdateFromResourceReport(resources); }, "GcsResourceManager::Update"); diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 3a2e396bc317..44b2bb0de1fb 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -477,7 +477,7 @@ void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { ray_syncer_ = std::make_unique(ray_syncer_io_context_, gcs_node_id_.Binary()); ray_syncer_->Register( - syncer::MessageType::RESOURCE_MANAGER, nullptr, gcs_resource_manager_.get()); + syncer::MessageType::RESOURCE_VIEW, nullptr, gcs_resource_manager_.get()); ray_syncer_thread_ = std::make_unique([this]() { boost::asio::io_service::work work(ray_syncer_io_context_); ray_syncer_io_context_.run(); diff --git a/src/ray/protobuf/ray_syncer.proto b/src/ray/protobuf/ray_syncer.proto index e9386278b11d..f171665ef6f0 100644 --- a/src/ray/protobuf/ray_syncer.proto +++ b/src/ray/protobuf/ray_syncer.proto @@ -17,7 +17,7 @@ option cc_enable_arenas = true; package ray.rpc.syncer; enum MessageType { - RESOURCE_MANAGER = 0; + RESOURCE_VIEW = 0; COMMANDS = 1; } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index d3a535f8007e..ba4ac6d5391f 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -550,7 +550,7 @@ ray::Status NodeManager::RegisterGcs() { if (RayConfig::instance().use_ray_syncer()) { // Register resource manager and scheduler ray_syncer_.Register( - /* message_type */ syncer::MessageType::RESOURCE_MANAGER, + /* message_type */ syncer::MessageType::RESOURCE_VIEW, /* reporter */ &cluster_resource_scheduler_->GetLocalResourceManager(), /* receiver */ this, /* pull_from_reporter_interval_ms */ @@ -1814,7 +1814,7 @@ void NodeManager::HandleCommitBundleResources( placement_group_resource_manager_->CommitBundles(bundle_specs); if (RayConfig::instance().use_ray_syncer()) { // To reduce the lag, we trigger a broadcasting immediately. - RAY_CHECK(ray_syncer_.OnDemandBroadcasting(syncer::MessageType::RESOURCE_MANAGER)); + RAY_CHECK(ray_syncer_.OnDemandBroadcasting(syncer::MessageType::RESOURCE_VIEW)); } send_reply_callback(Status::OK(), nullptr, nullptr); @@ -1855,7 +1855,7 @@ void NodeManager::HandleCancelResourceReserve( placement_group_resource_manager_->ReturnBundle(bundle_spec); if (RayConfig::instance().use_ray_syncer()) { // To reduce the lag, we trigger a broadcasting immediately. - RAY_CHECK(ray_syncer_.OnDemandBroadcasting(syncer::MessageType::RESOURCE_MANAGER)); + RAY_CHECK(ray_syncer_.OnDemandBroadcasting(syncer::MessageType::RESOURCE_VIEW)); } cluster_task_manager_->ScheduleAndDispatchTasks(); send_reply_callback(Status::OK(), nullptr, nullptr); @@ -2694,7 +2694,7 @@ void NodeManager::RecordMetrics() { void NodeManager::ConsumeSyncMessage( std::shared_ptr message) { - if (message->message_type() == syncer::MessageType::RESOURCE_MANAGER) { + if (message->message_type() == syncer::MessageType::RESOURCE_VIEW) { rpc::ResourcesData data; data.ParseFromString(message->sync_message()); NodeID node_id = NodeID::FromBinary(data.node_id()); diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index 75537ec7e4de..a74d7dbd5b67 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -367,7 +367,7 @@ double LocalResourceManager::GetLocalAvailableCpus() const { std::optional LocalResourceManager::CreateSyncMessage( int64_t after_version, syncer::MessageType message_type) const { - RAY_CHECK(message_type == syncer::MessageType::RESOURCE_MANAGER); + RAY_CHECK(message_type == syncer::MessageType::RESOURCE_VIEW); // We check the memory inside version, so version is not a const function. // Ideally, we need to move the memory check somewhere else. // TODO(iycheng): Make version as a const function. From 3825141269f72b4b391a58667e06bb73b887ac03 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Tue, 10 May 2022 00:49:13 +0000 Subject: [PATCH 74/77] fix compiling failure --- src/ray/gcs/gcs_server/gcs_server.cc | 5 ++--- src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc | 4 +++- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 44b2bb0de1fb..fc037f642774 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -48,6 +48,7 @@ GcsServer::GcsServer(const ray::gcs::GcsServerConfig &config, RayConfig::instance().gcs_server_rpc_client_thread_num()), raylet_client_pool_( std::make_shared(client_call_manager_)), + local_node_id_(NodeID::FromRandom()), pubsub_periodical_runner_(pubsub_io_service_), periodical_runner_(main_service), is_started_(false), @@ -302,7 +303,6 @@ void GcsServer::InitGcsResourceManager(const GcsInitData &gcs_init_data) { } void GcsServer::InitClusterResourceScheduler() { - local_node_id_ = NodeID::FromRandom(); cluster_resource_scheduler_ = std::make_shared( scheduling::NodeID(local_node_id_.Binary()), NodeResources(), @@ -473,9 +473,8 @@ void GcsServer::StoreGcsServerAddressInRedis() { void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { if (RayConfig::instance().use_ray_syncer()) { - gcs_node_id_ = NodeID::FromRandom(); ray_syncer_ = std::make_unique(ray_syncer_io_context_, - gcs_node_id_.Binary()); + local_node_id_.Binary()); ray_syncer_->Register( syncer::MessageType::RESOURCE_VIEW, nullptr, gcs_resource_manager_.get()); ray_syncer_thread_ = std::make_unique([this]() { diff --git a/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc b/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc index 132ca4dc8b51..5d112600ae87 100644 --- a/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc @@ -61,7 +61,9 @@ class GcsActorSchedulerTest : public ::testing::Test { /*local_task_manager=*/ nullptr); auto gcs_resource_manager = std::make_shared( - gcs_table_storage_, cluster_resource_scheduler->GetClusterResourceManager()); + io_service_, + gcs_table_storage_, + cluster_resource_scheduler->GetClusterResourceManager()); gcs_actor_scheduler_ = std::make_shared( io_service_, *gcs_actor_table_, From 77710ef3c6e4aa17348474bdeb230db674f2a557 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Tue, 10 May 2022 06:35:46 +0000 Subject: [PATCH 75/77] fix mis-merge --- src/ray/core_worker/profiling.cc | 4 ++-- src/ray/gcs/gcs_client/accessor.cc | 2 +- src/ray/gcs/gcs_server/stats_handler_impl.cc | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/ray/core_worker/profiling.cc b/src/ray/core_worker/profiling.cc index 4aa56a192b0f..1d9b4d306939 100644 --- a/src/ray/core_worker/profiling.cc +++ b/src/ray/core_worker/profiling.cc @@ -37,7 +37,7 @@ Profiler::Profiler(WorkerContext &worker_context, rpc_profile_data_(new rpc::ProfileTableData()), gcs_client_(gcs_client) { rpc_profile_data_->set_component_type(WorkerTypeString(worker_context.GetWorkerType())); - rpc_profile_data_->set_message_type(worker_context.GetWorkerID().Binary()); + rpc_profile_data_->set_component_id(worker_context.GetWorkerID().Binary()); rpc_profile_data_->set_node_ip_address(node_ip_address); periodical_runner_.RunFnPeriodically( [this] { FlushEvents(); }, @@ -56,7 +56,7 @@ void Profiler::FlushEvents() { absl::MutexLock lock(&mutex_); if (rpc_profile_data_->profile_events_size() != 0) { cur_profile_data->set_component_type(rpc_profile_data_->component_type()); - cur_profile_data->set_message_type(rpc_profile_data_->message_type()); + cur_profile_data->set_component_id(rpc_profile_data_->component_id()); cur_profile_data->set_node_ip_address(rpc_profile_data_->node_ip_address()); rpc_profile_data_.swap(cur_profile_data); } diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index 34a9bc85e4e5..0346e7951f2e 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -780,7 +780,7 @@ StatsInfoAccessor::StatsInfoAccessor(GcsClient *client_impl) Status StatsInfoAccessor::AsyncAddProfileData( const std::shared_ptr &data_ptr, const StatusCallback &callback) { - NodeID node_id = NodeID::FromBinary(data_ptr->message_type()); + NodeID node_id = NodeID::FromBinary(data_ptr->component_id()); RAY_LOG(DEBUG) << "Adding profile data, component type = " << data_ptr->component_type() << ", node id = " << node_id; rpc::AddProfileDataRequest request; diff --git a/src/ray/gcs/gcs_server/stats_handler_impl.cc b/src/ray/gcs/gcs_server/stats_handler_impl.cc index 628299b92f54..2d9425140630 100644 --- a/src/ray/gcs/gcs_server/stats_handler_impl.cc +++ b/src/ray/gcs/gcs_server/stats_handler_impl.cc @@ -22,7 +22,7 @@ namespace rpc { void DefaultStatsHandler::HandleAddProfileData(const AddProfileDataRequest &request, AddProfileDataReply *reply, SendReplyCallback send_reply_callback) { - NodeID node_id = NodeID::FromBinary(request.profile_data().message_type()); + NodeID node_id = NodeID::FromBinary(request.profile_data().component_id()); RAY_LOG(DEBUG) << "Adding profile data, component type = " << request.profile_data().component_type() << ", node id = " << node_id; auto profile_table_data = std::make_shared(); From eba9fc001b6b4f201d5e397b9f8082644dce12aa Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Tue, 10 May 2022 06:40:46 +0000 Subject: [PATCH 76/77] fix mis-update --- src/ray/gcs/gcs_server/test/gcs_server_rpc_test.cc | 2 +- src/ray/gcs/test/gcs_test_util.h | 2 +- src/ray/protobuf/gcs.proto | 2 +- 3 files changed, 3 insertions(+), 3 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 7f04673918b5..fe7863baec23 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 @@ -378,7 +378,7 @@ TEST_F(GcsServerTest, TestHeartbeatWithNoRegistering) { TEST_F(GcsServerTest, TestStats) { rpc::ProfileTableData profile_table_data; - profile_table_data.set_message_type(NodeID::FromRandom().Binary()); + profile_table_data.set_component_id(NodeID::FromRandom().Binary()); rpc::AddProfileDataRequest add_profile_data_request; add_profile_data_request.mutable_profile_data()->CopyFrom(profile_table_data); ASSERT_TRUE(AddProfileData(add_profile_data_request)); diff --git a/src/ray/gcs/test/gcs_test_util.h b/src/ray/gcs/test/gcs_test_util.h index d9a6db039090..e9cca3866307 100644 --- a/src/ray/gcs/test/gcs_test_util.h +++ b/src/ray/gcs/test/gcs_test_util.h @@ -222,7 +222,7 @@ struct Mocker { static std::shared_ptr GenProfileTableData( const NodeID &node_id) { auto profile_table_data = std::make_shared(); - profile_table_data->set_message_type(node_id.Binary()); + profile_table_data->set_component_id(node_id.Binary()); return profile_table_data; } diff --git a/src/ray/protobuf/gcs.proto b/src/ray/protobuf/gcs.proto index 57ad632ece7d..34dec8bd56b5 100644 --- a/src/ray/protobuf/gcs.proto +++ b/src/ray/protobuf/gcs.proto @@ -186,7 +186,7 @@ message ProfileTableData { // object_manager, or node_manager. string component_type = 1; // An identifier for the component that generated the event. - bytes message_type = 2; + bytes component_id = 2; // An identifier for the node that generated the event. string node_ip_address = 3; // This is a batch of profiling events. We batch these together for From ff56f8b2a8c2b0a958e087ead2cda510214f1d38 Mon Sep 17 00:00:00 2001 From: Yi Cheng Date: Tue, 10 May 2022 06:47:17 +0000 Subject: [PATCH 77/77] fix a bad merge --- src/ray/gcs/gcs_server/gcs_server.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index fc037f642774..6ea1a6f5dcb7 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -257,8 +257,8 @@ void GcsServer::InitGcsResourceManager(const GcsInitData &gcs_init_data) { gcs_resource_manager_ = std::make_shared( main_service_, gcs_table_storage_, - cluster_resource_scheduler_->GetClusterResourceManager()); - scheduling::NodeID(local_node_id_.Binary()); + cluster_resource_scheduler_->GetClusterResourceManager(), + scheduling::NodeID(local_node_id_.Binary())); // Initialize by gcs tables data. gcs_resource_manager_->Initialize(gcs_init_data);