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

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

commit c0d970c2c7a5044b5cc11dc4b2c155a87b61d11e
Author: ZhangJian He <[email protected]>
AuthorDate: Fri Jul 29 18:38:27 2022 +0800

    [ISSUE 2637] Fix jvm_memory_direct_bytes_used metrics when using jdk11+ 
(#3252)
    
    Fix #2637 #3247
    
    ### Motivation
    The mertics about `jvm_memory_direct_bytes_used` is acquired by netty's 
`PlatformDependent#DIRECT_MEMORY_COUNTER`. Which can only acquired the memory 
used by netty.
    
    ### Changes
    - use `java.nio.Bits#RESERVED_MEMORY` for jvm direct memory metrics.
    - add tests to ensure `jvm_memory_direct_bytes_max` and 
`jvm_memory_direct_bytes_used` gets value.
    
    (cherry picked from commit cefe9d44dad3558cd4cc30547007c3c41b9bfbec)
---
 .../prometheus/PrometheusMetricsProvider.java      | 30 ++++++----------
 .../prometheus/TestPrometheusMetricsProvider.java  | 40 ++++++++++++++++++++++
 2 files changed, 50 insertions(+), 20 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 974d914e3e..ab81992531 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
@@ -30,19 +30,19 @@ 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.lang.management.BufferPoolMXBean;
+import java.lang.management.ManagementFactory;
 import java.net.InetSocketAddress;
 import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 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.StatsLogger;
 import org.apache.bookkeeper.stats.StatsProvider;
 import org.apache.bookkeeper.stats.ThreadRegistry;
@@ -130,7 +130,7 @@ public class PrometheusMetricsProvider implements 
StatsProvider {
         registerMetrics(Gauge.build("jvm_memory_direct_bytes_used", 
"-").create().setChild(new Child() {
             @Override
             public double get() {
-                return directMemoryUsage != null ? 
directMemoryUsage.longValue() : Double.NaN;
+                return poolMxBeanOp.isPresent() ? 
poolMxBeanOp.get().getMemoryUsed() : Double.NaN;
             }
         }));
 
@@ -215,21 +215,11 @@ public class PrometheusMetricsProvider implements 
StatsProvider {
 
     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;
+    private static final Optional<BufferPoolMXBean> poolMxBeanOp;
 
-        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;
+    static {
+        List<BufferPoolMXBean> platformMXBeans = 
ManagementFactory.getPlatformMXBeans(BufferPoolMXBean.class);
+        poolMxBeanOp = platformMXBeans.stream()
+                .filter(bufferPoolMXBean -> 
bufferPoolMXBean.getName().equals("direct")).findAny();
     }
 }
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 df954e64d9..999be26cbb 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
@@ -21,11 +21,16 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 
+import java.io.StringWriter;
+import java.nio.ByteBuffer;
 import java.util.Collections;
+import java.util.HashMap;
+
 import lombok.Cleanup;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.commons.configuration.PropertiesConfiguration;
+import org.junit.Assert;
 import org.junit.Test;
 
 /**
@@ -111,4 +116,39 @@ public class TestPrometheusMetricsProvider {
         assertEquals(1, provider.counters.size());
     }
 
+    @Test
+    public void testJvmDirectMemoryMetrics() throws Exception {
+        PropertiesConfiguration config = new PropertiesConfiguration();
+        
config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, 
true);
+        
config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_PORT, 0);
+        
config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ADDRESS, 
"127.0.0.1");
+        ByteBuffer byteBuffer = ByteBuffer.allocateDirect(25);
+        PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+        try {
+            provider.start(config);
+            assertNotNull(provider.server);
+            StringWriter writer = new StringWriter();
+            provider.writeAllMetrics(writer);
+            String s = writer.toString();
+            String[] split = s.split(System.lineSeparator());
+            HashMap<String, String> map = new HashMap<>();
+            for (String str : split) {
+                String[] aux = str.split(" ");
+                map.put(aux[0], aux[1]);
+            }
+            String directBytesMax = map.get("jvm_memory_direct_bytes_max{}");
+            Assert.assertNotNull(directBytesMax);
+            Assert.assertNotEquals("Nan", directBytesMax);
+            Assert.assertNotEquals("-1", directBytesMax);
+            String directBytesUsed = map.get("jvm_memory_direct_bytes_used{}");
+            Assert.assertNotNull(directBytesUsed);
+            Assert.assertNotEquals("Nan", directBytesUsed);
+            Assert.assertTrue(Double.parseDouble(directBytesUsed) > 25);
+            // ensure byteBuffer doesn't gc
+            byteBuffer.clear();
+        } finally {
+            provider.stop();
+        }
+    }
+
 }

Reply via email to