You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2018/11/03 18:02:48 UTC

[bookkeeper] branch branch-4.8 updated: Use default metric registry in Prometheus exporter

This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch branch-4.8
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/branch-4.8 by this push:
     new ccf0f09  Use default metric registry in Prometheus exporter
ccf0f09 is described below

commit ccf0f09cafc01ebba32d6dbc5cfeb57d6160a176
Author: Matteo Merli <mm...@apache.org>
AuthorDate: Sat Nov 3 10:42:49 2018 -0700

    Use default metric registry in Prometheus exporter
    
    ### Motivation
    
    Currently the Prometheus exporter is using a private metrics register. This doesn't work well for BK client where we want to expose in the same registry of application, or when we want to interact with other tools that use the default static registry (eg: log4j, jetty integrations for prometheus).
    
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Sijie Guo <si...@apache.org>
    
    This closes #1765 from merlimat/prometheus
---
 .../prometheus/PrometheusMetricsProvider.java      | 32 +++++++++++++++-------
 1 file changed, 22 insertions(+), 10 deletions(-)

diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java
index 645a686..df8279e 100644
--- a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java
@@ -22,7 +22,7 @@ import io.netty.util.concurrent.DefaultThreadFactory;
 //CHECKSTYLE.OFF: IllegalImport
 import io.netty.util.internal.PlatformDependent;
 //CHECKSTYLE.ON: IllegalImport
-
+import io.prometheus.client.Collector;
 import io.prometheus.client.CollectorRegistry;
 import io.prometheus.client.Gauge;
 import io.prometheus.client.Gauge.Child;
@@ -81,7 +81,7 @@ public class PrometheusMetricsProvider implements StatsProvider {
     final ConcurrentMap<String, DataSketchesOpStatsLogger> opStats = new ConcurrentSkipListMap<>();
 
     public PrometheusMetricsProvider() {
-        this(new CollectorRegistry());
+        this(CollectorRegistry.defaultRegistry);
     }
 
     public PrometheusMetricsProvider(CollectorRegistry registry) {
@@ -128,25 +128,25 @@ public class PrometheusMetricsProvider implements StatsProvider {
         }
 
         // Include standard JVM stats
-        new StandardExports().register(registry);
-        new MemoryPoolsExports().register(registry);
-        new GarbageCollectorExports().register(registry);
-        new ThreadExports().register(registry);
+        registerMetrics(new StandardExports());
+        registerMetrics(new MemoryPoolsExports());
+        registerMetrics(new GarbageCollectorExports());
+        registerMetrics(new ThreadExports());
 
         // Add direct memory allocated through unsafe
-        Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() {
+        registerMetrics(Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() {
             @Override
             public double get() {
                 return directMemoryUsage != null ? directMemoryUsage.longValue() : Double.NaN;
             }
-        }).register(registry);
+        }));
 
-        Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() {
+        registerMetrics(Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() {
             @Override
             public double get() {
                 return PlatformDependent.maxDirectMemory();
             }
-        }).register(registry);
+        }));
 
         executor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("metrics"));
 
@@ -191,6 +191,18 @@ public class PrometheusMetricsProvider implements StatsProvider {
         });
     }
 
+    private void registerMetrics(Collector collector) {
+        try {
+            collector.register(registry);
+        } catch (Exception e) {
+            // Ignore if these were already registered
+            if (log.isDebugEnabled()) {
+                log.debug("Failed to register Prometheus collector exports", e);
+            }
+        }
+    }
+
+
     private static final Logger log = LoggerFactory.getLogger(PrometheusMetricsProvider.class);
 
     /*