Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions src/ray/raylet/scheduling/cluster_lease_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -412,6 +412,7 @@ const RayLease *ClusterLeaseManager::AnyPendingLeasesForResourceAcquisition(
void ClusterLeaseManager::RecordMetrics() const {
internal_stats_.RecordMetrics();
cluster_resource_scheduler_.GetLocalResourceManager().RecordMetrics();
cluster_resource_scheduler_.GetClusterResourceManager().RecordMetrics();
}

std::string ClusterLeaseManager::DebugStr() const {
Expand Down
5 changes: 5 additions & 0 deletions src/ray/raylet/scheduling/cluster_resource_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

#include "ray/common/grpc_util.h"
#include "ray/common/ray_config.h"
#include "ray/stats/metric_defs.h"

namespace ray {

Expand Down Expand Up @@ -302,4 +303,8 @@ void ClusterResourceManager::SetNodeLabels(
it->second.GetMutableLocalView()->labels = std::move(labels);
}

void ClusterResourceManager::RecordMetrics() const {
ray::stats::STATS_local_resource_view_node_count.Record(nodes_.size());
}

} // namespace ray
3 changes: 3 additions & 0 deletions src/ray/raylet/scheduling/cluster_resource_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,9 @@ class ClusterResourceManager {
std::string DebugString(
std::optional<size_t> max_num_nodes_to_include = std::nullopt) const;

/// Record metrics for the cluster resource manager.
void RecordMetrics() const;

BundleLocationIndex &GetBundleLocationIndex();

void SetNodeLabels(const scheduling::NodeID &node_id,
Expand Down
5 changes: 5 additions & 0 deletions src/ray/stats/metric_defs.cc
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,11 @@ DEFINE_stats(scheduler_placement_time_s,
("WorkloadType"),
({0.1, 1, 10, 100, 1000, 10000}, ),
ray::stats::HISTOGRAM);
DEFINE_stats(local_resource_view_node_count,
"Number of nodes tracked in the ClusterResourceManager.",
(),
(),
ray::stats::GAUGE);

/// Local Object Manager
DEFINE_stats(
Expand Down
1 change: 1 addition & 0 deletions src/ray/stats/metric_defs.h
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ DECLARE_stats(scheduler_failed_worker_startup_total);
DECLARE_stats(scheduler_tasks);
DECLARE_stats(scheduler_unscheduleable_tasks);
DECLARE_stats(scheduler_placement_time_s);
DECLARE_stats(local_resource_view_node_count);

/// Raylet Resource Manager
DECLARE_stats(resources);
Expand Down