This is an automated email from the ASF dual-hosted git repository.
abhishekrb pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git
The following commit(s) were added to refs/heads/master by this push:
new 33a9c290879 prometheus-emitter: disable label tracking when the TTL
isn't set (#18718)
33a9c290879 is described below
commit 33a9c2908792e66f0e6b524db21ea19b953da73d
Author: aho135 <[email protected]>
AuthorDate: Wed Nov 5 11:35:38 2025 -0800
prometheus-emitter: disable label tracking when the TTL isn't set (#18718)
Follow up to #18689 to not track labels if the TTL isn't set
Co-authored-by: Andrew Ho <[email protected]>
---
.../emitter/prometheus/DimensionsAndCollector.java | 5 ++++-
.../emitter/prometheus/PrometheusEmitterTest.java | 23 ++++++++++++++++++++++
2 files changed, 27 insertions(+), 1 deletion(-)
diff --git
a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java
b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java
index 0d75d45034b..bea5c02599f 100644
---
a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java
+++
b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java
@@ -72,10 +72,13 @@ public class DimensionsAndCollector
/**
* For each unique set of labelValues, keeps track of the amount of time
that has elapsed since its metric
- * value has been updated.
+ * value has been updated. Label tracking is only required if a metric TTL
has been configured
*/
public void resetLastUpdateTime(List<String> labelValues)
{
+ if (ttlSeconds == null) {
+ return;
+ }
labelValuesToStopwatch.compute(labelValues, (k, v) -> {
if (v != null) {
v.restart();
diff --git
a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java
b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java
index d7e1bcfcb94..88ce81629c4 100644
---
a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java
+++
b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java
@@ -575,4 +575,27 @@ public class PrometheusEmitterTest
emitter.close();
}
+ @Test
+ public void testLabelsNotTrackedWithTtlUnset()
+ {
+ PrometheusEmitterConfig flushPeriodNull = new
PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.exporter, "test",
null, 0, null, true, true, null, null, false, null);
+ PrometheusEmitterConfig flushPeriodSet = new
PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.exporter, "test",
null, 0, null, true, true, 3, null, false, null);
+
+ PrometheusEmitter emitter = new PrometheusEmitter(flushPeriodNull);
+ ServiceMetricEvent event = ServiceMetricEvent.builder()
+ .setMetric("segment/loadQueue/count", 10)
+ .setDimension("server", "historical1")
+ .build(ImmutableMap.of("service", "historical", "host",
"druid.test.cn"));
+ emitter.emit(event);
+ DimensionsAndCollector metric =
emitter.getMetrics().getRegisteredMetrics().get("segment/loadQueue/count");
+ Assert.assertEquals(0, metric.getLabelValuesToStopwatch().size());
+ emitter.close();
+ CollectorRegistry.defaultRegistry.clear();
+
+ emitter = new PrometheusEmitter(flushPeriodSet);
+ emitter.emit(event);
+ metric =
emitter.getMetrics().getRegisteredMetrics().get("segment/loadQueue/count");
+ Assert.assertEquals(1, metric.getLabelValuesToStopwatch().size());
+ emitter.close();
+ }
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]