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

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

commit 0175d99f90ac5f963cdc80f45c4260a307e9c498
Author: ZhangJian He <[email protected]>
AuthorDate: Tue Mar 29 14:29:00 2022 +0800

    fix duplicate typeline for prometheus type (#3137)
    
    (cherry picked from commit 38dc3281f5f175a7d194d4c0d4b255472886187b)
---
 .../prometheus/PrometheusMetricsProvider.java      |  9 ++---
 ...xtFormatUtil.java => PrometheusTextFormat.java} | 38 ++++++++++++++-------
 .../stats/prometheus/PrometheusTextFormatTest.java | 39 ++++++++++++++++++++++
 .../org/apache/bookkeeper/stats/StatsProvider.java |  2 +-
 4 files changed, 71 insertions(+), 17 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 502aa888f7..2acbfb6ad3 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
@@ -168,12 +168,13 @@ public class PrometheusMetricsProvider implements 
StatsProvider {
 
     @Override
     public void writeAllMetrics(Writer writer) throws IOException {
-        
PrometheusTextFormatUtil.writeMetricsCollectedByPrometheusClient(writer, 
registry);
+        PrometheusTextFormat prometheusTextFormat = new PrometheusTextFormat();
+        PrometheusTextFormat.writeMetricsCollectedByPrometheusClient(writer, 
registry);
 
-        gauges.forEach((sc, gauge) -> 
PrometheusTextFormatUtil.writeGauge(writer, sc.getScope(), gauge));
-        counters.forEach((sc, counter) -> 
PrometheusTextFormatUtil.writeCounter(writer, sc.getScope(), counter));
+        gauges.forEach((sc, gauge) -> prometheusTextFormat.writeGauge(writer, 
sc.getScope(), gauge));
+        counters.forEach((sc, counter) -> 
prometheusTextFormat.writeCounter(writer, sc.getScope(), counter));
         opStats.forEach((sc, opStatLogger) ->
-                PrometheusTextFormatUtil.writeOpStat(writer, sc.getScope(), 
opStatLogger));
+                prometheusTextFormat.writeOpStat(writer, sc.getScope(), 
opStatLogger));
     }
 
     @Override
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/PrometheusTextFormat.java
similarity index 85%
rename from 
bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusTextFormatUtil.java
rename to 
bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusTextFormat.java
index 1259f658ff..c27e5b3706 100644
--- 
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/PrometheusTextFormat.java
@@ -24,18 +24,23 @@ import io.prometheus.client.CollectorRegistry;
 import java.io.IOException;
 import java.io.Writer;
 import java.util.Enumeration;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 
 /**
  * Logic to write metrics in Prometheus text format.
  */
-public class PrometheusTextFormatUtil {
-    static void writeGauge(Writer w, String name, SimpleGauge<? extends 
Number> gauge) {
+public class PrometheusTextFormat {
+
+    Set<String> metricNameSet = new HashSet<>();
+
+    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");
+            writeType(w, name, "gauge");
             w.append(name);
             writeLabels(w, gauge.getLabels());
             w.append(' ').append(gauge.getSample().toString()).append('\n');
@@ -44,12 +49,12 @@ public class PrometheusTextFormatUtil {
         }
     }
 
-    static void writeCounter(Writer w, String name, LongAdderCounter counter) {
+    void writeCounter(Writer w, String name, LongAdderCounter counter) {
         // Example:
         // # TYPE jvm_threads_started_total counter
         // jvm_threads_started_total 59
         try {
-            w.append("# TYPE ").append(name).append(" counter\n");
+            writeType(w, name, "counter");
             w.append(name);
             writeLabels(w, counter.getLabels());
             w.append(' ').append(counter.get().toString()).append('\n');
@@ -58,7 +63,7 @@ public class PrometheusTextFormatUtil {
         }
     }
 
-    static void writeOpStat(Writer w, String name, DataSketchesOpStatsLogger 
opStat) {
+    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
@@ -80,7 +85,7 @@ public class PrometheusTextFormatUtil {
         // 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");
+            writeType(w, name, "summary");
             writeQuantile(w, opStat, name, false, 0.5);
             writeQuantile(w, opStat, name, false, 0.75);
             writeQuantile(w, opStat, name, false, 0.95);
@@ -106,7 +111,7 @@ public class PrometheusTextFormatUtil {
         }
     }
 
-    private static void writeLabels(Writer w, Map<String, String> labels) 
throws IOException {
+    private void writeLabels(Writer w, Map<String, String> labels) throws 
IOException {
         if (labels.isEmpty()) {
             return;
         }
@@ -116,7 +121,7 @@ public class PrometheusTextFormatUtil {
         w.append('}');
     }
 
-    private static void writeLabelsNoBraces(Writer w, Map<String, String> 
labels) throws IOException {
+    private void writeLabelsNoBraces(Writer w, Map<String, String> labels) 
throws IOException {
         if (labels.isEmpty()) {
             return;
         }
@@ -134,7 +139,7 @@ public class PrometheusTextFormatUtil {
         }
     }
 
-    private static void writeQuantile(Writer w, DataSketchesOpStatsLogger 
opStat, String name, Boolean success,
+    private void writeQuantile(Writer w, DataSketchesOpStatsLogger opStat, 
String name, Boolean success,
             double quantile) throws IOException {
         w.append(name)
                 .append("{success=\"").append(success.toString())
@@ -148,7 +153,7 @@ public class PrometheusTextFormatUtil {
                 .append(Double.toString(opStat.getQuantileValue(success, 
quantile))).append('\n');
     }
 
-    private static void writeCount(Writer w, DataSketchesOpStatsLogger opStat, 
String name, Boolean success)
+    private void writeCount(Writer w, DataSketchesOpStatsLogger opStat, String 
name, Boolean success)
             throws IOException {
         
w.append(name).append("_count{success=\"").append(success.toString()).append("\"");
         if (!opStat.getLabels().isEmpty()) {
@@ -159,7 +164,7 @@ public class PrometheusTextFormatUtil {
                 .append(Long.toString(opStat.getCount(success))).append('\n');
     }
 
-    private static void writeSum(Writer w, DataSketchesOpStatsLogger opStat, 
String name, Boolean success)
+    private void writeSum(Writer w, DataSketchesOpStatsLogger opStat, String 
name, Boolean success)
             throws IOException {
         
w.append(name).append("_sum{success=\"").append(success.toString()).append("\"");
         if (!opStat.getLabels().isEmpty()) {
@@ -195,4 +200,13 @@ public class PrometheusTextFormatUtil {
             }
         }
     }
+
+    void writeType(Writer w, String name, String type) throws IOException {
+        if (metricNameSet.contains(name)) {
+            return;
+        }
+        metricNameSet.add(name);
+        w.append("# TYPE ").append(name).append(" ").append(type).append("\n");
+    }
+
 }
diff --git 
a/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/PrometheusTextFormatTest.java
 
b/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/PrometheusTextFormatTest.java
new file mode 100644
index 0000000000..2c6d373468
--- /dev/null
+++ 
b/bookkeeper-stats-providers/prometheus-metrics-provider/src/test/java/org/apache/bookkeeper/stats/prometheus/PrometheusTextFormatTest.java
@@ -0,0 +1,39 @@
+/**
+ * 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 java.io.IOException;
+import java.io.StringWriter;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link PrometheusTextFormat}.
+ */
+public class PrometheusTextFormatTest {
+
+    @Test
+    public void testPrometheusTypeDuplicate() throws IOException {
+        PrometheusTextFormat prometheusTextFormat = new PrometheusTextFormat();
+        StringWriter writer = new StringWriter();
+        prometheusTextFormat.writeType(writer, "counter", "gauge");
+        prometheusTextFormat.writeType(writer, "counter", "gauge");
+        String string = writer.toString();
+        Assert.assertEquals("# TYPE counter gauge\n", string);
+    }
+
+}
diff --git 
a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java 
b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java
index 0bb236a6e5..0d500f1fd4 100644
--- 
a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java
+++ 
b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java
@@ -26,7 +26,7 @@ import org.apache.commons.lang.StringUtils;
  */
 public interface StatsProvider {
     /**
-     * Intialize the stats provider by loading the given configuration 
<i>conf</i>.
+     * Initialize the stats provider by loading the given configuration 
<i>conf</i>.
      *
      * @param conf
      *          Configuration to configure the stats provider.

Reply via email to