abhishekrb19 commented on code in PR #18689:
URL: https://github.com/apache/druid/pull/18689#discussion_r2482655432


##########
docs/development/extensions-contrib/prometheus.md:
##########
@@ -44,7 +44,7 @@ All the configuration parameters for the Prometheus emitter 
are under `druid.emi
 | `druid.emitter.prometheus.addHostAsLabel`     | Flag to include the hostname 
as a prometheus label.                                                          
                                                                                
                                          | no        | false                   
             |
 | `druid.emitter.prometheus.addServiceAsLabel`  | Flag to include the druid 
service name (e.g. `druid/broker`, `druid/coordinator`, etc.) as a prometheus 
label.                                                                          
                                               | no        | false              
                  |
 | `druid.emitter.prometheus.pushGatewayAddress` | Pushgateway address. 
Required if using `pushgateway` strategy.                                       
                                                                                
                                                  | no        | none            
                     |
-| `druid.emitter.prometheus.flushPeriod`        | When using the `pushgateway` 
strategy metrics are emitted every `flushPeriod` seconds. <br/>When using the 
`exporter` strategy this configures the metric TTL such that if the metric 
value is not updated within `flushPeriod` seconds then it will stop being 
emitted. Note that unique label combinations per metric are currently not 
subject to TTL expiration. It is recommended to set this to at least 3 * 
`scrape_interval`. | Required if `pushgateway` strategy is used, optional 
otherwise. | 15 seconds for `pushgateway` strategy. <br/>None for `exporter` 
strategy. |
+| `druid.emitter.prometheus.flushPeriod`        | When using the `pushgateway` 
strategy metrics are emitted every `flushPeriod` seconds. <br/>When using the 
`exporter` strategy this configures the metric TTL such that if the metric 
value is not updated within `flushPeriod` seconds then it will stop being 
emitted. It is recommended to set this to at least 3 * `scrape_interval`. | 
Required if `pushgateway` strategy is used, optional otherwise. | 15 seconds 
for `pushgateway` strategy. <br/>None for `exporter` strategy. |

Review Comment:
   Should we perhaps amend this to say that this reset also applies to unique 
label combinations?
   



##########
extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java:
##########
@@ -507,14 +510,81 @@ public void testMetricTtlUpdate()
 
     Assert.assertFalse(
         "Metric should not be expired",
-        testMetric.isExpired()
+        testMetric.removeIfExpired(Arrays.asList("historical", 
"druid.test.cn", "historical1"))
     );
-    emitter.emit(event);
+    emitter.close();
+  }
+
+  @Test
+  public void testMetricTtlUpdateWithDifferentLabels()
+  {
+    int flushPeriod = 3;
+    PrometheusEmitterConfig config = new 
PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.exporter, "test", 
null, 0, null, true, true, flushPeriod, null, false, null);
+    PrometheusEmitter emitter = new PrometheusEmitter(config);
+    emitter.start();
+
+    ServiceMetricEvent event1 = ServiceMetricEvent.builder()
+                                                 
.setMetric("segment/loadQueue/count", 10)
+                                                 .setDimension("server", 
"historical1")
+                                                 
.build(ImmutableMap.of("service", "historical", "host", "druid.test.cn"));
+    ServiceMetricEvent event2 = ServiceMetricEvent.builder()
+                                                  
.setMetric("segment/loadQueue/count", 10)
+                                                  .setDimension("server", 
"historical2")
+                                                  
.build(ImmutableMap.of("service", "historical", "host", "druid.test.cn"));
+    emitter.emit(event1);
+    emitter.emit(event2);
+
+    // Get the metrics and check that it's not expired initially
+    Map<String, DimensionsAndCollector> registeredMetrics = 
emitter.getMetrics().getRegisteredMetrics();
+    DimensionsAndCollector testMetric = 
registeredMetrics.get("segment/loadQueue/count");
 
-    long timeSinceLastUpdate = testMetric.getMillisSinceLastUpdate();
+    Assert.assertNotNull(
+        "Test metric should be registered",
+        testMetric
+    );
+    Assert.assertFalse(
+        "Metric should not be expired initially",
+        testMetric.removeIfExpired(Arrays.asList("historical", 
"druid.test.cn", "historical1"))
+    );
+    Assert.assertFalse(
+        "Metric should not be expired initially",
+        testMetric.removeIfExpired(Arrays.asList("historical", 
"druid.test.cn", "historical2"))
+    );
+
+    // Wait for a little, but not long enough for the metric to expire
+    long waitTime = TimeUnit.SECONDS.toMillis(flushPeriod) / 5;
+    try {
+      Thread.sleep(waitTime);
+    }
+    catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+
+    Assert.assertFalse(
+        "Metric should not be expired",
+        testMetric.removeIfExpired(Arrays.asList("historical", 
"druid.test.cn", "historical1"))
+    );
+    Assert.assertFalse(
+        "Metric should not be expired",
+        testMetric.removeIfExpired(Arrays.asList("historical", 
"druid.test.cn", "historical2"))
+    );
+    // Reset update time only for event2
+    emitter.emit(event2);
+
+    try {
+      // Wait for the remainder of the TTL to allow event1 to expire
+      Thread.sleep(waitTime * 4);
+    }
+    catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
     Assert.assertTrue(
-        "Update time should have been refreshed",
-        timeSinceLastUpdate < waitTime
+        "Metric should be expired",
+        testMetric.removeIfExpired(Arrays.asList("historical", 
"druid.test.cn", "historical1"))
+    );
+    Assert.assertFalse(

Review Comment:
   Gotcha, the SimpleCollector 
[exposes](https://www.javadoc.io/doc/io.prometheus/simpleclient/0.0.5/io/prometheus/client/Collector.html#collect())
 a `collect()` API to get the metrics collected/samples. Perhaps something like 
this should work?
   ```
   List<Collector.MetricFamilySamples.Sample> samples = 
testMetric.getCollector().collect().get(0).samples;
   // assert the labels & the sample size is 1 during a valid period
   ```
   After the TTL period, the sample size would be 0? I think that would give us 
sufficient functionality validation on `cleanUpStaleMetrics()`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to