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/03/15 06:48:39 UTC

[bookkeeper] branch master updated: Collect Prometheus latency stats using DataSketches

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 3bff199  Collect Prometheus latency stats using DataSketches
3bff199 is described below

commit 3bff19956e70e37c025a8e29aa8428937af77aa1
Author: Matteo Merli <mm...@apache.org>
AuthorDate: Wed Mar 14 23:48:32 2018 -0700

    Collect Prometheus latency stats using DataSketches
    
    The implementation for collecting and estimating the latency quantiles in the Prometheus Java client library is very slow and it is impacting the the bookie performance.
    
    I have added a micro-benchmark that tests our various stats providers. These tests are simulating 16 concurrent threads updating the stats.
    
    #### Counter increment
    ```
    Benchmark                              (statsProvider)   Mode  Cnt    Score      Error   Units
    StatsLoggerBenchmark.counterIncrement       Prometheus  thrpt    3  391.882 ±  786.987  ops/us
    StatsLoggerBenchmark.counterIncrement         Codahale  thrpt    3  449.341 ± 1337.736  ops/us
    StatsLoggerBenchmark.counterIncrement          Twitter  thrpt    3   43.354 ±    9.331  ops/us
    StatsLoggerBenchmark.counterIncrement          Ostrich  thrpt    3   43.790 ±    1.332  ops/us
    ```
    
    Here prometheus is fast, though not as fast as a simple `LongAdder` which can reach ~500M ops/sec.
    
    #### Latency quantiles
    
    ```
    Benchmark                              (statsProvider)   Mode  Cnt    Score      Error   Units
    StatsLoggerBenchmark.recordLatency          Prometheus  thrpt    3    0.255 ±    0.667  ops/us
    StatsLoggerBenchmark.recordLatency            Codahale  thrpt    3    4.963 ±    1.671  ops/us
    StatsLoggerBenchmark.recordLatency             Twitter  thrpt    3    4.793 ±    0.766  ops/us
    StatsLoggerBenchmark.recordLatency             Ostrich  thrpt    3    2.473 ±    6.394  ops/us
    ```
    
    Here is where Prometheus is super-slow: 250K ops/second max, mostly due to contention and GC pressure.
    
    ## Modification
    
    I have re-adapted a stats collector I had done in the Yahoo branch:
    https://github.com/yahoo/bookkeeper/tree/yahoo-4.3/bookkeeper-stats-providers/datasketches-metrics-provider/src/main/java/org/apache/bokkeeper/stats/datasketches
    
    This is based on the [DataSketches](https://datasketches.github.io/) library to have very fast and lightweight quantile estimates (along with a number of other operations), plus some tricks to avoid concurrency issues by using thread local collectors and aggregating when needed in background.
    
    After the change, the throughput is 150x the original prometheus collector.
    
    ```
    Benchmark                              (statsProvider)   Mode  Cnt    Score     Error   Units
    StatsLoggerBenchmark.counterIncrement       Prometheus  thrpt    3  531.906 ± 129.602  ops/us
    StatsLoggerBenchmark.recordLatency          Prometheus  thrpt    3   27.538 ±   5.893  ops/us
    ```
    
    It is worth noting that the main bottle-neck in the `recordLatency` test is now the `System.nanoTime()`
    call used to pass different samples to the stat logger.
    
    `System.nanoTime()` is not super fast:
    
    ```
    Benchmark                               (statsProvider)   Mode  Cnt    Score     Error   Units
    StatsLoggerBenchmark.currentTimeMillis              N/A  thrpt    3  161.502 ± 267.238  ops/us
    StatsLoggerBenchmark.nanoTime                       N/A  thrpt    3   32.822 ±   2.256  ops/us
    ```
    
    By removing the `System.nanoTime()` call from the benchmark, the Prometheus+DataSketches collector results in:
    
    ```
    Benchmark                               (statsProvider)   Mode  Cnt    Score     Error   Units
    StatsLoggerBenchmark.recordLatency           Prometheus  thrpt    3  108.542 ±  31.848  ops/us
    ```
    
    Author: Matteo Merli <mm...@apache.org>
    
    Reviewers: Jia Zhai <None>, Sijie Guo <si...@apache.org>
    
    This closes #1245 from merlimat/prometheus-datasketches
---
 .../src/main/resources/LICENSE-all.bin.txt         |   3 +
 .../src/main/resources/LICENSE-server.bin.txt      |   3 +
 .../prometheus-metrics-provider/pom.xml            |  10 ++
 .../prometheus/DataSketchesOpStatsLogger.java      | 199 +++++++++++++++++++++
 ...rometheusCounter.java => LongAdderCounter.java} |  31 ++--
 .../prometheus/PrometheusMetricsProvider.java      | 139 +++++++++++---
 .../stats/prometheus/PrometheusOpStatsLogger.java  |  84 ---------
 .../stats/prometheus/PrometheusServlet.java        |  60 +++++++
 .../stats/prometheus/PrometheusStatsLogger.java    |  33 +---
 .../stats/prometheus/PrometheusTextFormatUtil.java | 150 ++++++++++++++++
 .../stats/prometheus/PrometheusUtil.java           |  67 -------
 .../bookkeeper/stats/prometheus/SimpleGauge.java   |  40 +++++
 .../stats/prometheus/TestPrometheusFormatter.java  | 177 ++++++++++++++++++
 .../prometheus/TestPrometheusMetricsProvider.java  |  24 ++-
 microbenchmarks/pom.xml                            |  20 +++
 .../proto/checksum/StatsLoggerBenchmark.java       |  96 ++++++++++
 pom.xml                                            |   8 +
 17 files changed, 913 insertions(+), 231 deletions(-)

diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
index 7762efd..e4bb88b 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
@@ -271,6 +271,8 @@ Apache Software License, Version 2.
 - lib/org.eclipse.jetty-jetty-util-9.4.5.v20170502.jar [34]
 - lib/org.rocksdb-rocksdbjni-5.8.6.jar [35]
 - lib/com.beust-jcommander-1.48.jar [36]
+- lib/com.yahoo.datasketches-memory-0.8.3.jar [37]
+- lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [37]
 
 [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9
 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9
@@ -307,6 +309,7 @@ Apache Software License, Version 2.
 [34] Source available at https://github.com/eclipse/jetty.project/tree/jetty-9.4.5.v20170502
 [35] Source available at https://github.com/facebook/rocksdb/tree/v5.8.6
 [36] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48
+[37] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3
 
 ------------------------------------------------------------------------------------
 lib/io.netty-netty-3.10.1.Final.jar contains the extensions to Java Collections Framework which has
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
index d477fdb..bd0dba2 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
@@ -236,6 +236,8 @@ Apache Software License, Version 2.
 - lib/org.eclipse.jetty-jetty-util-9.4.5.v20170502.jar [21]
 - lib/org.rocksdb-rocksdbjni-5.8.6.jar [22]
 - lib/com.beust-jcommander-1.48.jar [23]
+- lib/com.yahoo.datasketches-memory-0.8.3.jar [24]
+- lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [24]
 
 [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9
 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9
@@ -260,6 +262,7 @@ Apache Software License, Version 2.
 [21] Source available at https://github.com/eclipse/jetty.project/tree/jetty-9.4.5.v20170502
 [22] Source available at https://github.com/facebook/rocksdb/tree/v5.8.6
 [23] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48
+[24] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3
 
 ------------------------------------------------------------------------------------
 lib/io.netty-netty-all-4.1.12.Final.jar bundles some 3rd party dependencies
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml b/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml
index f7f33cd..111a319 100644
--- a/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml
@@ -56,6 +56,11 @@
       <groupId>io.prometheus</groupId>
       <artifactId>simpleclient_servlet</artifactId>
     </dependency>
+    
+    <dependency>
+       <groupId>io.netty</groupId>
+       <artifactId>netty-all</artifactId>
+    </dependency>
 
     <dependency>
       <groupId>org.eclipse.jetty</groupId>
@@ -66,6 +71,11 @@
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>
+    
+    <dependency>
+      <groupId>com.yahoo.datasketches</groupId>
+      <artifactId>sketches-core</artifactId>
+    </dependency>
 
   </dependencies>
 </project>
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/DataSketchesOpStatsLogger.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/DataSketchesOpStatsLogger.java
new file mode 100644
index 0000000..ad0e7d4
--- /dev/null
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/DataSketchesOpStatsLogger.java
@@ -0,0 +1,199 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.bookkeeper.stats.prometheus;
+
+import com.yahoo.sketches.quantiles.DoublesSketch;
+import com.yahoo.sketches.quantiles.DoublesSketchBuilder;
+import com.yahoo.sketches.quantiles.DoublesUnion;
+import com.yahoo.sketches.quantiles.DoublesUnionBuilder;
+
+import io.netty.util.concurrent.FastThreadLocal;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.concurrent.locks.StampedLock;
+
+import org.apache.bookkeeper.stats.OpStatsData;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+
+/**
+ * OpStatsLogger implementation that uses DataSketches library to calculate the approximated latency quantiles.
+ */
+public class DataSketchesOpStatsLogger implements OpStatsLogger {
+
+    /*
+     * Use 2 rotating thread local accessor so that we can safely swap them.
+     */
+    private volatile ThreadLocalAccessor current;
+    private volatile ThreadLocalAccessor replacement;
+
+    /*
+     * These are the sketches where all the aggregated results are published.
+     */
+    private volatile DoublesSketch successResult;
+    private volatile DoublesSketch failResult;
+
+    private final LongAdder successCountAdder = new LongAdder();
+    private final LongAdder failCountAdder = new LongAdder();
+
+    private final LongAdder successSumAdder = new LongAdder();
+    private final LongAdder failSumAdder = new LongAdder();
+
+    DataSketchesOpStatsLogger() {
+        this.current = new ThreadLocalAccessor();
+        this.replacement = new ThreadLocalAccessor();
+    }
+
+    @Override
+    public void registerFailedEvent(long eventLatency, TimeUnit unit) {
+        double valueMillis = unit.toMicros(eventLatency) / 1000.0;
+
+        failCountAdder.increment();
+        failSumAdder.add((long) valueMillis);
+
+        LocalData localData = current.localData.get();
+
+        long stamp = localData.lock.readLock();
+        try {
+            localData.failSketch.update(valueMillis);
+        } finally {
+            localData.lock.unlockRead(stamp);
+        }
+    }
+
+    @Override
+    public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) {
+        double valueMillis = unit.toMicros(eventLatency) / 1000.0;
+
+        successCountAdder.increment();
+        successSumAdder.add((long) valueMillis);
+
+        LocalData localData = current.localData.get();
+
+        long stamp = localData.lock.readLock();
+        try {
+            localData.successSketch.update(valueMillis);
+        } finally {
+            localData.lock.unlockRead(stamp);
+        }
+    }
+
+    @Override
+    public void registerSuccessfulValue(long value) {
+        successCountAdder.increment();
+        successSumAdder.add(value);
+
+        LocalData localData = current.localData.get();
+
+        long stamp = localData.lock.readLock();
+        try {
+            localData.successSketch.update(value);
+        } finally {
+            localData.lock.unlockRead(stamp);
+        }
+    }
+
+    @Override
+    public void registerFailedValue(long value) {
+        failCountAdder.increment();
+        failSumAdder.add(value);
+
+        LocalData localData = current.localData.get();
+
+        long stamp = localData.lock.readLock();
+        try {
+            localData.failSketch.update(value);
+        } finally {
+            localData.lock.unlockRead(stamp);
+        }
+    }
+
+    @Override
+    public OpStatsData toOpStatsData() {
+        // Not relevant as we don't use JMX here
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void clear() {
+        // Not relevant as we don't use JMX here
+        throw new UnsupportedOperationException();
+    }
+
+    public void rotateLatencyCollection() {
+        // Swap current with replacement
+        ThreadLocalAccessor local = current;
+        current = replacement;
+        replacement = local;
+
+        final DoublesUnion aggregateSuccesss = new DoublesUnionBuilder().build();
+        final DoublesUnion aggregateFail = new DoublesUnionBuilder().build();
+        local.map.forEach((localData, b) -> {
+            long stamp = localData.lock.writeLock();
+            try {
+                aggregateSuccesss.update(localData.successSketch);
+                localData.successSketch.reset();
+                aggregateFail.update(localData.failSketch);
+                localData.failSketch.reset();
+            } finally {
+                localData.lock.unlockWrite(stamp);
+            }
+        });
+
+        successResult = aggregateSuccesss.getResultAndReset();
+        failResult = aggregateFail.getResultAndReset();
+    }
+
+    public long getCount(boolean success) {
+        return success ? successCountAdder.sum() : failCountAdder.sum();
+    }
+
+    public long getSum(boolean success) {
+        return success ? successSumAdder.sum() : failSumAdder.sum();
+    }
+
+    public double getQuantileValue(boolean success, double quantile) {
+        DoublesSketch s = success ? successResult : failResult;
+        return s != null ? s.getQuantile(quantile) : Double.NaN;
+    }
+
+    private static class LocalData {
+        private final DoublesSketch successSketch = new DoublesSketchBuilder().build();
+        private final DoublesSketch failSketch = new DoublesSketchBuilder().build();
+        private final StampedLock lock = new StampedLock();
+    }
+
+    private static class ThreadLocalAccessor {
+        private final Map<LocalData, Boolean> map = new ConcurrentHashMap<>();
+        private final FastThreadLocal<LocalData> localData = new FastThreadLocal<LocalData>() {
+
+            @Override
+            protected LocalData initialValue() throws Exception {
+                LocalData localData = new LocalData();
+                map.put(localData, Boolean.TRUE);
+                return localData;
+            }
+
+            @Override
+            protected void onRemoval(LocalData value) throws Exception {
+                map.remove(value);
+            }
+        };
+    }
+}
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusCounter.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/LongAdderCounter.java
similarity index 63%
rename from bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusCounter.java
rename to bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/LongAdderCounter.java
index db5b13d..4b67703 100644
--- a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusCounter.java
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/LongAdderCounter.java
@@ -16,46 +16,41 @@
  */
 package org.apache.bookkeeper.stats.prometheus;
 
-import io.prometheus.client.Collector;
-import io.prometheus.client.CollectorRegistry;
-import io.prometheus.client.Gauge;
+import java.util.concurrent.atomic.LongAdder;
+
 import org.apache.bookkeeper.stats.Counter;
 
 /**
- * A {@link Counter} implementation based on <i>Prometheus</i> metrics library.
+ * {@link Counter} implementation based on {@link LongAdder}.
+ *
+ * <p>LongAdder keeps a counter per-thread and then aggregates to get the result, in order to avoid contention between
+ * multiple threads.
  */
-public class PrometheusCounter implements Counter {
-
-    private final Gauge gauge;
-
-    public PrometheusCounter(CollectorRegistry registry, String name) {
-        this.gauge = PrometheusUtil.safeRegister(registry,
-                Gauge.build().name(Collector.sanitizeMetricName(name)).help("-").create());
-    }
+public class LongAdderCounter implements Counter {
+    private final LongAdder counter = new LongAdder();
 
     @Override
     public void clear() {
-        gauge.clear();
+        counter.reset();
     }
 
     @Override
     public void inc() {
-        gauge.inc();
+        counter.increment();
     }
 
     @Override
     public void dec() {
-        gauge.dec();
+        counter.decrement();
     }
 
     @Override
     public void add(long delta) {
-        gauge.inc(delta);
+        counter.add(delta);
     }
 
     @Override
     public Long get() {
-        return (long) gauge.get();
+        return counter.sum();
     }
-
 }
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 4008c04..a79d05d 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
@@ -16,13 +16,32 @@
  */
 package org.apache.bookkeeper.stats.prometheus;
 
+import com.google.common.annotations.VisibleForTesting;
+
+import io.netty.util.concurrent.DefaultThreadFactory;
+//CHECKSTYLE.OFF: IllegalImport
+import io.netty.util.internal.PlatformDependent;
+//CHECKSTYLE.ON: IllegalImport
+
 import io.prometheus.client.CollectorRegistry;
-import io.prometheus.client.exporter.MetricsServlet;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.Gauge.Child;
 import io.prometheus.client.hotspot.GarbageCollectorExports;
 import io.prometheus.client.hotspot.MemoryPoolsExports;
 import io.prometheus.client.hotspot.StandardExports;
 import io.prometheus.client.hotspot.ThreadExports;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.lang.reflect.Field;
 import java.net.InetSocketAddress;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
 import org.apache.bookkeeper.stats.CachingStatsProvider;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.stats.StatsProvider;
@@ -38,41 +57,55 @@ import org.slf4j.LoggerFactory;
  */
 public class PrometheusMetricsProvider implements StatsProvider {
 
-    private final CollectorRegistry registry = new CollectorRegistry();
+    private ScheduledExecutorService executor;
+
+    private static final String PROMETHEUS_STATS_HTTP_PORT = "prometheusStatsHttpPort";
+    private static final int DEFAULT_PROMETHEUS_STATS_HTTP_PORT = 8000;
+
+    private static final String PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS = "prometheusStatsLatencyRolloverSeconds";
+    private static final int DEFAULT_PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS = 60;
+
+    final CollectorRegistry registry = new CollectorRegistry();
+
     private Server server;
     private final CachingStatsProvider cachingStatsProvider;
 
+    /*
+     * These acts a registry of the metrics defined in this provider
+     */
+    final ConcurrentMap<String, LongAdderCounter> counters = new ConcurrentSkipListMap<>();
+    final ConcurrentMap<String, SimpleGauge<? extends Number>> gauges = new ConcurrentSkipListMap<>();
+    final ConcurrentMap<String, DataSketchesOpStatsLogger> opStats = new ConcurrentSkipListMap<>();
+
     public PrometheusMetricsProvider() {
-        this.cachingStatsProvider = new CachingStatsProvider(
-            new StatsProvider() {
-                @Override
-                public void start(Configuration conf) {
-                    // nop
-                }
-
-                @Override
-                public void stop() {
-                    // nop
-                }
-
-                @Override
-                public StatsLogger getStatsLogger(String scope) {
-                    return new PrometheusStatsLogger(registry, scope);
-                }
+        this.cachingStatsProvider = new CachingStatsProvider(new StatsProvider() {
+            @Override
+            public void start(Configuration conf) {
+                // nop
             }
-        );
+
+            @Override
+            public void stop() {
+                // nop
+            }
+
+            @Override
+            public StatsLogger getStatsLogger(String scope) {
+                return new PrometheusStatsLogger(PrometheusMetricsProvider.this, scope);
+            }
+        });
     }
 
     @Override
     public void start(Configuration conf) {
-        int httpPort = conf.getInt("prometheusStatsHttpPort", 8000);
+        int httpPort = conf.getInt(PROMETHEUS_STATS_HTTP_PORT, DEFAULT_PROMETHEUS_STATS_HTTP_PORT);
         InetSocketAddress httpEndpoint = InetSocketAddress.createUnresolved("0.0.0.0", httpPort);
         this.server = new Server(httpEndpoint);
         ServletContextHandler context = new ServletContextHandler();
         context.setContextPath("/");
         server.setHandler(context);
 
-        context.addServlet(new ServletHolder(new MetricsServlet(registry)), "/metrics");
+        context.addServlet(new ServletHolder(new PrometheusServlet(this)), "/metrics");
 
         try {
             server.start();
@@ -80,13 +113,37 @@ public class PrometheusMetricsProvider implements StatsProvider {
             throw new RuntimeException(e);
         }
 
-        log.info("Started Prometheus stats endpoint at {}", httpEndpoint);
-
         // Include standard JVM stats
         new StandardExports().register(registry);
         new MemoryPoolsExports().register(registry);
         new GarbageCollectorExports().register(registry);
         new ThreadExports().register(registry);
+
+        // Add direct memory allocated through unsafe
+        Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() {
+            @Override
+            public double get() {
+                return directMemoryUsage != null ? directMemoryUsage.longValue() : Double.NaN;
+            }
+        }).register(CollectorRegistry.defaultRegistry);
+
+        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"));
+
+        int latencyRolloverSeconds = conf.getInt(PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS,
+                DEFAULT_PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS);
+
+        executor.scheduleAtFixedRate(() -> {
+            rotateLatencyCollection();
+        }, 1, latencyRolloverSeconds, TimeUnit.SECONDS);
+
+        log.info("Started Prometheus stats endpoint at {}", httpEndpoint);
     }
 
     @Override
@@ -105,5 +162,39 @@ public class PrometheusMetricsProvider implements StatsProvider {
         return this.cachingStatsProvider.getStatsLogger(scope);
     }
 
+    @VisibleForTesting
+    void writeAllMetrics(Writer writer) throws IOException {
+        PrometheusTextFormatUtil.writeMetricsCollectedByPrometheusClient(writer, registry);
+
+        gauges.forEach((name, gauge) -> PrometheusTextFormatUtil.writeGauge(writer, name, gauge));
+        counters.forEach((name, counter) -> PrometheusTextFormatUtil.writeCounter(writer, name, counter));
+        opStats.forEach((name, opStatLogger) -> PrometheusTextFormatUtil.writeOpStat(writer, name, opStatLogger));
+    }
+
+    @VisibleForTesting
+    void rotateLatencyCollection() {
+        opStats.forEach((name, metric) -> {
+            metric.rotateLatencyCollection();
+        });
+    }
+
     private static final Logger log = LoggerFactory.getLogger(PrometheusMetricsProvider.class);
-}
+
+    /*
+     * Try to get Netty counter of used direct memory. This will be correct, unlike the JVM values.
+     */
+    private static final AtomicLong directMemoryUsage;
+    static {
+        AtomicLong tmpDirectMemoryUsage = null;
+
+        try {
+            Field field = PlatformDependent.class.getDeclaredField("DIRECT_MEMORY_COUNTER");
+            field.setAccessible(true);
+            tmpDirectMemoryUsage = (AtomicLong) field.get(null);
+        } catch (Throwable t) {
+            log.warn("Failed to access netty DIRECT_MEMORY_COUNTER field {}", t.getMessage());
+        }
+
+        directMemoryUsage = tmpDirectMemoryUsage;
+    }
+}
\ No newline at end of file
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusOpStatsLogger.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusOpStatsLogger.java
deleted file mode 100644
index a44ad5a..0000000
--- a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusOpStatsLogger.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.bookkeeper.stats.prometheus;
-
-import io.prometheus.client.CollectorRegistry;
-import io.prometheus.client.Summary;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.OpStatsData;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-
-/**
- * A {@code Prometheus} based {@link OpStatsLogger} implementation.
- */
-public class PrometheusOpStatsLogger implements OpStatsLogger {
-
-    private final Summary summary;
-    private final Summary.Child success;
-    private final Summary.Child fail;
-
-    public PrometheusOpStatsLogger(CollectorRegistry registry, String name) {
-        this.summary = PrometheusUtil.safeRegister(registry,
-                Summary.build().name(name).help("-") //
-                        .quantile(0.50, 0.01) //
-                        .quantile(0.75, 0.01) //
-                        .quantile(0.95, 0.01) //
-                        .quantile(0.99, 0.01) //
-                        .quantile(0.999, 0.01) //
-                        .quantile(0.9999, 0.01) //
-                        .quantile(1.0, 0.01) //
-                        .maxAgeSeconds(60) //
-                        .labelNames("success") //
-                        .create());
-
-        this.success = summary.labels("true");
-        this.fail = summary.labels("false");
-    }
-
-    @Override
-    public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) {
-        // Collect latency in millis, truncating anything below micros
-        success.observe(unit.toMicros(eventLatency) / 1000.0);
-    }
-
-    @Override
-    public void registerFailedEvent(long eventLatency, TimeUnit unit) {
-        fail.observe(unit.toMicros(eventLatency) / 1000.0);
-    }
-
-    @Override
-    public void registerSuccessfulValue(long value) {
-        success.observe(value);
-    }
-
-    @Override
-    public void registerFailedValue(long value) {
-        fail.observe(value);
-    }
-
-    @Override
-    public OpStatsData toOpStatsData() {
-        // Not relevant as we don't use JMX here
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void clear() {
-        summary.clear();
-    }
-
-}
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusServlet.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusServlet.java
new file mode 100644
index 0000000..31f1d32
--- /dev/null
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusServlet.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.bookkeeper.stats.prometheus;
+
+import io.prometheus.client.exporter.common.TextFormat;
+
+import java.io.IOException;
+import java.io.Writer;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+/**
+ * Servlet used to export metrics in prometheus text format.
+ */
+public class PrometheusServlet extends HttpServlet {
+    private static final long serialVersionUID = 1L;
+
+    private final transient PrometheusMetricsProvider provider;
+
+    public PrometheusServlet(PrometheusMetricsProvider provider) {
+        this.provider = provider;
+    }
+
+    @Override
+    protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
+        resp.setStatus(HttpServletResponse.SC_OK);
+        resp.setContentType(TextFormat.CONTENT_TYPE_004);
+
+        Writer writer = resp.getWriter();
+        try {
+            provider.writeAllMetrics(writer);
+            writer.flush();
+        } finally {
+            writer.close();
+        }
+    }
+
+    @Override
+    protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
+        doGet(req, resp);
+    }
+
+}
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusStatsLogger.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusStatsLogger.java
index 4f2cdfe..472a3fb 100644
--- a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusStatsLogger.java
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusStatsLogger.java
@@ -17,8 +17,9 @@
 package org.apache.bookkeeper.stats.prometheus;
 
 import com.google.common.base.Joiner;
+
 import io.prometheus.client.Collector;
-import io.prometheus.client.CollectorRegistry;
+
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.Gauge;
 import org.apache.bookkeeper.stats.OpStatsLogger;
@@ -29,43 +30,27 @@ import org.apache.bookkeeper.stats.StatsLogger;
  */
 public class PrometheusStatsLogger implements StatsLogger {
 
-    private final CollectorRegistry registry;
+    private final PrometheusMetricsProvider provider;
     private final String scope;
 
-    PrometheusStatsLogger(CollectorRegistry registry, String scope) {
-        this.registry = registry;
+    PrometheusStatsLogger(PrometheusMetricsProvider provider, String scope) {
+        this.provider = provider;
         this.scope = scope;
     }
 
     @Override
     public OpStatsLogger getOpStatsLogger(String name) {
-        return new PrometheusOpStatsLogger(registry, completeName(name));
+        return provider.opStats.computeIfAbsent(completeName(name), x -> new DataSketchesOpStatsLogger());
     }
 
     @Override
     public Counter getCounter(String name) {
-        return new PrometheusCounter(registry, completeName(name));
+        return provider.counters.computeIfAbsent(completeName(name), x -> new LongAdderCounter());
     }
 
     @Override
     public <T extends Number> void registerGauge(String name, Gauge<T> gauge) {
-        PrometheusUtil.safeRegister(registry, io.prometheus.client.Gauge.build().name(completeName(name)).help("-")
-                .create().setChild(new io.prometheus.client.Gauge.Child() {
-                    @Override
-                    public double get() {
-                        Number value = null;
-                        try {
-                            value = gauge.getSample();
-                        } catch (Exception e) {
-                            // no-op
-                        }
-
-                        if (value == null) {
-                            value = gauge.getDefaultValue();
-                        }
-                        return value.doubleValue();
-                    }
-                }));
+        provider.gauges.computeIfAbsent(completeName(name), x -> new SimpleGauge<T>(gauge));
     }
 
     @Override
@@ -80,7 +65,7 @@ public class PrometheusStatsLogger implements StatsLogger {
 
     @Override
     public StatsLogger scope(String name) {
-        return new PrometheusStatsLogger(registry, completeName(name));
+        return new PrometheusStatsLogger(provider, completeName(name));
     }
 
     private String completeName(String name) {
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusTextFormatUtil.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusTextFormatUtil.java
new file mode 100644
index 0000000..d2fae28
--- /dev/null
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusTextFormatUtil.java
@@ -0,0 +1,150 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.bookkeeper.stats.prometheus;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.Collector.MetricFamilySamples;
+import io.prometheus.client.Collector.MetricFamilySamples.Sample;
+import io.prometheus.client.CollectorRegistry;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.Enumeration;
+
+import org.apache.bookkeeper.stats.Counter;
+
+/**
+ * Logic to write metrics in Prometheus text format.
+ */
+public class PrometheusTextFormatUtil {
+    static void writeGauge(Writer w, String name, SimpleGauge<? extends Number> gauge) {
+        // Example:
+        // # TYPE bookie_storage_entries_count gauge
+        // bookie_storage_entries_count 519
+        try {
+            w.append("# TYPE ").append(name).append(" gauge\n");
+            w.append(name).append(' ').append(gauge.getSample().toString()).append('\n');
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    static void writeCounter(Writer w, String name, Counter counter) {
+        // Example:
+        // # TYPE jvm_threads_started_total counter
+        // jvm_threads_started_total 59
+        try {
+            w.append("# TYPE ").append(name).append(" counter\n");
+            w.append(name).append(' ').append(counter.get().toString()).append('\n');
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    static void writeOpStat(Writer w, String name, DataSketchesOpStatsLogger opStat) {
+        // Example:
+        // # TYPE bookie_journal_JOURNAL_ADD_ENTRY summary
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.5",} NaN
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.75",} NaN
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.95",} NaN
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.99",} NaN
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.999",} NaN
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.9999",} NaN
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="1.0",} NaN
+        // bookie_journal_JOURNAL_ADD_ENTRY_count{success="false",} 0.0
+        // bookie_journal_JOURNAL_ADD_ENTRY_sum{success="false",} 0.0
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.5",} 1.706
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.75",} 1.89
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.95",} 2.121
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.99",} 10.708
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.999",} 10.902
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.9999",} 10.902
+        // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="1.0",} 10.902
+        // bookie_journal_JOURNAL_ADD_ENTRY_count{success="true",} 658.0
+        // bookie_journal_JOURNAL_ADD_ENTRY_sum{success="true",} 1265.0800000000002
+        try {
+            w.append("# TYPE ").append(name).append(" summary\n");
+            writeQuantile(w, opStat, name, false, 0.5);
+            writeQuantile(w, opStat, name, false, 0.75);
+            writeQuantile(w, opStat, name, false, 0.95);
+            writeQuantile(w, opStat, name, false, 0.99);
+            writeQuantile(w, opStat, name, false, 0.999);
+            writeQuantile(w, opStat, name, false, 0.9999);
+            writeQuantile(w, opStat, name, false, 1.0);
+            writeCount(w, opStat, name, false);
+            writeSum(w, opStat, name, false);
+
+            writeQuantile(w, opStat, name, true, 0.5);
+            writeQuantile(w, opStat, name, true, 0.75);
+            writeQuantile(w, opStat, name, true, 0.95);
+            writeQuantile(w, opStat, name, true, 0.99);
+            writeQuantile(w, opStat, name, true, 0.999);
+            writeQuantile(w, opStat, name, true, 0.9999);
+            writeQuantile(w, opStat, name, true, 1.0);
+            writeCount(w, opStat, name, true);
+            writeSum(w, opStat, name, true);
+
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static void writeQuantile(Writer w, DataSketchesOpStatsLogger opStat, String name, Boolean success,
+            double quantile) throws IOException {
+        w.append(name).append("{success=\"").append(success.toString()).append("\",quantile=\"")
+                .append(Double.toString(quantile)).append("\"} ")
+                .append(Double.toString(opStat.getQuantileValue(success, quantile))).append('\n');
+    }
+
+    private static void writeCount(Writer w, DataSketchesOpStatsLogger opStat, String name, Boolean success)
+            throws IOException {
+        w.append(name).append("_count{success=\"").append(success.toString()).append("\"} ")
+                .append(Long.toString(opStat.getCount(success))).append('\n');
+    }
+
+    private static void writeSum(Writer w, DataSketchesOpStatsLogger opStat, String name, Boolean success)
+            throws IOException {
+        w.append(name).append("_sum{success=\"").append(success.toString()).append("\"} ")
+                .append(Double.toString(opStat.getSum(success))).append('\n');
+    }
+
+    static void writeMetricsCollectedByPrometheusClient(Writer w, CollectorRegistry registry) throws IOException {
+        Enumeration<MetricFamilySamples> metricFamilySamples = registry.metricFamilySamples();
+        while (metricFamilySamples.hasMoreElements()) {
+            MetricFamilySamples metricFamily = metricFamilySamples.nextElement();
+
+            for (int i = 0; i < metricFamily.samples.size(); i++) {
+                Sample sample = metricFamily.samples.get(i);
+                w.write(sample.name);
+                w.write('{');
+                for (int j = 0; j < sample.labelNames.size(); j++) {
+                    if (j != 0) {
+                        w.write(", ");
+                    }
+                    w.write(sample.labelNames.get(j));
+                    w.write("=\"");
+                    w.write(sample.labelValues.get(j));
+                    w.write('"');
+                }
+
+                w.write("} ");
+                w.write(Collector.doubleToGoString(sample.value));
+                w.write('\n');
+            }
+        }
+    }
+}
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusUtil.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusUtil.java
deleted file mode 100644
index dcb1a71..0000000
--- a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusUtil.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.bookkeeper.stats.prometheus;
-
-import io.prometheus.client.Collector;
-import io.prometheus.client.CollectorRegistry;
-
-import java.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Container for Prometheus utility methods.
- *
- */
-public class PrometheusUtil {
-
-    private static final Field collectorsMapField;
-    private static final Method collectorsNamesMethod;
-
-    static {
-        try {
-            collectorsMapField = CollectorRegistry.class.getDeclaredField("namesToCollectors");
-            collectorsMapField.setAccessible(true);
-
-            collectorsNamesMethod = CollectorRegistry.class.getDeclaredMethod("collectorNames", Collector.class);
-            collectorsNamesMethod.setAccessible(true);
-
-        } catch (NoSuchFieldException | SecurityException | NoSuchMethodException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    @SuppressWarnings("unchecked")
-    public static <T extends Collector> T safeRegister(CollectorRegistry registry, T collector) {
-        try {
-            registry.register(collector);
-            return collector;
-        } catch (IllegalArgumentException e) {
-            // Collector is already registered. Return the existing instance
-            try {
-                Map<String, Collector> collectorsMap = (Map<String, Collector>) collectorsMapField.get(registry);
-                List<String> collectorNames = (List<String>) collectorsNamesMethod.invoke(registry, collector);
-                return (T) collectorsMap.get(collectorNames.get(0));
-
-            } catch (IllegalArgumentException | IllegalAccessException | InvocationTargetException e1) {
-                throw new RuntimeException(e1);
-            }
-        }
-    }
-}
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/SimpleGauge.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/SimpleGauge.java
new file mode 100644
index 0000000..1f30872
--- /dev/null
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/SimpleGauge.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.bookkeeper.stats.prometheus;
+
+import org.apache.bookkeeper.stats.Gauge;
+
+/**
+ * A {@link Gauge} implementation that forwards on the value supplier.
+ */
+public class SimpleGauge<T extends Number> {
+
+    // public SimpleGauge(CollectorRegistry registry, String name) {
+    // this.gauge = PrometheusUtil.safeRegister(registry,
+    // Gauge.build().name(Collector.sanitizeMetricName(name)).help("-").create());
+    // }
+
+    private final Gauge<T> gauge;
+
+    public SimpleGauge(final Gauge<T> gauge) {
+        this.gauge = gauge;
+    }
+
+    Number getSample() {
+        return gauge.getSample();
+    }
+}
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/TestPrometheusFormatter.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/TestPrometheusFormatter.java
new file mode 100644
index 0000000..f7b6ebe
--- /dev/null
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/TestPrometheusFormatter.java
@@ -0,0 +1,177 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.bookkeeper.stats.prometheus;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Splitter;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+import java.io.StringWriter;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.junit.Test;
+
+/**
+ * Test for {@link PrometheusMetricsProvider}.
+ */
+public class TestPrometheusFormatter {
+
+    @Test
+    public void testStatsOutput() throws Exception {
+        PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+        StatsLogger statsLogger = provider.getStatsLogger("test");
+        Counter counter = statsLogger.getCounter("my_counter");
+
+        counter.inc();
+        counter.inc();
+
+        OpStatsLogger opStats = statsLogger.getOpStatsLogger("op");
+        opStats.registerSuccessfulEvent(10, TimeUnit.MILLISECONDS);
+        opStats.registerSuccessfulEvent(5, TimeUnit.MILLISECONDS);
+
+        provider.rotateLatencyCollection();
+
+        StringWriter writer = new StringWriter();
+        provider.writeAllMetrics(writer);
+        System.out.println(writer);
+        Multimap<String, Metric> metrics = parseMetrics(writer.toString());
+        System.out.println(metrics);
+
+        List<Metric> cm = (List<Metric>) metrics.get("test_my_counter");
+        assertEquals(1, cm.size());
+        assertEquals(0, cm.get(0).tags.size());
+        assertEquals(2.0, cm.get(0).value, 0.0);
+
+        // test_op_sum
+        cm = (List<Metric>) metrics.get("test_op_sum");
+        assertEquals(2, cm.size());
+        Metric m = cm.get(0);
+        assertEquals(1, cm.get(0).tags.size());
+        assertEquals(0.0, m.value, 0.0);
+        assertEquals(1, m.tags.size());
+        assertEquals("false", m.tags.get("success"));
+
+        m = cm.get(1);
+        assertEquals(1, cm.get(0).tags.size());
+        assertEquals(15.0, m.value, 0.0);
+        assertEquals(1, m.tags.size());
+        assertEquals("true", m.tags.get("success"));
+
+        // test_op_count
+        cm = (List<Metric>) metrics.get("test_op_count");
+        assertEquals(2, cm.size());
+        m = cm.get(0);
+        assertEquals(1, cm.get(0).tags.size());
+        assertEquals(0.0, m.value, 0.0);
+        assertEquals(1, m.tags.size());
+        assertEquals("false", m.tags.get("success"));
+
+        m = cm.get(1);
+        assertEquals(1, cm.get(0).tags.size());
+        assertEquals(2.0, m.value, 0.0);
+        assertEquals(1, m.tags.size());
+        assertEquals("true", m.tags.get("success"));
+
+        // Latency
+        cm = (List<Metric>) metrics.get("test_op");
+        assertEquals(14, cm.size());
+
+        boolean found = false;
+        for (Metric mt  : cm) {
+            if ("true".equals(mt.tags.get("success")) && "1.0".equals(mt.tags.get("quantile"))) {
+                assertEquals(10.0, mt.value, 0.0);
+                found = true;
+            }
+        }
+
+        assertTrue(found);
+    }
+
+    /**
+     * Hacky parsing of Prometheus text format. Sould be good enough for unit tests
+     */
+    private static Multimap<String, Metric> parseMetrics(String metrics) {
+        Multimap<String, Metric> parsed = ArrayListMultimap.create();
+
+        // Example of lines are
+        // jvm_threads_current{cluster="standalone",} 203.0
+        // or
+        // pulsar_subscriptions_count{cluster="standalone", namespace="sample/standalone/ns1",
+        // topic="persistent://sample/standalone/ns1/test-2"} 0.0 1517945780897
+        Pattern pattern = Pattern.compile("^(\\w+)(\\{([^\\}]+)\\})?\\s(-?[\\d\\w\\.]+)(\\s(\\d+))?$");
+        Pattern tagsPattern = Pattern.compile("(\\w+)=\"([^\"]+)\"(,\\s?)?");
+
+        Splitter.on("\n").split(metrics).forEach(line -> {
+            if (line.isEmpty() || line.startsWith("#")) {
+                return;
+            }
+
+            System.err.println("LINE: '" + line + "'");
+            Matcher matcher = pattern.matcher(line);
+            System.err.println("Matches: " + matcher.matches());
+            System.err.println(matcher);
+
+            System.err.println("groups: " + matcher.groupCount());
+            for (int i = 0; i < matcher.groupCount(); i++) {
+                System.err.println("   GROUP " + i + " -- " + matcher.group(i));
+            }
+
+            checkArgument(matcher.matches());
+            String name = matcher.group(1);
+
+            Metric m = new Metric();
+            m.value = Double.valueOf(matcher.group(4));
+
+            String tags = matcher.group(3);
+            if (tags != null) {
+                Matcher tagsMatcher = tagsPattern.matcher(tags);
+                while (tagsMatcher.find()) {
+                    String tag = tagsMatcher.group(1);
+                    String value = tagsMatcher.group(2);
+                    m.tags.put(tag, value);
+                }
+            }
+
+            parsed.put(name, m);
+        });
+
+        return parsed;
+    }
+
+    static class Metric {
+        Map<String, String> tags = new TreeMap<>();
+        double value;
+
+        @Override
+        public String toString() {
+            return MoreObjects.toStringHelper(this).add("tags", tags).add("value", value).toString();
+        }
+    }
+}
diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/TestPrometheusMetricsProvider.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/TestPrometheusMetricsProvider.java
index 3ee8cf2..b969970 100644
--- a/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/TestPrometheusMetricsProvider.java
+++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/TestPrometheusMetricsProvider.java
@@ -19,10 +19,6 @@ package org.apache.bookkeeper.stats.prometheus;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertSame;
 
-import io.prometheus.client.Collector;
-import io.prometheus.client.CollectorRegistry;
-import java.lang.reflect.Field;
-import java.util.Map;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
@@ -33,8 +29,6 @@ import org.junit.Test;
  */
 public class TestPrometheusMetricsProvider {
 
-    private final CollectorRegistry registry = new CollectorRegistry();
-
     @Test
     public void testCache() {
         PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
@@ -56,7 +50,7 @@ public class TestPrometheusMetricsProvider {
 
     @Test
     public void testCounter() {
-        PrometheusCounter counter = new PrometheusCounter(registry, "testcounter");
+        LongAdderCounter counter = new LongAdderCounter();
         long value = counter.get();
         assertEquals(0L, value);
         counter.inc();
@@ -68,14 +62,16 @@ public class TestPrometheusMetricsProvider {
     }
 
     @Test
-    @SuppressWarnings("unchecked")
     public void testTwoCounters() throws Exception {
-        PrometheusCounter counter1 = new PrometheusCounter(registry, "testcounter");
-        PrometheusCounter counter2 = new PrometheusCounter(registry, "testcounter");
-        Field collectorsMapField = CollectorRegistry.class.getDeclaredField("namesToCollectors");
-        collectorsMapField.setAccessible(true);
-        Map<String, Collector> collectorMap = (Map<String, Collector>) collectorsMapField.get(registry);
-        assertEquals(1, collectorMap.size());
+        PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+        StatsLogger statsLogger =  provider.getStatsLogger("test");
+
+        Counter counter1 = statsLogger.getCounter("counter");
+        Counter counter2 = statsLogger.getCounter("counter");
+        assertEquals(counter1, counter2);
+        assertSame(counter1, counter2);
+
+        assertEquals(1, provider.counters.size());
     }
 
 }
diff --git a/microbenchmarks/pom.xml b/microbenchmarks/pom.xml
index a4ed471..0200465 100644
--- a/microbenchmarks/pom.xml
+++ b/microbenchmarks/pom.xml
@@ -58,6 +58,26 @@
       <scope>compile</scope>
       <type>jar</type>
     </dependency>
+    <dependency>
+      <groupId>org.apache.bookkeeper.stats</groupId>
+      <artifactId>prometheus-metrics-provider</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.bookkeeper.stats</groupId>
+      <artifactId>codahale-metrics-provider</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.bookkeeper.stats</groupId>
+      <artifactId>twitter-science-provider</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.bookkeeper.stats</groupId>
+      <artifactId>twitter-ostrich-provider</artifactId>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
   <build>
     <plugins>
diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/StatsLoggerBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/StatsLoggerBenchmark.java
new file mode 100644
index 0000000..3388933
--- /dev/null
+++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/StatsLoggerBenchmark.java
@@ -0,0 +1,96 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+package org.apache.bookkeeper.proto.checksum;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.stats.codahale.CodahaleMetricsProvider;
+import org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider;
+import org.apache.bookkeeper.stats.twitter.ostrich.OstrichProvider;
+import org.apache.bookkeeper.stats.twitter.science.TwitterStatsProvider;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+/**
+ * Microbenchmarks for different stats backend providers.
+ */
+@OutputTimeUnit(TimeUnit.MICROSECONDS)
+@Threads(16)
+@Fork(1)
+@Warmup(iterations = 1, time = 10, timeUnit = TimeUnit.SECONDS)
+@Measurement(iterations = 3, time = 10, timeUnit = TimeUnit.SECONDS)
+public class StatsLoggerBenchmark {
+
+    private static Map<String, Supplier<StatsProvider>> providers = new HashMap<>();
+
+    static {
+        providers.put("Prometheus", PrometheusMetricsProvider::new);
+        providers.put("Codahale", CodahaleMetricsProvider::new);
+        providers.put("Twitter", TwitterStatsProvider::new);
+        providers.put("Ostrich", OstrichProvider::new);
+    }
+
+    @State(Scope.Benchmark)
+    public static class LoggerState {
+        @Param({ "Prometheus" })//, "Codahale", "Twitter", "Ostrich" })
+        private String statsProvider;
+
+        private Counter counter;
+        private OpStatsLogger opStats;
+
+        private long startTime = System.nanoTime();
+
+        @Setup(Level.Trial)
+        public void setup() {
+            StatsProvider provider = providers.get(statsProvider).get();
+            StatsLogger logger = provider.getStatsLogger("test");
+            counter = logger.getCounter("counter");
+            opStats = logger.getOpStatsLogger("opstats");
+        }
+    }
+
+     @Benchmark
+    public void counterIncrement(LoggerState s) {
+        s.counter.inc();
+    }
+
+    @Benchmark
+    public void recordLatency(LoggerState s) {
+        s.opStats.registerSuccessfulValue(System.nanoTime() - s.startTime);
+    }
+}
diff --git a/pom.xml b/pom.xml
index c0f3e9b..d958172 100644
--- a/pom.xml
+++ b/pom.xml
@@ -141,6 +141,7 @@
     <ostrich.version>9.1.3</ostrich.version>
     <powermock.version>2.0.0-beta.5</powermock.version>
     <prometheus.version>0.0.21</prometheus.version>
+    <datasketches.version>0.8.3</datasketches.version>
     <protobuf.version>3.4.0</protobuf.version>
     <protoc-gen-grpc-java.version>1.0.0</protoc-gen-grpc-java.version>
     <rocksdb.version>5.8.6</rocksdb.version>
@@ -492,6 +493,13 @@
         <artifactId>simpleclient_servlet</artifactId>
         <version>${prometheus.version}</version>
       </dependency>
+      <!-- data-sketches -->
+      <dependency>
+        <groupId>com.yahoo.datasketches</groupId>
+        <artifactId>sketches-core</artifactId>
+        <version>${datasketches.version}</version>
+      </dependency>
+
       <!-- ostrich -->
       <dependency>
         <groupId>com.twitter</groupId>

-- 
To stop receiving notification emails like this one, please contact
sijie@apache.org.