Skip to content

[Core] Optimize open telemetry metric recording calls#59337

Merged
edoakes merged 8 commits intoray-project:masterfrom
sampan-s-nayak:optimize-otel
Dec 17, 2025
Merged

[Core] Optimize open telemetry metric recording calls#59337
edoakes merged 8 commits intoray-project:masterfrom
sampan-s-nayak:optimize-otel

Conversation

@sampan-s-nayak
Copy link
Contributor

@sampan-s-nayak sampan-s-nayak commented Dec 10, 2025

Description

this pr introduces the following optimizations in the opentelemetryMetricsRecorder and some of its consumers:

  • use asynchronous instruments wherever available (counter and up down counter)
  • introduce a batch api to record histogram metrics (to prevent lock contention caused by repeated set_metric_value() calls)
  • batch events received metric update in aggregator_agent instead of making individual calls

Signed-off-by: sampan <sampan@anyscale.com>
@sampan-s-nayak sampan-s-nayak requested a review from a team as a code owner December 10, 2025 05:39
@gemini-code-assist
Copy link
Contributor

Warning

You have reached your daily quota limit. Please wait up to 24 hours and I will start processing your requests again!

@sampan-s-nayak sampan-s-nayak added the go add ONLY when ready to merge, run all tests label Dec 10, 2025
filtered = frozenset(
(k, v) for k, v in tag_set if k not in high_cardinality_labels
)
aggregated[filtered] += val
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bug: Gauge aggregation now sums values instead of metric-specific function

The new _create_observable_callback method uses aggregated[filtered] += val for all metric types, including gauges. The previous implementation used MetricCardinality.get_aggregation_function(name)(values) which applies a metric-specific aggregation function. For metrics like "tasks" and "actors", this returns sum(values), but for other gauges it returns values[0] (the first value). The new code incorrectly sums all gauge values when high-cardinality labels are dropped and observations are aggregated, changing the semantics for gauges that aren't configured to use sum aggregation.

Fix in Cursor Fix in Web

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

only a problem when dealing with high cardinality tags, but handling this just in case

@jjyao
Copy link
Collaborator

jjyao commented Dec 10, 2025

Could you add perf numbers

@sampan-s-nayak
Copy link
Contributor Author

sampan-s-nayak commented Dec 10, 2025

@jjyao with these changes (and when enabling aggregator to gcs), stress_test_many_tasks.aws (None) (0) stage 3 time comes down to around normal numbers (1978.6315276622772, run: https://buildkite.com/ray-project/release/builds/71111) from stage_3_time = 3084.111141204834 (run: https://buildkite.com/ray-project/release/builds/69176#019ab3f6-8bef-4fe2-aeae-4571adab0090).

@sampan-s-nayak
Copy link
Contributor Author

sampan-s-nayak commented Dec 10, 2025

also personally I feel we should revisit this again in the future and in the long term consider moving either the reporter agent/aggregator agent out of the dashboard agent event loop (I prefer moving reporters metric handling logic out of the event loop as it is performing synchronous work). The way we emit histogram metrics can also be further improved (this is the part adding the most overhead right now).

@sampan-s-nayak sampan-s-nayak changed the title [Core] Optimise open telemetry metric recording calls [Core] Optimize open telemetry metric recording calls Dec 10, 2025
Signed-off-by: sampan <sampan@anyscale.com>
@ray-gardener ray-gardener bot added core Issues that should be addressed in Ray Core observability Issues related to the Ray Dashboard, Logging, Metrics, Tracing, and/or Profiling labels Dec 10, 2025
observations = self._counter_observations_by_name.get(
metric_name, {}
)
# Don't clear - counters are cumulative
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this might not be correct btw; if the sub-component (worker for example, but can also a ray data concept like operator) no longer emit metrics, this should be cleared (otherwise the metric will show as not changing in grafana instead of stop emitting values)

might be an acceptable behavior but i'm not sure

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

might also cause memory leak, explosion if a counter metric is never emitted from the set

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

in my experience with counter metrics, once you stop publishing metrics for a counter it just shows up as a straight line (with the value equal to the last emitted value), it dosent automatically get cleared.

also from prometheus docs:

A counter is a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart.

so reset is only possible during process restarts. https://prometheus.io/docs/concepts/metric_types/#counter

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

to hit memory issues we would need to be emitting a very large number of counters. do you think it is possible today? If yes then I can use cachetools.TTLCache to auto delete entries after a configured timeout

elif isinstance(instrument, metrics.Histogram):
instrument.record(value, attributes=tags)
if isinstance(instrument, metrics.Histogram):
# Filter out high cardinality labels.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this implements cardinality for histogram, which is a good but I would recommend to split it in a separated PR (dropping label for histogram probably needs a separated discussion, to make sure the logic is correct)

Copy link
Contributor Author

@sampan-s-nayak sampan-s-nayak Dec 11, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the existing implementation of set_metric_value() is also filtering out high cardinality labels for histogram metrics(we create tags dict at the start and then use it for all metric types).

Signed-off-by: sampan <sampan@anyscale.com>
continue
bucket_midpoint = bucket_midpoints[i]
for _ in range(bucket_count):
self._open_telemetry_metric_recorder.set_metric_value(
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

previously we would call set_metric_value() for every histogram observation (we dont have the actual value with us so we call set_metric_value() with value = bucket_midpoint bucket_count number of times). this is the primary codeblock responsible for blocking the dashboard event loop.

return sum
# Gauge metrics use metric-specific aggregation or default to first value
if metric_name in HIGH_CARDINALITY_GAUGE_AGGREGATION:
return HIGH_CARDINALITY_GAUGE_AGGREGATION[metric_name]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we flag in the documentation that this implementation just doesn't work for histogram types?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added doc + I now throw an error if metric type is histogram.

class MetricType(Enum):
"""Types of metrics supported by the telemetry system."""

GAUGE = 0
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe also call out why SUMMARY isn't supported (doesn't aggregate). Just for completeness.

Signed-off-by: sampan <sampan@anyscale.com>
@sampan-s-nayak
Copy link
Contributor Author

Addressed @ZacAttack comment

# Sum - add the value for the given tags.
self._sum_observations_by_name[name][tag_key] = (
self._sum_observations_by_name[name].get(tag_key, 0) + value
)
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bug: Unbounded counter tag storage growth

set_metric_value() accumulates counter/sum values in _counter_observations_by_name/_sum_observations_by_name keyed by the full tag set and the observable callback never clears or evicts them. If tag values churn or are high-cardinality, this can cause unbounded in-memory growth in long-running processes even when labels are later dropped during export.

Additional Locations (1)

Fix in Cursor Fix in Web

Signed-off-by: sampan <sampan@anyscale.com>
Copy link
Contributor

@ZacAttack ZacAttack left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good!

@edoakes edoakes merged commit 05e7efd into ray-project:master Dec 17, 2025
6 checks passed
zzchun pushed a commit to zzchun/ray that referenced this pull request Dec 18, 2025
)

## Description
this pr introduces the following optimizations in the
`opentelemetryMetricsRecorder` and some of its consumers:
- use asynchronous instruments wherever available (counter and up down
counter)
- introduce a batch api to record histogram metrics (to prevent lock
contention caused by repeated `set_metric_value()` calls)
- batch events received metric update in aggregator_agent instead of
making individual calls

---------

Signed-off-by: sampan <sampan@anyscale.com>
Co-authored-by: sampan <sampan@anyscale.com>
Yicheng-Lu-llll pushed a commit to Yicheng-Lu-llll/ray that referenced this pull request Dec 22, 2025
)

## Description
this pr introduces the following optimizations in the
`opentelemetryMetricsRecorder` and some of its consumers:
- use asynchronous instruments wherever available (counter and up down
counter)
- introduce a batch api to record histogram metrics (to prevent lock
contention caused by repeated `set_metric_value()` calls)
- batch events received metric update in aggregator_agent instead of
making individual calls

---------

Signed-off-by: sampan <sampan@anyscale.com>
Co-authored-by: sampan <sampan@anyscale.com>
edoakes pushed a commit that referenced this pull request Jan 7, 2026
## Description
update metrics export docs based on changes in
#59337

## Related issues
> Link related issues: "Fixes #1234", "Closes #1234", or "Related to
#1234".

## Additional information
> Optional: Add implementation details, API changes, usage examples,
screenshots, etc.

---------

Signed-off-by: sampan <sampan@anyscale.com>
Co-authored-by: sampan <sampan@anyscale.com>
AYou0207 pushed a commit to AYou0207/ray that referenced this pull request Jan 13, 2026
## Description
update metrics export docs based on changes in
ray-project#59337

## Related issues
> Link related issues: "Fixes ray-project#1234", "Closes ray-project#1234", or "Related to
ray-project#1234".

## Additional information
> Optional: Add implementation details, API changes, usage examples,
screenshots, etc.

---------

Signed-off-by: sampan <sampan@anyscale.com>
Co-authored-by: sampan <sampan@anyscale.com>
Signed-off-by: jasonwrwang <jasonwrwang@tencent.com>
lee1258561 pushed a commit to pinterest/ray that referenced this pull request Feb 3, 2026
## Description
update metrics export docs based on changes in
ray-project#59337

## Related issues
> Link related issues: "Fixes ray-project#1234", "Closes ray-project#1234", or "Related to
ray-project#1234".

## Additional information
> Optional: Add implementation details, API changes, usage examples,
screenshots, etc.

---------

Signed-off-by: sampan <sampan@anyscale.com>
Co-authored-by: sampan <sampan@anyscale.com>
ryanaoleary pushed a commit to ryanaoleary/ray that referenced this pull request Feb 3, 2026
## Description
update metrics export docs based on changes in
ray-project#59337

## Related issues
> Link related issues: "Fixes ray-project#1234", "Closes ray-project#1234", or "Related to
ray-project#1234".

## Additional information
> Optional: Add implementation details, API changes, usage examples,
screenshots, etc.

---------

Signed-off-by: sampan <sampan@anyscale.com>
Co-authored-by: sampan <sampan@anyscale.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Issues that should be addressed in Ray Core go add ONLY when ready to merge, run all tests observability Issues related to the Ray Dashboard, Logging, Metrics, Tracing, and/or Profiling

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants