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/pulsar.git
The following commit(s) were added to refs/heads/master by this push:
new 7bf7cdd Add producer metrics for Prometheus (#9541)
7bf7cdd is described below
commit 7bf7cdda1a2181c329176b403f7a60fc84f557fa
Author: WangJialing <[email protected]>
AuthorDate: Wed Feb 17 11:06:27 2021 +0800
Add producer metrics for Prometheus (#9541)
### Motivation
There are subscriptions and consumers metrics for prometheus but miss
producers' metrics.
### Modifications
1. Add producer metrics for prometheus.
2. Add a configration for whether export producer metrics, default is false.
### Verifying this change
Add test case testPerProducerStats() in PrometheusMetricsTest
---
conf/broker.conf | 3 +
.../apache/pulsar/broker/ServiceConfiguration.java | 5 ++
.../org/apache/pulsar/broker/PulsarService.java | 3 +-
.../stats/prometheus/AggregatedProducerStats.java | 31 +++++++
.../stats/prometheus/NamespaceStatsAggregator.java | 15 +++-
.../prometheus/PrometheusMetricsGenerator.java | 10 ++-
.../stats/prometheus/PrometheusMetricsServlet.java | 7 +-
.../pulsar/broker/stats/prometheus/TopicStats.java | 20 +++++
.../pulsar/broker/stats/PrometheusMetricsTest.java | 94 +++++++++++++++++++---
9 files changed, 166 insertions(+), 22 deletions(-)
diff --git a/conf/broker.conf b/conf/broker.conf
index 52b63d9..6b458c0 100644
--- a/conf/broker.conf
+++ b/conf/broker.conf
@@ -1088,6 +1088,9 @@ exposeTopicLevelMetricsInPrometheus=true
# Enable consumer level metrics. default is false
exposeConsumerLevelMetricsInPrometheus=false
+# Enable producer level metrics. default is false
+exposeProducerLevelMetricsInPrometheus=false
+
# Classname of Pluggable JVM GC metrics logger that can log GC specific metrics
# jvmGCMetricsLoggerClassName=
diff --git
a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
index 7820d61..ba2fcd8 100644
---
a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
+++
b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
@@ -1828,6 +1828,11 @@ public class ServiceConfiguration implements
PulsarConfiguration {
private boolean exposeConsumerLevelMetricsInPrometheus = false;
@FieldContext(
category = CATEGORY_METRICS,
+ doc = "If true, export producer level metrics otherwise namespace
level"
+ )
+ private boolean exposeProducerLevelMetricsInPrometheus = false;
+ @FieldContext(
+ category = CATEGORY_METRICS,
doc = "Classname of Pluggable JVM GC metrics logger that can log
GC specific metrics")
private String jvmGCMetricsLoggerClassName;
diff --git
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
index c820741..aedd965 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
@@ -568,7 +568,8 @@ public class PulsarService implements AutoCloseable {
"org.apache.pulsar.broker.lookup", true, attributeMap);
this.metricsServlet = new PrometheusMetricsServlet(
this, config.isExposeTopicLevelMetricsInPrometheus(),
- config.isExposeConsumerLevelMetricsInPrometheus());
+ config.isExposeConsumerLevelMetricsInPrometheus(),
+ config.isExposeProducerLevelMetricsInPrometheus());
if (pendingMetricsProviders != null) {
pendingMetricsProviders.forEach(provider ->
metricsServlet.addRawMetricsProvider(provider));
this.pendingMetricsProviders = null;
diff --git
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedProducerStats.java
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedProducerStats.java
new file mode 100644
index 0000000..d21e49d
--- /dev/null
+++
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedProducerStats.java
@@ -0,0 +1,31 @@
+/**
+ * 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.pulsar.broker.stats.prometheus;
+
+public class AggregatedProducerStats {
+
+ public long producerId;
+
+ public double msgRateIn;
+
+ public double msgThroughputIn;
+
+ public double averageMsgSize;
+
+}
diff --git
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java
index 2e728a6..0a55d81 100644
---
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java
+++
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java
@@ -47,7 +47,7 @@ public class NamespaceStatsAggregator {
};
public static void generate(PulsarService pulsar, boolean
includeTopicMetrics, boolean includeConsumerMetrics,
- SimpleTextOutputStream stream) {
+ boolean includeProducerMetrics, SimpleTextOutputStream stream) {
String cluster = pulsar.getConfiguration().getClusterName();
AggregatedNamespaceStats namespaceStats = localNamespaceStats.get();
TopicStats.resetTypes();
@@ -62,7 +62,7 @@ public class NamespaceStatsAggregator {
bundlesMap.forEach((bundle, topicsMap) -> {
topicsMap.forEach((name, topic) -> {
- getTopicStats(topic, topicStats, includeConsumerMetrics,
+ getTopicStats(topic, topicStats, includeConsumerMetrics,
includeProducerMetrics,
pulsar.getConfiguration().isExposePreciseBacklogInPrometheus(),
pulsar.getConfiguration().isExposeSubscriptionBacklogSizeInPrometheus());
@@ -86,7 +86,7 @@ public class NamespaceStatsAggregator {
}
private static void getTopicStats(Topic topic, TopicStats stats, boolean
includeConsumerMetrics,
- boolean getPreciseBacklog, boolean
subscriptionBacklogSize) {
+ boolean includeProducerMetrics, boolean getPreciseBacklog, boolean
subscriptionBacklogSize) {
stats.reset();
if (topic instanceof PersistentTopic) {
@@ -131,6 +131,15 @@ public class NamespaceStatsAggregator {
stats.producersCount++;
stats.rateIn += producer.getStats().msgRateIn;
stats.throughputIn += producer.getStats().msgThroughputIn;
+
+ if (includeProducerMetrics) {
+ AggregatedProducerStats producerStats =
stats.producerStats.computeIfAbsent(
+ producer.getProducerName(), k -> new
AggregatedProducerStats());
+ producerStats.producerId = producer.getStats().producerId;
+ producerStats.msgRateIn = producer.getStats().msgRateIn;
+ producerStats.msgThroughputIn =
producer.getStats().msgThroughputIn;
+ producerStats.averageMsgSize =
producer.getStats().averageMsgSize;
+ }
}
});
diff --git
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java
index 5a97ebf..676eac1 100644
---
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java
+++
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java
@@ -84,19 +84,21 @@ public class PrometheusMetricsGenerator {
}
public static void generate(PulsarService pulsar, boolean
includeTopicMetrics, boolean includeConsumerMetrics,
- OutputStream out) throws IOException {
- generate(pulsar, includeTopicMetrics, includeConsumerMetrics, out,
null);
+ boolean includeProducerMetrics, OutputStream out) throws IOException {
+ generate(pulsar, includeTopicMetrics, includeConsumerMetrics,
includeProducerMetrics, out, null);
}
public static void generate(PulsarService pulsar, boolean
includeTopicMetrics, boolean includeConsumerMetrics,
- OutputStream out, List<PrometheusRawMetricsProvider> metricsProviders)
throws IOException {
+ boolean includeProducerMetrics, OutputStream out,
List<PrometheusRawMetricsProvider> metricsProviders)
+ throws IOException {
ByteBuf buf = ByteBufAllocator.DEFAULT.heapBuffer();
try {
SimpleTextOutputStream stream = new SimpleTextOutputStream(buf);
generateSystemMetrics(stream,
pulsar.getConfiguration().getClusterName());
- NamespaceStatsAggregator.generate(pulsar, includeTopicMetrics,
includeConsumerMetrics, stream);
+ NamespaceStatsAggregator.generate(pulsar, includeTopicMetrics,
includeConsumerMetrics,
+ includeProducerMetrics, stream);
if (pulsar.getWorkerServiceOpt().isPresent()) {
pulsar.getWorkerService().generateFunctionsStats(stream);
diff --git
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java
index a142e9a..7e469a2 100644
---
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java
+++
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java
@@ -42,14 +42,17 @@ public class PrometheusMetricsServlet extends HttpServlet {
private final PulsarService pulsar;
private final boolean shouldExportTopicMetrics;
private final boolean shouldExportConsumerMetrics;
+ private final boolean shouldExportProducerMetrics;
private List<PrometheusRawMetricsProvider> metricsProviders;
private ExecutorService executor = null;
- public PrometheusMetricsServlet(PulsarService pulsar, boolean
includeTopicMetrics, boolean includeConsumerMetrics) {
+ public PrometheusMetricsServlet(PulsarService pulsar, boolean
includeTopicMetrics, boolean includeConsumerMetrics,
+ boolean shouldExportProducerMetrics) {
this.pulsar = pulsar;
this.shouldExportTopicMetrics = includeTopicMetrics;
this.shouldExportConsumerMetrics = includeConsumerMetrics;
+ this.shouldExportProducerMetrics = shouldExportProducerMetrics;
}
@Override
@@ -67,7 +70,7 @@ public class PrometheusMetricsServlet extends HttpServlet {
res.setStatus(HttpStatus.OK_200);
res.setContentType("text/plain");
PrometheusMetricsGenerator.generate(pulsar,
shouldExportTopicMetrics, shouldExportConsumerMetrics,
- res.getOutputStream(), metricsProviders);
+ shouldExportProducerMetrics, res.getOutputStream(),
metricsProviders);
context.complete();
} catch (Exception e) {
diff --git
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java
index 020a82b..416136b 100644
---
a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java
+++
b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java
@@ -54,6 +54,7 @@ class TopicStats {
Map<String, AggregatedReplicationStats> replicationStats = new HashMap<>();
Map<String, AggregatedSubscriptionStats> subscriptionStats = new
HashMap<>();
+ Map<String, AggregatedProducerStats> producerStats = new HashMap<>();
// Used for tracking duplicate TYPE definitions
static Map<String, String> metricWithTypeDefinition = new HashMap<>();
@@ -82,6 +83,7 @@ class TopicStats {
replicationStats.clear();
subscriptionStats.clear();
+ producerStats.clear();
storageWriteLatencyBuckets.reset();
storageLedgerWriteLatencyBuckets.reset();
entrySizeBuckets.reset();
@@ -163,6 +165,15 @@ class TopicStats {
metric(stream, cluster, namespace, topic, "pulsar_entry_size_count",
stats.entrySizeBuckets.getCount());
metric(stream, cluster, namespace, topic, "pulsar_entry_size_sum",
stats.entrySizeBuckets.getSum());
+ stats.producerStats.forEach((p, producerStats) -> {
+ metric(stream, cluster, namespace, topic, p,
producerStats.producerId, "pulsar_producer_msg_rate_in",
+ producerStats.msgRateIn);
+ metric(stream, cluster, namespace, topic, p,
producerStats.producerId, "pulsar_producer_msg_throughput_in",
+ producerStats.msgThroughputIn);
+ metric(stream, cluster, namespace, topic, p,
producerStats.producerId, "pulsar_producer_msg_average_Size",
+ producerStats.averageMsgSize);
+ });
+
stats.subscriptionStats.forEach((n, subsStats) -> {
metric(stream, cluster, namespace, topic, n,
"pulsar_subscription_back_log",
subsStats.msgBacklog);
@@ -266,6 +277,15 @@ class TopicStats {
}
private static void metric(SimpleTextOutputStream stream, String cluster,
String namespace, String topic,
+ String producerName, long produceId, String
name, double value) {
+ metricType(stream, name);
+
stream.write(name).write("{cluster=\"").write(cluster).write("\",namespace=\"").write(namespace)
+
.write("\",topic=\"").write(topic).write("\",producer_name=\"").write(producerName)
+ .write("\",producer_id=\"").write(produceId).write("\"} ");
+ stream.write(value).write('
').write(System.currentTimeMillis()).write('\n');
+ }
+
+ private static void metric(SimpleTextOutputStream stream, String cluster,
String namespace, String topic,
String subscription, String name, double value)
{
metricType(stream, name);
stream.write(name).write("{cluster=\"").write(cluster).write("\",namespace=\"").write(namespace)
diff --git
a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java
b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java
index 95291e3..511f43e 100644
---
a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java
+++
b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java
@@ -97,7 +97,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
}
Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT);
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, true, false, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, true, false, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
Collection<Metric> metric = metrics.get("pulsar_topics_count");
@@ -140,7 +140,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
}
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, true, false, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, true, false, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
@@ -247,7 +247,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
Awaitility.await().atMost(2, TimeUnit.SECONDS).until(() ->
sub2.getExpiredMessageRate() != 0.0);
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, true, false, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, true, false, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
// There should be 2 metrics with different tags for each topic
@@ -328,7 +328,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
}
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, false, false, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, false, false, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
@@ -371,6 +371,76 @@ public class PrometheusMetricsTest extends BrokerTestBase {
}
@Test
+ public void testPerProducerStats() throws Exception {
+ Producer<byte[]> p1 =
pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic1")
+ .producerName("producer1").create();
+ Producer<byte[]> p2 =
pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic2")
+ .producerName("producer2").create();
+
+ Consumer<byte[]> c1 = pulsarClient.newConsumer()
+ .topic("persistent://my-property/use/my-ns/my-topic1")
+ .subscriptionName("Test")
+ .subscribe();
+
+ Consumer<byte[]> c2 = pulsarClient.newConsumer()
+ .topic("persistent://my-property/use/my-ns/my-topic2")
+ .subscriptionName("Test")
+ .subscribe();
+
+ final int messages = 10;
+
+ for (int i = 0; i < messages; i++) {
+ String message = "my-message-" + i;
+ p1.send(message.getBytes());
+ p2.send(message.getBytes());
+ }
+
+ for (int i = 0; i < messages; i++) {
+ c1.acknowledge(c1.receive());
+ c2.acknowledge(c2.receive());
+ }
+
+ ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
+ PrometheusMetricsGenerator.generate(pulsar, true, false, true,
statsOut);
+ String metricsStr = new String(statsOut.toByteArray());
+
+ Multimap<String, Metric> metrics = parseMetrics(metricsStr);
+
+ metrics.entries().forEach(e -> {
+ System.out.println(e.getKey() + ": " + e.getValue());
+ });
+
+ List<Metric> cm = (List<Metric>)
metrics.get("pulsar_producer_msg_rate_in");
+ assertEquals(cm.size(), 2);
+ assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns");
+ assertEquals(cm.get(0).tags.get("topic"),
"persistent://my-property/use/my-ns/my-topic2");
+ assertEquals(cm.get(0).tags.get("producer_name"), "producer2");
+ assertEquals(cm.get(0).tags.get("producer_id"), "1");
+
+ assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns");
+ assertEquals(cm.get(1).tags.get("topic"),
"persistent://my-property/use/my-ns/my-topic1");
+ assertEquals(cm.get(1).tags.get("producer_name"), "producer1");
+ assertEquals(cm.get(1).tags.get("producer_id"), "0");
+
+ cm = (List<Metric>) metrics.get("pulsar_producer_msg_throughput_in");
+ assertEquals(cm.size(), 2);
+ assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns");
+ assertEquals(cm.get(0).tags.get("topic"),
"persistent://my-property/use/my-ns/my-topic2");
+ assertEquals(cm.get(0).tags.get("producer_name"), "producer2");
+ assertEquals(cm.get(0).tags.get("producer_id"), "1");
+
+ assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns");
+ assertEquals(cm.get(1).tags.get("topic"),
"persistent://my-property/use/my-ns/my-topic1");
+ assertEquals(cm.get(1).tags.get("producer_name"), "producer1");
+ assertEquals(cm.get(1).tags.get("producer_id"), "0");
+
+ p1.close();
+ p2.close();
+ c1.close();
+ c2.close();
+ }
+
+ @Test
public void testPerConsumerStats() throws Exception {
Producer<byte[]> p1 =
pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic1").create();
Producer<byte[]> p2 =
pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic2").create();
@@ -399,7 +469,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
}
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, true, true, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, true, true, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
@@ -479,7 +549,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
}
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, false, false, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, false, false, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Map<String, String> typeDefs = new HashMap<String, String>();
@@ -565,7 +635,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
}
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, false, false, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, false, false, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
@@ -601,7 +671,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
}
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, false, false, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, false, false, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
@@ -676,7 +746,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
}
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, false, false, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, false, false, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
@@ -749,7 +819,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
});
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, false, false, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, false, false, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
List<Metric> cm = (List<Metric>)
metrics.get("pulsar_authentication_success_count");
@@ -809,7 +879,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
}
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, false, false, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, false, false, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
List<Metric> cm = (List<Metric>)
metrics.get("pulsar_expired_token_count");
@@ -850,7 +920,7 @@ public class PrometheusMetricsTest extends BrokerTestBase {
}
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
- PrometheusMetricsGenerator.generate(pulsar, false, false, statsOut);
+ PrometheusMetricsGenerator.generate(pulsar, false, false, false,
statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
Metric countMetric = ((List<Metric>)
metrics.get("pulsar_expiring_token_minutes_count")).get(0);