Skip to content

Commit dce9219

Browse files
committed
[core][stats-die/04] kill STATS in the common component
Signed-off-by: Cuong Nguyen <can@anyscale.com>
1 parent ef36a1c commit dce9219

File tree

8 files changed

+88
-53
lines changed

8 files changed

+88
-53
lines changed

src/ray/common/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -281,6 +281,7 @@ ray_cc_library(
281281
],
282282
deps = [
283283
":event_stats",
284+
":metrics",
284285
":ray_config",
285286
"//src/ray/util:array",
286287
"//src/ray/util:function_traits",
@@ -311,6 +312,7 @@ ray_cc_library(
311312
"event_stats.h",
312313
],
313314
deps = [
315+
":metrics",
314316
":ray_config",
315317
"//src/ray/stats:stats_metric",
316318
"//src/ray/util:time",

src/ray/common/asio/instrumented_io_context.cc

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@
2020
#include "ray/common/asio/asio_chaos.h"
2121
#include "ray/common/asio/asio_util.h"
2222
#include "ray/stats/metric.h"
23-
#include "ray/stats/metric_defs.h"
2423

2524
namespace {
2625

@@ -35,7 +34,7 @@ void LagProbeLoop(instrumented_io_context &io_context,
3534
auto end = std::chrono::steady_clock::now();
3635
auto duration =
3736
std::chrono::duration_cast<std::chrono::milliseconds>(end - begin);
38-
ray::stats::STATS_io_context_event_loop_lag_ms.Record(
37+
io_context.io_context_event_loop_lag_ms_gauge_metric.Record(
3938
duration.count(),
4039
{
4140
{"Name", context_name.value_or(GetThreadName())},

src/ray/common/asio/instrumented_io_context.h

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
#include <string>
2020

2121
#include "ray/common/event_stats.h"
22+
#include "ray/common/metrics.h"
2223
#include "ray/common/ray_config.h"
2324
#include "ray/util/logging.h"
2425

@@ -58,6 +59,9 @@ class instrumented_io_context : public boost::asio::io_context {
5859

5960
EventTracker &stats() const { return *event_stats_; };
6061

62+
ray::stats::Gauge io_context_event_loop_lag_ms_gauge_metric{
63+
ray::GetIoContextEventLoopLagMsGaugeMetric()};
64+
6165
private:
6266
/// The event stats tracker to use to record asio handler stats to.
6367
std::shared_ptr<EventTracker> event_stats_;

src/ray/common/event_stats.cc

Lines changed: 25 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@
2121
#include <utility>
2222

2323
#include "ray/stats/metric.h"
24-
#include "ray/stats/metric_defs.h"
2524
#include "ray/util/time.h"
2625

2726
namespace {
@@ -64,20 +63,22 @@ std::shared_ptr<StatsHandle> EventTracker::RecordStart(
6463
++stats->stats.cum_count;
6564
curr_count = ++stats->stats.curr_count;
6665
}
66+
auto handle =
67+
std::make_shared<StatsHandle>(std::move(name),
68+
ray::current_time_ns() + expected_queueing_delay_ns,
69+
std::move(stats),
70+
global_stats_,
71+
emit_metrics,
72+
event_context_name);
6773

6874
if (emit_metrics) {
69-
ray::stats::STATS_operation_count.Record(1, event_context_name.value_or(name));
70-
ray::stats::STATS_operation_active_count.Record(curr_count,
71-
event_context_name.value_or(name));
75+
handle->operation_count_metric_.Record(1,
76+
{{"Name", event_context_name.value_or(name)}});
77+
handle->operation_active_gauge_metric_.Record(
78+
curr_count, {{"Name", event_context_name.value_or(name)}});
7279
}
7380

74-
return std::make_shared<StatsHandle>(
75-
std::move(name),
76-
ray::current_time_ns() + expected_queueing_delay_ns,
77-
std::move(stats),
78-
global_stats_,
79-
emit_metrics,
80-
event_context_name);
81+
return handle;
8182
}
8283

8384
void EventTracker::RecordEnd(std::shared_ptr<StatsHandle> handle) {
@@ -89,10 +90,11 @@ void EventTracker::RecordEnd(std::shared_ptr<StatsHandle> handle) {
8990

9091
if (handle->emit_stats) {
9192
// Update event-specific stats.
92-
ray::stats::STATS_operation_run_time_ms.Record(
93-
execution_time_ns / 1000000, handle->context_name.value_or(handle->event_name));
94-
ray::stats::STATS_operation_active_count.Record(
95-
curr_count, handle->context_name.value_or(handle->event_name));
93+
handle->operation_run_time_ms_histogram_metric_.Record(
94+
execution_time_ns / 1000000,
95+
{{"Name", handle->context_name.value_or(handle->event_name)}});
96+
handle->operation_active_gauge_metric_.Record(
97+
curr_count, {{"Name", handle->context_name.value_or(handle->event_name)}});
9698
}
9799

98100
handle->end_or_execution_recorded = true;
@@ -135,13 +137,15 @@ void EventTracker::RecordExecution(const std::function<void()> &fn,
135137

136138
if (handle->emit_stats) {
137139
// Update event-specific stats.
138-
ray::stats::STATS_operation_run_time_ms.Record(
139-
execution_time_ns / 1000000, handle->context_name.value_or(handle->event_name));
140-
ray::stats::STATS_operation_active_count.Record(
141-
curr_count, handle->context_name.value_or(handle->event_name));
140+
handle->operation_run_time_ms_histogram_metric_.Record(
141+
execution_time_ns / 1000000,
142+
{{"Name", handle->context_name.value_or(handle->event_name)}});
143+
handle->operation_active_gauge_metric_.Record(
144+
curr_count, {{"Name", handle->context_name.value_or(handle->event_name)}});
142145
// Update global stats.
143-
ray::stats::STATS_operation_queue_time_ms.Record(
144-
queue_time_ns / 1000000, handle->context_name.value_or(handle->event_name));
146+
handle->operation_queue_time_ms_histogram_metric_.Record(
147+
queue_time_ns / 1000000,
148+
{{"Name", handle->context_name.value_or(handle->event_name)}});
145149
}
146150

147151
{

src/ray/common/event_stats.h

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818

1919
#include "absl/container/flat_hash_map.h"
2020
#include "absl/synchronization/mutex.h"
21+
#include "ray/common/metrics.h"
2122
#include "ray/common/ray_config.h"
2223
#include "ray/util/logging.h"
2324

@@ -77,6 +78,14 @@ struct StatsHandle {
7778
const bool emit_stats;
7879
const std::optional<std::string> context_name;
7980

81+
ray::stats::Count operation_count_metric_{ray::GetOperationCountCounterMetric()};
82+
ray::stats::Gauge operation_active_gauge_metric_{
83+
ray::GetOperationActiveCountGaugeMetric()};
84+
ray::stats::Histogram operation_run_time_ms_histogram_metric_{
85+
ray::GetOperationRunTimeMsHistogramMetric()};
86+
ray::stats::Histogram operation_queue_time_ms_histogram_metric_{
87+
ray::GetOperationQueueTimeMsHistogramMetric()};
88+
8089
StatsHandle(std::string event_name_,
8190
const int64_t start_time_,
8291
std::shared_ptr<GuardedEventStats> handler_stats_,

src/ray/common/metrics.h

Lines changed: 47 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -86,4 +86,51 @@ inline ray::stats::Histogram GetSchedulerPlacementTimeSHistogramMetric() {
8686
};
8787
}
8888

89+
inline ray::stats::Gauge GetIoContextEventLoopLagMsGaugeMetric() {
90+
return ray::stats::Gauge{
91+
/*name=*/"io_context_event_loop_lag_ms",
92+
/*description=*/"The latency of a task from post to execution",
93+
/*unit=*/"ms",
94+
/*tag_keys=*/{"Name"},
95+
};
96+
}
97+
98+
inline ray::stats::Count GetOperationCountCounterMetric() {
99+
return ray::stats::Count{
100+
/*name=*/"operation_count",
101+
/*description=*/"operation count",
102+
/*unit=*/"",
103+
/*tag_keys=*/{"Name"},
104+
};
105+
}
106+
107+
inline ray::stats::Histogram GetOperationRunTimeMsHistogramMetric() {
108+
return ray::stats::Histogram{
109+
/*name=*/"operation_run_time_ms",
110+
/*description=*/"operation execution time",
111+
/*unit=*/"ms",
112+
/*boundaries=*/{1, 10, 100, 1000, 10000},
113+
/*tag_keys=*/{"Name"},
114+
};
115+
}
116+
117+
inline ray::stats::Histogram GetOperationQueueTimeMsHistogramMetric() {
118+
return ray::stats::Histogram{
119+
/*name=*/"operation_queue_time_ms",
120+
/*description=*/"operation queuing time",
121+
/*unit=*/"ms",
122+
/*boundaries=*/{1, 10, 100, 1000, 10000},
123+
/*tag_keys=*/{"Name"},
124+
};
125+
}
126+
127+
inline ray::stats::Gauge GetOperationActiveCountGaugeMetric() {
128+
return ray::stats::Gauge{
129+
/*name=*/"operation_active_count",
130+
/*description=*/"active operation number",
131+
/*unit=*/"",
132+
/*tag_keys=*/{"Name"},
133+
};
134+
}
135+
89136
} // namespace ray

src/ray/stats/metric_defs.cc

Lines changed: 0 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -74,27 +74,6 @@ DEFINE_stats(
7474
/// ===================== INTERNAL SYSTEM METRICS =================================
7575
/// ===============================================================================
7676

77-
DEFINE_stats(io_context_event_loop_lag_ms,
78-
"Latency of a task from post to execution",
79-
("Name"), // Name of the instrumented_io_context.
80-
(),
81-
ray::stats::GAUGE);
82-
83-
/// Event stats
84-
DEFINE_stats(operation_count, "operation count", ("Name"), (), ray::stats::COUNT);
85-
DEFINE_stats(operation_run_time_ms,
86-
"operation execution time",
87-
("Name"),
88-
({1, 10, 100, 1000, 10000}),
89-
ray::stats::HISTOGRAM);
90-
DEFINE_stats(operation_queue_time_ms,
91-
"operation queuing time",
92-
("Name"),
93-
({1, 10, 100, 1000, 10000}),
94-
ray::stats::HISTOGRAM);
95-
DEFINE_stats(
96-
operation_active_count, "active operation number", ("Name"), (), ray::stats::GAUGE);
97-
9877
/// Scheduler
9978
DEFINE_stats(
10079
scheduler_tasks,

src/ray/stats/metric_defs.h

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -42,15 +42,6 @@ namespace stats {
4242
/// ray_[component]_[metrics_name]_total (e.g., ray_pull_manager_total)
4343
///
4444

45-
/// ASIO stats
46-
DECLARE_stats(io_context_event_loop_lag_ms);
47-
48-
/// Event stats
49-
DECLARE_stats(operation_count);
50-
DECLARE_stats(operation_run_time_ms);
51-
DECLARE_stats(operation_queue_time_ms);
52-
DECLARE_stats(operation_active_count);
53-
5445
/// Scheduler
5546
DECLARE_stats(scheduler_failed_worker_startup_total);
5647
DECLARE_stats(scheduler_tasks);

0 commit comments

Comments
 (0)