From 1f160a43cfcc661d2bfda1f872b7ec24fb2b3060 Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Wed, 11 May 2022 17:38:56 +0100 Subject: [PATCH 01/19] [fix][broker] Cherry pick changes --- .../prometheus/NamespaceStatsAggregator.java | 268 ++++++------ .../broker/stats/prometheus/TopicStats.java | 389 ++++++++++-------- .../broker/stats/PrometheusMetricsTest.java | 54 +++ 3 files changed, 405 insertions(+), 306 deletions(-) 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 16e438e2a2eb3..4e3d0dc202291 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 @@ -19,6 +19,10 @@ package org.apache.pulsar.broker.stats.prometheus; import io.netty.util.concurrent.FastThreadLocal; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.LongAdder; import lombok.extern.slf4j.Slf4j; @@ -56,13 +60,16 @@ protected TopicStats initialValue() throws Exception { }; public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, SimpleTextOutputStream stream) { + boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, + SimpleTextOutputStream stream) { String cluster = pulsar.getConfiguration().getClusterName(); AggregatedNamespaceStats namespaceStats = localNamespaceStats.get(); TopicStats.resetTypes(); TopicStats topicStats = localTopicStats.get(); - printDefaultBrokerStats(stream, cluster); + Map> metrics = new HashMap<>(); + + printDefaultBrokerStats(metrics, cluster); Optional compactorMXBean = getCompactorMXBean(pulsar); LongAdder topicsCount = new LongAdder(); @@ -70,41 +77,45 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b namespaceStats.reset(); topicsCount.reset(); - bundlesMap.forEach((bundle, topicsMap) -> { - topicsMap.forEach((name, topic) -> { - getTopicStats(topic, topicStats, includeConsumerMetrics, includeProducerMetrics, - pulsar.getConfiguration().isExposePreciseBacklogInPrometheus(), - pulsar.getConfiguration().isExposeSubscriptionBacklogSizeInPrometheus(), - compactorMXBean - ); - - if (includeTopicMetrics) { - topicsCount.add(1); - TopicStats.printTopicStats(stream, cluster, namespace, name, topicStats, compactorMXBean, - splitTopicAndPartitionIndexLabel); - } else { - namespaceStats.updateStats(topicStats); - } - }); - }); + bundlesMap.forEach((bundle, topicsMap) -> topicsMap.forEach((name, topic) -> { + getTopicStats(topic, topicStats, includeConsumerMetrics, includeProducerMetrics, + pulsar.getConfiguration().isExposePreciseBacklogInPrometheus(), + pulsar.getConfiguration().isExposeSubscriptionBacklogSizeInPrometheus(), + compactorMXBean + ); + + if (includeTopicMetrics) { + topicsCount.add(1); + TopicStats.printTopicStats(metrics, cluster, namespace, name, topicStats, compactorMXBean, + splitTopicAndPartitionIndexLabel); + } else { + namespaceStats.updateStats(topicStats); + } + })); if (!includeTopicMetrics) { // Only include namespace level stats if we don't have the per-topic, otherwise we're going to report // the same data twice, and it will make the aggregation difficult - printNamespaceStats(stream, cluster, namespace, namespaceStats); + printNamespaceStats(metrics, cluster, namespace, namespaceStats); } else { - printTopicsCountStats(stream, cluster, namespace, topicsCount); + printTopicsCountStats(metrics, cluster, namespace, topicsCount); } }); + + metrics.forEach((type, lines) -> { + TopicStats.metricType(stream, type); + lines.forEach(stream::write); + }); } private static Optional getCompactorMXBean(PulsarService pulsar) { Compactor compactor = pulsar.getNullableCompactor(); - return Optional.ofNullable(compactor).map(c -> c.getStats()); + return Optional.ofNullable(compactor).map(Compactor::getStats); } private static void getTopicStats(Topic topic, TopicStats stats, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean getPreciseBacklog, boolean subscriptionBacklogSize, + boolean includeProducerMetrics, boolean getPreciseBacklog, + boolean subscriptionBacklogSize, Optional compactorMXBean) { stats.reset(); @@ -267,161 +278,164 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include }); } - private static void printDefaultBrokerStats(SimpleTextOutputStream stream, String cluster) { + private static void printDefaultBrokerStats(Map> metrics, String cluster) { // Print metrics with 0 values. This is necessary to have the available brokers being // reported in the brokers dashboard even if they don't have any topic or traffic - metric(stream, cluster, "pulsar_topics_count", 0); - metric(stream, cluster, "pulsar_subscriptions_count", 0); - metric(stream, cluster, "pulsar_producers_count", 0); - metric(stream, cluster, "pulsar_consumers_count", 0); - metric(stream, cluster, "pulsar_rate_in", 0); - metric(stream, cluster, "pulsar_rate_out", 0); - metric(stream, cluster, "pulsar_throughput_in", 0); - metric(stream, cluster, "pulsar_throughput_out", 0); - metric(stream, cluster, "pulsar_storage_size", 0); - metric(stream, cluster, "pulsar_storage_logical_size", 0); - metric(stream, cluster, "pulsar_storage_write_rate", 0); - metric(stream, cluster, "pulsar_storage_read_rate", 0); - metric(stream, cluster, "pulsar_msg_backlog", 0); + metric(metrics, cluster, "pulsar_topics_count", 0); + metric(metrics, cluster, "pulsar_subscriptions_count", 0); + metric(metrics, cluster, "pulsar_producers_count", 0); + metric(metrics, cluster, "pulsar_consumers_count", 0); + metric(metrics, cluster, "pulsar_rate_in", 0); + metric(metrics, cluster, "pulsar_rate_out", 0); + metric(metrics, cluster, "pulsar_throughput_in", 0); + metric(metrics, cluster, "pulsar_throughput_out", 0); + metric(metrics, cluster, "pulsar_storage_size", 0); + metric(metrics, cluster, "pulsar_storage_logical_size", 0); + metric(metrics, cluster, "pulsar_storage_write_rate", 0); + metric(metrics, cluster, "pulsar_storage_read_rate", 0); + metric(metrics, cluster, "pulsar_msg_backlog", 0); } - private static void printTopicsCountStats(SimpleTextOutputStream stream, String cluster, String namespace, + private static void printTopicsCountStats(Map> metrics, String cluster, String namespace, LongAdder topicsCount) { - metric(stream, cluster, namespace, "pulsar_topics_count", topicsCount.sum()); + metric(metrics, cluster, namespace, "pulsar_topics_count", topicsCount.sum()); } - private static void printNamespaceStats(SimpleTextOutputStream stream, String cluster, String namespace, + private static void printNamespaceStats(Map> metrics, String cluster, String namespace, AggregatedNamespaceStats stats) { - metric(stream, cluster, namespace, "pulsar_topics_count", stats.topicsCount); - metric(stream, cluster, namespace, "pulsar_subscriptions_count", stats.subscriptionsCount); - metric(stream, cluster, namespace, "pulsar_producers_count", stats.producersCount); - metric(stream, cluster, namespace, "pulsar_consumers_count", stats.consumersCount); - - metric(stream, cluster, namespace, "pulsar_rate_in", stats.rateIn); - metric(stream, cluster, namespace, "pulsar_rate_out", stats.rateOut); - metric(stream, cluster, namespace, "pulsar_throughput_in", stats.throughputIn); - metric(stream, cluster, namespace, "pulsar_throughput_out", stats.throughputOut); - metric(stream, cluster, namespace, "pulsar_consumer_msg_ack_rate", stats.messageAckRate); - - metric(stream, cluster, namespace, "pulsar_in_bytes_total", stats.bytesInCounter); - metric(stream, cluster, namespace, "pulsar_in_messages_total", stats.msgInCounter); - metric(stream, cluster, namespace, "pulsar_out_bytes_total", stats.bytesOutCounter); - metric(stream, cluster, namespace, "pulsar_out_messages_total", stats.msgOutCounter); - - metric(stream, cluster, namespace, "pulsar_storage_size", stats.managedLedgerStats.storageSize); - metric(stream, cluster, namespace, "pulsar_storage_logical_size", stats.managedLedgerStats.storageLogicalSize); - metric(stream, cluster, namespace, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize); - metric(stream, cluster, namespace, "pulsar_storage_offloaded_size", + metric(metrics, cluster, namespace, "pulsar_topics_count", stats.topicsCount); + metric(metrics, cluster, namespace, "pulsar_subscriptions_count", stats.subscriptionsCount); + metric(metrics, cluster, namespace, "pulsar_producers_count", stats.producersCount); + metric(metrics, cluster, namespace, "pulsar_consumers_count", stats.consumersCount); + + metric(metrics, cluster, namespace, "pulsar_rate_in", stats.rateIn); + metric(metrics, cluster, namespace, "pulsar_rate_out", stats.rateOut); + metric(metrics, cluster, namespace, "pulsar_throughput_in", stats.throughputIn); + metric(metrics, cluster, namespace, "pulsar_throughput_out", stats.throughputOut); + metric(metrics, cluster, namespace, "pulsar_consumer_msg_ack_rate", stats.messageAckRate); + + metric(metrics, cluster, namespace, "pulsar_in_bytes_total", stats.bytesInCounter); + metric(metrics, cluster, namespace, "pulsar_in_messages_total", stats.msgInCounter); + metric(metrics, cluster, namespace, "pulsar_out_bytes_total", stats.bytesOutCounter); + metric(metrics, cluster, namespace, "pulsar_out_messages_total", stats.msgOutCounter); + + metric(metrics, cluster, namespace, "pulsar_storage_size", stats.managedLedgerStats.storageSize); + metric(metrics, cluster, namespace, "pulsar_storage_logical_size", stats.managedLedgerStats.storageLogicalSize); + metric(metrics, cluster, namespace, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize); + metric(metrics, cluster, namespace, "pulsar_storage_offloaded_size", stats.managedLedgerStats.offloadedStorageUsed); - metric(stream, cluster, namespace, "pulsar_storage_write_rate", stats.managedLedgerStats.storageWriteRate); - metric(stream, cluster, namespace, "pulsar_storage_read_rate", stats.managedLedgerStats.storageReadRate); + metric(metrics, cluster, namespace, "pulsar_storage_write_rate", stats.managedLedgerStats.storageWriteRate); + metric(metrics, cluster, namespace, "pulsar_storage_read_rate", stats.managedLedgerStats.storageReadRate); - metric(stream, cluster, namespace, "pulsar_subscription_delayed", stats.msgDelayed); + metric(metrics, cluster, namespace, "pulsar_subscription_delayed", stats.msgDelayed); - metricWithRemoteCluster(stream, cluster, namespace, "pulsar_msg_backlog", "local", stats.msgBacklog); + metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_msg_backlog", "local", stats.msgBacklog); stats.managedLedgerStats.storageWriteLatencyBuckets.refresh(); long[] latencyBuckets = stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets(); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_le_0_5", latencyBuckets[0]); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_le_1", latencyBuckets[1]); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_le_5", latencyBuckets[2]); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_le_10", latencyBuckets[3]); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_le_20", latencyBuckets[4]); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_le_50", latencyBuckets[5]); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_le_100", latencyBuckets[6]); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_le_200", latencyBuckets[7]); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_le_1000", latencyBuckets[8]); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_overflow", latencyBuckets[9]); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_count", + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_0_5", latencyBuckets[0]); + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_1", latencyBuckets[1]); + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_5", latencyBuckets[2]); + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_10", latencyBuckets[3]); + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_20", latencyBuckets[4]); + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_50", latencyBuckets[5]); + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_100", latencyBuckets[6]); + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_200", latencyBuckets[7]); + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_1000", latencyBuckets[8]); + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_overflow", latencyBuckets[9]); + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_count", stats.managedLedgerStats.storageWriteLatencyBuckets.getCount()); - metric(stream, cluster, namespace, "pulsar_storage_write_latency_sum", + metric(metrics, cluster, namespace, "pulsar_storage_write_latency_sum", stats.managedLedgerStats.storageWriteLatencyBuckets.getSum()); stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.refresh(); long[] ledgerWritelatencyBuckets = stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets(); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_le_0_5", ledgerWritelatencyBuckets[0]); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_le_1", ledgerWritelatencyBuckets[1]); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_le_5", ledgerWritelatencyBuckets[2]); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_le_10", ledgerWritelatencyBuckets[3]); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_le_20", ledgerWritelatencyBuckets[4]); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_le_50", ledgerWritelatencyBuckets[5]); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_le_100", ledgerWritelatencyBuckets[6]); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_le_200", ledgerWritelatencyBuckets[7]); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_le_1000", ledgerWritelatencyBuckets[8]); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_overflow", + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_0_5", ledgerWritelatencyBuckets[0]); + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_1", ledgerWritelatencyBuckets[1]); + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_5", ledgerWritelatencyBuckets[2]); + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_10", ledgerWritelatencyBuckets[3]); + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_20", ledgerWritelatencyBuckets[4]); + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_50", ledgerWritelatencyBuckets[5]); + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_100", ledgerWritelatencyBuckets[6]); + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_200", ledgerWritelatencyBuckets[7]); + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_1000", + ledgerWritelatencyBuckets[8]); + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_overflow", ledgerWritelatencyBuckets[9]); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_count", + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_count", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount()); - metric(stream, cluster, namespace, "pulsar_storage_ledger_write_latency_sum", + metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_sum", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum()); stats.managedLedgerStats.entrySizeBuckets.refresh(); long[] entrySizeBuckets = stats.managedLedgerStats.entrySizeBuckets.getBuckets(); - metric(stream, cluster, namespace, "pulsar_entry_size_le_128", entrySizeBuckets[0]); - metric(stream, cluster, namespace, "pulsar_entry_size_le_512", entrySizeBuckets[1]); - metric(stream, cluster, namespace, "pulsar_entry_size_le_1_kb", entrySizeBuckets[2]); - metric(stream, cluster, namespace, "pulsar_entry_size_le_2_kb", entrySizeBuckets[3]); - metric(stream, cluster, namespace, "pulsar_entry_size_le_4_kb", entrySizeBuckets[4]); - metric(stream, cluster, namespace, "pulsar_entry_size_le_16_kb", entrySizeBuckets[5]); - metric(stream, cluster, namespace, "pulsar_entry_size_le_100_kb", entrySizeBuckets[6]); - metric(stream, cluster, namespace, "pulsar_entry_size_le_1_mb", entrySizeBuckets[7]); - metric(stream, cluster, namespace, "pulsar_entry_size_le_overflow", entrySizeBuckets[8]); - metric(stream, cluster, namespace, "pulsar_entry_size_count", + metric(metrics, cluster, namespace, "pulsar_entry_size_le_128", entrySizeBuckets[0]); + metric(metrics, cluster, namespace, "pulsar_entry_size_le_512", entrySizeBuckets[1]); + metric(metrics, cluster, namespace, "pulsar_entry_size_le_1_kb", entrySizeBuckets[2]); + metric(metrics, cluster, namespace, "pulsar_entry_size_le_2_kb", entrySizeBuckets[3]); + metric(metrics, cluster, namespace, "pulsar_entry_size_le_4_kb", entrySizeBuckets[4]); + metric(metrics, cluster, namespace, "pulsar_entry_size_le_16_kb", entrySizeBuckets[5]); + metric(metrics, cluster, namespace, "pulsar_entry_size_le_100_kb", entrySizeBuckets[6]); + metric(metrics, cluster, namespace, "pulsar_entry_size_le_1_mb", entrySizeBuckets[7]); + metric(metrics, cluster, namespace, "pulsar_entry_size_le_overflow", entrySizeBuckets[8]); + metric(metrics, cluster, namespace, "pulsar_entry_size_count", stats.managedLedgerStats.entrySizeBuckets.getCount()); - metric(stream, cluster, namespace, "pulsar_entry_size_sum", + metric(metrics, cluster, namespace, "pulsar_entry_size_sum", stats.managedLedgerStats.entrySizeBuckets.getSum()); if (!stats.replicationStats.isEmpty()) { stats.replicationStats.forEach((remoteCluster, replStats) -> { - metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_rate_in", remoteCluster, + metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_rate_in", remoteCluster, replStats.msgRateIn); - metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_rate_out", remoteCluster, + metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_rate_out", remoteCluster, replStats.msgRateOut); - metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_throughput_in", remoteCluster, + metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_throughput_in", remoteCluster, replStats.msgThroughputIn); - metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_throughput_out", remoteCluster, + metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_throughput_out", remoteCluster, replStats.msgThroughputOut); - metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_backlog", remoteCluster, + metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_backlog", remoteCluster, replStats.replicationBacklog); - metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_connected_count", remoteCluster, + metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_connected_count", + remoteCluster, replStats.connectedCount); - metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_rate_expired", remoteCluster, + metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_rate_expired", remoteCluster, replStats.msgRateExpired); - metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_delay_in_seconds", + metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_delay_in_seconds", remoteCluster, replStats.replicationDelayInSeconds); }); } } - private static void metric(SimpleTextOutputStream stream, String cluster, String name, - long value) { - TopicStats.metricType(stream, name); - stream.write(name) - .write("{cluster=\"").write(cluster).write("\"} ") - .write(value).write(' ').write(System.currentTimeMillis()) - .write('\n'); + private static void metric(Map> metrics, String cluster, String name, + long value) { + addMetric(metrics, name, + name + "{cluster=\"" + cluster + "\"} " + value + ' ' + System.currentTimeMillis() + '\n'); } - private static void metric(SimpleTextOutputStream stream, String cluster, String namespace, String name, + private static void metric(Map> metrics, String cluster, String namespace, String name, long value) { - TopicStats.metricType(stream, name); - stream.write(name).write("{cluster=\"").write(cluster).write("\",namespace=\"").write(namespace).write("\"} "); - stream.write(value).write(' ').write(System.currentTimeMillis()).write('\n'); + addMetric(metrics, name, name + "{cluster=\"" + cluster + "\",namespace=\"" + namespace + "\"} " + value + ' ' + + System.currentTimeMillis() + '\n'); } - private static void metric(SimpleTextOutputStream stream, String cluster, String namespace, String name, + private static void metric(Map> metrics, String cluster, String namespace, String name, double value) { - TopicStats.metricType(stream, name); - stream.write(name).write("{cluster=\"").write(cluster).write("\",namespace=\"").write(namespace).write("\"} "); - stream.write(value).write(' ').write(System.currentTimeMillis()).write('\n'); + addMetric(metrics, name, name + "{cluster=\"" + cluster + "\",namespace=\"" + namespace + "\"} " + value + ' ' + + System.currentTimeMillis() + '\n'); } - private static void metricWithRemoteCluster(SimpleTextOutputStream stream, String cluster, String namespace, + private static void metricWithRemoteCluster(Map> metrics, String cluster, String namespace, String name, String remoteCluster, double value) { - TopicStats.metricType(stream, name); - stream.write(name).write("{cluster=\"").write(cluster).write("\",namespace=\"").write(namespace); - stream.write("\",remote_cluster=\"").write(remoteCluster).write("\"} "); - stream.write(value).write(' ').write(System.currentTimeMillis()).write('\n'); + addMetric(metrics, name, + name + "{cluster=\"" + cluster + "\",namespace=\"" + namespace + "\",remote_cluster=\"" + remoteCluster + + "\"} " + value + ' ' + System.currentTimeMillis() + '\n'); + } + + static void addMetric(Map> metrics, String type, String metricLine) { + if (!metrics.containsKey(type)) { + metrics.put(type, new ArrayList<>()); + } + metrics.get(type).add(metricLine); } } 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 e6e5883847df2..3966a07c25f11 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 @@ -18,10 +18,16 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import static org.apache.pulsar.broker.stats.prometheus.NamespaceStatsAggregator.addMetric; import static org.apache.pulsar.common.naming.TopicName.PARTITIONED_TOPIC_SUFFIX; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import java.nio.charset.StandardCharsets; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.Consumer; import org.apache.bookkeeper.mledger.util.StatsBuckets; import org.apache.pulsar.common.util.SimpleTextOutputStream; import org.apache.pulsar.compaction.CompactionRecord; @@ -107,199 +113,199 @@ static void resetTypes() { metricWithTypeDefinition.clear(); } - static void printTopicStats(SimpleTextOutputStream stream, String cluster, String namespace, String topic, + static void printTopicStats(Map> metrics, String cluster, String namespace, String topic, TopicStats stats, Optional compactorMXBean, boolean splitTopicAndPartitionIndexLabel) { - metric(stream, cluster, namespace, topic, "pulsar_subscriptions_count", stats.subscriptionsCount, + metric(metrics, cluster, namespace, topic, "pulsar_subscriptions_count", stats.subscriptionsCount, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_producers_count", stats.producersCount, + metric(metrics, cluster, namespace, topic, "pulsar_producers_count", stats.producersCount, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_consumers_count", stats.consumersCount, + metric(metrics, cluster, namespace, topic, "pulsar_consumers_count", stats.consumersCount, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_rate_in", stats.rateIn, + metric(metrics, cluster, namespace, topic, "pulsar_rate_in", stats.rateIn, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_rate_out", stats.rateOut, + metric(metrics, cluster, namespace, topic, "pulsar_rate_out", stats.rateOut, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_throughput_in", stats.throughputIn, + metric(metrics, cluster, namespace, topic, "pulsar_throughput_in", stats.throughputIn, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_throughput_out", stats.throughputOut, + metric(metrics, cluster, namespace, topic, "pulsar_throughput_out", stats.throughputOut, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_average_msg_size", stats.averageMsgSize, + metric(metrics, cluster, namespace, topic, "pulsar_average_msg_size", stats.averageMsgSize, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_size", stats.managedLedgerStats.storageSize, + metric(metrics, cluster, namespace, topic, "pulsar_storage_size", stats.managedLedgerStats.storageSize, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_logical_size", + metric(metrics, cluster, namespace, topic, "pulsar_storage_logical_size", stats.managedLedgerStats.storageLogicalSize, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_msg_backlog", stats.msgBacklog, + metric(metrics, cluster, namespace, topic, "pulsar_msg_backlog", stats.msgBacklog, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_rate", + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_rate", stats.managedLedgerStats.storageWriteRate, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_read_rate", stats.managedLedgerStats.storageReadRate, + metric(metrics, cluster, namespace, topic, "pulsar_storage_read_rate", stats.managedLedgerStats.storageReadRate, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_backlog_size", + metric(metrics, cluster, namespace, topic, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_publish_rate_limit_times", stats.publishRateLimitedTimes, + metric(metrics, cluster, namespace, topic, "pulsar_publish_rate_limit_times", stats.publishRateLimitedTimes, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_offloaded_size", stats.managedLedgerStats + metric(metrics, cluster, namespace, topic, "pulsar_storage_offloaded_size", stats.managedLedgerStats .offloadedStorageUsed, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_backlog_quota_limit", stats.backlogQuotaLimit, + metric(metrics, cluster, namespace, topic, "pulsar_storage_backlog_quota_limit", stats.backlogQuotaLimit, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_backlog_quota_limit_time", + metric(metrics, cluster, namespace, topic, "pulsar_storage_backlog_quota_limit_time", stats.backlogQuotaLimitTime, splitTopicAndPartitionIndexLabel); long[] latencyBuckets = stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets(); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_le_0_5", latencyBuckets[0], + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_0_5", latencyBuckets[0], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_le_1", latencyBuckets[1], + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_1", latencyBuckets[1], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_le_5", latencyBuckets[2], + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_5", latencyBuckets[2], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_le_10", latencyBuckets[3], + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_10", latencyBuckets[3], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_le_20", latencyBuckets[4], + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_20", latencyBuckets[4], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_le_50", latencyBuckets[5], + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_50", latencyBuckets[5], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_le_100", latencyBuckets[6], + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_100", latencyBuckets[6], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_le_200", latencyBuckets[7], + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_200", latencyBuckets[7], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_le_1000", latencyBuckets[8], + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_1000", latencyBuckets[8], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_overflow", latencyBuckets[9], + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_overflow", latencyBuckets[9], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_count", + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_count", stats.managedLedgerStats.storageWriteLatencyBuckets.getCount(), splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_write_latency_sum", + metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_sum", stats.managedLedgerStats.storageWriteLatencyBuckets.getSum(), splitTopicAndPartitionIndexLabel); long[] ledgerWriteLatencyBuckets = stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets(); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_0_5", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_0_5", ledgerWriteLatencyBuckets[0], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_1", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_1", ledgerWriteLatencyBuckets[1], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_5", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_5", ledgerWriteLatencyBuckets[2], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_10", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_10", ledgerWriteLatencyBuckets[3], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_20", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_20", ledgerWriteLatencyBuckets[4], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_50", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_50", ledgerWriteLatencyBuckets[5], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_100", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_100", ledgerWriteLatencyBuckets[6], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_200", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_200", ledgerWriteLatencyBuckets[7], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_1000", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_1000", ledgerWriteLatencyBuckets[8], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_overflow", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_overflow", ledgerWriteLatencyBuckets[9], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_count", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_count", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount(), splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_sum", + metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_sum", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum(), splitTopicAndPartitionIndexLabel); long[] entrySizeBuckets = stats.managedLedgerStats.entrySizeBuckets.getBuckets(); - metric(stream, cluster, namespace, topic, "pulsar_entry_size_le_128", entrySizeBuckets[0], + metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_128", entrySizeBuckets[0], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_entry_size_le_512", entrySizeBuckets[1], + metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_512", entrySizeBuckets[1], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_entry_size_le_1_kb", entrySizeBuckets[2], + metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_1_kb", entrySizeBuckets[2], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_entry_size_le_2_kb", entrySizeBuckets[3], + metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_2_kb", entrySizeBuckets[3], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_entry_size_le_4_kb", entrySizeBuckets[4], + metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_4_kb", entrySizeBuckets[4], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_entry_size_le_16_kb", entrySizeBuckets[5], + metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_16_kb", entrySizeBuckets[5], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_entry_size_le_100_kb", entrySizeBuckets[6], + metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_100_kb", entrySizeBuckets[6], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_entry_size_le_1_mb", entrySizeBuckets[7], + metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_1_mb", entrySizeBuckets[7], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_entry_size_le_overflow", entrySizeBuckets[8], + metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_overflow", entrySizeBuckets[8], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_entry_size_count", + metric(metrics, cluster, namespace, topic, "pulsar_entry_size_count", stats.managedLedgerStats.entrySizeBuckets.getCount(), splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_entry_size_sum", + metric(metrics, cluster, namespace, topic, "pulsar_entry_size_sum", stats.managedLedgerStats.entrySizeBuckets.getSum(), splitTopicAndPartitionIndexLabel); stats.producerStats.forEach((p, producerStats) -> { - metric(stream, cluster, namespace, topic, p, producerStats.producerId, "pulsar_producer_msg_rate_in", + metric(metrics, cluster, namespace, topic, p, producerStats.producerId, "pulsar_producer_msg_rate_in", producerStats.msgRateIn, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, p, producerStats.producerId, "pulsar_producer_msg_throughput_in", + metric(metrics, cluster, namespace, topic, p, producerStats.producerId, "pulsar_producer_msg_throughput_in", producerStats.msgThroughputIn, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, p, producerStats.producerId, "pulsar_producer_msg_average_Size", + metric(metrics, cluster, namespace, topic, p, producerStats.producerId, "pulsar_producer_msg_average_Size", producerStats.averageMsgSize, splitTopicAndPartitionIndexLabel); }); stats.subscriptionStats.forEach((n, subsStats) -> { - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_back_log", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_back_log", subsStats.msgBacklog, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_back_log_no_delayed", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_back_log_no_delayed", subsStats.msgBacklogNoDelayed, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_delayed", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_delayed", subsStats.msgDelayed, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_redeliver", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_redeliver", subsStats.msgRateRedeliver, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_unacked_messages", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_unacked_messages", subsStats.unackedMessages, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_blocked_on_unacked_messages", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_blocked_on_unacked_messages", subsStats.blockedSubscriptionOnUnackedMsgs ? 1 : 0, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_out", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_out", subsStats.msgRateOut, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_msg_ack_rate", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_msg_ack_rate", subsStats.messageAckRate, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_msg_throughput_out", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_msg_throughput_out", subsStats.msgThroughputOut, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_out_bytes_total", + metric(metrics, cluster, namespace, topic, n, "pulsar_out_bytes_total", subsStats.bytesOutCounter, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_out_messages_total", + metric(metrics, cluster, namespace, topic, n, "pulsar_out_messages_total", subsStats.msgOutCounter, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_last_expire_timestamp", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_last_expire_timestamp", subsStats.lastExpireTimestamp, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_last_acked_timestamp", - subsStats.lastAckedTimestamp, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_last_consumed_flow_timestamp", - subsStats.lastConsumedFlowTimestamp, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_last_consumed_timestamp", - subsStats.lastConsumedTimestamp, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_last_mark_delete_advanced_timestamp", - subsStats.lastMarkDeleteAdvancedTimestamp, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_expired", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_last_acked_timestamp", + subsStats.lastAckedTimestamp, splitTopicAndPartitionIndexLabel); + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_last_consumed_flow_timestamp", + subsStats.lastConsumedFlowTimestamp, splitTopicAndPartitionIndexLabel); + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_last_consumed_timestamp", + subsStats.lastConsumedTimestamp, splitTopicAndPartitionIndexLabel); + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_last_mark_delete_advanced_timestamp", + subsStats.lastMarkDeleteAdvancedTimestamp, splitTopicAndPartitionIndexLabel); + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_expired", subsStats.msgRateExpired, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, "pulsar_subscription_total_msg_expired", + metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_total_msg_expired", subsStats.totalMsgExpired, splitTopicAndPartitionIndexLabel); subsStats.consumerStat.forEach((c, consumerStats) -> { - metric(stream, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), + metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), "pulsar_consumer_msg_rate_redeliver", consumerStats.msgRateRedeliver, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), + metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), "pulsar_consumer_unacked_messages", consumerStats.unackedMessages, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), + metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), "pulsar_consumer_blocked_on_unacked_messages", consumerStats.blockedSubscriptionOnUnackedMsgs ? 1 : 0, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), + metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), "pulsar_consumer_msg_rate_out", consumerStats.msgRateOut, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), + metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), "pulsar_consumer_msg_ack_rate", consumerStats.msgAckRate, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), + metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), "pulsar_consumer_msg_throughput_out", consumerStats.msgThroughputOut, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), + metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), "pulsar_consumer_available_permits", consumerStats.availablePermits, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), + metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), "pulsar_out_bytes_total", consumerStats.bytesOutCounter, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), + metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), "pulsar_out_messages_total", consumerStats.msgOutCounter, splitTopicAndPartitionIndexLabel); }); @@ -307,74 +313,75 @@ static void printTopicStats(SimpleTextOutputStream stream, String cluster, Strin if (!stats.replicationStats.isEmpty()) { stats.replicationStats.forEach((remoteCluster, replStats) -> { - metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_rate_in", remoteCluster, + metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_rate_in", remoteCluster, replStats.msgRateIn, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_rate_out", remoteCluster, + metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_rate_out", + remoteCluster, replStats.msgRateOut, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_throughput_in", + metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_throughput_in", remoteCluster, replStats.msgThroughputIn, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_throughput_out", + metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_throughput_out", remoteCluster, replStats.msgThroughputOut, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_backlog", remoteCluster, + metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_backlog", remoteCluster, replStats.replicationBacklog, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_connected_count", + metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_connected_count", remoteCluster, replStats.connectedCount, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_rate_expired", + metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_rate_expired", remoteCluster, replStats.msgRateExpired, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_delay_in_seconds", + metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_delay_in_seconds", remoteCluster, replStats.replicationDelayInSeconds, splitTopicAndPartitionIndexLabel); }); } - metric(stream, cluster, namespace, topic, "pulsar_in_bytes_total", stats.bytesInCounter, + metric(metrics, cluster, namespace, topic, "pulsar_in_bytes_total", stats.bytesInCounter, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_in_messages_total", stats.msgInCounter, + metric(metrics, cluster, namespace, topic, "pulsar_in_messages_total", stats.msgInCounter, splitTopicAndPartitionIndexLabel); // Compaction - boolean hasCompaction = compactorMXBean.flatMap(mxBean -> mxBean.getCompactionRecordForTopic(topic)) - .map(__ -> true).orElse(false); + boolean hasCompaction = + compactorMXBean.flatMap(mxBean -> mxBean.getCompactionRecordForTopic(topic)).isPresent(); if (hasCompaction) { - metric(stream, cluster, namespace, topic, "pulsar_compaction_removed_event_count", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_removed_event_count", stats.compactionRemovedEventCount, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_succeed_count", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_succeed_count", stats.compactionSucceedCount, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_failed_count", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_failed_count", stats.compactionFailedCount, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_duration_time_in_mills", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_duration_time_in_mills", stats.compactionDurationTimeInMills, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_read_throughput", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_read_throughput", stats.compactionReadThroughput, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_write_throughput", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_write_throughput", stats.compactionWriteThroughput, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_compacted_entries_count", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_compacted_entries_count", stats.compactionCompactedEntriesCount, splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_compacted_entries_size", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_compacted_entries_size", stats.compactionCompactedEntriesSize, splitTopicAndPartitionIndexLabel); long[] compactionLatencyBuckets = stats.compactionLatencyBuckets.getBuckets(); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_le_0_5", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_0_5", compactionLatencyBuckets[0], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_le_1", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_1", compactionLatencyBuckets[1], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_le_5", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_5", compactionLatencyBuckets[2], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_le_10", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_10", compactionLatencyBuckets[3], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_le_20", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_20", compactionLatencyBuckets[4], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_le_50", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_50", compactionLatencyBuckets[5], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_le_100", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_100", compactionLatencyBuckets[6], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_le_200", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_200", compactionLatencyBuckets[7], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_le_1000", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_1000", compactionLatencyBuckets[8], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_overflow", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_overflow", compactionLatencyBuckets[9], splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_sum", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_sum", stats.compactionLatencyBuckets.getSum(), splitTopicAndPartitionIndexLabel); - metric(stream, cluster, namespace, topic, "pulsar_compaction_latency_count", + metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_count", stats.compactionLatencyBuckets.getCount(), splitTopicAndPartitionIndexLabel); } } @@ -388,77 +395,102 @@ static void metricType(SimpleTextOutputStream stream, String name) { } - private static void metric(SimpleTextOutputStream stream, String cluster, String namespace, String topic, - String name, double value, boolean splitTopicAndPartitionIndexLabel) { - metricType(stream, name); - appendRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel).write("\"} "); - stream.write(value); - appendEndings(stream); + private static void metric(Map> metrics, String cluster, String namespace, String topic, + String name, double value, boolean splitTopicAndPartitionIndexLabel) { + writeMetric(metrics, name, stream -> { + writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); + stream.write("\"} ") + .write(value); + writeEndings(stream); + }); } - private static void metric(SimpleTextOutputStream stream, String cluster, String namespace, String topic, - String subscription, String name, long value, boolean splitTopicAndPartitionIndexLabel) { - metricType(stream, name); - appendRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel) - .write("\",subscription=\"").write(subscription).write("\"} "); - stream.write(value); - appendEndings(stream); + private static void metric(Map> metrics, String cluster, String namespace, String topic, + String subscription, String name, long value, boolean splitTopicAndPartitionIndexLabel) { + writeMetric(metrics, name, stream -> { + writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); + stream.write("\",subscription=\"").write(subscription) + .write("\"} ") + .write(value); + writeEndings(stream); + }); + } + + private static void metric(Map> metrics, String cluster, String namespace, String topic, + String producerName, long produceId, String name, double value, + boolean splitTopicAndPartitionIndexLabel) { + writeMetric(metrics, name, stream -> { + writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); + stream.write("\",producer_name=\"").write(producerName) + .write("\",producer_id=\"").write(produceId) + .write("\"} ") + .write(value); + writeEndings(stream); + }); } - private static void metric(SimpleTextOutputStream stream, String cluster, String namespace, String topic, - String producerName, long produceId, String name, double value, boolean splitTopicAndPartitionIndexLabel) { - metricType(stream, name); - appendRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel) - .write("\",producer_name=\"").write(producerName) - .write("\",producer_id=\"").write(produceId).write("\"} "); - stream.write(value); - appendEndings(stream); + private static void metric(Map> metrics, String cluster, String namespace, String topic, + String subscription, String name, double value, + boolean splitTopicAndPartitionIndexLabel) { + writeMetric(metrics, name, stream -> { + writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); + stream.write("\",subscription=\"").write(subscription) + .write("\"} ") + .write(value); + writeEndings(stream); + }); } - private static void metric(SimpleTextOutputStream stream, String cluster, String namespace, String topic, - String subscription, String name, double value, boolean splitTopicAndPartitionIndexLabel) { - metricType(stream, name); - appendRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel) - .write("\",subscription=\"").write(subscription).write("\"} "); - stream.write(value); - appendEndings(stream); + private static void metric(Map> metrics, String cluster, String namespace, String topic, + String subscription, String consumerName, long consumerId, String name, long value, + boolean splitTopicAndPartitionIndexLabel) { + writeMetric(metrics, name, stream -> { + writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); + stream.write("\",subscription=\"").write(subscription) + .write("\",consumer_name=\"").write(consumerName).write("\",consumer_id=\"").write(consumerId) + .write("\"} ") + .write(value); + writeEndings(stream); + }); } - private static void metric(SimpleTextOutputStream stream, String cluster, String namespace, String topic, - String subscription, String consumerName, long consumerId, String name, long value, - boolean splitTopicAndPartitionIndexLabel) { - metricType(stream, name); - appendRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel) - .write("\",subscription=\"").write(subscription) - .write("\",consumer_name=\"").write(consumerName).write("\",consumer_id=\"").write(consumerId) - .write("\"} "); - stream.write(value); - appendEndings(stream); + private static void metric(Map> metrics, String cluster, String namespace, String topic, + String subscription, String consumerName, long consumerId, String name, double value, + boolean splitTopicAndPartitionIndexLabel) { + writeMetric(metrics, name, stream -> { + writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); + stream.write("\",subscription=\"").write(subscription) + .write("\",consumer_name=\"").write(consumerName).write("\",consumer_id=\"") + .write(consumerId).write("\"} ") + .write(value); + writeEndings(stream); + }); } - private static void metric(SimpleTextOutputStream stream, String cluster, String namespace, String topic, - String subscription, String consumerName, long consumerId, String name, double value, - boolean splitTopicAndPartitionIndexLabel) { - metricType(stream, name); - appendRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel) - .write("\",subscription=\"").write(subscription) - .write("\",consumer_name=\"").write(consumerName).write("\",consumer_id=\"") - .write(consumerId).write("\"} "); - stream.write(value); - appendEndings(stream); + private static void metricWithRemoteCluster(Map> metrics, String cluster, String namespace, + String topic, String name, String remoteCluster, double value, + boolean splitTopicAndPartitionIndexLabel) { + writeMetric(metrics, name, stream -> { + writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); + stream.write("\",remote_cluster=\"").write(remoteCluster).write("\"} ").write(value); + writeEndings(stream); + }); } - private static void metricWithRemoteCluster(SimpleTextOutputStream stream, String cluster, String namespace, - String topic, String name, String remoteCluster, double value, boolean splitTopicAndPartitionIndexLabel) { - metricType(stream, name); - appendRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel) - .write("\",remote_cluster=\"").write(remoteCluster).write("\"} "); - stream.write(value); - appendEndings(stream); + private static void writeMetric(Map> metrics, String name, + Consumer metricWriter) { + ByteBuf buf = ByteBufAllocator.DEFAULT.heapBuffer(); + try { + SimpleTextOutputStream stream = new SimpleTextOutputStream(buf); + metricWriter.accept(stream); + addMetric(metrics, name, buf.toString(StandardCharsets.UTF_8)); + } finally { + buf.release(); + } } - private static SimpleTextOutputStream appendRequiredLabels(SimpleTextOutputStream stream, String cluster, - String namespace, String topic, String name, boolean splitTopicAndPartitionIndexLabel) { + private static void writeRequiredLabels(SimpleTextOutputStream stream, String cluster, String namespace, + String topic, String name, boolean splitTopicAndPartitionIndexLabel) { stream.write(name).write("{cluster=\"").write(cluster).write("\",namespace=\"").write(namespace); if (splitTopicAndPartitionIndexLabel) { int index = topic.indexOf(PARTITIONED_TOPIC_SUFFIX); @@ -471,10 +503,9 @@ private static SimpleTextOutputStream appendRequiredLabels(SimpleTextOutputStrea } else { stream.write("\",topic=\"").write(topic); } - return stream; } - private static void appendEndings(SimpleTextOutputStream stream) { + private static void writeEndings(SimpleTextOutputStream stream) { stream.write(' ').write(System.currentTimeMillis()).write('\n'); } } 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 f28412ea75160..2ead6fecacf74 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 @@ -50,6 +50,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; import javax.crypto.SecretKey; @@ -1404,6 +1405,59 @@ private void compareCompactionStateCount(List cm, double count) { assertEquals(cm.get(0).value, count); } + @Test + public void testMetricsGroupedByTypeDefinitions() throws Exception { + Producer p1 = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic1").create(); + Producer p2 = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic2").create(); + for (int i = 0; i < 10; i++) { + String message = "my-message-" + i; + p1.send(message.getBytes()); + p2.send(message.getBytes()); + } + + ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); + PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + String metricsStr = statsOut.toString(); + + Pattern typePattern = Pattern.compile("^#\\s+TYPE\\s+(\\w+)\\s+(\\w+)"); + Pattern metricNamePattern = Pattern.compile("^(\\w+)\\{.+"); + + AtomicReference currentMetric = new AtomicReference<>(); + Splitter.on("\n").split(metricsStr).forEach(line -> { + if (line.isEmpty()) { + return; + } + if (line.startsWith("#")) { + // Get the current type definition + Matcher typeMatcher = typePattern.matcher(line); + checkArgument(typeMatcher.matches()); + String metricName = typeMatcher.group(1); + currentMetric.set(metricName); + } else { + Matcher metricMatcher = metricNamePattern.matcher(line); + checkArgument(metricMatcher.matches()); + String metricName = metricMatcher.group(1); + + if (metricName.endsWith("_bucket")) { + metricName = metricName.substring(0, metricName.indexOf("_bucket")); + } else if (metricName.endsWith("_count") && !currentMetric.get().endsWith("_count")) { + metricName = metricName.substring(0, metricName.indexOf("_count")); + } else if (metricName.endsWith("_sum") && !currentMetric.get().endsWith("_sum")) { + metricName = metricName.substring(0, metricName.indexOf("_sum")); + } + + if (!metricName.equals(currentMetric.get())) { + System.out.println(metricsStr); + fail("Metric not grouped under it's type definition: " + line); + } + + } + }); + + p1.close(); + p2.close(); + } + /** * Hacky parsing of Prometheus text format. Should be good enough for unit tests */ From ccf8ca014996dbde21852851b6b6279fd7d29b24 Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Mon, 8 Aug 2022 09:43:18 +0100 Subject: [PATCH 02/19] [fix][broker] Updates from PR comments --- .../prometheus/PrometheusMetricStreams.java | 75 +++++++++++++++++++ 1 file changed, 75 insertions(+) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java new file mode 100644 index 0000000000000..6b6b972c175f0 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java @@ -0,0 +1,75 @@ +/** + * 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; + +import java.util.HashMap; +import java.util.Map; +import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; +import org.apache.pulsar.common.util.SimpleTextOutputStream; + +/** + * Helper class to ensure that metrics of the same name are grouped together under the same TYPE header when written. + * Those are the requirements of the + * Prometheus Exposition Format. + */ +public class PrometheusMetricStreams { + private final Map metricStreamMap = new HashMap<>(); + + /** + * Write the given metric and sample value to the stream. Will write #TYPE header if metric not seen before. + * @param metricName name of the metric. + * @param value value of the sample + * @param labelsAndValuesArray varargs of label and label value + */ + void writeSample(String metricName, Number value, String... labelsAndValuesArray) { + SimpleTextOutputStream stream = initGaugeType(metricName); + stream.write(metricName).write('{'); + for (int i = 0; i < labelsAndValuesArray.length; i += 2) { + stream.write(labelsAndValuesArray[i]).write("=\"").write(labelsAndValuesArray[i + 1]).write('\"'); + if (i + 2 != labelsAndValuesArray.length) { + stream.write(','); + } + } + stream.write("} ").write(value).write(' ').write(System.currentTimeMillis()).write('\n'); + } + + /** + * Flush all the stored metrics to the supplied stream. + * @param stream the stream to write to. + */ + void flushAllToStream(SimpleTextOutputStream stream) { + metricStreamMap.values().forEach(s -> stream.write(s.getBuffer())); + } + + /** + * Release all the streams to clean up resources. + */ + void releaseAll() { + metricStreamMap.values().forEach(s -> s.getBuffer().release()); + metricStreamMap.clear(); + } + + private SimpleTextOutputStream initGaugeType(String metricName) { + return metricStreamMap.computeIfAbsent(metricName, s -> { + SimpleTextOutputStream stream = new SimpleTextOutputStream(PulsarByteBufAllocator.DEFAULT.directBuffer()); + stream.write("# TYPE ").write(metricName).write(" gauge\n"); + return stream; + }); + } +} From 3fafc1e5464fdd2f06099aee4c446e0075875af9 Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Thu, 12 May 2022 11:03:09 +0100 Subject: [PATCH 03/19] [fix][functions-worker] Ensure prometheus metrics are grouped by type (#13865) --- .../functions/instance/stats/PrometheusTextFormat.java | 5 +++++ .../apache/pulsar/functions/worker/WorkerStatsManager.java | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PrometheusTextFormat.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PrometheusTextFormat.java index f7a205c7db02c..f5aa273f656ac 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PrometheusTextFormat.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PrometheusTextFormat.java @@ -37,6 +37,11 @@ public static void write004(Writer writer, Enumeration 0) { diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerStatsManager.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerStatsManager.java index c8b411cbf5706..2ad407b2e5e3e 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerStatsManager.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerStatsManager.java @@ -328,6 +328,11 @@ private void generateLeaderMetrics(StringWriter stream) { } private void writeMetric(String metricName, long value, StringWriter stream) { + stream.write("# TYPE "); + stream.write(PULSAR_FUNCTION_WORKER_METRICS_PREFIX); + stream.write(metricName); + stream.write(" gauge \n"); + stream.write(PULSAR_FUNCTION_WORKER_METRICS_PREFIX); stream.write(metricName); stream.write("{"); From 32f97ea7ddd9e37df8c1737e078a55e95caa9b59 Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Fri, 13 May 2022 16:02:57 +0100 Subject: [PATCH 04/19] [fix][broker] remove extra SimpleTextOutputStream and use prometheus model --- .../prometheus/NamespaceStatsAggregator.java | 63 +++---- .../broker/stats/prometheus/TopicStats.java | 164 +++++++----------- .../metrics/PrometheusTextFormatUtil.java | 50 +++--- 3 files changed, 111 insertions(+), 166 deletions(-) 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 4e3d0dc202291..f986885c7b4c1 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 @@ -18,10 +18,10 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import static org.apache.pulsar.broker.stats.prometheus.TopicStats.addMetric; import io.netty.util.concurrent.FastThreadLocal; -import java.util.ArrayList; +import io.prometheus.client.Collector; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.LongAdder; @@ -32,6 +32,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.prometheus.metrics.PrometheusTextFormatUtil; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; @@ -45,14 +46,14 @@ public class NamespaceStatsAggregator { private static FastThreadLocal localNamespaceStats = - new FastThreadLocal() { + new FastThreadLocal<>() { @Override protected AggregatedNamespaceStats initialValue() throws Exception { return new AggregatedNamespaceStats(); } }; - private static FastThreadLocal localTopicStats = new FastThreadLocal() { + private static FastThreadLocal localTopicStats = new FastThreadLocal<>() { @Override protected TopicStats initialValue() throws Exception { return new TopicStats(); @@ -67,9 +68,9 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b TopicStats.resetTypes(); TopicStats topicStats = localTopicStats.get(); - Map> metrics = new HashMap<>(); + Map metrics = new HashMap<>(); - printDefaultBrokerStats(metrics, cluster); + buildDefaultBrokerStats(metrics, cluster); Optional compactorMXBean = getCompactorMXBean(pulsar); LongAdder topicsCount = new LongAdder(); @@ -86,7 +87,7 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b if (includeTopicMetrics) { topicsCount.add(1); - TopicStats.printTopicStats(metrics, cluster, namespace, name, topicStats, compactorMXBean, + TopicStats.buildTopicStats(metrics, cluster, namespace, name, topicStats, compactorMXBean, splitTopicAndPartitionIndexLabel); } else { namespaceStats.updateStats(topicStats); @@ -102,10 +103,7 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b } }); - metrics.forEach((type, lines) -> { - TopicStats.metricType(stream, type); - lines.forEach(stream::write); - }); + PrometheusTextFormatUtil.writeMetrics(stream, metrics.values()); } private static Optional getCompactorMXBean(PulsarService pulsar) { @@ -278,7 +276,7 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include }); } - private static void printDefaultBrokerStats(Map> metrics, String cluster) { + private static void buildDefaultBrokerStats(Map metrics, String cluster) { // Print metrics with 0 values. This is necessary to have the available brokers being // reported in the brokers dashboard even if they don't have any topic or traffic metric(metrics, cluster, "pulsar_topics_count", 0); @@ -296,12 +294,14 @@ private static void printDefaultBrokerStats(Map> metrics, S metric(metrics, cluster, "pulsar_msg_backlog", 0); } - private static void printTopicsCountStats(Map> metrics, String cluster, String namespace, + private static void printTopicsCountStats(Map metrics, String cluster, + String namespace, LongAdder topicsCount) { metric(metrics, cluster, namespace, "pulsar_topics_count", topicsCount.sum()); } - private static void printNamespaceStats(Map> metrics, String cluster, String namespace, + private static void printNamespaceStats(Map metrics, String cluster, + String namespace, AggregatedNamespaceStats stats) { metric(metrics, cluster, namespace, "pulsar_topics_count", stats.topicsCount); metric(metrics, cluster, namespace, "pulsar_subscriptions_count", stats.subscriptionsCount); @@ -407,35 +407,24 @@ private static void printNamespaceStats(Map> metrics, Strin } } - private static void metric(Map> metrics, String cluster, String name, + private static void metric(Map metrics, String cluster, String name, long value) { - addMetric(metrics, name, - name + "{cluster=\"" + cluster + "\"} " + value + ' ' + System.currentTimeMillis() + '\n'); + addMetric(metrics, Map.of("cluster", cluster), name, value); } - private static void metric(Map> metrics, String cluster, String namespace, String name, - long value) { - addMetric(metrics, name, name + "{cluster=\"" + cluster + "\",namespace=\"" + namespace + "\"} " + value + ' ' - + System.currentTimeMillis() + '\n'); + private static void metric(Map metrics, String cluster, String namespace, + String name, long value) { + addMetric(metrics, Map.of("cluster", cluster, "namespace", namespace), name, value); } - private static void metric(Map> metrics, String cluster, String namespace, String name, - double value) { - addMetric(metrics, name, name + "{cluster=\"" + cluster + "\",namespace=\"" + namespace + "\"} " + value + ' ' - + System.currentTimeMillis() + '\n'); + private static void metric(Map metrics, String cluster, String namespace, + String name, double value) { + addMetric(metrics, Map.of("cluster", cluster, "namespace", namespace), name, value); } - private static void metricWithRemoteCluster(Map> metrics, String cluster, String namespace, - String name, String remoteCluster, double value) { - addMetric(metrics, name, - name + "{cluster=\"" + cluster + "\",namespace=\"" + namespace + "\",remote_cluster=\"" + remoteCluster - + "\"} " + value + ' ' + System.currentTimeMillis() + '\n'); - } - - static void addMetric(Map> metrics, String type, String metricLine) { - if (!metrics.containsKey(type)) { - metrics.put(type, new ArrayList<>()); - } - metrics.get(type).add(metricLine); + private static void metricWithRemoteCluster(Map metrics, String cluster, + String namespace, String name, String remoteCluster, double value) { + addMetric(metrics, Map.of("cluster", cluster, "namespace", namespace, "remote_cluster", remoteCluster), name, + value); } } 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 3966a07c25f11..c7a7ce916a80f 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 @@ -18,18 +18,13 @@ */ package org.apache.pulsar.broker.stats.prometheus; -import static org.apache.pulsar.broker.stats.prometheus.NamespaceStatsAggregator.addMetric; import static org.apache.pulsar.common.naming.TopicName.PARTITIONED_TOPIC_SUFFIX; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufAllocator; -import java.nio.charset.StandardCharsets; +import io.prometheus.client.Collector; +import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.function.Consumer; import org.apache.bookkeeper.mledger.util.StatsBuckets; -import org.apache.pulsar.common.util.SimpleTextOutputStream; import org.apache.pulsar.compaction.CompactionRecord; import org.apache.pulsar.compaction.CompactorMXBean; @@ -113,7 +108,8 @@ static void resetTypes() { metricWithTypeDefinition.clear(); } - static void printTopicStats(Map> metrics, String cluster, String namespace, String topic, + static void buildTopicStats(Map metrics, String cluster, String namespace, + String topic, TopicStats stats, Optional compactorMXBean, boolean splitTopicAndPartitionIndexLabel) { metric(metrics, cluster, namespace, topic, "pulsar_subscriptions_count", stats.subscriptionsCount, @@ -386,126 +382,84 @@ static void printTopicStats(Map> metrics, String cluster, S } } - static void metricType(SimpleTextOutputStream stream, String name) { - - if (!metricWithTypeDefinition.containsKey(name)) { - metricWithTypeDefinition.put(name, "gauge"); - stream.write("# TYPE ").write(name).write(" gauge\n"); - } - - } - - private static void metric(Map> metrics, String cluster, String namespace, String topic, - String name, double value, boolean splitTopicAndPartitionIndexLabel) { - writeMetric(metrics, name, stream -> { - writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); - stream.write("\"} ") - .write(value); - writeEndings(stream); - }); - } - - private static void metric(Map> metrics, String cluster, String namespace, String topic, - String subscription, String name, long value, boolean splitTopicAndPartitionIndexLabel) { - writeMetric(metrics, name, stream -> { - writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); - stream.write("\",subscription=\"").write(subscription) - .write("\"} ") - .write(value); - writeEndings(stream); - }); + private static void metric(Map metrics, String cluster, String namespace, + String topic, String name, double value, boolean splitTopicAndPartitionIndexLabel) { + Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + addMetric(metrics, labels, name, value); } - private static void metric(Map> metrics, String cluster, String namespace, String topic, - String producerName, long produceId, String name, double value, + private static void metric(Map metrics, String cluster, String namespace, + String topic, String subscription, String name, long value, boolean splitTopicAndPartitionIndexLabel) { - writeMetric(metrics, name, stream -> { - writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); - stream.write("\",producer_name=\"").write(producerName) - .write("\",producer_id=\"").write(produceId) - .write("\"} ") - .write(value); - writeEndings(stream); - }); + Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + labels.put("subscription", subscription); + addMetric(metrics, labels, name, value); } - private static void metric(Map> metrics, String cluster, String namespace, String topic, - String subscription, String name, double value, + private static void metric(Map metrics, String cluster, String namespace, + String topic, String producerName, long produceId, String name, double value, boolean splitTopicAndPartitionIndexLabel) { - writeMetric(metrics, name, stream -> { - writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); - stream.write("\",subscription=\"").write(subscription) - .write("\"} ") - .write(value); - writeEndings(stream); - }); + Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + labels.put("producer_name", producerName); + labels.put("producer_id", String.valueOf(produceId)); + addMetric(metrics, labels, name, value); } - private static void metric(Map> metrics, String cluster, String namespace, String topic, - String subscription, String consumerName, long consumerId, String name, long value, + private static void metric(Map metrics, String cluster, String namespace, + String topic, String subscription, String name, double value, boolean splitTopicAndPartitionIndexLabel) { - writeMetric(metrics, name, stream -> { - writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); - stream.write("\",subscription=\"").write(subscription) - .write("\",consumer_name=\"").write(consumerName).write("\",consumer_id=\"").write(consumerId) - .write("\"} ") - .write(value); - writeEndings(stream); - }); + Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + labels.put("subscription", subscription); + addMetric(metrics, labels, name, value); } - private static void metric(Map> metrics, String cluster, String namespace, String topic, - String subscription, String consumerName, long consumerId, String name, double value, - boolean splitTopicAndPartitionIndexLabel) { - writeMetric(metrics, name, stream -> { - writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); - stream.write("\",subscription=\"").write(subscription) - .write("\",consumer_name=\"").write(consumerName).write("\",consumer_id=\"") - .write(consumerId).write("\"} ") - .write(value); - writeEndings(stream); - }); + private static void metric(Map metrics, String cluster, String namespace, + String topic, String subscription, String consumerName, long consumerId, String name, + double value, boolean splitTopicAndPartitionIndexLabel) { + Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + labels.put("subscription", subscription); + labels.put("consumer_name", consumerName); + labels.put("consumer_id", String.valueOf(consumerId)); + addMetric(metrics, labels, name, value); } - private static void metricWithRemoteCluster(Map> metrics, String cluster, String namespace, - String topic, String name, String remoteCluster, double value, - boolean splitTopicAndPartitionIndexLabel) { - writeMetric(metrics, name, stream -> { - writeRequiredLabels(stream, cluster, namespace, topic, name, splitTopicAndPartitionIndexLabel); - stream.write("\",remote_cluster=\"").write(remoteCluster).write("\"} ").write(value); - writeEndings(stream); - }); - } - - private static void writeMetric(Map> metrics, String name, - Consumer metricWriter) { - ByteBuf buf = ByteBufAllocator.DEFAULT.heapBuffer(); - try { - SimpleTextOutputStream stream = new SimpleTextOutputStream(buf); - metricWriter.accept(stream); - addMetric(metrics, name, buf.toString(StandardCharsets.UTF_8)); - } finally { - buf.release(); - } + private static void metricWithRemoteCluster(Map metrics, String cluster, + String namespace, String topic, String name, String remoteCluster, + double value, boolean splitTopicAndPartitionIndexLabel) { + Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + labels.put("remote_cluster", remoteCluster); + addMetric(metrics, labels, name, value); } - private static void writeRequiredLabels(SimpleTextOutputStream stream, String cluster, String namespace, - String topic, String name, boolean splitTopicAndPartitionIndexLabel) { - stream.write(name).write("{cluster=\"").write(cluster).write("\",namespace=\"").write(namespace); + private static Map buildLabels(String cluster, String namespace, String topic, + boolean splitTopicAndPartitionIndexLabel) { + Map labels = new HashMap<>(); + labels.put("cluster", cluster); + labels.put("namespace", namespace); if (splitTopicAndPartitionIndexLabel) { int index = topic.indexOf(PARTITIONED_TOPIC_SUFFIX); if (index > 0) { - stream.write("\",topic=\"").write(topic.substring(0, index)).write("\",partition=\"") - .write(topic.substring(index + PARTITIONED_TOPIC_SUFFIX.length())); + labels.put("topic", topic.substring(0, index)); + labels.put("partition", topic.substring(index + PARTITIONED_TOPIC_SUFFIX.length())); } else { - stream.write("\",topic=\"").write(topic).write("\",partition=\"").write("-1"); + labels.put("topic", topic); + labels.put("partition", "-1"); } } else { - stream.write("\",topic=\"").write(topic); + labels.put("topic", topic); } + return labels; } - private static void writeEndings(SimpleTextOutputStream stream) { - stream.write(' ').write(System.currentTimeMillis()).write('\n'); + static void addMetric(Map metrics, Map labels, + String name, double value) { + Collector.MetricFamilySamples familySamples = metrics.getOrDefault(name, + new Collector.MetricFamilySamples(name, Collector.Type.GAUGE, null, new ArrayList<>())); + familySamples.samples.add(new Collector.MetricFamilySamples.Sample(name, + labels.keySet().stream().toList(), + labels.values().stream().toList(), + value, + System.currentTimeMillis())); + metrics.put(name, familySamples); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java index 7550096c2b584..073f7744db7df 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java @@ -20,12 +20,11 @@ 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 java.util.Collection; import org.apache.bookkeeper.stats.Counter; +import org.apache.pulsar.common.util.SimpleTextOutputStream; /** * Logic to write metrics in Prometheus text format. @@ -141,29 +140,32 @@ private static void writeSum(Writer w, DataSketchesOpStatsLogger opStat, String .append(Double.toString(opStat.getSum(success))).append('\n'); } - public static void writeMetricsCollectedByPrometheusClient(Writer w, CollectorRegistry registry) - throws IOException { - Enumeration 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(", "); + public static void writeMetrics(SimpleTextOutputStream stream, Collection familySamples) { + for (MetricFamilySamples familySample : familySamples) { + stream.write("# TYPE "); + stream.write(familySample.name); + stream.write(' '); + stream.write(familySample.type.name().toLowerCase()); + stream.write('\n'); + for (Collector.MetricFamilySamples.Sample sample : familySample.samples) { + stream.write(sample.name); + if (sample.labelNames.size() > 0) { + stream.write('{'); + for (int i = 0; i < sample.labelNames.size(); ++i) { + stream.write(sample.labelNames.get(i)); + stream.write("=\""); + stream.write(sample.labelValues.get(i)); + stream.write("\","); } - w.write(sample.labelNames.get(j)); - w.write("=\""); - w.write(sample.labelValues.get(j)); - w.write('"'); + stream.write('}'); } - - w.write("} "); - w.write(Collector.doubleToGoString(sample.value)); - w.write('\n'); + stream.write(' '); + stream.write(sample.value); + if (sample.timestampMs != null) { + stream.write(' '); + stream.write(sample.timestampMs.toString()); + } + stream.write('\n'); } } } From f335b720510ee7606dec96b369faec496c7d10c0 Mon Sep 17 00:00:00 2001 From: Mark Silcox <63227862+marksilcox@users.noreply.github.com> Date: Sat, 14 May 2022 04:11:58 +0100 Subject: [PATCH 05/19] [fix][broker] ensure new line added before appending new keys from env (#15596) (cherry picked from commit 74b2c70bb0f661ef2f6b6689232c10850024d03d) --- docker/pulsar/scripts/apply-config-from-env-with-prefix.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/pulsar/scripts/apply-config-from-env-with-prefix.py b/docker/pulsar/scripts/apply-config-from-env-with-prefix.py index 3f6bc2e4d3b85..b1c479343e01d 100755 --- a/docker/pulsar/scripts/apply-config-from-env-with-prefix.py +++ b/docker/pulsar/scripts/apply-config-from-env-with-prefix.py @@ -73,6 +73,10 @@ lines[idx] = '%s=%s\n' % (k, v) + # Ensure we have a new-line at the end of the file, to avoid issue + # when appending more lines to the config + lines.append('\n') + # Add new keys from Env for k in sorted(os.environ.keys()): v = os.environ[k] From 93622cebd9094e4ac2eb8aa5f862a067cbe1f184 Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Wed, 15 Jun 2022 16:32:31 +0100 Subject: [PATCH 06/19] [fix][broker] Use `Collector.doubleToGoString` --- .../stats/prometheus/metrics/PrometheusTextFormatUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java index 073f7744db7df..5a991f7e0998b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java @@ -160,7 +160,7 @@ public static void writeMetrics(SimpleTextOutputStream stream, Collection Date: Thu, 16 Jun 2022 14:09:43 +0100 Subject: [PATCH 07/19] [fix][broker] Update to ensure topic values are still rounded --- .../pulsar/broker/stats/prometheus/TopicStats.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) 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 c7a7ce916a80f..721db508177dc 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 @@ -7,7 +7,7 @@ * "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 + * 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 @@ -20,6 +20,8 @@ import static org.apache.pulsar.common.naming.TopicName.PARTITIONED_TOPIC_SUFFIX; import io.prometheus.client.Collector; +import java.math.BigDecimal; +import java.math.RoundingMode; import java.util.ArrayList; import java.util.HashMap; import java.util.Map; @@ -458,8 +460,16 @@ static void addMetric(Map metrics, Map Date: Thu, 16 Jun 2022 15:13:58 +0100 Subject: [PATCH 08/19] [fix][broker] Fix licence header (again) --- .../org/apache/pulsar/broker/stats/prometheus/TopicStats.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 721db508177dc..fe4b1e36e7eb3 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 @@ -7,7 +7,7 @@ * "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 + * 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 From edbac235d7d10d26b163f3e26e820375bbd3a66b Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Thu, 23 Jun 2022 17:49:50 +0100 Subject: [PATCH 09/19] [fix][broker] Cherry pick changes --- .../prometheus/AggregatedNamespaceStats.java | 4 +- .../prometheus/NamespaceStatsAggregator.java | 454 +++++++----- .../broker/stats/prometheus/TopicStats.java | 691 ++++++++++-------- .../broker/stats/PrometheusMetricsTest.java | 12 +- .../AggregatedNamespaceStatsTest.java | 1 - 5 files changed, 672 insertions(+), 490 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index 5610dbab218e0..6a6a535f2015e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -24,6 +24,8 @@ import org.apache.pulsar.compaction.CompactionRecord; public class AggregatedNamespaceStats { + public String name; + public int topicsCount; public int subscriptionsCount; public int producersCount; @@ -96,7 +98,7 @@ void updateStats(TopicStats stats) { stats.replicationStats.forEach((n, as) -> { AggregatedReplicationStats replStats = - replicationStats.computeIfAbsent(n, k -> new AggregatedReplicationStats()); + replicationStats.computeIfAbsent(n, k -> new AggregatedReplicationStats()); replStats.msgRateIn += as.msgRateIn; replStats.msgRateOut += as.msgRateOut; replStats.msgThroughputIn += as.msgThroughputIn; 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 f986885c7b4c1..a2cc95ce84ea5 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 @@ -18,13 +18,14 @@ */ package org.apache.pulsar.broker.stats.prometheus; -import static org.apache.pulsar.broker.stats.prometheus.TopicStats.addMetric; import io.netty.util.concurrent.FastThreadLocal; -import io.prometheus.client.Collector; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.LongAdder; +import java.util.function.Function; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.mledger.ManagedLedger; @@ -32,7 +33,6 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.broker.stats.prometheus.metrics.PrometheusTextFormatUtil; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; @@ -45,17 +45,31 @@ @Slf4j public class NamespaceStatsAggregator { - private static FastThreadLocal localNamespaceStats = + private static final FastThreadLocal> localAllNamespaceStats = new FastThreadLocal<>() { @Override - protected AggregatedNamespaceStats initialValue() throws Exception { - return new AggregatedNamespaceStats(); + protected List initialValue() { + return new ArrayList<>(); } }; - private static FastThreadLocal localTopicStats = new FastThreadLocal<>() { + private static final FastThreadLocal> localAllTopicStats = new FastThreadLocal<>() { @Override - protected TopicStats initialValue() throws Exception { + protected List initialValue() { + return new ArrayList<>(); + } + }; + + private static final FastThreadLocal> localNamespaceTopicCount = new FastThreadLocal<>() { + @Override + protected Map initialValue() { + return new HashMap<>(); + } + }; + + private static final FastThreadLocal localTopicStats = new FastThreadLocal<>() { + @Override + protected TopicStats initialValue() { return new TopicStats(); } }; @@ -64,46 +78,59 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, SimpleTextOutputStream stream) { String cluster = pulsar.getConfiguration().getClusterName(); - AggregatedNamespaceStats namespaceStats = localNamespaceStats.get(); - TopicStats.resetTypes(); - TopicStats topicStats = localTopicStats.get(); - - Map metrics = new HashMap<>(); - - buildDefaultBrokerStats(metrics, cluster); - Optional compactorMXBean = getCompactorMXBean(pulsar); LongAdder topicsCount = new LongAdder(); - pulsar.getBrokerService().getMultiLayerTopicMap().forEach((namespace, bundlesMap) -> { - namespaceStats.reset(); - topicsCount.reset(); - - bundlesMap.forEach((bundle, topicsMap) -> topicsMap.forEach((name, topic) -> { - getTopicStats(topic, topicStats, includeConsumerMetrics, includeProducerMetrics, - pulsar.getConfiguration().isExposePreciseBacklogInPrometheus(), - pulsar.getConfiguration().isExposeSubscriptionBacklogSizeInPrometheus(), - compactorMXBean - ); + try { + pulsar.getBrokerService().getMultiLayerTopicMap().forEach((namespace, bundlesMap) -> { + // only need to create if we are not including topic metrics + AggregatedNamespaceStats namespaceStats = includeTopicMetrics ? null : new AggregatedNamespaceStats(); + topicsCount.reset(); + + bundlesMap.forEach((bundle, topicsMap) -> topicsMap.forEach((name, topic) -> { + //If not includeTopicMetrics then use a single thread local, so not assigning lots of objects + TopicStats topicStats = includeTopicMetrics ? new TopicStats() : localTopicStats.get(); + topicStats.reset(); + topicStats.name = name; + topicStats.namespace = namespace; + getTopicStats(topic, topicStats, includeConsumerMetrics, includeProducerMetrics, + pulsar.getConfiguration().isExposePreciseBacklogInPrometheus(), + pulsar.getConfiguration().isExposeSubscriptionBacklogSizeInPrometheus(), + compactorMXBean + ); + + if (includeTopicMetrics) { + topicsCount.add(1); + localAllTopicStats.get().add(topicStats); + } else { + namespaceStats.updateStats(topicStats); + } + })); - if (includeTopicMetrics) { - topicsCount.add(1); - TopicStats.buildTopicStats(metrics, cluster, namespace, name, topicStats, compactorMXBean, - splitTopicAndPartitionIndexLabel); + if (!includeTopicMetrics) { + // Only include namespace level stats if we don't have the per-topic, otherwise we're going to + // report the same data twice, and it will make the aggregation difficult + namespaceStats.name = namespace; + localAllNamespaceStats.get().add(namespaceStats); } else { - namespaceStats.updateStats(topicStats); + localNamespaceTopicCount.get().put(namespace, topicsCount.sum()); } - })); + }); - if (!includeTopicMetrics) { - // Only include namespace level stats if we don't have the per-topic, otherwise we're going to report - // the same data twice, and it will make the aggregation difficult - printNamespaceStats(metrics, cluster, namespace, namespaceStats); + if (includeTopicMetrics) { + printTopicsCountStats(stream, cluster, localNamespaceTopicCount.get()); + TopicStats.printTopicStats(stream, cluster, localAllTopicStats.get(), compactorMXBean, + splitTopicAndPartitionIndexLabel); } else { - printTopicsCountStats(metrics, cluster, namespace, topicsCount); + printNamespaceStats(stream, cluster, localAllNamespaceStats.get()); } - }); - - PrometheusTextFormatUtil.writeMetrics(stream, metrics.values()); + } finally { + if (includeTopicMetrics) { + localNamespaceTopicCount.get().clear(); + localAllTopicStats.get().clear(); + } else { + localAllNamespaceStats.get().clear(); + } + } } private static Optional getCompactorMXBean(PulsarService pulsar) { @@ -115,7 +142,7 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include boolean includeProducerMetrics, boolean getPreciseBacklog, boolean subscriptionBacklogSize, Optional compactorMXBean) { - stats.reset(); + stats.name = topic.getName(); if (topic instanceof PersistentTopic) { // Managed Ledger stats @@ -276,155 +303,224 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include }); } - private static void buildDefaultBrokerStats(Map metrics, String cluster) { - // Print metrics with 0 values. This is necessary to have the available brokers being - // reported in the brokers dashboard even if they don't have any topic or traffic - metric(metrics, cluster, "pulsar_topics_count", 0); - metric(metrics, cluster, "pulsar_subscriptions_count", 0); - metric(metrics, cluster, "pulsar_producers_count", 0); - metric(metrics, cluster, "pulsar_consumers_count", 0); - metric(metrics, cluster, "pulsar_rate_in", 0); - metric(metrics, cluster, "pulsar_rate_out", 0); - metric(metrics, cluster, "pulsar_throughput_in", 0); - metric(metrics, cluster, "pulsar_throughput_out", 0); - metric(metrics, cluster, "pulsar_storage_size", 0); - metric(metrics, cluster, "pulsar_storage_logical_size", 0); - metric(metrics, cluster, "pulsar_storage_write_rate", 0); - metric(metrics, cluster, "pulsar_storage_read_rate", 0); - metric(metrics, cluster, "pulsar_msg_backlog", 0); + private static void printTopicsCountStats(SimpleTextOutputStream stream, String cluster, + Map namespaceTopicsCount) { + stream.write("# TYPE ").write("pulsar_topics_count").write(" gauge\n"); + stream.write("pulsar_topics_count") + .write("{cluster=\"").write(cluster).write("\"} ") + .write(0).write(' ').write(System.currentTimeMillis()) + .write('\n'); + namespaceTopicsCount.forEach((ns, topicCount) -> stream.write("pulsar_topics_count") + .write("{cluster=\"").write(cluster) + .write("\",namespace=\"").write(ns) + .write("\"} ") + .write(topicCount).write(' ').write(System.currentTimeMillis()) + .write('\n') + ); } - private static void printTopicsCountStats(Map metrics, String cluster, - String namespace, - LongAdder topicsCount) { - metric(metrics, cluster, namespace, "pulsar_topics_count", topicsCount.sum()); + private static void printNamespaceStats(SimpleTextOutputStream stream, String cluster, + List stats) { + writeMetricWithBrokerDefault(stream, cluster, "pulsar_topics_count", stats, s -> s.topicsCount); + writeMetricWithBrokerDefault(stream, cluster, "pulsar_subscriptions_count", stats, s -> s.subscriptionsCount); + writeMetricWithBrokerDefault(stream, cluster, "pulsar_producers_count", stats, s -> s.producersCount); + writeMetricWithBrokerDefault(stream, cluster, "pulsar_consumers_count", stats, s -> s.consumersCount); + + writeMetricWithBrokerDefault(stream, cluster, "pulsar_rate_in", stats, s -> s.rateIn); + writeMetricWithBrokerDefault(stream, cluster, "pulsar_rate_out", stats, s -> s.rateOut); + writeMetricWithBrokerDefault(stream, cluster, "pulsar_throughput_in", stats, s -> s.throughputIn); + writeMetricWithBrokerDefault(stream, cluster, "pulsar_throughput_out", stats, s -> s.throughputOut); + writeMetric(stream, cluster, "pulsar_consumer_msg_ack_rate", stats, s -> s.messageAckRate); + + writeMetric(stream, cluster, "pulsar_in_bytes_total", stats, s -> s.bytesInCounter); + writeMetric(stream, cluster, "pulsar_in_messages_total", stats, s -> s.msgInCounter); + writeMetric(stream, cluster, "pulsar_out_bytes_total", stats, s -> s.bytesOutCounter); + writeMetric(stream, cluster, "pulsar_out_messages_total", stats, s -> s.msgOutCounter); + + writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_size", stats, + s -> s.managedLedgerStats.storageSize); + writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_logical_size", stats, + s -> s.managedLedgerStats.storageLogicalSize); + writeMetric(stream, cluster, "pulsar_storage_backlog_size", stats, s -> s.managedLedgerStats.backlogSize); + writeMetric(stream, cluster, "pulsar_storage_offloaded_size", + stats, s -> s.managedLedgerStats.offloadedStorageUsed); + + writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_write_rate", stats, + s -> s.managedLedgerStats.storageWriteRate); + writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_read_rate", stats, + s -> s.managedLedgerStats.storageReadRate); + + writeMetric(stream, cluster, "pulsar_subscription_delayed", stats, s -> s.msgDelayed); + + writeMsgBacklog(stream, cluster, stats, s -> s.msgBacklog); + + stats.forEach(s -> s.managedLedgerStats.storageWriteLatencyBuckets.refresh()); + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_0_5", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[0]); + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_1", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[1]); + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_5", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[2]); + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_10", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[3]); + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_20", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[4]); + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_50", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[5]); + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_100", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[6]); + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_200", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[7]); + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_1000", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[8]); + writeMetric(stream, cluster, "pulsar_storage_write_latency_overflow", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[9]); + writeMetric(stream, cluster, "pulsar_storage_write_latency_count", + stats, s -> s.managedLedgerStats.storageWriteLatencyBuckets.getCount()); + writeMetric(stream, cluster, "pulsar_storage_write_latency_sum", + stats, s -> s.managedLedgerStats.storageWriteLatencyBuckets.getSum()); + + stats.forEach(s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.refresh()); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_0_5", stats, + s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[0]); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_1", stats, + s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[1]); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_5", stats, + s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[2]); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_10", stats, + s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[3]); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_20", stats, + s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[4]); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_50", stats, + s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[5]); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_100", stats, + s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[6]); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_200", stats, + s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[7]); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_1000", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[8]); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_overflow", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[9]); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_count", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount()); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_sum", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum()); + + stats.forEach(s -> s.managedLedgerStats.entrySizeBuckets.refresh()); + writeMetric(stream, cluster, "pulsar_entry_size_le_128", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[0]); + writeMetric(stream, cluster, "pulsar_entry_size_le_512", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[1]); + writeMetric(stream, cluster, "pulsar_entry_size_le_1_kb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[2]); + writeMetric(stream, cluster, "pulsar_entry_size_le_2_kb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[3]); + writeMetric(stream, cluster, "pulsar_entry_size_le_4_kb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[4]); + writeMetric(stream, cluster, "pulsar_entry_size_le_16_kb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[5]); + writeMetric(stream, cluster, "pulsar_entry_size_le_100_kb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[6]); + writeMetric(stream, cluster, "pulsar_entry_size_le_1_mb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[7]); + writeMetric(stream, cluster, "pulsar_entry_size_le_overflow", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[8]); + writeMetric(stream, cluster, "pulsar_entry_size_count", + stats, s -> s.managedLedgerStats.entrySizeBuckets.getCount()); + writeMetric(stream, cluster, "pulsar_entry_size_sum", + stats, s -> s.managedLedgerStats.entrySizeBuckets.getSum()); + + writeReplicationStat(stream, cluster, "pulsar_replication_rate_in", stats, + replStats -> replStats.msgRateIn); + writeReplicationStat(stream, cluster, "pulsar_replication_rate_out", stats, + replStats -> replStats.msgRateOut); + writeReplicationStat(stream, cluster, "pulsar_replication_throughput_in", stats, + replStats -> replStats.msgThroughputIn); + writeReplicationStat(stream, cluster, "pulsar_replication_throughput_out", stats, + replStats -> replStats.msgThroughputOut); + writeReplicationStat(stream, cluster, "pulsar_replication_backlog", stats, + replStats -> replStats.replicationBacklog); + writeReplicationStat(stream, cluster, "pulsar_replication_connected_count", stats, + replStats -> replStats.connectedCount); + writeReplicationStat(stream, cluster, "pulsar_replication_rate_expired", stats, + replStats -> replStats.msgRateExpired); + writeReplicationStat(stream, cluster, "pulsar_replication_delay_in_seconds", stats, + replStats -> replStats.replicationDelayInSeconds); } - private static void printNamespaceStats(Map metrics, String cluster, - String namespace, - AggregatedNamespaceStats stats) { - metric(metrics, cluster, namespace, "pulsar_topics_count", stats.topicsCount); - metric(metrics, cluster, namespace, "pulsar_subscriptions_count", stats.subscriptionsCount); - metric(metrics, cluster, namespace, "pulsar_producers_count", stats.producersCount); - metric(metrics, cluster, namespace, "pulsar_consumers_count", stats.consumersCount); - - metric(metrics, cluster, namespace, "pulsar_rate_in", stats.rateIn); - metric(metrics, cluster, namespace, "pulsar_rate_out", stats.rateOut); - metric(metrics, cluster, namespace, "pulsar_throughput_in", stats.throughputIn); - metric(metrics, cluster, namespace, "pulsar_throughput_out", stats.throughputOut); - metric(metrics, cluster, namespace, "pulsar_consumer_msg_ack_rate", stats.messageAckRate); - - metric(metrics, cluster, namespace, "pulsar_in_bytes_total", stats.bytesInCounter); - metric(metrics, cluster, namespace, "pulsar_in_messages_total", stats.msgInCounter); - metric(metrics, cluster, namespace, "pulsar_out_bytes_total", stats.bytesOutCounter); - metric(metrics, cluster, namespace, "pulsar_out_messages_total", stats.msgOutCounter); - - metric(metrics, cluster, namespace, "pulsar_storage_size", stats.managedLedgerStats.storageSize); - metric(metrics, cluster, namespace, "pulsar_storage_logical_size", stats.managedLedgerStats.storageLogicalSize); - metric(metrics, cluster, namespace, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize); - metric(metrics, cluster, namespace, "pulsar_storage_offloaded_size", - stats.managedLedgerStats.offloadedStorageUsed); - - metric(metrics, cluster, namespace, "pulsar_storage_write_rate", stats.managedLedgerStats.storageWriteRate); - metric(metrics, cluster, namespace, "pulsar_storage_read_rate", stats.managedLedgerStats.storageReadRate); - - metric(metrics, cluster, namespace, "pulsar_subscription_delayed", stats.msgDelayed); - - metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_msg_backlog", "local", stats.msgBacklog); - - stats.managedLedgerStats.storageWriteLatencyBuckets.refresh(); - long[] latencyBuckets = stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets(); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_0_5", latencyBuckets[0]); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_1", latencyBuckets[1]); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_5", latencyBuckets[2]); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_10", latencyBuckets[3]); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_20", latencyBuckets[4]); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_50", latencyBuckets[5]); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_100", latencyBuckets[6]); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_200", latencyBuckets[7]); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_le_1000", latencyBuckets[8]); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_overflow", latencyBuckets[9]); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_count", - stats.managedLedgerStats.storageWriteLatencyBuckets.getCount()); - metric(metrics, cluster, namespace, "pulsar_storage_write_latency_sum", - stats.managedLedgerStats.storageWriteLatencyBuckets.getSum()); - - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.refresh(); - long[] ledgerWritelatencyBuckets = stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets(); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_0_5", ledgerWritelatencyBuckets[0]); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_1", ledgerWritelatencyBuckets[1]); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_5", ledgerWritelatencyBuckets[2]); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_10", ledgerWritelatencyBuckets[3]); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_20", ledgerWritelatencyBuckets[4]); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_50", ledgerWritelatencyBuckets[5]); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_100", ledgerWritelatencyBuckets[6]); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_200", ledgerWritelatencyBuckets[7]); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_le_1000", - ledgerWritelatencyBuckets[8]); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_overflow", - ledgerWritelatencyBuckets[9]); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_count", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount()); - metric(metrics, cluster, namespace, "pulsar_storage_ledger_write_latency_sum", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum()); - - stats.managedLedgerStats.entrySizeBuckets.refresh(); - long[] entrySizeBuckets = stats.managedLedgerStats.entrySizeBuckets.getBuckets(); - metric(metrics, cluster, namespace, "pulsar_entry_size_le_128", entrySizeBuckets[0]); - metric(metrics, cluster, namespace, "pulsar_entry_size_le_512", entrySizeBuckets[1]); - metric(metrics, cluster, namespace, "pulsar_entry_size_le_1_kb", entrySizeBuckets[2]); - metric(metrics, cluster, namespace, "pulsar_entry_size_le_2_kb", entrySizeBuckets[3]); - metric(metrics, cluster, namespace, "pulsar_entry_size_le_4_kb", entrySizeBuckets[4]); - metric(metrics, cluster, namespace, "pulsar_entry_size_le_16_kb", entrySizeBuckets[5]); - metric(metrics, cluster, namespace, "pulsar_entry_size_le_100_kb", entrySizeBuckets[6]); - metric(metrics, cluster, namespace, "pulsar_entry_size_le_1_mb", entrySizeBuckets[7]); - metric(metrics, cluster, namespace, "pulsar_entry_size_le_overflow", entrySizeBuckets[8]); - metric(metrics, cluster, namespace, "pulsar_entry_size_count", - stats.managedLedgerStats.entrySizeBuckets.getCount()); - metric(metrics, cluster, namespace, "pulsar_entry_size_sum", - stats.managedLedgerStats.entrySizeBuckets.getSum()); - - if (!stats.replicationStats.isEmpty()) { - stats.replicationStats.forEach((remoteCluster, replStats) -> { - metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_rate_in", remoteCluster, - replStats.msgRateIn); - metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_rate_out", remoteCluster, - replStats.msgRateOut); - metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_throughput_in", remoteCluster, - replStats.msgThroughputIn); - metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_throughput_out", remoteCluster, - replStats.msgThroughputOut); - metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_backlog", remoteCluster, - replStats.replicationBacklog); - metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_connected_count", - remoteCluster, - replStats.connectedCount); - metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_rate_expired", remoteCluster, - replStats.msgRateExpired); - metricWithRemoteCluster(metrics, cluster, namespace, "pulsar_replication_delay_in_seconds", - remoteCluster, replStats.replicationDelayInSeconds); - }); - } + private static void writeMetricWithBrokerDefault(SimpleTextOutputStream stream, String cluster, String name, + List allNamespaceStats, + Function namespaceFunction) { + stream.write("# TYPE ").write(name).write(" gauge\n"); + stream.write(name) + .write("{cluster=\"").write(cluster).write("\"} ") + .write(0).write(' ').write(System.currentTimeMillis()) + .write('\n'); + writeNamespaceStats(stream, cluster, name, allNamespaceStats, namespaceFunction); } - private static void metric(Map metrics, String cluster, String name, - long value) { - addMetric(metrics, Map.of("cluster", cluster), name, value); + private static void writeMetric(SimpleTextOutputStream stream, String cluster, String name, + List allNamespaceStats, + Function namespaceFunction) { + stream.write("# TYPE ").write(name).write(" gauge\n"); + writeNamespaceStats(stream, cluster, name, allNamespaceStats, namespaceFunction); } - private static void metric(Map metrics, String cluster, String namespace, - String name, long value) { - addMetric(metrics, Map.of("cluster", cluster, "namespace", namespace), name, value); + private static void writeNamespaceStats(SimpleTextOutputStream stream, String cluster, String name, + List allNamespaceStats, + Function namespaceFunction) { + allNamespaceStats.forEach(n -> { + Number value = namespaceFunction.apply(n); + if (value != null) { + stream.write(name) + .write("{cluster=\"").write(cluster) + .write("\",namespace=\"").write(n.name) + .write("\"} ") + .write(value).write(' ').write(System.currentTimeMillis()) + .write('\n'); + } + }); } - private static void metric(Map metrics, String cluster, String namespace, - String name, double value) { - addMetric(metrics, Map.of("cluster", cluster, "namespace", namespace), name, value); + private static void writeMsgBacklog(SimpleTextOutputStream stream, String cluster, + List allNamespaceStats, + Function namespaceFunction) { + stream.write("# TYPE ").write("pulsar_msg_backlog").write(" gauge\n"); + stream.write("pulsar_msg_backlog") + .write("{cluster=\"").write(cluster).write("\"} ") + .write(0).write(' ').write(System.currentTimeMillis()) + .write('\n'); + allNamespaceStats.forEach(n -> { + Number value = namespaceFunction.apply(n); + if (value != null) { + stream.write("pulsar_msg_backlog") + .write("{cluster=\"").write(cluster) + .write("\",namespace=\"").write(n.name) + .write("\",remote_cluster=\"").write("local") + .write("\"} ") + .write(value).write(' ').write(System.currentTimeMillis()) + .write('\n'); + } + }); } - private static void metricWithRemoteCluster(Map metrics, String cluster, - String namespace, String name, String remoteCluster, double value) { - addMetric(metrics, Map.of("cluster", cluster, "namespace", namespace, "remote_cluster", remoteCluster), name, - value); + private static void writeReplicationStat(SimpleTextOutputStream stream, String cluster, String name, + List allNamespaceStats, + Function replStatsFunction) { + stream.write("# TYPE ").write(name).write(" gauge\n"); + allNamespaceStats.forEach(n -> { + if (!n.replicationStats.isEmpty()) { + n.replicationStats.forEach((remoteCluster, replStats) -> + stream.write(name) + .write("{cluster=\"").write(cluster) + .write("\",namespace=\"").write(n.name) + .write("\",remote_cluster=\"").write(remoteCluster) + .write("\"} ") + .write(replStatsFunction.apply(replStats)).write(' ').write(System.currentTimeMillis()) + .write('\n') + ); + } + }); } + } 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 fe4b1e36e7eb3..15e130756399b 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 @@ -19,18 +19,19 @@ package org.apache.pulsar.broker.stats.prometheus; import static org.apache.pulsar.common.naming.TopicName.PARTITIONED_TOPIC_SUFFIX; -import io.prometheus.client.Collector; -import java.math.BigDecimal; -import java.math.RoundingMode; -import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.Function; import org.apache.bookkeeper.mledger.util.StatsBuckets; +import org.apache.pulsar.common.util.SimpleTextOutputStream; import org.apache.pulsar.compaction.CompactionRecord; import org.apache.pulsar.compaction.CompactorMXBean; class TopicStats { + String namespace; + String name; int subscriptionsCount; int producersCount; @@ -58,9 +59,6 @@ class TopicStats { Map subscriptionStats = new HashMap<>(); Map producerStats = new HashMap<>(); - // Used for tracking duplicate TYPE definitions - static Map metricWithTypeDefinition = new HashMap<>(); - // For compaction long compactionRemovedEventCount; long compactionSucceedCount; @@ -106,370 +104,457 @@ public void reset() { compactionLatencyBuckets.reset(); } - static void resetTypes() { - metricWithTypeDefinition.clear(); - } - - static void buildTopicStats(Map metrics, String cluster, String namespace, - String topic, - TopicStats stats, Optional compactorMXBean, - boolean splitTopicAndPartitionIndexLabel) { - metric(metrics, cluster, namespace, topic, "pulsar_subscriptions_count", stats.subscriptionsCount, + public static void printTopicStats(SimpleTextOutputStream stream, String cluster, List stats, + Optional compactorMXBean, + boolean splitTopicAndPartitionIndexLabel) { + writeMetricWithBrokerDefault(stream, cluster, "pulsar_subscriptions_count", stats, s -> s.subscriptionsCount, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_producers_count", stats.producersCount, + writeMetricWithBrokerDefault(stream, cluster, "pulsar_producers_count", stats, s -> s.producersCount, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_consumers_count", stats.consumersCount, + writeMetricWithBrokerDefault(stream, cluster, "pulsar_consumers_count", stats, s -> s.consumersCount, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_rate_in", stats.rateIn, + writeMetricWithBrokerDefault(stream, cluster, "pulsar_rate_in", stats, s -> s.rateIn, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_rate_out", stats.rateOut, + writeMetricWithBrokerDefault(stream, cluster, "pulsar_rate_out", stats, s -> s.rateOut, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_throughput_in", stats.throughputIn, + writeMetricWithBrokerDefault(stream, cluster, "pulsar_throughput_in", stats, s -> s.throughputIn, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_throughput_out", stats.throughputOut, + writeMetricWithBrokerDefault(stream, cluster, "pulsar_throughput_out", stats, s -> s.throughputOut, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_average_msg_size", stats.averageMsgSize, + writeMetric(stream, cluster, "pulsar_average_msg_size", stats, s -> s.averageMsgSize, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_size", stats.managedLedgerStats.storageSize, + writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_size", stats, + s -> s.managedLedgerStats.storageSize, + splitTopicAndPartitionIndexLabel); + writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_logical_size", + stats, s -> s.managedLedgerStats.storageLogicalSize, splitTopicAndPartitionIndexLabel); + writeMetricWithBrokerDefault(stream, cluster, "pulsar_msg_backlog", stats, s -> s.msgBacklog, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_logical_size", - stats.managedLedgerStats.storageLogicalSize, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_msg_backlog", stats.msgBacklog, + writeMetric(stream, cluster, "pulsar_storage_backlog_size", + stats, s -> s.managedLedgerStats.backlogSize, splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_write_rate", stats, s -> s.managedLedgerStats.storageWriteRate, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_rate", - stats.managedLedgerStats.storageWriteRate, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_read_rate", stats.managedLedgerStats.storageReadRate, + writeMetric(stream, cluster, "pulsar_storage_read_rate", stats, s -> s.managedLedgerStats.storageReadRate, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_backlog_size", - stats.managedLedgerStats.backlogSize, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_publish_rate_limit_times", stats.publishRateLimitedTimes, + writeMetric(stream, cluster, "pulsar_publish_rate_limit_times", stats, s -> s.publishRateLimitedTimes, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_offloaded_size", stats.managedLedgerStats + writeMetric(stream, cluster, "pulsar_storage_offloaded_size", stats, s -> s.managedLedgerStats .offloadedStorageUsed, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_backlog_quota_limit", stats.backlogQuotaLimit, + writeMetric(stream, cluster, "pulsar_storage_backlog_quota_limit", stats, s -> s.backlogQuotaLimit, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_backlog_quota_limit_time", - stats.backlogQuotaLimitTime, splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_backlog_quota_limit_time", + stats, s -> s.backlogQuotaLimitTime, splitTopicAndPartitionIndexLabel); - long[] latencyBuckets = stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets(); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_0_5", latencyBuckets[0], + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_0_5", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[0], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_1", latencyBuckets[1], + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_1", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[1], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_5", latencyBuckets[2], + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_5", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[2], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_10", latencyBuckets[3], + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_10", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[3], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_20", latencyBuckets[4], + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_20", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[4], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_50", latencyBuckets[5], + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_50", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[5], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_100", latencyBuckets[6], + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_100", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[6], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_200", latencyBuckets[7], + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_200", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[7], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_le_1000", latencyBuckets[8], + writeMetric(stream, cluster, "pulsar_storage_write_latency_le_1000", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[8], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_overflow", latencyBuckets[9], + writeMetric(stream, cluster, "pulsar_storage_write_latency_overflow", stats, + s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[9], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_count", - stats.managedLedgerStats.storageWriteLatencyBuckets.getCount(), splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_write_latency_sum", - stats.managedLedgerStats.storageWriteLatencyBuckets.getSum(), splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_write_latency_count", + stats, s -> s.managedLedgerStats.storageWriteLatencyBuckets.getCount(), + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_write_latency_sum", + stats, s -> s.managedLedgerStats.storageWriteLatencyBuckets.getSum(), splitTopicAndPartitionIndexLabel); - long[] ledgerWriteLatencyBuckets = stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets(); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_0_5", - ledgerWriteLatencyBuckets[0], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_1", - ledgerWriteLatencyBuckets[1], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_5", - ledgerWriteLatencyBuckets[2], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_10", - ledgerWriteLatencyBuckets[3], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_20", - ledgerWriteLatencyBuckets[4], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_50", - ledgerWriteLatencyBuckets[5], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_100", - ledgerWriteLatencyBuckets[6], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_200", - ledgerWriteLatencyBuckets[7], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_le_1000", - ledgerWriteLatencyBuckets[8], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_overflow", - ledgerWriteLatencyBuckets[9], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_count", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount(), + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_0_5", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[0], + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_1", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[1], + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_5", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[2], + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_10", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[3], + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_20", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[4], + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_50", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[5], + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_100", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[6], + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_200", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[7], + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_1000", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[8], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_storage_ledger_write_latency_sum", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum(), + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_overflow", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[9], + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_count", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount(), + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_sum", + stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum(), splitTopicAndPartitionIndexLabel); - long[] entrySizeBuckets = stats.managedLedgerStats.entrySizeBuckets.getBuckets(); - metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_128", entrySizeBuckets[0], + writeMetric(stream, cluster, "pulsar_entry_size_le_128", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[0], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_512", entrySizeBuckets[1], + writeMetric(stream, cluster, "pulsar_entry_size_le_512", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[1], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_1_kb", entrySizeBuckets[2], + writeMetric(stream, cluster, "pulsar_entry_size_le_1_kb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[2], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_2_kb", entrySizeBuckets[3], + writeMetric(stream, cluster, "pulsar_entry_size_le_2_kb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[3], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_4_kb", entrySizeBuckets[4], + writeMetric(stream, cluster, "pulsar_entry_size_le_4_kb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[4], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_16_kb", entrySizeBuckets[5], + writeMetric(stream, cluster, "pulsar_entry_size_le_16_kb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[5], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_100_kb", entrySizeBuckets[6], + writeMetric(stream, cluster, "pulsar_entry_size_le_100_kb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[6], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_1_mb", entrySizeBuckets[7], + writeMetric(stream, cluster, "pulsar_entry_size_le_1_mb", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[7], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_entry_size_le_overflow", entrySizeBuckets[8], + writeMetric(stream, cluster, "pulsar_entry_size_le_overflow", stats, + s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[8], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_entry_size_count", - stats.managedLedgerStats.entrySizeBuckets.getCount(), splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_entry_size_sum", - stats.managedLedgerStats.entrySizeBuckets.getSum(), splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_entry_size_count", + stats, s -> s.managedLedgerStats.entrySizeBuckets.getCount(), splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_entry_size_sum", + stats, s -> s.managedLedgerStats.entrySizeBuckets.getSum(), splitTopicAndPartitionIndexLabel); - stats.producerStats.forEach((p, producerStats) -> { - metric(metrics, cluster, namespace, topic, p, producerStats.producerId, "pulsar_producer_msg_rate_in", - producerStats.msgRateIn, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, p, producerStats.producerId, "pulsar_producer_msg_throughput_in", - producerStats.msgThroughputIn, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, p, producerStats.producerId, "pulsar_producer_msg_average_Size", - producerStats.averageMsgSize, splitTopicAndPartitionIndexLabel); - }); + writeProducerStat(stream, cluster, "pulsar_producer_msg_rate_in", stats, + p -> p.msgRateIn, splitTopicAndPartitionIndexLabel); + writeProducerStat(stream, cluster, "pulsar_producer_msg_throughput_in", stats, + p -> p.msgThroughputIn, splitTopicAndPartitionIndexLabel); + writeProducerStat(stream, cluster, "pulsar_producer_msg_average_Size", stats, + p -> p.averageMsgSize, splitTopicAndPartitionIndexLabel); - stats.subscriptionStats.forEach((n, subsStats) -> { - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_back_log", - subsStats.msgBacklog, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_back_log_no_delayed", - subsStats.msgBacklogNoDelayed, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_delayed", - subsStats.msgDelayed, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_redeliver", - subsStats.msgRateRedeliver, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_unacked_messages", - subsStats.unackedMessages, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_blocked_on_unacked_messages", - subsStats.blockedSubscriptionOnUnackedMsgs ? 1 : 0, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_out", - subsStats.msgRateOut, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_msg_ack_rate", - subsStats.messageAckRate, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_msg_throughput_out", - subsStats.msgThroughputOut, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_out_bytes_total", - subsStats.bytesOutCounter, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_out_messages_total", - subsStats.msgOutCounter, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_last_expire_timestamp", - subsStats.lastExpireTimestamp, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_last_acked_timestamp", - subsStats.lastAckedTimestamp, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_last_consumed_flow_timestamp", - subsStats.lastConsumedFlowTimestamp, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_last_consumed_timestamp", - subsStats.lastConsumedTimestamp, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_last_mark_delete_advanced_timestamp", - subsStats.lastMarkDeleteAdvancedTimestamp, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_expired", - subsStats.msgRateExpired, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, "pulsar_subscription_total_msg_expired", - subsStats.totalMsgExpired, splitTopicAndPartitionIndexLabel); - subsStats.consumerStat.forEach((c, consumerStats) -> { - metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), - "pulsar_consumer_msg_rate_redeliver", consumerStats.msgRateRedeliver, - splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), - "pulsar_consumer_unacked_messages", consumerStats.unackedMessages, - splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), - "pulsar_consumer_blocked_on_unacked_messages", - consumerStats.blockedSubscriptionOnUnackedMsgs ? 1 : 0, - splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), - "pulsar_consumer_msg_rate_out", consumerStats.msgRateOut, - splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), - "pulsar_consumer_msg_ack_rate", consumerStats.msgAckRate, - splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), - "pulsar_consumer_msg_throughput_out", consumerStats.msgThroughputOut, - splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), - "pulsar_consumer_available_permits", consumerStats.availablePermits, - splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), - "pulsar_out_bytes_total", consumerStats.bytesOutCounter, - splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, n, c.consumerName(), c.consumerId(), - "pulsar_out_messages_total", consumerStats.msgOutCounter, - splitTopicAndPartitionIndexLabel); - }); - }); - if (!stats.replicationStats.isEmpty()) { - stats.replicationStats.forEach((remoteCluster, replStats) -> { - metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_rate_in", remoteCluster, - replStats.msgRateIn, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_rate_out", - remoteCluster, - replStats.msgRateOut, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_throughput_in", - remoteCluster, replStats.msgThroughputIn, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_throughput_out", - remoteCluster, replStats.msgThroughputOut, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_backlog", remoteCluster, - replStats.replicationBacklog, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_connected_count", - remoteCluster, replStats.connectedCount, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_rate_expired", - remoteCluster, replStats.msgRateExpired, splitTopicAndPartitionIndexLabel); - metricWithRemoteCluster(metrics, cluster, namespace, topic, "pulsar_replication_delay_in_seconds", - remoteCluster, replStats.replicationDelayInSeconds, splitTopicAndPartitionIndexLabel); - }); - } + writeSubscriptionStat(stream, cluster, "pulsar_subscription_back_log", stats, + s -> s.msgBacklog, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_back_log_no_delayed", + stats, s -> s.msgBacklogNoDelayed, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_delayed", + stats, s -> s.msgDelayed, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_msg_rate_redeliver", + stats, s -> s.msgRateRedeliver, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_unacked_messages", + stats, s -> s.unackedMessages, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_blocked_on_unacked_messages", + stats, s -> s.blockedSubscriptionOnUnackedMsgs ? 1 : 0, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_msg_rate_out", + stats, s -> s.msgRateOut, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_msg_ack_rate", + stats, s -> s.messageAckRate, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_msg_throughput_out", + stats, s -> s.msgThroughputOut, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_out_bytes_total", + stats, s -> s.bytesOutCounter, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_out_messages_total", + stats, s -> s.msgOutCounter, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_last_expire_timestamp", + stats, s -> s.lastExpireTimestamp, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_last_acked_timestamp", + stats, s -> s.lastAckedTimestamp, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_last_consumed_flow_timestamp", + stats, s -> s.lastConsumedFlowTimestamp, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_last_consumed_timestamp", + stats, s -> s.lastConsumedTimestamp, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_last_mark_delete_advanced_timestamp", + stats, s -> s.lastMarkDeleteAdvancedTimestamp, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_msg_rate_expired", + stats, s -> s.msgRateExpired, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(stream, cluster, "pulsar_subscription_total_msg_expired", + stats, s -> s.totalMsgExpired, splitTopicAndPartitionIndexLabel); + + writeConsumerStat(stream, cluster, "pulsar_consumer_msg_rate_redeliver", stats, c -> c.msgRateRedeliver, + splitTopicAndPartitionIndexLabel); + writeConsumerStat(stream, cluster, "pulsar_consumer_unacked_messages", stats, c -> c.unackedMessages, + splitTopicAndPartitionIndexLabel); + writeConsumerStat(stream, cluster, "pulsar_consumer_blocked_on_unacked_messages", + stats, c -> c.blockedSubscriptionOnUnackedMsgs ? 1 : 0, + splitTopicAndPartitionIndexLabel); + writeConsumerStat(stream, cluster, "pulsar_consumer_msg_rate_out", stats, c -> c.msgRateOut, + splitTopicAndPartitionIndexLabel); + + writeConsumerStat(stream, cluster, "pulsar_consumer_msg_ack_rate", stats, c -> c.msgAckRate, + splitTopicAndPartitionIndexLabel); + + writeConsumerStat(stream, cluster, "pulsar_consumer_msg_throughput_out", stats, c -> c.msgThroughputOut, + splitTopicAndPartitionIndexLabel); + writeConsumerStat(stream, cluster, "pulsar_consumer_available_permits", stats, c -> c.availablePermits, + splitTopicAndPartitionIndexLabel); + writeConsumerStat(stream, cluster, "pulsar_out_bytes_total", stats, c -> c.bytesOutCounter, + splitTopicAndPartitionIndexLabel); + writeConsumerStat(stream, cluster, "pulsar_out_messages_total", stats, c -> c.msgOutCounter, + splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_in_bytes_total", stats.bytesInCounter, + writeReplicationStat(stream, cluster, "pulsar_replication_rate_in", stats, r -> r.msgRateIn, + splitTopicAndPartitionIndexLabel); + writeReplicationStat(stream, cluster, "pulsar_replication_rate_out", stats, r -> r.msgRateOut, + splitTopicAndPartitionIndexLabel); + writeReplicationStat(stream, cluster, "pulsar_replication_throughput_in", stats, r -> r.msgThroughputIn, + splitTopicAndPartitionIndexLabel); + writeReplicationStat(stream, cluster, "pulsar_replication_throughput_out", stats, r -> r.msgThroughputOut, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_in_messages_total", stats.msgInCounter, + writeReplicationStat(stream, cluster, "pulsar_replication_backlog", stats, r -> r.replicationBacklog, + splitTopicAndPartitionIndexLabel); + writeReplicationStat(stream, cluster, "pulsar_replication_connected_count", stats, r -> r.connectedCount, + splitTopicAndPartitionIndexLabel); + writeReplicationStat(stream, cluster, "pulsar_replication_rate_expired", stats, r -> r.msgRateExpired, + splitTopicAndPartitionIndexLabel); + writeReplicationStat(stream, cluster, "pulsar_replication_delay_in_seconds", stats, + r -> r.replicationDelayInSeconds, splitTopicAndPartitionIndexLabel); + + writeMetric(stream, cluster, "pulsar_in_bytes_total", stats, s -> s.bytesInCounter, + splitTopicAndPartitionIndexLabel); + writeMetric(stream, cluster, "pulsar_in_messages_total", stats, s -> s.msgInCounter, splitTopicAndPartitionIndexLabel); // Compaction - boolean hasCompaction = - compactorMXBean.flatMap(mxBean -> mxBean.getCompactionRecordForTopic(topic)).isPresent(); - if (hasCompaction) { - metric(metrics, cluster, namespace, topic, "pulsar_compaction_removed_event_count", - stats.compactionRemovedEventCount, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_succeed_count", - stats.compactionSucceedCount, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_failed_count", - stats.compactionFailedCount, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_duration_time_in_mills", - stats.compactionDurationTimeInMills, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_read_throughput", - stats.compactionReadThroughput, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_write_throughput", - stats.compactionWriteThroughput, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_compacted_entries_count", - stats.compactionCompactedEntriesCount, splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_compacted_entries_size", - stats.compactionCompactedEntriesSize, splitTopicAndPartitionIndexLabel); - long[] compactionLatencyBuckets = stats.compactionLatencyBuckets.getBuckets(); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_0_5", - compactionLatencyBuckets[0], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_1", - compactionLatencyBuckets[1], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_5", - compactionLatencyBuckets[2], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_10", - compactionLatencyBuckets[3], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_20", - compactionLatencyBuckets[4], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_50", - compactionLatencyBuckets[5], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_100", - compactionLatencyBuckets[6], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_200", - compactionLatencyBuckets[7], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_le_1000", - compactionLatencyBuckets[8], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_overflow", - compactionLatencyBuckets[9], splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_sum", - stats.compactionLatencyBuckets.getSum(), splitTopicAndPartitionIndexLabel); - metric(metrics, cluster, namespace, topic, "pulsar_compaction_latency_count", - stats.compactionLatencyBuckets.getCount(), splitTopicAndPartitionIndexLabel); - } + + writeCompactionStat(stream, cluster, "pulsar_compaction_removed_event_count", + stats, s -> s.compactionRemovedEventCount, compactorMXBean, splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_succeed_count", + stats, s -> s.compactionSucceedCount, compactorMXBean, splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_failed_count", + stats, s -> s.compactionFailedCount, compactorMXBean, splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_duration_time_in_mills", + stats, s -> s.compactionDurationTimeInMills, compactorMXBean, splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_read_throughput", + stats, s -> s.compactionReadThroughput, compactorMXBean, splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_write_throughput", + stats, s -> s.compactionWriteThroughput, compactorMXBean, splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_compacted_entries_count", + stats, s -> s.compactionCompactedEntriesCount, compactorMXBean, splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_compacted_entries_size", + stats, s -> s.compactionCompactedEntriesSize, compactorMXBean, splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_0_5", + stats, s -> s.compactionLatencyBuckets.getBuckets()[0], compactorMXBean, + splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_1", + stats, s -> s.compactionLatencyBuckets.getBuckets()[1], compactorMXBean, + splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_5", + stats, s -> s.compactionLatencyBuckets.getBuckets()[2], compactorMXBean, + splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_10", + stats, s -> s.compactionLatencyBuckets.getBuckets()[3], compactorMXBean, + splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_20", + stats, s -> s.compactionLatencyBuckets.getBuckets()[4], compactorMXBean, + splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_50", + stats, s -> s.compactionLatencyBuckets.getBuckets()[5], compactorMXBean, + splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_100", + stats, s -> s.compactionLatencyBuckets.getBuckets()[6], compactorMXBean, + splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_200", + stats, s -> s.compactionLatencyBuckets.getBuckets()[7], compactorMXBean, + splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_1000", + stats, s -> s.compactionLatencyBuckets.getBuckets()[8], compactorMXBean, + splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_overflow", + stats, s -> s.compactionLatencyBuckets.getBuckets()[9], compactorMXBean, + splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_sum", + stats, s -> s.compactionLatencyBuckets.getSum(), compactorMXBean, splitTopicAndPartitionIndexLabel); + writeCompactionStat(stream, cluster, "pulsar_compaction_latency_count", + stats, s -> s.compactionLatencyBuckets.getCount(), compactorMXBean, splitTopicAndPartitionIndexLabel); + } + + private static void writeMetricWithBrokerDefault(SimpleTextOutputStream stream, String cluster, String name, + List allTopicStats, + Function topicFunction, + boolean splitTopicAndPartitionIndexLabel) { + stream.write("# TYPE ").write(name).write(" gauge\n"); + stream.write(name) + .write("{cluster=\"").write(cluster).write("\"} ") + .write(0).write(' ').write(System.currentTimeMillis()) + .write('\n'); + writeTopicStats(stream, cluster, name, allTopicStats, topicFunction, splitTopicAndPartitionIndexLabel); } - private static void metric(Map metrics, String cluster, String namespace, - String topic, String name, double value, boolean splitTopicAndPartitionIndexLabel) { - Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - addMetric(metrics, labels, name, value); + private static void writeMetric(SimpleTextOutputStream stream, String cluster, String name, + List allTopicStats, + Function topicFunction, + boolean splitTopicAndPartitionIndexLabel) { + stream.write("# TYPE ").write(name).write(" gauge\n"); + writeTopicStats(stream, cluster, name, allTopicStats, topicFunction, splitTopicAndPartitionIndexLabel); } - private static void metric(Map metrics, String cluster, String namespace, - String topic, String subscription, String name, long value, - boolean splitTopicAndPartitionIndexLabel) { - Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - labels.put("subscription", subscription); - addMetric(metrics, labels, name, value); + private static void writeTopicStats(SimpleTextOutputStream stream, String cluster, String name, + List allTopicStats, Function valueFunction, + boolean splitTopicAndPartitionIndexLabel) { + allTopicStats.forEach(t -> { + writeCommonLabels(stream, cluster, t.namespace, t.name, name, splitTopicAndPartitionIndexLabel); + stream.write("\"} ") + .write(valueFunction.apply(t)).write(' ') + .write(System.currentTimeMillis()) + .write('\n'); + }); } - private static void metric(Map metrics, String cluster, String namespace, - String topic, String producerName, long produceId, String name, double value, - boolean splitTopicAndPartitionIndexLabel) { - Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - labels.put("producer_name", producerName); - labels.put("producer_id", String.valueOf(produceId)); - addMetric(metrics, labels, name, value); + private static void writeProducerStat(SimpleTextOutputStream stream, String cluster, String name, + List allTopicStats, + Function valueFunction, + boolean splitTopicAndPartitionIndexLabel) { + stream.write("# TYPE ").write(name).write(" gauge\n"); + allTopicStats.forEach(t -> + t.producerStats.forEach((p, producerStats) -> { + writeCommonLabels(stream, cluster, t.namespace, t.name, name, + splitTopicAndPartitionIndexLabel); + stream.write("\",producer_name=\"").write(p) + .write("\",producer_id=\"").write(String.valueOf(producerStats.producerId)) + .write("\"} ") + .write(valueFunction.apply(producerStats)).write(' ').write(System.currentTimeMillis()) + .write('\n'); + })); } - private static void metric(Map metrics, String cluster, String namespace, - String topic, String subscription, String name, double value, - boolean splitTopicAndPartitionIndexLabel) { - Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - labels.put("subscription", subscription); - addMetric(metrics, labels, name, value); + private static void writeSubscriptionStat(SimpleTextOutputStream stream, String cluster, String name, + List allTopicStats, + Function valueFunction, + boolean splitTopicAndPartitionIndexLabel) { + stream.write("# TYPE ").write(name).write(" gauge\n"); + allTopicStats.forEach(t -> + t.subscriptionStats.forEach((s, subStats) -> { + writeCommonLabels(stream, cluster, t.namespace, t.name, name, + splitTopicAndPartitionIndexLabel); + stream.write("\",subscription=\"").write(s) + .write("\"} ") + .write(valueFunction.apply(subStats)).write(' ').write(System.currentTimeMillis()) + .write('\n'); + })); } - private static void metric(Map metrics, String cluster, String namespace, - String topic, String subscription, String consumerName, long consumerId, String name, - double value, boolean splitTopicAndPartitionIndexLabel) { - Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - labels.put("subscription", subscription); - labels.put("consumer_name", consumerName); - labels.put("consumer_id", String.valueOf(consumerId)); - addMetric(metrics, labels, name, value); + private static void writeConsumerStat(SimpleTextOutputStream stream, String cluster, String name, + List allTopicStats, + Function valueFunction, + boolean splitTopicAndPartitionIndexLabel) { + stream.write("# TYPE ").write(name).write(" gauge\n"); + allTopicStats.forEach(t -> + t.subscriptionStats.forEach((s, subStats) -> + subStats.consumerStat.forEach((c, conStats) -> { + writeCommonLabels(stream, cluster, t.namespace, t.name, name, + splitTopicAndPartitionIndexLabel); + stream.write("\",subscription=\"").write(s) + .write("\",consumer_name=\"").write(c.consumerName()) + .write("\",consumer_id=\"").write(String.valueOf(c.consumerId())) + .write("\"} ") + .write(valueFunction.apply(conStats)).write(' ').write(System.currentTimeMillis()) + .write('\n'); + }))); } - private static void metricWithRemoteCluster(Map metrics, String cluster, - String namespace, String topic, String name, String remoteCluster, - double value, boolean splitTopicAndPartitionIndexLabel) { - Map labels = buildLabels(cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - labels.put("remote_cluster", remoteCluster); - addMetric(metrics, labels, name, value); + + private static void writeReplicationStat(SimpleTextOutputStream stream, String cluster, String name, + List allTopicStats, + Function replStatsFunction, + boolean splitTopicAndPartitionIndexLabel) { + stream.write("# TYPE ").write(name).write(" gauge\n"); + allTopicStats.forEach(t -> { + if (!t.replicationStats.isEmpty()) { + t.replicationStats.forEach((remoteCluster, replStats) -> { + stream.write(name) + .write("{cluster=\"").write(cluster) + .write("\",namespace=\"").write(t.namespace); + if (splitTopicAndPartitionIndexLabel) { + int index = t.name.indexOf(PARTITIONED_TOPIC_SUFFIX); + if (index > 0) { + stream.write("\",topic=\"").write(t.name.substring(0, index)); + stream.write("\",partition=\"") + .write(t.name.substring( + index + PARTITIONED_TOPIC_SUFFIX.length())); + } else { + stream.write("\",topic=\"").write(t.name); + stream.write("\",partition=\"").write("-1"); + } + } else { + stream.write("\",topic=\"").write(t.name); + } + stream.write("\",remote_cluster=\"").write(remoteCluster) + .write("\"} ") + .write(replStatsFunction.apply(replStats)).write(' ') + .write(System.currentTimeMillis()) + .write('\n'); + } + ); + } + }); } - private static Map buildLabels(String cluster, String namespace, String topic, - boolean splitTopicAndPartitionIndexLabel) { - Map labels = new HashMap<>(); - labels.put("cluster", cluster); - labels.put("namespace", namespace); + private static void writeCompactionStat(SimpleTextOutputStream stream, String cluster, String name, + List allTopicStats, + Function valueFunction, + Optional compactorMXBean, + boolean splitTopicAndPartitionIndexLabel) { + stream.write("# TYPE ").write(name).write(" gauge\n"); + allTopicStats.forEach(t -> { + boolean hasCompaction = + compactorMXBean.flatMap(mxBean -> mxBean.getCompactionRecordForTopic(t.name)) + .isPresent(); + if (hasCompaction) { + writeCommonLabels(stream, cluster, t.namespace, t.name, name, + splitTopicAndPartitionIndexLabel); + stream.write("\"} ") + .write(valueFunction.apply(t)).write(' ') + .write(System.currentTimeMillis()) + .write('\n'); + } + } + ); + } + + private static void writeCommonLabels(SimpleTextOutputStream stream, String cluster, String namespace, String topic, + String metricName, boolean splitTopicAndPartitionIndexLabel) { + stream.write(metricName) + .write("{cluster=\"").write(cluster) + .write("\",namespace=\"").write(namespace); if (splitTopicAndPartitionIndexLabel) { int index = topic.indexOf(PARTITIONED_TOPIC_SUFFIX); if (index > 0) { - labels.put("topic", topic.substring(0, index)); - labels.put("partition", topic.substring(index + PARTITIONED_TOPIC_SUFFIX.length())); + stream.write("\",topic=\"").write(topic.substring(0, index)); + stream.write("\",partition=\"") + .write(topic.substring(index + PARTITIONED_TOPIC_SUFFIX.length())); } else { - labels.put("topic", topic); - labels.put("partition", "-1"); + stream.write("\",topic=\"").write(topic); + stream.write("\",partition=\"").write("-1"); } } else { - labels.put("topic", topic); + stream.write("\",topic=\"").write(topic); } - return labels; - } - - static void addMetric(Map metrics, Map labels, - String name, double value) { - Collector.MetricFamilySamples familySamples = metrics.getOrDefault(name, - new Collector.MetricFamilySamples(name, Collector.Type.GAUGE, null, new ArrayList<>())); - familySamples.samples.add(new Collector.MetricFamilySamples.Sample(name, - labels.keySet().stream().toList(), - labels.values().stream().toList(), - round(value), - System.currentTimeMillis())); - metrics.put(name, familySamples); - } - - // This replicates the SimpleTextOutputStream.write(double d) String formatting - // to ensure values are rounded to 3 decimal places - static double round(double value) { - BigDecimal bd = new BigDecimal(Double.toString(value)); - bd = bd.setScale(3, RoundingMode.DOWN); - return bd.doubleValue(); } } 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 2ead6fecacf74..8911d66cdd6d2 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 @@ -634,13 +634,13 @@ public void testPerConsumerStats() throws Exception { assertEquals(cm.get(0).tags.get("subscription"), "test"); 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-topic2"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("subscription"), "test"); - assertEquals(cm.get(1).tags.get("consumer_id"), "1"); assertEquals(cm.get(2).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(2).tags.get("subscription"), "test"); + assertEquals(cm.get(2).tags.get("consumer_id"), "1"); assertEquals(cm.get(3).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(3).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); @@ -654,13 +654,13 @@ public void testPerConsumerStats() throws Exception { assertEquals(cm.get(0).tags.get("subscription"), "test"); 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-topic2"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("subscription"), "test"); - assertEquals(cm.get(1).tags.get("consumer_id"), "1"); assertEquals(cm.get(2).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(2).tags.get("subscription"), "test"); + assertEquals(cm.get(2).tags.get("consumer_id"), "1"); assertEquals(cm.get(3).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(3).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java index 39ecc4235888a..dffc065e985d3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java @@ -20,7 +20,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; - import org.testng.annotations.Test; @Test(groups = "broker") From 9f4163433f9a338fa295e8f9c3f70f77ed44d015 Mon Sep 17 00:00:00 2001 From: Mark Silcox <63227862+marksilcox@users.noreply.github.com> Date: Tue, 28 Jun 2022 09:34:16 +0100 Subject: [PATCH 10/19] Update pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java Co-authored-by: Dave Maughan --- .../org/apache/pulsar/broker/stats/PrometheusMetricsTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8911d66cdd6d2..1c834a6a50d0c 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 @@ -1448,7 +1448,7 @@ public void testMetricsGroupedByTypeDefinitions() throws Exception { if (!metricName.equals(currentMetric.get())) { System.out.println(metricsStr); - fail("Metric not grouped under it's type definition: " + line); + fail("Metric not grouped under its type definition: " + line); } } From 04923c047a94844f5a40de6174b55742f8b5f18e Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Fri, 1 Jul 2022 17:52:35 +0100 Subject: [PATCH 11/19] [fix][broker] Cherry pick changes --- .../prometheus/AggregatedNamespaceStats.java | 1 - .../prometheus/NamespaceStatsAggregator.java | 554 ++++++------- .../PrometheusMetricsGenerator.java | 2 + .../broker/stats/prometheus/TopicStats.java | 761 ++++++++---------- .../broker/stats/PrometheusMetricsTest.java | 12 +- .../pulsar/common/util/NumberFormat.java | 2 +- .../common/util/SimpleTextOutputStream.java | 5 + 7 files changed, 649 insertions(+), 688 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index 6a6a535f2015e..58f94c315e309 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -24,7 +24,6 @@ import org.apache.pulsar.compaction.CompactionRecord; public class AggregatedNamespaceStats { - public String name; public int topicsCount; public int subscriptionsCount; 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 a2cc95ce84ea5..caf0dd9ec8046 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 @@ -18,10 +18,11 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator.MAX_COMPONENTS; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.util.concurrent.FastThreadLocal; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.LongAdder; @@ -37,6 +38,7 @@ import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; +import org.apache.pulsar.common.util.NumberFormat; import org.apache.pulsar.common.util.SimpleTextOutputStream; import org.apache.pulsar.compaction.CompactedTopicContext; import org.apache.pulsar.compaction.Compactor; @@ -45,28 +47,14 @@ @Slf4j public class NamespaceStatsAggregator { - private static final FastThreadLocal> localAllNamespaceStats = + private static final FastThreadLocal localNamespaceStats = new FastThreadLocal<>() { @Override - protected List initialValue() { - return new ArrayList<>(); + protected AggregatedNamespaceStats initialValue() { + return new AggregatedNamespaceStats(); } }; - private static final FastThreadLocal> localAllTopicStats = new FastThreadLocal<>() { - @Override - protected List initialValue() { - return new ArrayList<>(); - } - }; - - private static final FastThreadLocal> localNamespaceTopicCount = new FastThreadLocal<>() { - @Override - protected Map initialValue() { - return new HashMap<>(); - } - }; - private static final FastThreadLocal localTopicStats = new FastThreadLocal<>() { @Override protected TopicStats initialValue() { @@ -78,59 +66,46 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, SimpleTextOutputStream stream) { String cluster = pulsar.getConfiguration().getClusterName(); + AggregatedNamespaceStats namespaceStats = localNamespaceStats.get(); + TopicStats topicStats = localTopicStats.get(); Optional compactorMXBean = getCompactorMXBean(pulsar); LongAdder topicsCount = new LongAdder(); - try { - pulsar.getBrokerService().getMultiLayerTopicMap().forEach((namespace, bundlesMap) -> { - // only need to create if we are not including topic metrics - AggregatedNamespaceStats namespaceStats = includeTopicMetrics ? null : new AggregatedNamespaceStats(); - topicsCount.reset(); - - bundlesMap.forEach((bundle, topicsMap) -> topicsMap.forEach((name, topic) -> { - //If not includeTopicMetrics then use a single thread local, so not assigning lots of objects - TopicStats topicStats = includeTopicMetrics ? new TopicStats() : localTopicStats.get(); - topicStats.reset(); - topicStats.name = name; - topicStats.namespace = namespace; - getTopicStats(topic, topicStats, includeConsumerMetrics, includeProducerMetrics, - pulsar.getConfiguration().isExposePreciseBacklogInPrometheus(), - pulsar.getConfiguration().isExposeSubscriptionBacklogSizeInPrometheus(), - compactorMXBean - ); - - if (includeTopicMetrics) { - topicsCount.add(1); - localAllTopicStats.get().add(topicStats); - } else { - namespaceStats.updateStats(topicStats); - } - })); + Map allMetrics = new HashMap<>(); + Map localNamespaceTopicCount = new HashMap<>(); + pulsar.getBrokerService().getMultiLayerTopicMap().forEach((namespace, bundlesMap) -> { + namespaceStats.reset(); + topicsCount.reset(); + + bundlesMap.forEach((bundle, topicsMap) -> topicsMap.forEach((name, topic) -> { + getTopicStats(topic, topicStats, includeConsumerMetrics, includeProducerMetrics, + pulsar.getConfiguration().isExposePreciseBacklogInPrometheus(), + pulsar.getConfiguration().isExposeSubscriptionBacklogSizeInPrometheus(), + compactorMXBean + ); - if (!includeTopicMetrics) { - // Only include namespace level stats if we don't have the per-topic, otherwise we're going to - // report the same data twice, and it will make the aggregation difficult - namespaceStats.name = namespace; - localAllNamespaceStats.get().add(namespaceStats); + if (includeTopicMetrics) { + topicsCount.add(1); + TopicStats.printTopicStats(allMetrics, topicStats, compactorMXBean, cluster, namespace, name, + splitTopicAndPartitionIndexLabel); } else { - localNamespaceTopicCount.get().put(namespace, topicsCount.sum()); + namespaceStats.updateStats(topicStats); } - }); + })); - if (includeTopicMetrics) { - printTopicsCountStats(stream, cluster, localNamespaceTopicCount.get()); - TopicStats.printTopicStats(stream, cluster, localAllTopicStats.get(), compactorMXBean, - splitTopicAndPartitionIndexLabel); - } else { - printNamespaceStats(stream, cluster, localAllNamespaceStats.get()); - } - } finally { - if (includeTopicMetrics) { - localNamespaceTopicCount.get().clear(); - localAllTopicStats.get().clear(); + if (!includeTopicMetrics) { + // Only include namespace level stats if we don't have the per-topic, otherwise we're going to + // report the same data twice, and it will make the aggregation difficult + printNamespaceStats(allMetrics, namespaceStats, cluster, namespace); } else { - localAllNamespaceStats.get().clear(); + localNamespaceTopicCount.put(namespace, topicsCount.sum()); } + }); + + if (includeTopicMetrics) { + printTopicsCountStats(allMetrics, localNamespaceTopicCount, cluster); } + + allMetrics.values().forEach(stream::write); } private static Optional getCompactorMXBean(PulsarService pulsar) { @@ -140,9 +115,8 @@ private static Optional getCompactorMXBean(PulsarService pulsar private static void getTopicStats(Topic topic, TopicStats stats, boolean includeConsumerMetrics, boolean includeProducerMetrics, boolean getPreciseBacklog, - boolean subscriptionBacklogSize, - Optional compactorMXBean) { - stats.name = topic.getName(); + boolean subscriptionBacklogSize, Optional compactorMXBean) { + stats.reset(); if (topic instanceof PersistentTopic) { // Managed Ledger stats @@ -303,224 +277,264 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include }); } - private static void printTopicsCountStats(SimpleTextOutputStream stream, String cluster, - Map namespaceTopicsCount) { - stream.write("# TYPE ").write("pulsar_topics_count").write(" gauge\n"); - stream.write("pulsar_topics_count") - .write("{cluster=\"").write(cluster).write("\"} ") - .write(0).write(' ').write(System.currentTimeMillis()) - .write('\n'); - namespaceTopicsCount.forEach((ns, topicCount) -> stream.write("pulsar_topics_count") - .write("{cluster=\"").write(cluster) - .write("\",namespace=\"").write(ns) - .write("\"} ") - .write(topicCount).write(' ').write(System.currentTimeMillis()) - .write('\n') + private static void printTopicsCountStats(Map allMetrics, Map namespaceTopicsCount, + String cluster) { + namespaceTopicsCount.forEach((ns, topicCount) -> + writeMetricWithBrokerDefault(allMetrics, "pulsar_topics_count", topicCount, cluster, ns) ); } - private static void printNamespaceStats(SimpleTextOutputStream stream, String cluster, - List stats) { - writeMetricWithBrokerDefault(stream, cluster, "pulsar_topics_count", stats, s -> s.topicsCount); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_subscriptions_count", stats, s -> s.subscriptionsCount); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_producers_count", stats, s -> s.producersCount); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_consumers_count", stats, s -> s.consumersCount); - - writeMetricWithBrokerDefault(stream, cluster, "pulsar_rate_in", stats, s -> s.rateIn); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_rate_out", stats, s -> s.rateOut); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_throughput_in", stats, s -> s.throughputIn); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_throughput_out", stats, s -> s.throughputOut); - writeMetric(stream, cluster, "pulsar_consumer_msg_ack_rate", stats, s -> s.messageAckRate); - - writeMetric(stream, cluster, "pulsar_in_bytes_total", stats, s -> s.bytesInCounter); - writeMetric(stream, cluster, "pulsar_in_messages_total", stats, s -> s.msgInCounter); - writeMetric(stream, cluster, "pulsar_out_bytes_total", stats, s -> s.bytesOutCounter); - writeMetric(stream, cluster, "pulsar_out_messages_total", stats, s -> s.msgOutCounter); - - writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_size", stats, - s -> s.managedLedgerStats.storageSize); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_logical_size", stats, - s -> s.managedLedgerStats.storageLogicalSize); - writeMetric(stream, cluster, "pulsar_storage_backlog_size", stats, s -> s.managedLedgerStats.backlogSize); - writeMetric(stream, cluster, "pulsar_storage_offloaded_size", - stats, s -> s.managedLedgerStats.offloadedStorageUsed); - - writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_write_rate", stats, - s -> s.managedLedgerStats.storageWriteRate); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_read_rate", stats, - s -> s.managedLedgerStats.storageReadRate); - - writeMetric(stream, cluster, "pulsar_subscription_delayed", stats, s -> s.msgDelayed); - - writeMsgBacklog(stream, cluster, stats, s -> s.msgBacklog); - - stats.forEach(s -> s.managedLedgerStats.storageWriteLatencyBuckets.refresh()); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_0_5", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[0]); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_1", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[1]); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_5", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[2]); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_10", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[3]); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_20", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[4]); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_50", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[5]); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_100", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[6]); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_200", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[7]); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_1000", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[8]); - writeMetric(stream, cluster, "pulsar_storage_write_latency_overflow", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[9]); - writeMetric(stream, cluster, "pulsar_storage_write_latency_count", - stats, s -> s.managedLedgerStats.storageWriteLatencyBuckets.getCount()); - writeMetric(stream, cluster, "pulsar_storage_write_latency_sum", - stats, s -> s.managedLedgerStats.storageWriteLatencyBuckets.getSum()); - - stats.forEach(s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.refresh()); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_0_5", stats, - s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[0]); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_1", stats, - s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[1]); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_5", stats, - s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[2]); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_10", stats, - s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[3]); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_20", stats, - s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[4]); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_50", stats, - s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[5]); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_100", stats, - s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[6]); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_200", stats, - s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[7]); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_1000", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[8]); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_overflow", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[9]); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_count", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount()); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_sum", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum()); - - stats.forEach(s -> s.managedLedgerStats.entrySizeBuckets.refresh()); - writeMetric(stream, cluster, "pulsar_entry_size_le_128", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[0]); - writeMetric(stream, cluster, "pulsar_entry_size_le_512", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[1]); - writeMetric(stream, cluster, "pulsar_entry_size_le_1_kb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[2]); - writeMetric(stream, cluster, "pulsar_entry_size_le_2_kb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[3]); - writeMetric(stream, cluster, "pulsar_entry_size_le_4_kb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[4]); - writeMetric(stream, cluster, "pulsar_entry_size_le_16_kb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[5]); - writeMetric(stream, cluster, "pulsar_entry_size_le_100_kb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[6]); - writeMetric(stream, cluster, "pulsar_entry_size_le_1_mb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[7]); - writeMetric(stream, cluster, "pulsar_entry_size_le_overflow", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[8]); - writeMetric(stream, cluster, "pulsar_entry_size_count", - stats, s -> s.managedLedgerStats.entrySizeBuckets.getCount()); - writeMetric(stream, cluster, "pulsar_entry_size_sum", - stats, s -> s.managedLedgerStats.entrySizeBuckets.getSum()); - - writeReplicationStat(stream, cluster, "pulsar_replication_rate_in", stats, - replStats -> replStats.msgRateIn); - writeReplicationStat(stream, cluster, "pulsar_replication_rate_out", stats, - replStats -> replStats.msgRateOut); - writeReplicationStat(stream, cluster, "pulsar_replication_throughput_in", stats, - replStats -> replStats.msgThroughputIn); - writeReplicationStat(stream, cluster, "pulsar_replication_throughput_out", stats, - replStats -> replStats.msgThroughputOut); - writeReplicationStat(stream, cluster, "pulsar_replication_backlog", stats, - replStats -> replStats.replicationBacklog); - writeReplicationStat(stream, cluster, "pulsar_replication_connected_count", stats, - replStats -> replStats.connectedCount); - writeReplicationStat(stream, cluster, "pulsar_replication_rate_expired", stats, - replStats -> replStats.msgRateExpired); - writeReplicationStat(stream, cluster, "pulsar_replication_delay_in_seconds", stats, - replStats -> replStats.replicationDelayInSeconds); + private static void printNamespaceStats(Map allMetrics, AggregatedNamespaceStats stats, + String cluster, String namespace) { + writeMetricWithBrokerDefault(allMetrics, "pulsar_topics_count", stats.topicsCount, cluster, namespace); + writeMetricWithBrokerDefault(allMetrics, "pulsar_subscriptions_count", stats.subscriptionsCount, cluster, + namespace); + writeMetricWithBrokerDefault(allMetrics, "pulsar_producers_count", stats.producersCount, cluster, namespace); + writeMetricWithBrokerDefault(allMetrics, "pulsar_consumers_count", stats.consumersCount, cluster, namespace); + + writeMetricWithBrokerDefault(allMetrics, "pulsar_rate_in", stats.rateIn, cluster, namespace); + writeMetricWithBrokerDefault(allMetrics, "pulsar_rate_out", stats.rateOut, cluster, namespace); + writeMetricWithBrokerDefault(allMetrics, "pulsar_throughput_in", stats.throughputIn, cluster, namespace); + writeMetricWithBrokerDefault(allMetrics, "pulsar_throughput_out", stats.throughputOut, cluster, namespace); + writeMetric(allMetrics, "pulsar_consumer_msg_ack_rate", stats.messageAckRate, cluster, namespace); + + writeMetric(allMetrics, "pulsar_in_bytes_total", stats.bytesInCounter, cluster, namespace); + writeMetric(allMetrics, "pulsar_in_messages_total", stats.msgInCounter, cluster, namespace); + writeMetric(allMetrics, "pulsar_out_bytes_total", stats.bytesOutCounter, cluster, namespace); + writeMetric(allMetrics, "pulsar_out_messages_total", stats.msgOutCounter, cluster, namespace); + + writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_size", stats.managedLedgerStats.storageSize, cluster, + namespace); + writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_logical_size", + stats.managedLedgerStats.storageLogicalSize, cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize, cluster, + namespace); + writeMetric(allMetrics, "pulsar_storage_offloaded_size", + stats.managedLedgerStats.offloadedStorageUsed, cluster, namespace); + + writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_write_rate", stats.managedLedgerStats.storageWriteRate, + cluster, namespace); + writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_read_rate", stats.managedLedgerStats.storageReadRate, + cluster, namespace); + + writeMetric(allMetrics, "pulsar_subscription_delayed", stats.msgDelayed, cluster, namespace); + + writePulsarMsgBacklog(allMetrics, stats.msgBacklog, cluster, namespace); + + stats.managedLedgerStats.storageWriteLatencyBuckets.refresh(); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_0_5", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[0], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_1", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[1], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_5", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[2], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_10", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[3], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_20", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[4], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_50", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[5], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_100", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[6], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_200", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[7], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_1000", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[8], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_write_latency_overflow", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[9], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_write_latency_count", + stats.managedLedgerStats.storageWriteLatencyBuckets.getCount(), cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_write_latency_sum", + stats.managedLedgerStats.storageWriteLatencyBuckets.getSum(), cluster, namespace); + + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.refresh(); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_0_5", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[0], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_1", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[1], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_5", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[2], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_10", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[3], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_20", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[4], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_50", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[5], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_100", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[6], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_200", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[7], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_1000", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[8], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_overflow", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[9], cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_count", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount(), cluster, namespace); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_sum", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum(), cluster, namespace); + + stats.managedLedgerStats.entrySizeBuckets.refresh(); + writeMetric(allMetrics, "pulsar_entry_size_le_128", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[0], + cluster, namespace); + writeMetric(allMetrics, "pulsar_entry_size_le_512", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[1], + cluster, namespace); + writeMetric(allMetrics, "pulsar_entry_size_le_1_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[2], + cluster, namespace); + writeMetric(allMetrics, "pulsar_entry_size_le_2_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[3], + cluster, namespace); + writeMetric(allMetrics, "pulsar_entry_size_le_4_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[4], + cluster, namespace); + writeMetric(allMetrics, "pulsar_entry_size_le_16_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[5], + cluster, namespace); + writeMetric(allMetrics, "pulsar_entry_size_le_100_kb", + stats.managedLedgerStats.entrySizeBuckets.getBuckets()[6], cluster, namespace); + writeMetric(allMetrics, "pulsar_entry_size_le_1_mb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[7], + cluster, namespace); + writeMetric(allMetrics, "pulsar_entry_size_le_overflow", + stats.managedLedgerStats.entrySizeBuckets.getBuckets()[8], cluster, namespace); + writeMetric(allMetrics, "pulsar_entry_size_count", + stats.managedLedgerStats.entrySizeBuckets.getCount(), cluster, namespace); + writeMetric(allMetrics, "pulsar_entry_size_sum", + stats.managedLedgerStats.entrySizeBuckets.getSum(), cluster, namespace); + + writeReplicationStat(allMetrics, "pulsar_replication_rate_in", stats, + replStats -> replStats.msgRateIn, cluster, namespace); + writeReplicationStat(allMetrics, "pulsar_replication_rate_out", stats, + replStats -> replStats.msgRateOut, cluster, namespace); + writeReplicationStat(allMetrics, "pulsar_replication_throughput_in", stats, + replStats -> replStats.msgThroughputIn, cluster, namespace); + writeReplicationStat(allMetrics, "pulsar_replication_throughput_out", stats, + replStats -> replStats.msgThroughputOut, cluster, namespace); + writeReplicationStat(allMetrics, "pulsar_replication_backlog", stats, + replStats -> replStats.replicationBacklog, cluster, namespace); + writeReplicationStat(allMetrics, "pulsar_replication_connected_count", stats, + replStats -> replStats.connectedCount, cluster, namespace); + writeReplicationStat(allMetrics, "pulsar_replication_rate_expired", stats, + replStats -> replStats.msgRateExpired, cluster, namespace); + writeReplicationStat(allMetrics, "pulsar_replication_delay_in_seconds", stats, + replStats -> replStats.replicationDelayInSeconds, cluster, namespace); } - private static void writeMetricWithBrokerDefault(SimpleTextOutputStream stream, String cluster, String name, - List allNamespaceStats, - Function namespaceFunction) { - stream.write("# TYPE ").write(name).write(" gauge\n"); - stream.write(name) - .write("{cluster=\"").write(cluster).write("\"} ") - .write(0).write(' ').write(System.currentTimeMillis()) - .write('\n'); - writeNamespaceStats(stream, cluster, name, allNamespaceStats, namespaceFunction); + private static void writeMetricWithBrokerDefault(Map allMetrics, String metricName, Number value, + String cluster, String namespace) { + ByteBuf buffer = writeGaugeTypeWithBrokerDefault(allMetrics, metricName, cluster); + writeSample(buffer, metricName, value, "cluster", cluster, "namespace", namespace); } - private static void writeMetric(SimpleTextOutputStream stream, String cluster, String name, - List allNamespaceStats, - Function namespaceFunction) { - stream.write("# TYPE ").write(name).write(" gauge\n"); - writeNamespaceStats(stream, cluster, name, allNamespaceStats, namespaceFunction); + private static void writePulsarMsgBacklog(Map allMetrics, Number value, + String cluster, String namespace) { + ByteBuf buffer = writeGaugeTypeWithBrokerDefault(allMetrics, "pulsar_msg_backlog", cluster); + writeSample(buffer, "pulsar_msg_backlog", value, "cluster", cluster, "namespace", namespace, "remote_cluster", + "local"); } - private static void writeNamespaceStats(SimpleTextOutputStream stream, String cluster, String name, - List allNamespaceStats, - Function namespaceFunction) { - allNamespaceStats.forEach(n -> { - Number value = namespaceFunction.apply(n); - if (value != null) { - stream.write(name) - .write("{cluster=\"").write(cluster) - .write("\",namespace=\"").write(n.name) - .write("\"} ") - .write(value).write(' ').write(System.currentTimeMillis()) - .write('\n'); - } - }); + private static void writeMetric(Map allMetrics, String metricName, Number value, String cluster, + String namespace) { + ByteBuf buffer = writeGaugeType(allMetrics, metricName); + writeSample(buffer, metricName, value, "cluster", cluster, "namespace", namespace); } - private static void writeMsgBacklog(SimpleTextOutputStream stream, String cluster, - List allNamespaceStats, - Function namespaceFunction) { - stream.write("# TYPE ").write("pulsar_msg_backlog").write(" gauge\n"); - stream.write("pulsar_msg_backlog") - .write("{cluster=\"").write(cluster).write("\"} ") - .write(0).write(' ').write(System.currentTimeMillis()) - .write('\n'); - allNamespaceStats.forEach(n -> { - Number value = namespaceFunction.apply(n); - if (value != null) { - stream.write("pulsar_msg_backlog") - .write("{cluster=\"").write(cluster) - .write("\",namespace=\"").write(n.name) - .write("\",remote_cluster=\"").write("local") - .write("\"} ") - .write(value).write(' ').write(System.currentTimeMillis()) - .write('\n'); - } - }); + private static void writeReplicationStat(Map allMetrics, String metricName, + AggregatedNamespaceStats namespaceStats, + Function replStatsFunction, + String cluster, String namespace) { + if (!namespaceStats.replicationStats.isEmpty()) { + ByteBuf buffer = writeGaugeType(allMetrics, metricName); + namespaceStats.replicationStats.forEach((remoteCluster, replStats) -> + writeSample(buffer, metricName, replStatsFunction.apply(replStats), + "cluster", cluster, + "namespace", namespace, + "remote_cluster", remoteCluster) + ); + } } - private static void writeReplicationStat(SimpleTextOutputStream stream, String cluster, String name, - List allNamespaceStats, - Function replStatsFunction) { - stream.write("# TYPE ").write(name).write(" gauge\n"); - allNamespaceStats.forEach(n -> { - if (!n.replicationStats.isEmpty()) { - n.replicationStats.forEach((remoteCluster, replStats) -> - stream.write(name) - .write("{cluster=\"").write(cluster) - .write("\",namespace=\"").write(n.name) - .write("\",remote_cluster=\"").write(remoteCluster) - .write("\"} ") - .write(replStatsFunction.apply(replStats)).write(' ').write(System.currentTimeMillis()) - .write('\n') - ); + static ByteBuf writeGaugeType(Map allMetrics, String metricName) { + if (!allMetrics.containsKey(metricName)) { + ByteBuf buffer = UnpooledByteBufAllocator.DEFAULT.compositeDirectBuffer(MAX_COMPONENTS); + write(buffer, "# TYPE "); + write(buffer, metricName); + write(buffer, " gauge\n"); + allMetrics.put(metricName, buffer); + } + return allMetrics.get(metricName); + } + + static ByteBuf writeGaugeTypeWithBrokerDefault(Map allMetrics, String metricName, String cluster) { + if (!allMetrics.containsKey(metricName)) { + ByteBuf buffer = writeGaugeType(allMetrics, metricName); + writeSample(buffer, metricName, 0, "cluster", cluster); + } + return allMetrics.get(metricName); + } + + static void writeSample(ByteBuf buffer, String metricName, Number value, String... labelsAndValuesArray) { + write(buffer, metricName); + write(buffer, '{'); + for (int i = 0; i < labelsAndValuesArray.length; i += 2) { + write(buffer, labelsAndValuesArray[i]); + write(buffer, "=\""); + write(buffer, labelsAndValuesArray[i + 1]); + write(buffer, '\"'); + if (labelsAndValuesArray.length != i + 2) { + write(buffer, ','); } - }); + } + write(buffer, "\"} "); + write(buffer, value); + write(buffer, ' '); + write(buffer, System.currentTimeMillis()); + write(buffer, '\n'); + } + + private static void write(ByteBuf buffer, String s) { + if (s == null) { + return; + } + int len = s.length(); + for (int i = 0; i < len; i++) { + write(buffer, s.charAt(i)); + } + } + + private static void write(ByteBuf buffer, Number n) { + if (n instanceof Integer) { + write(buffer, n.intValue()); + } else if (n instanceof Long) { + write(buffer, n.longValue()); + } else if (n instanceof Double) { + write(buffer, n.doubleValue()); + } else { + write(buffer, n.toString()); + } } + private static void write(ByteBuf buffer, long n) { + NumberFormat.format(buffer, n); + } + + private static void write(ByteBuf buffer, double d) { + long i = (long) d; + write(buffer, i); + + long r = Math.abs((long) (1000 * (d - i))); + write(buffer, '.'); + if (r == 0) { + write(buffer, '0'); + } + + if (r < 100) { + write(buffer, '0'); + } + + if (r < 10) { + write(buffer, '0'); + } + + write(buffer, r); + } + + private static void write(ByteBuf buffer, char c) { + buffer.writeByte((byte) c); + } } 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 cd6afd1535dec..437c1ea12a8ee 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 @@ -55,6 +55,8 @@ * Format specification can be found at {@link https://prometheus.io/docs/instrumenting/exposition_formats/} */ public class PrometheusMetricsGenerator { + private static volatile TimeWindow timeWindow; + protected static final int MAX_COMPONENTS = 64; static { DefaultExports.initialize(); 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 15e130756399b..3c515f71fd8a4 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 @@ -18,21 +18,22 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import static org.apache.pulsar.broker.stats.prometheus.NamespaceStatsAggregator.writeGaugeType; +import static org.apache.pulsar.broker.stats.prometheus.NamespaceStatsAggregator.writeGaugeTypeWithBrokerDefault; +import static org.apache.pulsar.broker.stats.prometheus.NamespaceStatsAggregator.writeSample; import static org.apache.pulsar.common.naming.TopicName.PARTITIONED_TOPIC_SUFFIX; +import io.netty.buffer.ByteBuf; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.function.Function; import org.apache.bookkeeper.mledger.util.StatsBuckets; -import org.apache.pulsar.common.util.SimpleTextOutputStream; import org.apache.pulsar.compaction.CompactionRecord; import org.apache.pulsar.compaction.CompactorMXBean; class TopicStats { - String namespace; - String name; - int subscriptionsCount; int producersCount; int consumersCount; @@ -104,457 +105,397 @@ public void reset() { compactionLatencyBuckets.reset(); } - public static void printTopicStats(SimpleTextOutputStream stream, String cluster, List stats, - Optional compactorMXBean, - boolean splitTopicAndPartitionIndexLabel) { - writeMetricWithBrokerDefault(stream, cluster, "pulsar_subscriptions_count", stats, s -> s.subscriptionsCount, - splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_producers_count", stats, s -> s.producersCount, - splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_consumers_count", stats, s -> s.consumersCount, - splitTopicAndPartitionIndexLabel); - - writeMetricWithBrokerDefault(stream, cluster, "pulsar_rate_in", stats, s -> s.rateIn, - splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_rate_out", stats, s -> s.rateOut, - splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_throughput_in", stats, s -> s.throughputIn, - splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_throughput_out", stats, s -> s.throughputOut, - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_average_msg_size", stats, s -> s.averageMsgSize, - splitTopicAndPartitionIndexLabel); - - writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_size", stats, - s -> s.managedLedgerStats.storageSize, - splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_storage_logical_size", - stats, s -> s.managedLedgerStats.storageLogicalSize, splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(stream, cluster, "pulsar_msg_backlog", stats, s -> s.msgBacklog, - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_backlog_size", - stats, s -> s.managedLedgerStats.backlogSize, splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_rate", stats, s -> s.managedLedgerStats.storageWriteRate, - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_read_rate", stats, s -> s.managedLedgerStats.storageReadRate, - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_publish_rate_limit_times", stats, s -> s.publishRateLimitedTimes, - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_offloaded_size", stats, s -> s.managedLedgerStats - .offloadedStorageUsed, splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_backlog_quota_limit", stats, s -> s.backlogQuotaLimit, - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_backlog_quota_limit_time", - stats, s -> s.backlogQuotaLimitTime, splitTopicAndPartitionIndexLabel); - - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_0_5", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[0], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_1", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[1], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_5", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[2], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_10", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[3], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_20", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[4], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_50", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[5], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_100", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[6], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_200", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[7], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_latency_le_1000", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[8], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_latency_overflow", stats, - s -> s.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[9], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_latency_count", - stats, s -> s.managedLedgerStats.storageWriteLatencyBuckets.getCount(), - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_write_latency_sum", - stats, s -> s.managedLedgerStats.storageWriteLatencyBuckets.getSum(), splitTopicAndPartitionIndexLabel); - - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_0_5", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[0], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_1", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[1], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_5", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[2], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_10", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[3], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_20", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[4], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_50", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[5], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_100", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[6], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_200", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[7], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_le_1000", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[8], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_overflow", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[9], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_count", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount(), - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_storage_ledger_write_latency_sum", - stats, s -> s.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum(), - splitTopicAndPartitionIndexLabel); - - writeMetric(stream, cluster, "pulsar_entry_size_le_128", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[0], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_entry_size_le_512", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[1], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_entry_size_le_1_kb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[2], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_entry_size_le_2_kb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[3], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_entry_size_le_4_kb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[4], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_entry_size_le_16_kb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[5], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_entry_size_le_100_kb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[6], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_entry_size_le_1_mb", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[7], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_entry_size_le_overflow", stats, - s -> s.managedLedgerStats.entrySizeBuckets.getBuckets()[8], - splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_entry_size_count", - stats, s -> s.managedLedgerStats.entrySizeBuckets.getCount(), splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_entry_size_sum", - stats, s -> s.managedLedgerStats.entrySizeBuckets.getSum(), splitTopicAndPartitionIndexLabel); - - writeProducerStat(stream, cluster, "pulsar_producer_msg_rate_in", stats, - p -> p.msgRateIn, splitTopicAndPartitionIndexLabel); - writeProducerStat(stream, cluster, "pulsar_producer_msg_throughput_in", stats, - p -> p.msgThroughputIn, splitTopicAndPartitionIndexLabel); - writeProducerStat(stream, cluster, "pulsar_producer_msg_average_Size", stats, - p -> p.averageMsgSize, splitTopicAndPartitionIndexLabel); - - - writeSubscriptionStat(stream, cluster, "pulsar_subscription_back_log", stats, - s -> s.msgBacklog, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_back_log_no_delayed", - stats, s -> s.msgBacklogNoDelayed, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_delayed", - stats, s -> s.msgDelayed, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_msg_rate_redeliver", - stats, s -> s.msgRateRedeliver, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_unacked_messages", - stats, s -> s.unackedMessages, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_blocked_on_unacked_messages", - stats, s -> s.blockedSubscriptionOnUnackedMsgs ? 1 : 0, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_msg_rate_out", - stats, s -> s.msgRateOut, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_msg_ack_rate", - stats, s -> s.messageAckRate, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_msg_throughput_out", - stats, s -> s.msgThroughputOut, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_out_bytes_total", - stats, s -> s.bytesOutCounter, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_out_messages_total", - stats, s -> s.msgOutCounter, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_last_expire_timestamp", - stats, s -> s.lastExpireTimestamp, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_last_acked_timestamp", - stats, s -> s.lastAckedTimestamp, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_last_consumed_flow_timestamp", - stats, s -> s.lastConsumedFlowTimestamp, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_last_consumed_timestamp", - stats, s -> s.lastConsumedTimestamp, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_last_mark_delete_advanced_timestamp", - stats, s -> s.lastMarkDeleteAdvancedTimestamp, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_msg_rate_expired", - stats, s -> s.msgRateExpired, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(stream, cluster, "pulsar_subscription_total_msg_expired", - stats, s -> s.totalMsgExpired, splitTopicAndPartitionIndexLabel); - - writeConsumerStat(stream, cluster, "pulsar_consumer_msg_rate_redeliver", stats, c -> c.msgRateRedeliver, - splitTopicAndPartitionIndexLabel); - writeConsumerStat(stream, cluster, "pulsar_consumer_unacked_messages", stats, c -> c.unackedMessages, - splitTopicAndPartitionIndexLabel); - writeConsumerStat(stream, cluster, "pulsar_consumer_blocked_on_unacked_messages", + public static void printTopicStats(Map allMetrics, TopicStats stats, + Optional compactorMXBean, String cluster, String namespace, + String name, boolean splitTopicAndPartitionIndexLabel) { + writeMetricWithBrokerDefault(allMetrics, "pulsar_subscriptions_count", stats.subscriptionsCount, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetricWithBrokerDefault(allMetrics, "pulsar_producers_count", stats.producersCount, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetricWithBrokerDefault(allMetrics, "pulsar_consumers_count", stats.consumersCount, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + + writeMetricWithBrokerDefault(allMetrics, "pulsar_rate_in", stats.rateIn, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetricWithBrokerDefault(allMetrics, "pulsar_rate_out", stats.rateOut, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetricWithBrokerDefault(allMetrics, "pulsar_throughput_in", stats.throughputIn, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetricWithBrokerDefault(allMetrics, "pulsar_throughput_out", stats.throughputOut, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_average_msg_size", stats.averageMsgSize, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + + writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_size", stats.managedLedgerStats.storageSize, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_logical_size", + stats.managedLedgerStats.storageLogicalSize, cluster, namespace, name, + splitTopicAndPartitionIndexLabel); + writeMetricWithBrokerDefault(allMetrics, "pulsar_msg_backlog", stats.msgBacklog, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_publish_rate_limit_times", stats.publishRateLimitedTimes, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_offloaded_size", stats.managedLedgerStats + .offloadedStorageUsed, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_backlog_quota_limit", stats.backlogQuotaLimit, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_backlog_quota_limit_time", stats.backlogQuotaLimitTime, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + + writeMetric(allMetrics, "pulsar_storage_write_latency_le_0_5", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[0], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_1", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[1], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_5", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[2], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_10", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[3], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_20", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[4], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_50", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[5], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_100", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[6], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_200", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[7], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_write_latency_le_1000", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[8], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_write_latency_overflow", + stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[9], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_write_latency_count", + stats.managedLedgerStats.storageWriteLatencyBuckets.getCount(), + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_write_latency_sum", + stats.managedLedgerStats.storageWriteLatencyBuckets.getSum(), cluster, namespace, name, + splitTopicAndPartitionIndexLabel); + + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_0_5", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[0], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_1", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[1], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_5", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[2], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_10", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[3], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_20", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[4], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_50", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[5], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_100", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[6], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_200", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[7], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_1000", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[8], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_overflow", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[9], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_count", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount(), + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_sum", + stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum(), + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + + writeMetric(allMetrics, "pulsar_entry_size_le_128", + stats.managedLedgerStats.entrySizeBuckets.getBuckets()[0], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_entry_size_le_512", + stats.managedLedgerStats.entrySizeBuckets.getBuckets()[1], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_entry_size_le_1_kb", + stats.managedLedgerStats.entrySizeBuckets.getBuckets()[2], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_entry_size_le_2_kb", + stats.managedLedgerStats.entrySizeBuckets.getBuckets()[3], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_entry_size_le_4_kb", + stats.managedLedgerStats.entrySizeBuckets.getBuckets()[4], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_entry_size_le_16_kb", + stats.managedLedgerStats.entrySizeBuckets.getBuckets()[5], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_entry_size_le_100_kb", + stats.managedLedgerStats.entrySizeBuckets.getBuckets()[6], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_entry_size_le_1_mb", + stats.managedLedgerStats.entrySizeBuckets.getBuckets()[7], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_entry_size_le_overflow", + stats.managedLedgerStats.entrySizeBuckets.getBuckets()[8], + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_entry_size_count", stats.managedLedgerStats.entrySizeBuckets.getCount(), + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_entry_size_sum", stats.managedLedgerStats.entrySizeBuckets.getSum(), + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + + writeProducerStat(allMetrics, "pulsar_producer_msg_rate_in", stats, + p -> p.msgRateIn, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeProducerStat(allMetrics, "pulsar_producer_msg_throughput_in", stats, + p -> p.msgThroughputIn, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeProducerStat(allMetrics, "pulsar_producer_msg_average_Size", stats, + p -> p.averageMsgSize, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + + + writeSubscriptionStat(allMetrics, "pulsar_subscription_back_log", stats, s -> s.msgBacklog, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_back_log_no_delayed", + stats, s -> s.msgBacklogNoDelayed, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_delayed", + stats, s -> s.msgDelayed, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_msg_rate_redeliver", + stats, s -> s.msgRateRedeliver, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_unacked_messages", + stats, s -> s.unackedMessages, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_blocked_on_unacked_messages", + stats, s -> s.blockedSubscriptionOnUnackedMsgs ? 1 : 0, cluster, namespace, name, + splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_msg_rate_out", + stats, s -> s.msgRateOut, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_msg_ack_rate", + stats, s -> s.messageAckRate, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_msg_throughput_out", + stats, s -> s.msgThroughputOut, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_out_bytes_total", + stats, s -> s.bytesOutCounter, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_out_messages_total", + stats, s -> s.msgOutCounter, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_last_expire_timestamp", + stats, s -> s.lastExpireTimestamp, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_last_acked_timestamp", + stats, s -> s.lastAckedTimestamp, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_last_consumed_flow_timestamp", + stats, s -> s.lastConsumedFlowTimestamp, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_last_consumed_timestamp", + stats, s -> s.lastConsumedTimestamp, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_last_mark_delete_advanced_timestamp", + stats, s -> s.lastMarkDeleteAdvancedTimestamp, cluster, namespace, name, + splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_msg_rate_expired", + stats, s -> s.msgRateExpired, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeSubscriptionStat(allMetrics, "pulsar_subscription_total_msg_expired", + stats, s -> s.totalMsgExpired, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + + writeConsumerStat(allMetrics, "pulsar_consumer_msg_rate_redeliver", stats, c -> c.msgRateRedeliver, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeConsumerStat(allMetrics, "pulsar_consumer_unacked_messages", stats, c -> c.unackedMessages, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeConsumerStat(allMetrics, "pulsar_consumer_blocked_on_unacked_messages", stats, c -> c.blockedSubscriptionOnUnackedMsgs ? 1 : 0, - splitTopicAndPartitionIndexLabel); - writeConsumerStat(stream, cluster, "pulsar_consumer_msg_rate_out", stats, c -> c.msgRateOut, - splitTopicAndPartitionIndexLabel); - - writeConsumerStat(stream, cluster, "pulsar_consumer_msg_ack_rate", stats, c -> c.msgAckRate, - splitTopicAndPartitionIndexLabel); - - writeConsumerStat(stream, cluster, "pulsar_consumer_msg_throughput_out", stats, c -> c.msgThroughputOut, - splitTopicAndPartitionIndexLabel); - writeConsumerStat(stream, cluster, "pulsar_consumer_available_permits", stats, c -> c.availablePermits, - splitTopicAndPartitionIndexLabel); - writeConsumerStat(stream, cluster, "pulsar_out_bytes_total", stats, c -> c.bytesOutCounter, - splitTopicAndPartitionIndexLabel); - writeConsumerStat(stream, cluster, "pulsar_out_messages_total", stats, c -> c.msgOutCounter, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeConsumerStat(allMetrics, "pulsar_consumer_msg_rate_out", stats, c -> c.msgRateOut, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + + writeConsumerStat(allMetrics, "pulsar_consumer_msg_ack_rate", stats, c -> c.msgAckRate, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + + writeConsumerStat(allMetrics, "pulsar_consumer_msg_throughput_out", stats, c -> c.msgThroughputOut, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeConsumerStat(allMetrics, "pulsar_consumer_available_permits", stats, c -> c.availablePermits, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeConsumerStat(allMetrics, "pulsar_out_bytes_total", stats, c -> c.bytesOutCounter, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeConsumerStat(allMetrics, "pulsar_out_messages_total", stats, c -> c.msgOutCounter, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + + writeReplicationStat(allMetrics, "pulsar_replication_rate_in", stats, r -> r.msgRateIn, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeReplicationStat(allMetrics, "pulsar_replication_rate_out", stats, r -> r.msgRateOut, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeReplicationStat(allMetrics, "pulsar_replication_throughput_in", stats, r -> r.msgThroughputIn, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeReplicationStat(allMetrics, "pulsar_replication_throughput_out", stats, r -> r.msgThroughputOut, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeReplicationStat(allMetrics, "pulsar_replication_backlog", stats, r -> r.replicationBacklog, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeReplicationStat(allMetrics, "pulsar_replication_connected_count", stats, r -> r.connectedCount, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeReplicationStat(allMetrics, "pulsar_replication_rate_expired", stats, r -> r.msgRateExpired, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeReplicationStat(allMetrics, "pulsar_replication_delay_in_seconds", stats, + r -> r.replicationDelayInSeconds, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + + writeMetric(allMetrics, "pulsar_in_bytes_total", stats.bytesInCounter, cluster, namespace, name, + splitTopicAndPartitionIndexLabel); + writeMetric(allMetrics, "pulsar_in_messages_total", stats.msgInCounter, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeReplicationStat(stream, cluster, "pulsar_replication_rate_in", stats, r -> r.msgRateIn, - splitTopicAndPartitionIndexLabel); - writeReplicationStat(stream, cluster, "pulsar_replication_rate_out", stats, r -> r.msgRateOut, - splitTopicAndPartitionIndexLabel); - writeReplicationStat(stream, cluster, "pulsar_replication_throughput_in", stats, r -> r.msgThroughputIn, - splitTopicAndPartitionIndexLabel); - writeReplicationStat(stream, cluster, "pulsar_replication_throughput_out", stats, r -> r.msgThroughputOut, - splitTopicAndPartitionIndexLabel); - writeReplicationStat(stream, cluster, "pulsar_replication_backlog", stats, r -> r.replicationBacklog, - splitTopicAndPartitionIndexLabel); - writeReplicationStat(stream, cluster, "pulsar_replication_connected_count", stats, r -> r.connectedCount, - splitTopicAndPartitionIndexLabel); - writeReplicationStat(stream, cluster, "pulsar_replication_rate_expired", stats, r -> r.msgRateExpired, - splitTopicAndPartitionIndexLabel); - writeReplicationStat(stream, cluster, "pulsar_replication_delay_in_seconds", stats, - r -> r.replicationDelayInSeconds, splitTopicAndPartitionIndexLabel); + // Compaction - writeMetric(stream, cluster, "pulsar_in_bytes_total", stats, s -> s.bytesInCounter, + writeCompactionStat(allMetrics, "pulsar_compaction_removed_event_count", stats.compactionRemovedEventCount, + compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeCompactionStat(allMetrics, "pulsar_compaction_succeed_count", stats.compactionSucceedCount, + compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeCompactionStat(allMetrics, "pulsar_compaction_failed_count", stats.compactionFailedCount, compactorMXBean, + cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeCompactionStat(allMetrics, "pulsar_compaction_duration_time_in_mills", stats.compactionDurationTimeInMills, + compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeCompactionStat(allMetrics, "pulsar_compaction_read_throughput", stats.compactionReadThroughput, + compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeCompactionStat(allMetrics, "pulsar_compaction_write_throughput", stats.compactionWriteThroughput, + compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeCompactionStat(allMetrics, "pulsar_compaction_compacted_entries_count", + stats.compactionCompactedEntriesCount, compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(stream, cluster, "pulsar_in_messages_total", stats, s -> s.msgInCounter, + writeCompactionStat(allMetrics, "pulsar_compaction_compacted_entries_size", + stats.compactionCompactedEntriesSize, compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - // Compaction - - writeCompactionStat(stream, cluster, "pulsar_compaction_removed_event_count", - stats, s -> s.compactionRemovedEventCount, compactorMXBean, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_succeed_count", - stats, s -> s.compactionSucceedCount, compactorMXBean, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_failed_count", - stats, s -> s.compactionFailedCount, compactorMXBean, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_duration_time_in_mills", - stats, s -> s.compactionDurationTimeInMills, compactorMXBean, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_read_throughput", - stats, s -> s.compactionReadThroughput, compactorMXBean, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_write_throughput", - stats, s -> s.compactionWriteThroughput, compactorMXBean, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_compacted_entries_count", - stats, s -> s.compactionCompactedEntriesCount, compactorMXBean, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_compacted_entries_size", - stats, s -> s.compactionCompactedEntriesSize, compactorMXBean, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_0_5", - stats, s -> s.compactionLatencyBuckets.getBuckets()[0], compactorMXBean, + writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_0_5", + stats.compactionLatencyBuckets.getBuckets()[0], compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_1", - stats, s -> s.compactionLatencyBuckets.getBuckets()[1], compactorMXBean, + writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_1", + stats.compactionLatencyBuckets.getBuckets()[1], compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_5", - stats, s -> s.compactionLatencyBuckets.getBuckets()[2], compactorMXBean, + writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_5", + stats.compactionLatencyBuckets.getBuckets()[2], compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_10", - stats, s -> s.compactionLatencyBuckets.getBuckets()[3], compactorMXBean, + writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_10", + stats.compactionLatencyBuckets.getBuckets()[3], compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_20", - stats, s -> s.compactionLatencyBuckets.getBuckets()[4], compactorMXBean, + writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_20", + stats.compactionLatencyBuckets.getBuckets()[4], compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_50", - stats, s -> s.compactionLatencyBuckets.getBuckets()[5], compactorMXBean, + writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_50", + stats.compactionLatencyBuckets.getBuckets()[5], compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_100", - stats, s -> s.compactionLatencyBuckets.getBuckets()[6], compactorMXBean, + writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_100", + stats.compactionLatencyBuckets.getBuckets()[6], compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_200", - stats, s -> s.compactionLatencyBuckets.getBuckets()[7], compactorMXBean, + writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_200", + stats.compactionLatencyBuckets.getBuckets()[7], compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_le_1000", - stats, s -> s.compactionLatencyBuckets.getBuckets()[8], compactorMXBean, + writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_1000", + stats.compactionLatencyBuckets.getBuckets()[8], compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_overflow", - stats, s -> s.compactionLatencyBuckets.getBuckets()[9], compactorMXBean, + writeCompactionStat(allMetrics, "pulsar_compaction_latency_overflow", + stats.compactionLatencyBuckets.getBuckets()[9], compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_sum", - stats, s -> s.compactionLatencyBuckets.getSum(), compactorMXBean, splitTopicAndPartitionIndexLabel); - writeCompactionStat(stream, cluster, "pulsar_compaction_latency_count", - stats, s -> s.compactionLatencyBuckets.getCount(), compactorMXBean, splitTopicAndPartitionIndexLabel); + writeCompactionStat(allMetrics, "pulsar_compaction_latency_sum", stats.compactionLatencyBuckets.getSum(), + compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); + writeCompactionStat(allMetrics, "pulsar_compaction_latency_count", stats.compactionLatencyBuckets.getCount(), + compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); } - private static void writeMetricWithBrokerDefault(SimpleTextOutputStream stream, String cluster, String name, - List allTopicStats, - Function topicFunction, - boolean splitTopicAndPartitionIndexLabel) { - stream.write("# TYPE ").write(name).write(" gauge\n"); - stream.write(name) - .write("{cluster=\"").write(cluster).write("\"} ") - .write(0).write(' ').write(System.currentTimeMillis()) - .write('\n'); - writeTopicStats(stream, cluster, name, allTopicStats, topicFunction, splitTopicAndPartitionIndexLabel); + private static void writeMetric(Map allMetrics, String metricName, Number value, String cluster, + String namespace, String topic, boolean splitTopicAndPartitionIndexLabel) { + ByteBuf buffer = writeGaugeType(allMetrics, metricName); + writeTopicSample(buffer, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); } - private static void writeMetric(SimpleTextOutputStream stream, String cluster, String name, - List allTopicStats, - Function topicFunction, - boolean splitTopicAndPartitionIndexLabel) { - stream.write("# TYPE ").write(name).write(" gauge\n"); - writeTopicStats(stream, cluster, name, allTopicStats, topicFunction, splitTopicAndPartitionIndexLabel); - } - - private static void writeTopicStats(SimpleTextOutputStream stream, String cluster, String name, - List allTopicStats, Function valueFunction, - boolean splitTopicAndPartitionIndexLabel) { - allTopicStats.forEach(t -> { - writeCommonLabels(stream, cluster, t.namespace, t.name, name, splitTopicAndPartitionIndexLabel); - stream.write("\"} ") - .write(valueFunction.apply(t)).write(' ') - .write(System.currentTimeMillis()) - .write('\n'); - }); - } - - private static void writeProducerStat(SimpleTextOutputStream stream, String cluster, String name, - List allTopicStats, + private static void writeProducerStat(Map allMetrics, String metricName, TopicStats topicStats, Function valueFunction, + String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel) { - stream.write("# TYPE ").write(name).write(" gauge\n"); - allTopicStats.forEach(t -> - t.producerStats.forEach((p, producerStats) -> { - writeCommonLabels(stream, cluster, t.namespace, t.name, name, - splitTopicAndPartitionIndexLabel); - stream.write("\",producer_name=\"").write(p) - .write("\",producer_id=\"").write(String.valueOf(producerStats.producerId)) - .write("\"} ") - .write(valueFunction.apply(producerStats)).write(' ').write(System.currentTimeMillis()) - .write('\n'); - })); + ByteBuf buffer = writeGaugeType(allMetrics, metricName); + topicStats.producerStats.forEach((p, producerStats) -> + writeTopicSample(buffer, metricName, valueFunction.apply(producerStats), cluster, namespace, topic, + splitTopicAndPartitionIndexLabel, "producer_name", p, "producer_id", + String.valueOf(producerStats.producerId))); } - private static void writeSubscriptionStat(SimpleTextOutputStream stream, String cluster, String name, - List allTopicStats, + private static void writeSubscriptionStat(Map allMetrics, String metricName, TopicStats topicStats, Function valueFunction, + String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel) { - stream.write("# TYPE ").write(name).write(" gauge\n"); - allTopicStats.forEach(t -> - t.subscriptionStats.forEach((s, subStats) -> { - writeCommonLabels(stream, cluster, t.namespace, t.name, name, - splitTopicAndPartitionIndexLabel); - stream.write("\",subscription=\"").write(s) - .write("\"} ") - .write(valueFunction.apply(subStats)).write(' ').write(System.currentTimeMillis()) - .write('\n'); - })); + ByteBuf buffer = writeGaugeType(allMetrics, metricName); + topicStats.subscriptionStats.forEach((s, subStats) -> + writeTopicSample(buffer, metricName, valueFunction.apply(subStats), cluster, namespace, topic, + splitTopicAndPartitionIndexLabel, "subscription", s)); } - private static void writeConsumerStat(SimpleTextOutputStream stream, String cluster, String name, - List allTopicStats, + private static void writeConsumerStat(Map allMetrics, String metricName, TopicStats topicStats, Function valueFunction, + String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel) { - stream.write("# TYPE ").write(name).write(" gauge\n"); - allTopicStats.forEach(t -> - t.subscriptionStats.forEach((s, subStats) -> - subStats.consumerStat.forEach((c, conStats) -> { - writeCommonLabels(stream, cluster, t.namespace, t.name, name, - splitTopicAndPartitionIndexLabel); - stream.write("\",subscription=\"").write(s) - .write("\",consumer_name=\"").write(c.consumerName()) - .write("\",consumer_id=\"").write(String.valueOf(c.consumerId())) - .write("\"} ") - .write(valueFunction.apply(conStats)).write(' ').write(System.currentTimeMillis()) - .write('\n'); - }))); + ByteBuf buffer = writeGaugeType(allMetrics, metricName); + topicStats.subscriptionStats.forEach((s, subStats) -> + subStats.consumerStat.forEach((c, conStats) -> + writeTopicSample(buffer, metricName, valueFunction.apply(conStats), cluster, namespace, topic, + splitTopicAndPartitionIndexLabel, "subscription", s, "consumer_name", c.consumerName(), + "consumer_id", String.valueOf(c.consumerId())) + )); } - private static void writeReplicationStat(SimpleTextOutputStream stream, String cluster, String name, - List allTopicStats, - Function replStatsFunction, + private static void writeReplicationStat(Map allMetrics, String metricName, TopicStats topicStats, + Function valueFunction, + String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel) { - stream.write("# TYPE ").write(name).write(" gauge\n"); - allTopicStats.forEach(t -> { - if (!t.replicationStats.isEmpty()) { - t.replicationStats.forEach((remoteCluster, replStats) -> { - stream.write(name) - .write("{cluster=\"").write(cluster) - .write("\",namespace=\"").write(t.namespace); - if (splitTopicAndPartitionIndexLabel) { - int index = t.name.indexOf(PARTITIONED_TOPIC_SUFFIX); - if (index > 0) { - stream.write("\",topic=\"").write(t.name.substring(0, index)); - stream.write("\",partition=\"") - .write(t.name.substring( - index + PARTITIONED_TOPIC_SUFFIX.length())); - } else { - stream.write("\",topic=\"").write(t.name); - stream.write("\",partition=\"").write("-1"); - } - } else { - stream.write("\",topic=\"").write(t.name); - } - stream.write("\",remote_cluster=\"").write(remoteCluster) - .write("\"} ") - .write(replStatsFunction.apply(replStats)).write(' ') - .write(System.currentTimeMillis()) - .write('\n'); - } - ); - } - }); + if (!topicStats.replicationStats.isEmpty()) { + ByteBuf buffer = writeGaugeType(allMetrics, metricName); + topicStats.replicationStats.forEach((remoteCluster, replStats) -> + writeTopicSample(buffer, metricName, valueFunction.apply(replStats), cluster, namespace, topic, + splitTopicAndPartitionIndexLabel, "remote_cluster", remoteCluster) + ); + } } - private static void writeCompactionStat(SimpleTextOutputStream stream, String cluster, String name, - List allTopicStats, - Function valueFunction, - Optional compactorMXBean, + private static void writeCompactionStat(Map allMetrics, String metricName, + Number value, Optional compactorMXBean, + String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel) { - stream.write("# TYPE ").write(name).write(" gauge\n"); - allTopicStats.forEach(t -> { - boolean hasCompaction = - compactorMXBean.flatMap(mxBean -> mxBean.getCompactionRecordForTopic(t.name)) - .isPresent(); - if (hasCompaction) { - writeCommonLabels(stream, cluster, t.namespace, t.name, name, - splitTopicAndPartitionIndexLabel); - stream.write("\"} ") - .write(valueFunction.apply(t)).write(' ') - .write(System.currentTimeMillis()) - .write('\n'); - } - } - ); + boolean hasCompaction = compactorMXBean.flatMap(mxBean -> mxBean.getCompactionRecordForTopic(topic)) + .isPresent(); + if (hasCompaction) { + ByteBuf buffer = writeGaugeType(allMetrics, metricName); + writeTopicSample(buffer, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + } } - private static void writeCommonLabels(SimpleTextOutputStream stream, String cluster, String namespace, String topic, - String metricName, boolean splitTopicAndPartitionIndexLabel) { - stream.write(metricName) - .write("{cluster=\"").write(cluster) - .write("\",namespace=\"").write(namespace); + static void writeMetricWithBrokerDefault(Map allMetrics, String metricName, Number value, + String cluster, String namespace, String topic, + boolean splitTopicAndPartitionIndexLabel) { + ByteBuf buffer = writeGaugeTypeWithBrokerDefault(allMetrics, metricName, cluster); + writeTopicSample(buffer, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + } + + static void writeTopicSample(ByteBuf buffer, String metricName, Number value, String cluster, + String namespace, String topic, boolean splitTopicAndPartitionIndexLabel, + String... extraLabelsAndValues) { + List labelsAndValues = new ArrayList<>(); + labelsAndValues.add("cluster"); + labelsAndValues.add(cluster); + labelsAndValues.add("namespace"); + labelsAndValues.add(namespace); + labelsAndValues.add("topic"); if (splitTopicAndPartitionIndexLabel) { int index = topic.indexOf(PARTITIONED_TOPIC_SUFFIX); if (index > 0) { - stream.write("\",topic=\"").write(topic.substring(0, index)); - stream.write("\",partition=\"") - .write(topic.substring(index + PARTITIONED_TOPIC_SUFFIX.length())); + labelsAndValues.add(topic.substring(0, index)); + labelsAndValues.add("partition"); + labelsAndValues.add(topic.substring(index + PARTITIONED_TOPIC_SUFFIX.length())); } else { - stream.write("\",topic=\"").write(topic); - stream.write("\",partition=\"").write("-1"); + labelsAndValues.add(topic); + labelsAndValues.add("partition"); + labelsAndValues.add("-1"); } } else { - stream.write("\",topic=\"").write(topic); + labelsAndValues.add(topic); + } + if (extraLabelsAndValues != null) { + labelsAndValues.addAll(List.of(extraLabelsAndValues)); } + writeSample(buffer, metricName, value, labelsAndValues.toArray(new String[0])); } } 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 1c834a6a50d0c..1769e547f09ca 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 @@ -634,13 +634,13 @@ public void testPerConsumerStats() throws Exception { assertEquals(cm.get(0).tags.get("subscription"), "test"); 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("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(1).tags.get("subscription"), "test"); + assertEquals(cm.get(1).tags.get("consumer_id"), "1"); assertEquals(cm.get(2).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(2).tags.get("subscription"), "test"); - assertEquals(cm.get(2).tags.get("consumer_id"), "1"); assertEquals(cm.get(3).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(3).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); @@ -654,13 +654,13 @@ public void testPerConsumerStats() throws Exception { assertEquals(cm.get(0).tags.get("subscription"), "test"); 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("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(1).tags.get("subscription"), "test"); + assertEquals(cm.get(1).tags.get("consumer_id"), "1"); assertEquals(cm.get(2).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(2).tags.get("subscription"), "test"); - assertEquals(cm.get(2).tags.get("consumer_id"), "1"); assertEquals(cm.get(3).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(3).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NumberFormat.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NumberFormat.java index 502af8251e512..0eae68314f809 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NumberFormat.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NumberFormat.java @@ -25,7 +25,7 @@ */ public class NumberFormat { - static void format(ByteBuf out, long num) { + public static void format(ByteBuf out, long num) { if (num == 0) { out.writeByte('0'); return; diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java index 9fc4b347c854f..f7e17eed6ed20 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java @@ -131,4 +131,9 @@ public SimpleTextOutputStream write(double d) { write(r); return this; } + + public void write(ByteBuf byteBuf) { + buffer.writeBytes(byteBuf); + byteBuf.release(); + } } From 6aa7142d56c4c881f0b3ca979b49c42fbc2bae25 Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Mon, 4 Jul 2022 18:32:03 +0100 Subject: [PATCH 12/19] [fix][broker] Updated stream output --- .../prometheus/AggregatedNamespaceStats.java | 1 - .../prometheus/NamespaceStatsAggregator.java | 289 +++----- .../prometheus/PrometheusMetricStreams.java | 9 +- .../PrometheusMetricsGenerator.java | 109 ++- .../broker/stats/prometheus/TopicStats.java | 648 ++++++++---------- .../prometheus/TransactionAggregator.java | 325 ++++----- .../metrics/PrometheusTextFormatUtil.java | 34 - .../AggregatedNamespaceStatsTest.java | 1 + .../pulsar/common/util/NumberFormat.java | 2 +- .../common/util/SimpleTextOutputStream.java | 10 +- 10 files changed, 642 insertions(+), 786 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index 58f94c315e309..1980af91b7b54 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -24,7 +24,6 @@ import org.apache.pulsar.compaction.CompactionRecord; public class AggregatedNamespaceStats { - public int topicsCount; public int subscriptionsCount; public int producersCount; 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 caf0dd9ec8046..17512c82178db 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 @@ -18,9 +18,6 @@ */ package org.apache.pulsar.broker.stats.prometheus; -import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator.MAX_COMPONENTS; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.util.concurrent.FastThreadLocal; import java.util.HashMap; import java.util.Map; @@ -38,8 +35,6 @@ import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; -import org.apache.pulsar.common.util.NumberFormat; -import org.apache.pulsar.common.util.SimpleTextOutputStream; import org.apache.pulsar.compaction.CompactedTopicContext; import org.apache.pulsar.compaction.Compactor; import org.apache.pulsar.compaction.CompactorMXBean; @@ -64,14 +59,16 @@ protected TopicStats initialValue() { public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, - SimpleTextOutputStream stream) { + PrometheusMetricStreams stream) { String cluster = pulsar.getConfiguration().getClusterName(); AggregatedNamespaceStats namespaceStats = localNamespaceStats.get(); TopicStats topicStats = localTopicStats.get(); Optional compactorMXBean = getCompactorMXBean(pulsar); LongAdder topicsCount = new LongAdder(); - Map allMetrics = new HashMap<>(); Map localNamespaceTopicCount = new HashMap<>(); + + printDefaultBrokerStats(stream, cluster); + pulsar.getBrokerService().getMultiLayerTopicMap().forEach((namespace, bundlesMap) -> { namespaceStats.reset(); topicsCount.reset(); @@ -85,7 +82,7 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b if (includeTopicMetrics) { topicsCount.add(1); - TopicStats.printTopicStats(allMetrics, topicStats, compactorMXBean, cluster, namespace, name, + TopicStats.printTopicStats(stream, topicStats, compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); } else { namespaceStats.updateStats(topicStats); @@ -95,17 +92,15 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b if (!includeTopicMetrics) { // Only include namespace level stats if we don't have the per-topic, otherwise we're going to // report the same data twice, and it will make the aggregation difficult - printNamespaceStats(allMetrics, namespaceStats, cluster, namespace); + printNamespaceStats(stream, namespaceStats, cluster, namespace); } else { localNamespaceTopicCount.put(namespace, topicsCount.sum()); } }); if (includeTopicMetrics) { - printTopicsCountStats(allMetrics, localNamespaceTopicCount, cluster); + printTopicsCountStats(stream, localNamespaceTopicCount, cluster); } - - allMetrics.values().forEach(stream::write); } private static Optional getCompactorMXBean(PulsarService pulsar) { @@ -277,171 +272,186 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include }); } - private static void printTopicsCountStats(Map allMetrics, Map namespaceTopicsCount, + private static void printDefaultBrokerStats(PrometheusMetricStreams stream, String cluster) { + // Print metrics with 0 values. This is necessary to have the available brokers being + // reported in the brokers dashboard even if they don't have any topic or traffic + writeMetric(stream, "pulsar_topics_count", 0, cluster); + writeMetric(stream, "pulsar_subscriptions_count", 0, cluster); + writeMetric(stream, "pulsar_producers_count", 0, cluster); + writeMetric(stream, "pulsar_consumers_count", 0, cluster); + writeMetric(stream, "pulsar_rate_in", 0, cluster); + writeMetric(stream, "pulsar_rate_out", 0, cluster); + writeMetric(stream, "pulsar_throughput_in", 0, cluster); + writeMetric(stream, "pulsar_throughput_out", 0, cluster); + writeMetric(stream, "pulsar_storage_size", 0, cluster); + writeMetric(stream, "pulsar_storage_logical_size", 0, cluster); + writeMetric(stream, "pulsar_storage_write_rate", 0, cluster); + writeMetric(stream, "pulsar_storage_read_rate", 0, cluster); + writeMetric(stream, "pulsar_msg_backlog", 0, cluster); + } + + private static void printTopicsCountStats(PrometheusMetricStreams stream, Map namespaceTopicsCount, String cluster) { - namespaceTopicsCount.forEach((ns, topicCount) -> - writeMetricWithBrokerDefault(allMetrics, "pulsar_topics_count", topicCount, cluster, ns) + namespaceTopicsCount.forEach( + (ns, topicCount) -> writeMetric(stream, "pulsar_topics_count", topicCount, cluster, ns) ); } - private static void printNamespaceStats(Map allMetrics, AggregatedNamespaceStats stats, + private static void printNamespaceStats(PrometheusMetricStreams stream, AggregatedNamespaceStats stats, String cluster, String namespace) { - writeMetricWithBrokerDefault(allMetrics, "pulsar_topics_count", stats.topicsCount, cluster, namespace); - writeMetricWithBrokerDefault(allMetrics, "pulsar_subscriptions_count", stats.subscriptionsCount, cluster, + writeMetric(stream, "pulsar_topics_count", stats.topicsCount, cluster, namespace); + writeMetric(stream, "pulsar_subscriptions_count", stats.subscriptionsCount, cluster, namespace); - writeMetricWithBrokerDefault(allMetrics, "pulsar_producers_count", stats.producersCount, cluster, namespace); - writeMetricWithBrokerDefault(allMetrics, "pulsar_consumers_count", stats.consumersCount, cluster, namespace); + writeMetric(stream, "pulsar_producers_count", stats.producersCount, cluster, namespace); + writeMetric(stream, "pulsar_consumers_count", stats.consumersCount, cluster, namespace); - writeMetricWithBrokerDefault(allMetrics, "pulsar_rate_in", stats.rateIn, cluster, namespace); - writeMetricWithBrokerDefault(allMetrics, "pulsar_rate_out", stats.rateOut, cluster, namespace); - writeMetricWithBrokerDefault(allMetrics, "pulsar_throughput_in", stats.throughputIn, cluster, namespace); - writeMetricWithBrokerDefault(allMetrics, "pulsar_throughput_out", stats.throughputOut, cluster, namespace); - writeMetric(allMetrics, "pulsar_consumer_msg_ack_rate", stats.messageAckRate, cluster, namespace); + writeMetric(stream, "pulsar_rate_in", stats.rateIn, cluster, namespace); + writeMetric(stream, "pulsar_rate_out", stats.rateOut, cluster, namespace); + writeMetric(stream, "pulsar_throughput_in", stats.throughputIn, cluster, namespace); + writeMetric(stream, "pulsar_throughput_out", stats.throughputOut, cluster, namespace); + writeMetric(stream, "pulsar_consumer_msg_ack_rate", stats.messageAckRate, cluster, namespace); - writeMetric(allMetrics, "pulsar_in_bytes_total", stats.bytesInCounter, cluster, namespace); - writeMetric(allMetrics, "pulsar_in_messages_total", stats.msgInCounter, cluster, namespace); - writeMetric(allMetrics, "pulsar_out_bytes_total", stats.bytesOutCounter, cluster, namespace); - writeMetric(allMetrics, "pulsar_out_messages_total", stats.msgOutCounter, cluster, namespace); + writeMetric(stream, "pulsar_in_bytes_total", stats.bytesInCounter, cluster, namespace); + writeMetric(stream, "pulsar_in_messages_total", stats.msgInCounter, cluster, namespace); + writeMetric(stream, "pulsar_out_bytes_total", stats.bytesOutCounter, cluster, namespace); + writeMetric(stream, "pulsar_out_messages_total", stats.msgOutCounter, cluster, namespace); - writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_size", stats.managedLedgerStats.storageSize, cluster, + writeMetric(stream, "pulsar_storage_size", stats.managedLedgerStats.storageSize, cluster, namespace); - writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_logical_size", + writeMetric(stream, "pulsar_storage_logical_size", stats.managedLedgerStats.storageLogicalSize, cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize, cluster, + writeMetric(stream, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize, cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_offloaded_size", + writeMetric(stream, "pulsar_storage_offloaded_size", stats.managedLedgerStats.offloadedStorageUsed, cluster, namespace); - writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_write_rate", stats.managedLedgerStats.storageWriteRate, + writeMetric(stream, "pulsar_storage_write_rate", stats.managedLedgerStats.storageWriteRate, cluster, namespace); - writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_read_rate", stats.managedLedgerStats.storageReadRate, + writeMetric(stream, "pulsar_storage_read_rate", stats.managedLedgerStats.storageReadRate, cluster, namespace); - writeMetric(allMetrics, "pulsar_subscription_delayed", stats.msgDelayed, cluster, namespace); + writeMetric(stream, "pulsar_subscription_delayed", stats.msgDelayed, cluster, namespace); - writePulsarMsgBacklog(allMetrics, stats.msgBacklog, cluster, namespace); + writePulsarMsgBacklog(stream, stats.msgBacklog, cluster, namespace); stats.managedLedgerStats.storageWriteLatencyBuckets.refresh(); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_0_5", + writeMetric(stream, "pulsar_storage_write_latency_le_0_5", stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[0], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_1", + writeMetric(stream, "pulsar_storage_write_latency_le_1", stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[1], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_5", + writeMetric(stream, "pulsar_storage_write_latency_le_5", stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[2], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_10", + writeMetric(stream, "pulsar_storage_write_latency_le_10", stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[3], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_20", + writeMetric(stream, "pulsar_storage_write_latency_le_20", stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[4], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_50", + writeMetric(stream, "pulsar_storage_write_latency_le_50", stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[5], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_100", + writeMetric(stream, "pulsar_storage_write_latency_le_100", stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[6], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_200", + writeMetric(stream, "pulsar_storage_write_latency_le_200", stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[7], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_1000", + writeMetric(stream, "pulsar_storage_write_latency_le_1000", stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[8], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_write_latency_overflow", + writeMetric(stream, "pulsar_storage_write_latency_overflow", stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[9], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_write_latency_count", + writeMetric(stream, "pulsar_storage_write_latency_count", stats.managedLedgerStats.storageWriteLatencyBuckets.getCount(), cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_write_latency_sum", + writeMetric(stream, "pulsar_storage_write_latency_sum", stats.managedLedgerStats.storageWriteLatencyBuckets.getSum(), cluster, namespace); stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.refresh(); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_0_5", + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_0_5", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[0], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_1", + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_1", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[1], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_5", + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_5", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[2], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_10", + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_10", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[3], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_20", + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_20", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[4], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_50", + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_50", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[5], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_100", + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_100", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[6], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_200", + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_200", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[7], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_1000", + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_1000", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[8], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_overflow", + writeMetric(stream, "pulsar_storage_ledger_write_latency_overflow", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[9], cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_count", + writeMetric(stream, "pulsar_storage_ledger_write_latency_count", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount(), cluster, namespace); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_sum", + writeMetric(stream, "pulsar_storage_ledger_write_latency_sum", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum(), cluster, namespace); stats.managedLedgerStats.entrySizeBuckets.refresh(); - writeMetric(allMetrics, "pulsar_entry_size_le_128", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[0], + writeMetric(stream, "pulsar_entry_size_le_128", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[0], cluster, namespace); - writeMetric(allMetrics, "pulsar_entry_size_le_512", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[1], + writeMetric(stream, "pulsar_entry_size_le_512", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[1], cluster, namespace); - writeMetric(allMetrics, "pulsar_entry_size_le_1_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[2], + writeMetric(stream, "pulsar_entry_size_le_1_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[2], cluster, namespace); - writeMetric(allMetrics, "pulsar_entry_size_le_2_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[3], + writeMetric(stream, "pulsar_entry_size_le_2_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[3], cluster, namespace); - writeMetric(allMetrics, "pulsar_entry_size_le_4_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[4], + writeMetric(stream, "pulsar_entry_size_le_4_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[4], cluster, namespace); - writeMetric(allMetrics, "pulsar_entry_size_le_16_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[5], + writeMetric(stream, "pulsar_entry_size_le_16_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[5], cluster, namespace); - writeMetric(allMetrics, "pulsar_entry_size_le_100_kb", + writeMetric(stream, "pulsar_entry_size_le_100_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[6], cluster, namespace); - writeMetric(allMetrics, "pulsar_entry_size_le_1_mb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[7], + writeMetric(stream, "pulsar_entry_size_le_1_mb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[7], cluster, namespace); - writeMetric(allMetrics, "pulsar_entry_size_le_overflow", + writeMetric(stream, "pulsar_entry_size_le_overflow", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[8], cluster, namespace); - writeMetric(allMetrics, "pulsar_entry_size_count", + writeMetric(stream, "pulsar_entry_size_count", stats.managedLedgerStats.entrySizeBuckets.getCount(), cluster, namespace); - writeMetric(allMetrics, "pulsar_entry_size_sum", + writeMetric(stream, "pulsar_entry_size_sum", stats.managedLedgerStats.entrySizeBuckets.getSum(), cluster, namespace); - writeReplicationStat(allMetrics, "pulsar_replication_rate_in", stats, + writeReplicationStat(stream, "pulsar_replication_rate_in", stats, replStats -> replStats.msgRateIn, cluster, namespace); - writeReplicationStat(allMetrics, "pulsar_replication_rate_out", stats, + writeReplicationStat(stream, "pulsar_replication_rate_out", stats, replStats -> replStats.msgRateOut, cluster, namespace); - writeReplicationStat(allMetrics, "pulsar_replication_throughput_in", stats, + writeReplicationStat(stream, "pulsar_replication_throughput_in", stats, replStats -> replStats.msgThroughputIn, cluster, namespace); - writeReplicationStat(allMetrics, "pulsar_replication_throughput_out", stats, + writeReplicationStat(stream, "pulsar_replication_throughput_out", stats, replStats -> replStats.msgThroughputOut, cluster, namespace); - writeReplicationStat(allMetrics, "pulsar_replication_backlog", stats, + writeReplicationStat(stream, "pulsar_replication_backlog", stats, replStats -> replStats.replicationBacklog, cluster, namespace); - writeReplicationStat(allMetrics, "pulsar_replication_connected_count", stats, + writeReplicationStat(stream, "pulsar_replication_connected_count", stats, replStats -> replStats.connectedCount, cluster, namespace); - writeReplicationStat(allMetrics, "pulsar_replication_rate_expired", stats, + writeReplicationStat(stream, "pulsar_replication_rate_expired", stats, replStats -> replStats.msgRateExpired, cluster, namespace); - writeReplicationStat(allMetrics, "pulsar_replication_delay_in_seconds", stats, + writeReplicationStat(stream, "pulsar_replication_delay_in_seconds", stats, replStats -> replStats.replicationDelayInSeconds, cluster, namespace); } - private static void writeMetricWithBrokerDefault(Map allMetrics, String metricName, Number value, - String cluster, String namespace) { - ByteBuf buffer = writeGaugeTypeWithBrokerDefault(allMetrics, metricName, cluster); - writeSample(buffer, metricName, value, "cluster", cluster, "namespace", namespace); - } - - private static void writePulsarMsgBacklog(Map allMetrics, Number value, + private static void writePulsarMsgBacklog(PrometheusMetricStreams stream, Number value, String cluster, String namespace) { - ByteBuf buffer = writeGaugeTypeWithBrokerDefault(allMetrics, "pulsar_msg_backlog", cluster); - writeSample(buffer, "pulsar_msg_backlog", value, "cluster", cluster, "namespace", namespace, "remote_cluster", + stream.writeSample("pulsar_msg_backlog", value, "cluster", cluster, "namespace", namespace, + "remote_cluster", "local"); } - private static void writeMetric(Map allMetrics, String metricName, Number value, String cluster, + private static void writeMetric(PrometheusMetricStreams stream, String metricName, Number value, + String cluster) { + stream.writeSample(metricName, value, "cluster", cluster); + } + + private static void writeMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, String namespace) { - ByteBuf buffer = writeGaugeType(allMetrics, metricName); - writeSample(buffer, metricName, value, "cluster", cluster, "namespace", namespace); + stream.writeSample(metricName, value, "cluster", cluster, "namespace", namespace); } - private static void writeReplicationStat(Map allMetrics, String metricName, + private static void writeReplicationStat(PrometheusMetricStreams stream, String metricName, AggregatedNamespaceStats namespaceStats, - Function replStatsFunction, + Function sampleValueFunction, String cluster, String namespace) { if (!namespaceStats.replicationStats.isEmpty()) { - ByteBuf buffer = writeGaugeType(allMetrics, metricName); namespaceStats.replicationStats.forEach((remoteCluster, replStats) -> - writeSample(buffer, metricName, replStatsFunction.apply(replStats), + stream.writeSample(metricName, sampleValueFunction.apply(replStats), "cluster", cluster, "namespace", namespace, "remote_cluster", remoteCluster) @@ -449,92 +459,5 @@ private static void writeReplicationStat(Map allMetrics, String } } - static ByteBuf writeGaugeType(Map allMetrics, String metricName) { - if (!allMetrics.containsKey(metricName)) { - ByteBuf buffer = UnpooledByteBufAllocator.DEFAULT.compositeDirectBuffer(MAX_COMPONENTS); - write(buffer, "# TYPE "); - write(buffer, metricName); - write(buffer, " gauge\n"); - allMetrics.put(metricName, buffer); - } - return allMetrics.get(metricName); - } - static ByteBuf writeGaugeTypeWithBrokerDefault(Map allMetrics, String metricName, String cluster) { - if (!allMetrics.containsKey(metricName)) { - ByteBuf buffer = writeGaugeType(allMetrics, metricName); - writeSample(buffer, metricName, 0, "cluster", cluster); - } - return allMetrics.get(metricName); - } - - static void writeSample(ByteBuf buffer, String metricName, Number value, String... labelsAndValuesArray) { - write(buffer, metricName); - write(buffer, '{'); - for (int i = 0; i < labelsAndValuesArray.length; i += 2) { - write(buffer, labelsAndValuesArray[i]); - write(buffer, "=\""); - write(buffer, labelsAndValuesArray[i + 1]); - write(buffer, '\"'); - if (labelsAndValuesArray.length != i + 2) { - write(buffer, ','); - } - } - write(buffer, "\"} "); - write(buffer, value); - write(buffer, ' '); - write(buffer, System.currentTimeMillis()); - write(buffer, '\n'); - } - - private static void write(ByteBuf buffer, String s) { - if (s == null) { - return; - } - int len = s.length(); - for (int i = 0; i < len; i++) { - write(buffer, s.charAt(i)); - } - } - - private static void write(ByteBuf buffer, Number n) { - if (n instanceof Integer) { - write(buffer, n.intValue()); - } else if (n instanceof Long) { - write(buffer, n.longValue()); - } else if (n instanceof Double) { - write(buffer, n.doubleValue()); - } else { - write(buffer, n.toString()); - } - } - - private static void write(ByteBuf buffer, long n) { - NumberFormat.format(buffer, n); - } - - private static void write(ByteBuf buffer, double d) { - long i = (long) d; - write(buffer, i); - - long r = Math.abs((long) (1000 * (d - i))); - write(buffer, '.'); - if (r == 0) { - write(buffer, '0'); - } - - if (r < 100) { - write(buffer, '0'); - } - - if (r < 10) { - write(buffer, '0'); - } - - write(buffer, r); - } - - private static void write(ByteBuf buffer, char c) { - buffer.writeByte((byte) c); - } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java index 6b6b972c175f0..7766031886b1c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java @@ -18,15 +18,13 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import io.netty.buffer.ByteBufAllocator; import java.util.HashMap; import java.util.Map; -import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.util.SimpleTextOutputStream; /** * Helper class to ensure that metrics of the same name are grouped together under the same TYPE header when written. - * Those are the requirements of the - * Prometheus Exposition Format. */ public class PrometheusMetricStreams { private final Map metricStreamMap = new HashMap<>(); @@ -46,7 +44,7 @@ void writeSample(String metricName, Number value, String... labelsAndValuesArray stream.write(','); } } - stream.write("} ").write(value).write(' ').write(System.currentTimeMillis()).write('\n'); + stream.write("\"} ").write(value).write(' ').write(System.currentTimeMillis()).write('\n'); } /** @@ -55,6 +53,7 @@ void writeSample(String metricName, Number value, String... labelsAndValuesArray */ void flushAllToStream(SimpleTextOutputStream stream) { metricStreamMap.values().forEach(s -> stream.write(s.getBuffer())); + releaseAll(); } /** @@ -67,7 +66,7 @@ void releaseAll() { private SimpleTextOutputStream initGaugeType(String metricName) { return metricStreamMap.computeIfAbsent(metricName, s -> { - SimpleTextOutputStream stream = new SimpleTextOutputStream(PulsarByteBufAllocator.DEFAULT.directBuffer()); + SimpleTextOutputStream stream = new SimpleTextOutputStream(ByteBufAllocator.DEFAULT.directBuffer()); stream.write("# TYPE ").write(metricName).write(" gauge\n"); return stream; }); 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 437c1ea12a8ee..1a8d9f6a10854 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 @@ -52,11 +52,12 @@ /** * Generate metrics aggregated at the namespace level and optionally at a topic level and formats them out * in a text format suitable to be consumed by Prometheus. - * Format specification can be found at {@link https://prometheus.io/docs/instrumenting/exposition_formats/} + * Format specification can be found at Exposition Formats */ public class PrometheusMetricsGenerator { private static volatile TimeWindow timeWindow; - protected static final int MAX_COMPONENTS = 64; + private static final int MAX_COMPONENTS = 64; static { DefaultExports.initialize(); @@ -88,38 +89,110 @@ public double get() { } public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, OutputStream out) throws IOException { + boolean includeProducerMetrics, OutputStream out) throws IOException { generate(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, false, out, null); } public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, - OutputStream out) throws IOException { + boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, + OutputStream out) throws IOException { generate(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, splitTopicAndPartitionIndexLabel, out, null); } - public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, OutputStream out, - List metricsProviders) - throws IOException { - ByteBuf buf = ByteBufAllocator.DEFAULT.heapBuffer(); + public static synchronized void generate(PulsarService pulsar, boolean includeTopicMetrics, + boolean includeConsumerMetrics, boolean includeProducerMetrics, + boolean splitTopicAndPartitionIndexLabel, OutputStream out, + List metricsProviders) throws IOException { + ByteBuf buffer; + boolean exposeBufferMetrics = pulsar.getConfiguration().isMetricsBufferResponse(); + + if (!exposeBufferMetrics) { + buffer = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, + splitTopicAndPartitionIndexLabel, metricsProviders); + } else { + if (null == timeWindow) { + int period = pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds(); + timeWindow = new TimeWindow<>(1, (int) TimeUnit.SECONDS.toMillis(period)); + } + WindowWrap window = timeWindow.current(oldBuf -> { + // release expired buffer, in case of memory leak + if (oldBuf != null && oldBuf.refCnt() > 0) { + oldBuf.release(); + log.debug("Cached metrics buffer released"); + } + + try { + ByteBuf buf = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, + splitTopicAndPartitionIndexLabel, metricsProviders); + log.debug("Generated metrics buffer size {}", buf.readableBytes()); + return buf; + } catch (IOException e) { + log.error("Generate metrics failed", e); + //return empty buffer if exception happens + return ByteBufAllocator.DEFAULT.heapBuffer(0); + } + }); + + if (null == window || null == window.value()) { + return; + } + buffer = window.value(); + log.debug("Current window start {}, current cached buf size {}", window.start(), buffer.readableBytes()); + } + + try { + if (out instanceof HttpOutput) { + HttpOutput output = (HttpOutput) out; + //no mem_copy and memory allocations here + ByteBuffer[] buffers = buffer.nioBuffers(); + for (ByteBuffer buffer0 : buffers) { + output.write(buffer0); + } + } else { + //read data from buffer and write it to output stream, with no more heap buffer(byte[]) allocation. + //not modify buffer readIndex/writeIndex here. + int readIndex = buffer.readerIndex(); + int readableBytes = buffer.readableBytes(); + for (int i = 0; i < readableBytes; i++) { + out.write(buffer.getByte(readIndex + i)); + } + } + } finally { + if (!exposeBufferMetrics && buffer.refCnt() > 0) { + buffer.release(); + log.debug("Metrics buffer released."); + } + } + } + + private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, + boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, + List metricsProviders) throws IOException { + //Use unpooled buffers here to avoid direct buffer usage increasing. + //when write out 200MB data, MAX_COMPONENTS = 64 needn't mem_copy. see: CompositeByteBuf#consolidateIfNeeded() + ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.compositeDirectBuffer(MAX_COMPONENTS); + boolean exceptionHappens = false; + //Used in namespace/topic and transaction aggregators as share metric names + PrometheusMetricStreams metricStream = new PrometheusMetricStreams(); try { SimpleTextOutputStream stream = new SimpleTextOutputStream(buf); generateSystemMetrics(stream, pulsar.getConfiguration().getClusterName()); NamespaceStatsAggregator.generate(pulsar, includeTopicMetrics, includeConsumerMetrics, - includeProducerMetrics, splitTopicAndPartitionIndexLabel, stream); + includeProducerMetrics, splitTopicAndPartitionIndexLabel, metricStream); if (pulsar.getWorkerServiceOpt().isPresent()) { pulsar.getWorkerService().generateFunctionsStats(stream); } if (pulsar.getConfiguration().isTransactionCoordinatorEnabled()) { - TransactionAggregator.generate(pulsar, stream, includeTopicMetrics); + TransactionAggregator.generate(pulsar, metricStream, includeTopicMetrics); } + metricStream.flushAllToStream(stream); + generateBrokerBasicMetrics(pulsar, stream); generateManagedLedgerBookieClientMetrics(pulsar, stream); @@ -131,7 +204,11 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b } out.write(buf.array(), buf.arrayOffset(), buf.readableBytes()); } finally { - buf.release(); + //if exception happens, release buffer + if (exceptionHappens) { + buf.release(); + metricStream.releaseAll(); + } } } @@ -144,17 +221,17 @@ private static void generateBrokerBasicMetrics(PulsarService pulsar, SimpleTextO if (pulsar.getConfiguration().isExposeManagedLedgerMetricsInPrometheus()) { // generate managedLedger metrics parseMetricsToPrometheusMetrics(new ManagedLedgerMetrics(pulsar).generate(), - clusterName, Collector.Type.GAUGE, stream); + clusterName, Collector.Type.GAUGE, stream); } if (pulsar.getConfiguration().isExposeManagedCursorMetricsInPrometheus()) { // generate managedCursor metrics parseMetricsToPrometheusMetrics(new ManagedCursorMetrics(pulsar).generate(), - clusterName, Collector.Type.GAUGE, stream); + clusterName, Collector.Type.GAUGE, stream); } parseMetricsToPrometheusMetrics(Collections.singletonList(pulsar.getBrokerService() - .getPulsarStats().getBrokerOperabilityMetrics().generateConnectionMetrics()), + .getPulsarStats().getBrokerOperabilityMetrics().generateConnectionMetrics()), clusterName, Collector.Type.GAUGE, stream); // generate loadBalance metrics 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 3c515f71fd8a4..b50a349bba381 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 @@ -18,18 +18,13 @@ */ package org.apache.pulsar.broker.stats.prometheus; -import static org.apache.pulsar.broker.stats.prometheus.NamespaceStatsAggregator.writeGaugeType; -import static org.apache.pulsar.broker.stats.prometheus.NamespaceStatsAggregator.writeGaugeTypeWithBrokerDefault; -import static org.apache.pulsar.broker.stats.prometheus.NamespaceStatsAggregator.writeSample; import static org.apache.pulsar.common.naming.TopicName.PARTITIONED_TOPIC_SUFFIX; -import io.netty.buffer.ByteBuf; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.function.Function; import org.apache.bookkeeper.mledger.util.StatsBuckets; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.compaction.CompactionRecord; import org.apache.pulsar.compaction.CompactorMXBean; @@ -105,397 +100,336 @@ public void reset() { compactionLatencyBuckets.reset(); } - public static void printTopicStats(Map allMetrics, TopicStats stats, + public static void printTopicStats(PrometheusMetricStreams stream, TopicStats stats, Optional compactorMXBean, String cluster, String namespace, - String name, boolean splitTopicAndPartitionIndexLabel) { - writeMetricWithBrokerDefault(allMetrics, "pulsar_subscriptions_count", stats.subscriptionsCount, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(allMetrics, "pulsar_producers_count", stats.producersCount, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(allMetrics, "pulsar_consumers_count", stats.consumersCount, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - writeMetricWithBrokerDefault(allMetrics, "pulsar_rate_in", stats.rateIn, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(allMetrics, "pulsar_rate_out", stats.rateOut, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(allMetrics, "pulsar_throughput_in", stats.throughputIn, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(allMetrics, "pulsar_throughput_out", stats.throughputOut, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_average_msg_size", stats.averageMsgSize, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_size", stats.managedLedgerStats.storageSize, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(allMetrics, "pulsar_storage_logical_size", - stats.managedLedgerStats.storageLogicalSize, cluster, namespace, name, + String topic, boolean splitTopicAndPartitionIndexLabel) { + writeMetric(stream, "pulsar_subscriptions_count", stats.subscriptionsCount, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_producers_count", stats.producersCount, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_consumers_count", stats.consumersCount, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + + writeMetric(stream, "pulsar_rate_in", stats.rateIn, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_rate_out", stats.rateOut, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_throughput_in", stats.throughputIn, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_throughput_out", stats.throughputOut, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_average_msg_size", stats.averageMsgSize, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + + writeMetric(stream, "pulsar_storage_size", stats.managedLedgerStats.storageSize, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_logical_size", + stats.managedLedgerStats.storageLogicalSize, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeMetricWithBrokerDefault(allMetrics, "pulsar_msg_backlog", stats.msgBacklog, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_publish_rate_limit_times", stats.publishRateLimitedTimes, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_offloaded_size", stats.managedLedgerStats - .offloadedStorageUsed, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_backlog_quota_limit", stats.backlogQuotaLimit, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_backlog_quota_limit_time", stats.backlogQuotaLimitTime, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - writeMetric(allMetrics, "pulsar_storage_write_latency_le_0_5", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[0], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_1", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[1], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_5", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[2], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_10", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[3], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_20", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[4], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_50", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[5], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_100", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[6], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_200", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[7], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_write_latency_le_1000", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[8], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_write_latency_overflow", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[9], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_write_latency_count", + writeMetric(stream, "pulsar_msg_backlog", stats.msgBacklog, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_publish_rate_limit_times", stats.publishRateLimitedTimes, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_offloaded_size", stats.managedLedgerStats + .offloadedStorageUsed, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_backlog_quota_limit", stats.backlogQuotaLimit, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_backlog_quota_limit_time", stats.backlogQuotaLimitTime, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + + long[] latencyBuckets = stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets(); + writeMetric(stream, "pulsar_storage_write_latency_le_0_5", + latencyBuckets[0], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_latency_le_1", + latencyBuckets[1], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_latency_le_5", + latencyBuckets[2], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_latency_le_10", + latencyBuckets[3], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_latency_le_20", + latencyBuckets[4], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_latency_le_50", + latencyBuckets[5], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_latency_le_100", + latencyBuckets[6], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_latency_le_200", + latencyBuckets[7], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_latency_le_1000", + latencyBuckets[8], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_latency_overflow", + latencyBuckets[9], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_latency_count", stats.managedLedgerStats.storageWriteLatencyBuckets.getCount(), - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_write_latency_sum", - stats.managedLedgerStats.storageWriteLatencyBuckets.getSum(), cluster, namespace, name, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_latency_sum", + stats.managedLedgerStats.storageWriteLatencyBuckets.getSum(), cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_0_5", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[0], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_1", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[1], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_5", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[2], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_10", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[3], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_20", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[4], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_50", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[5], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_100", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[6], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_200", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[7], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_le_1000", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[8], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_overflow", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[9], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_count", + long[] ledgerWriteLatencyBuckets = stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets(); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_0_5", + ledgerWriteLatencyBuckets[0], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_1", + ledgerWriteLatencyBuckets[1], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_5", + ledgerWriteLatencyBuckets[2], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_10", + ledgerWriteLatencyBuckets[3], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_20", + ledgerWriteLatencyBuckets[4], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_50", + ledgerWriteLatencyBuckets[5], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_100", + ledgerWriteLatencyBuckets[6], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_200", + ledgerWriteLatencyBuckets[7], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_1000", + ledgerWriteLatencyBuckets[8], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_ledger_write_latency_overflow", + ledgerWriteLatencyBuckets[9], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_ledger_write_latency_count", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount(), - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_storage_ledger_write_latency_sum", + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_ledger_write_latency_sum", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum(), - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - writeMetric(allMetrics, "pulsar_entry_size_le_128", - stats.managedLedgerStats.entrySizeBuckets.getBuckets()[0], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_entry_size_le_512", - stats.managedLedgerStats.entrySizeBuckets.getBuckets()[1], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_entry_size_le_1_kb", - stats.managedLedgerStats.entrySizeBuckets.getBuckets()[2], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_entry_size_le_2_kb", - stats.managedLedgerStats.entrySizeBuckets.getBuckets()[3], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_entry_size_le_4_kb", - stats.managedLedgerStats.entrySizeBuckets.getBuckets()[4], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_entry_size_le_16_kb", - stats.managedLedgerStats.entrySizeBuckets.getBuckets()[5], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_entry_size_le_100_kb", - stats.managedLedgerStats.entrySizeBuckets.getBuckets()[6], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_entry_size_le_1_mb", - stats.managedLedgerStats.entrySizeBuckets.getBuckets()[7], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_entry_size_le_overflow", - stats.managedLedgerStats.entrySizeBuckets.getBuckets()[8], - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_entry_size_count", stats.managedLedgerStats.entrySizeBuckets.getCount(), - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_entry_size_sum", stats.managedLedgerStats.entrySizeBuckets.getSum(), - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - writeProducerStat(allMetrics, "pulsar_producer_msg_rate_in", stats, - p -> p.msgRateIn, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeProducerStat(allMetrics, "pulsar_producer_msg_throughput_in", stats, - p -> p.msgThroughputIn, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeProducerStat(allMetrics, "pulsar_producer_msg_average_Size", stats, - p -> p.averageMsgSize, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - - writeSubscriptionStat(allMetrics, "pulsar_subscription_back_log", stats, s -> s.msgBacklog, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_back_log_no_delayed", - stats, s -> s.msgBacklogNoDelayed, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_delayed", - stats, s -> s.msgDelayed, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_msg_rate_redeliver", - stats, s -> s.msgRateRedeliver, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_unacked_messages", - stats, s -> s.unackedMessages, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_blocked_on_unacked_messages", - stats, s -> s.blockedSubscriptionOnUnackedMsgs ? 1 : 0, cluster, namespace, name, - splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_msg_rate_out", - stats, s -> s.msgRateOut, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_msg_ack_rate", - stats, s -> s.messageAckRate, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_msg_throughput_out", - stats, s -> s.msgThroughputOut, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_out_bytes_total", - stats, s -> s.bytesOutCounter, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_out_messages_total", - stats, s -> s.msgOutCounter, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_last_expire_timestamp", - stats, s -> s.lastExpireTimestamp, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_last_acked_timestamp", - stats, s -> s.lastAckedTimestamp, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_last_consumed_flow_timestamp", - stats, s -> s.lastConsumedFlowTimestamp, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_last_consumed_timestamp", - stats, s -> s.lastConsumedTimestamp, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_last_mark_delete_advanced_timestamp", - stats, s -> s.lastMarkDeleteAdvancedTimestamp, cluster, namespace, name, - splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_msg_rate_expired", - stats, s -> s.msgRateExpired, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeSubscriptionStat(allMetrics, "pulsar_subscription_total_msg_expired", - stats, s -> s.totalMsgExpired, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - writeConsumerStat(allMetrics, "pulsar_consumer_msg_rate_redeliver", stats, c -> c.msgRateRedeliver, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeConsumerStat(allMetrics, "pulsar_consumer_unacked_messages", stats, c -> c.unackedMessages, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeConsumerStat(allMetrics, "pulsar_consumer_blocked_on_unacked_messages", - stats, c -> c.blockedSubscriptionOnUnackedMsgs ? 1 : 0, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeConsumerStat(allMetrics, "pulsar_consumer_msg_rate_out", stats, c -> c.msgRateOut, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - writeConsumerStat(allMetrics, "pulsar_consumer_msg_ack_rate", stats, c -> c.msgAckRate, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - writeConsumerStat(allMetrics, "pulsar_consumer_msg_throughput_out", stats, c -> c.msgThroughputOut, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeConsumerStat(allMetrics, "pulsar_consumer_available_permits", stats, c -> c.availablePermits, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeConsumerStat(allMetrics, "pulsar_out_bytes_total", stats, c -> c.bytesOutCounter, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeConsumerStat(allMetrics, "pulsar_out_messages_total", stats, c -> c.msgOutCounter, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - writeReplicationStat(allMetrics, "pulsar_replication_rate_in", stats, r -> r.msgRateIn, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeReplicationStat(allMetrics, "pulsar_replication_rate_out", stats, r -> r.msgRateOut, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeReplicationStat(allMetrics, "pulsar_replication_throughput_in", stats, r -> r.msgThroughputIn, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeReplicationStat(allMetrics, "pulsar_replication_throughput_out", stats, r -> r.msgThroughputOut, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeReplicationStat(allMetrics, "pulsar_replication_backlog", stats, r -> r.replicationBacklog, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeReplicationStat(allMetrics, "pulsar_replication_connected_count", stats, r -> r.connectedCount, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeReplicationStat(allMetrics, "pulsar_replication_rate_expired", stats, r -> r.msgRateExpired, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeReplicationStat(allMetrics, "pulsar_replication_delay_in_seconds", stats, - r -> r.replicationDelayInSeconds, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - - writeMetric(allMetrics, "pulsar_in_bytes_total", stats.bytesInCounter, cluster, namespace, name, - splitTopicAndPartitionIndexLabel); - writeMetric(allMetrics, "pulsar_in_messages_total", stats.msgInCounter, cluster, namespace, name, - splitTopicAndPartitionIndexLabel); + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - // Compaction - - writeCompactionStat(allMetrics, "pulsar_compaction_removed_event_count", stats.compactionRemovedEventCount, - compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_succeed_count", stats.compactionSucceedCount, - compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_failed_count", stats.compactionFailedCount, compactorMXBean, - cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_duration_time_in_mills", stats.compactionDurationTimeInMills, - compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_read_throughput", stats.compactionReadThroughput, - compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_write_throughput", stats.compactionWriteThroughput, - compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_compacted_entries_count", - stats.compactionCompactedEntriesCount, compactorMXBean, cluster, namespace, name, - splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_compacted_entries_size", - stats.compactionCompactedEntriesSize, compactorMXBean, cluster, namespace, name, + long[] entrySizeBuckets = stats.managedLedgerStats.entrySizeBuckets.getBuckets(); + writeMetric(stream, "pulsar_entry_size_le_128", entrySizeBuckets[0], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_0_5", - stats.compactionLatencyBuckets.getBuckets()[0], compactorMXBean, cluster, namespace, name, + writeMetric(stream, "pulsar_entry_size_le_512", entrySizeBuckets[1], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_1", - stats.compactionLatencyBuckets.getBuckets()[1], compactorMXBean, cluster, namespace, name, + writeMetric(stream, "pulsar_entry_size_le_1_kb", entrySizeBuckets[2], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_5", - stats.compactionLatencyBuckets.getBuckets()[2], compactorMXBean, cluster, namespace, name, + writeMetric(stream, "pulsar_entry_size_le_2_kb", entrySizeBuckets[3], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_10", - stats.compactionLatencyBuckets.getBuckets()[3], compactorMXBean, cluster, namespace, name, + writeMetric(stream, "pulsar_entry_size_le_4_kb", entrySizeBuckets[4], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_20", - stats.compactionLatencyBuckets.getBuckets()[4], compactorMXBean, cluster, namespace, name, + writeMetric(stream, "pulsar_entry_size_le_16_kb", entrySizeBuckets[5], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_50", - stats.compactionLatencyBuckets.getBuckets()[5], compactorMXBean, cluster, namespace, name, + writeMetric(stream, "pulsar_entry_size_le_100_kb", entrySizeBuckets[6], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_100", - stats.compactionLatencyBuckets.getBuckets()[6], compactorMXBean, cluster, namespace, name, + writeMetric(stream, "pulsar_entry_size_le_1_mb", entrySizeBuckets[7], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_200", - stats.compactionLatencyBuckets.getBuckets()[7], compactorMXBean, cluster, namespace, name, + writeMetric(stream, "pulsar_entry_size_le_overflow", entrySizeBuckets[8], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_le_1000", - stats.compactionLatencyBuckets.getBuckets()[8], compactorMXBean, cluster, namespace, name, + writeMetric(stream, "pulsar_entry_size_count", stats.managedLedgerStats.entrySizeBuckets.getCount(), + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_entry_size_sum", stats.managedLedgerStats.entrySizeBuckets.getSum(), + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + + stats.producerStats.forEach((p, producerStats) -> { + writeProducerMetric(stream, "pulsar_producer_msg_rate_in", producerStats.msgRateIn, + cluster, namespace, topic, p, producerStats.producerId, splitTopicAndPartitionIndexLabel); + writeProducerMetric(stream, "pulsar_producer_msg_throughput_in", producerStats.msgThroughputIn, + cluster, namespace, topic, p, producerStats.producerId, splitTopicAndPartitionIndexLabel); + writeProducerMetric(stream, "pulsar_producer_msg_average_Size", producerStats.averageMsgSize, + cluster, namespace, topic, p, producerStats.producerId, splitTopicAndPartitionIndexLabel); + }); + + stats.subscriptionStats.forEach((sub, subsStats) -> { + writeSubscriptionMetric(stream, "pulsar_subscription_back_log", subsStats.msgBacklog, + cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_back_log_no_delayed", + subsStats.msgBacklogNoDelayed, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_delayed", + subsStats.msgDelayed, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_msg_rate_redeliver", + subsStats.msgRateRedeliver, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_unacked_messages", + subsStats.unackedMessages, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_blocked_on_unacked_messages", + subsStats.blockedSubscriptionOnUnackedMsgs ? 1 : 0, cluster, namespace, topic, sub, + splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_msg_rate_out", + subsStats.msgRateOut, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_msg_ack_rate", + subsStats.messageAckRate, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_msg_throughput_out", + subsStats.msgThroughputOut, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_out_bytes_total", + subsStats.bytesOutCounter, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_out_messages_total", + subsStats.msgOutCounter, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_last_expire_timestamp", + subsStats.lastExpireTimestamp, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_last_acked_timestamp", + subsStats.lastAckedTimestamp, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_last_consumed_flow_timestamp", + subsStats.lastConsumedFlowTimestamp, cluster, namespace, topic, sub, + splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_last_consumed_timestamp", + subsStats.lastConsumedTimestamp, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_last_mark_delete_advanced_timestamp", + subsStats.lastMarkDeleteAdvancedTimestamp, cluster, namespace, topic, sub, + splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_msg_rate_expired", + subsStats.msgRateExpired, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_total_msg_expired", + subsStats.totalMsgExpired, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + + subsStats.consumerStat.forEach((c, consumerStats) -> { + writeConsumerMetric(stream, "pulsar_consumer_msg_rate_redeliver", consumerStats.msgRateRedeliver, + cluster, namespace, topic, sub, c, splitTopicAndPartitionIndexLabel); + writeConsumerMetric(stream, "pulsar_consumer_unacked_messages", consumerStats.unackedMessages, + cluster, namespace, topic, sub, c, splitTopicAndPartitionIndexLabel); + writeConsumerMetric(stream, "pulsar_consumer_blocked_on_unacked_messages", + consumerStats.blockedSubscriptionOnUnackedMsgs ? 1 : 0, + cluster, namespace, topic, sub, c, splitTopicAndPartitionIndexLabel); + writeConsumerMetric(stream, "pulsar_consumer_msg_rate_out", consumerStats.msgRateOut, + cluster, namespace, topic, sub, c, splitTopicAndPartitionIndexLabel); + + writeConsumerMetric(stream, "pulsar_consumer_msg_ack_rate", consumerStats.msgAckRate, + cluster, namespace, topic, sub, c, splitTopicAndPartitionIndexLabel); + + writeConsumerMetric(stream, "pulsar_consumer_msg_throughput_out", consumerStats.msgThroughputOut, + cluster, namespace, topic, sub, c, splitTopicAndPartitionIndexLabel); + writeConsumerMetric(stream, "pulsar_consumer_available_permits", consumerStats.availablePermits, + cluster, namespace, topic, sub, c, splitTopicAndPartitionIndexLabel); + writeConsumerMetric(stream, "pulsar_out_bytes_total", consumerStats.bytesOutCounter, + cluster, namespace, topic, sub, c, splitTopicAndPartitionIndexLabel); + writeConsumerMetric(stream, "pulsar_out_messages_total", consumerStats.msgOutCounter, + cluster, namespace, topic, sub, c, splitTopicAndPartitionIndexLabel); + }); + }); + + if (!stats.replicationStats.isEmpty()) { + stats.replicationStats.forEach((remoteCluster, replStats) -> { + writeMetric(stream, "pulsar_replication_rate_in", replStats.msgRateIn, + cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_replication_rate_out", replStats.msgRateOut, + cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_replication_throughput_in", replStats.msgThroughputIn, + cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_replication_throughput_out", replStats.msgThroughputOut, + cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_replication_backlog", replStats.replicationBacklog, + cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_replication_connected_count", replStats.connectedCount, + cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_replication_rate_expired", replStats.msgRateExpired, + cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_replication_delay_in_seconds", replStats.replicationDelayInSeconds, + cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); + }); + } + + writeMetric(stream, "pulsar_in_bytes_total", stats.bytesInCounter, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_overflow", - stats.compactionLatencyBuckets.getBuckets()[9], compactorMXBean, cluster, namespace, name, + writeMetric(stream, "pulsar_in_messages_total", stats.msgInCounter, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_sum", stats.compactionLatencyBuckets.getSum(), - compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - writeCompactionStat(allMetrics, "pulsar_compaction_latency_count", stats.compactionLatencyBuckets.getCount(), - compactorMXBean, cluster, namespace, name, splitTopicAndPartitionIndexLabel); - } - private static void writeMetric(Map allMetrics, String metricName, Number value, String cluster, - String namespace, String topic, boolean splitTopicAndPartitionIndexLabel) { - ByteBuf buffer = writeGaugeType(allMetrics, metricName); - writeTopicSample(buffer, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + // Compaction + boolean hasCompaction = compactorMXBean.flatMap(mxBean -> mxBean.getCompactionRecordForTopic(topic)) + .isPresent(); + if (hasCompaction) { + writeMetric(stream, "pulsar_compaction_removed_event_count", + stats.compactionRemovedEventCount, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_succeed_count", + stats.compactionSucceedCount, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_failed_count", + stats.compactionFailedCount, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_duration_time_in_mills", + stats.compactionDurationTimeInMills, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_read_throughput", + stats.compactionReadThroughput, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_write_throughput", + stats.compactionWriteThroughput, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_compacted_entries_count", + stats.compactionCompactedEntriesCount, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_compacted_entries_size", + stats.compactionCompactedEntriesSize, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + + long[] compactionBuckets = stats.compactionLatencyBuckets.getBuckets(); + writeMetric(stream, "pulsar_compaction_latency_le_0_5", + compactionBuckets[0], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_latency_le_1", + compactionBuckets[1], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_latency_le_5", + compactionBuckets[2], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_latency_le_10", + compactionBuckets[3], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_latency_le_20", + compactionBuckets[4], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_latency_le_50", + compactionBuckets[5], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_latency_le_100", + compactionBuckets[6], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_latency_le_200", + compactionBuckets[7], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_latency_le_1000", + compactionBuckets[8], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_latency_overflow", + compactionBuckets[9], cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_latency_sum", + stats.compactionLatencyBuckets.getSum(), cluster, namespace, topic, + splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_compaction_latency_count", + stats.compactionLatencyBuckets.getCount(), cluster, namespace, topic, + splitTopicAndPartitionIndexLabel); + } } - private static void writeProducerStat(Map allMetrics, String metricName, TopicStats topicStats, - Function valueFunction, - String cluster, String namespace, String topic, - boolean splitTopicAndPartitionIndexLabel) { - ByteBuf buffer = writeGaugeType(allMetrics, metricName); - topicStats.producerStats.forEach((p, producerStats) -> - writeTopicSample(buffer, metricName, valueFunction.apply(producerStats), cluster, namespace, topic, - splitTopicAndPartitionIndexLabel, "producer_name", p, "producer_id", - String.valueOf(producerStats.producerId))); + private static void writeMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, + String namespace, String topic, boolean splitTopicAndPartitionIndexLabel) { + writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); } - private static void writeSubscriptionStat(Map allMetrics, String metricName, TopicStats topicStats, - Function valueFunction, - String cluster, String namespace, String topic, - boolean splitTopicAndPartitionIndexLabel) { - ByteBuf buffer = writeGaugeType(allMetrics, metricName); - topicStats.subscriptionStats.forEach((s, subStats) -> - writeTopicSample(buffer, metricName, valueFunction.apply(subStats), cluster, namespace, topic, - splitTopicAndPartitionIndexLabel, "subscription", s)); + private static void writeMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, + String namespace, String topic, String remoteCluster, + boolean splitTopicAndPartitionIndexLabel) { + writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel, + "remote_cluster", remoteCluster); } - private static void writeConsumerStat(Map allMetrics, String metricName, TopicStats topicStats, - Function valueFunction, - String cluster, String namespace, String topic, - boolean splitTopicAndPartitionIndexLabel) { - ByteBuf buffer = writeGaugeType(allMetrics, metricName); - topicStats.subscriptionStats.forEach((s, subStats) -> - subStats.consumerStat.forEach((c, conStats) -> - writeTopicSample(buffer, metricName, valueFunction.apply(conStats), cluster, namespace, topic, - splitTopicAndPartitionIndexLabel, "subscription", s, "consumer_name", c.consumerName(), - "consumer_id", String.valueOf(c.consumerId())) - )); + private static void writeProducerMetric(PrometheusMetricStreams stream, String metricName, Number value, + String cluster, String namespace, String topic, String producer, + long producerId, boolean splitTopicAndPartitionIndexLabel) { + writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel, + "producer_name", producer, "producer_id", String.valueOf(producerId)); } - private static void writeReplicationStat(Map allMetrics, String metricName, TopicStats topicStats, - Function valueFunction, - String cluster, String namespace, String topic, - boolean splitTopicAndPartitionIndexLabel) { - if (!topicStats.replicationStats.isEmpty()) { - ByteBuf buffer = writeGaugeType(allMetrics, metricName); - topicStats.replicationStats.forEach((remoteCluster, replStats) -> - writeTopicSample(buffer, metricName, valueFunction.apply(replStats), cluster, namespace, topic, - splitTopicAndPartitionIndexLabel, "remote_cluster", remoteCluster) - ); - } - } - - private static void writeCompactionStat(Map allMetrics, String metricName, - Number value, Optional compactorMXBean, - String cluster, String namespace, String topic, - boolean splitTopicAndPartitionIndexLabel) { - boolean hasCompaction = compactorMXBean.flatMap(mxBean -> mxBean.getCompactionRecordForTopic(topic)) - .isPresent(); - if (hasCompaction) { - ByteBuf buffer = writeGaugeType(allMetrics, metricName); - writeTopicSample(buffer, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); - } + private static void writeSubscriptionMetric(PrometheusMetricStreams stream, String metricName, Number value, + String cluster, String namespace, String topic, String subscription, + boolean splitTopicAndPartitionIndexLabel) { + writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel, + "subscription", subscription); } - static void writeMetricWithBrokerDefault(Map allMetrics, String metricName, Number value, - String cluster, String namespace, String topic, - boolean splitTopicAndPartitionIndexLabel) { - ByteBuf buffer = writeGaugeTypeWithBrokerDefault(allMetrics, metricName, cluster); - writeTopicSample(buffer, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + private static void writeConsumerMetric(PrometheusMetricStreams stream, String metricName, Number value, + String cluster, String namespace, String topic, String subscription, + Consumer consumer, boolean splitTopicAndPartitionIndexLabel) { + writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel, + "subscription", subscription, "consumer_name", consumer.consumerName(), + "consumer_id", String.valueOf(consumer.consumerId())); } - static void writeTopicSample(ByteBuf buffer, String metricName, Number value, String cluster, + static void writeTopicMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel, String... extraLabelsAndValues) { - List labelsAndValues = new ArrayList<>(); - labelsAndValues.add("cluster"); - labelsAndValues.add(cluster); - labelsAndValues.add("namespace"); - labelsAndValues.add(namespace); - labelsAndValues.add("topic"); + String[] labelsAndValues = new String[splitTopicAndPartitionIndexLabel ? 8 : 6]; + labelsAndValues[0] = "cluster"; + labelsAndValues[1] = cluster; + labelsAndValues[2] = "namespace"; + labelsAndValues[3] = namespace; + labelsAndValues[4] = "topic"; if (splitTopicAndPartitionIndexLabel) { int index = topic.indexOf(PARTITIONED_TOPIC_SUFFIX); if (index > 0) { - labelsAndValues.add(topic.substring(0, index)); - labelsAndValues.add("partition"); - labelsAndValues.add(topic.substring(index + PARTITIONED_TOPIC_SUFFIX.length())); + labelsAndValues[5] = topic.substring(0, index); + labelsAndValues[6] = "partition"; + labelsAndValues[7] = topic.substring(index + PARTITIONED_TOPIC_SUFFIX.length()); } else { - labelsAndValues.add(topic); - labelsAndValues.add("partition"); - labelsAndValues.add("-1"); + labelsAndValues[5] = topic; + labelsAndValues[6] = "partition"; + labelsAndValues[7] = "-1"; } } else { - labelsAndValues.add(topic); - } - if (extraLabelsAndValues != null) { - labelsAndValues.addAll(List.of(extraLabelsAndValues)); + labelsAndValues[5] = topic; } - writeSample(buffer, metricName, value, labelsAndValues.toArray(new String[0])); + String[] labels = ArrayUtils.addAll(labelsAndValues, extraLabelsAndValues); + stream.writeSample(metricName, value, labels); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java index e6ac1535f43c7..9dc4240c2aba5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java @@ -20,8 +20,6 @@ import static org.apache.pulsar.common.events.EventsTopicNames.checkTopicIsEventsNames; import io.netty.util.concurrent.FastThreadLocal; -import java.util.HashMap; -import java.util.Map; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.impl.ManagedLedgerMBeanImpl; @@ -30,7 +28,6 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.util.SimpleTextOutputStream; import org.apache.pulsar.transaction.coordinator.impl.MLTransactionLogImpl; import org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStore; import org.apache.pulsar.transaction.coordinator.impl.TransactionMetadataStoreStats; @@ -38,43 +35,29 @@ @Slf4j public class TransactionAggregator { - /** - * Used for tracking duplicate TYPE definitions. - */ - private static final FastThreadLocal> threadLocalMetricWithTypeDefinition = - new FastThreadLocal() { - @Override - protected Map initialValue() { - return new HashMap<>(); - } - }; - private static final FastThreadLocal localTransactionCoordinatorStats = - new FastThreadLocal() { + new FastThreadLocal<>() { @Override - protected AggregatedTransactionCoordinatorStats initialValue() throws Exception { + protected AggregatedTransactionCoordinatorStats initialValue() { return new AggregatedTransactionCoordinatorStats(); } }; private static final FastThreadLocal localManageLedgerStats = - new FastThreadLocal() { + new FastThreadLocal<>() { @Override - protected ManagedLedgerStats initialValue() throws Exception { + protected ManagedLedgerStats initialValue() { return new ManagedLedgerStats(); } }; - public static void generate(PulsarService pulsar, SimpleTextOutputStream stream, boolean includeTopicMetrics) { + public static void generate(PulsarService pulsar, PrometheusMetricStreams stream, boolean includeTopicMetrics) { String cluster = pulsar.getConfiguration().getClusterName(); - Map metricWithTypeDefinition = threadLocalMetricWithTypeDefinition.get(); - metricWithTypeDefinition.clear(); if (includeTopicMetrics) { - pulsar.getBrokerService().getMultiLayerTopicMap().forEach((namespace, bundlesMap) -> { - bundlesMap.forEach((bundle, topicsMap) -> { - topicsMap.forEach((name, topic) -> { + pulsar.getBrokerService().getMultiLayerTopicMap().forEach((namespace, bundlesMap) -> + bundlesMap.forEach((bundle, topicsMap) -> topicsMap.forEach((name, topic) -> { if (topic instanceof PersistentTopic) { topic.getSubscriptions().values().forEach(subscription -> { try { @@ -82,9 +65,8 @@ public static void generate(PulsarService pulsar, SimpleTextOutputStream stream, if (!checkTopicIsEventsNames(TopicName.get(subscription.getTopic().getName())) && subscription instanceof PersistentSubscription && ((PersistentSubscription) subscription).checkIfPendingAckStoreInit()) { - ManagedLedger managedLedger = - ((PersistentSubscription) subscription) - .getPendingAckManageLedger().get(); + ManagedLedger managedLedger = ((PersistentSubscription) subscription) + .getPendingAckManageLedger().get(); generateManageLedgerStats(managedLedger, stream, cluster, namespace, name, subscription.getName()); } @@ -93,9 +75,7 @@ public static void generate(PulsarService pulsar, SimpleTextOutputStream stream, } }); } - }); - }); - }); + }))); } AggregatedTransactionCoordinatorStats transactionCoordinatorStats = localTransactionCoordinatorStats.get(); @@ -124,18 +104,18 @@ public static void generate(PulsarService pulsar, SimpleTextOutputStream stream, localManageLedgerStats.get().reset(); if (transactionMetadataStore instanceof MLTransactionMetadataStore) { - ManagedLedger managedLedger = - ((MLTransactionMetadataStore) transactionMetadataStore).getManagedLedger(); + ManagedLedger managedLedger = + ((MLTransactionMetadataStore) transactionMetadataStore).getManagedLedger(); generateManageLedgerStats(managedLedger, stream, cluster, NamespaceName.SYSTEM_NAMESPACE.toString(), MLTransactionLogImpl.TRANSACTION_LOG_PREFIX + transactionCoordinatorID.getId(), MLTransactionLogImpl.TRANSACTION_SUBSCRIPTION_NAME); } - }); + }); } - private static void generateManageLedgerStats(ManagedLedger managedLedger, SimpleTextOutputStream stream, + private static void generateManageLedgerStats(ManagedLedger managedLedger, PrometheusMetricStreams stream, String cluster, String namespace, String topic, String subscription) { ManagedLedgerStats managedLedgerStats = localManageLedgerStats.get(); ManagedLedgerMBeanImpl mlStats = (ManagedLedgerMBeanImpl) managedLedger.getStats(); @@ -157,174 +137,149 @@ private static void generateManageLedgerStats(ManagedLedger managedLedger, Simpl managedLedgerStats.storageWriteRate = mlStats.getAddEntryMessagesRate(); managedLedgerStats.storageReadRate = mlStats.getReadEntriesRate(); - printManageLedgerStats(stream, cluster, namespace, topic, - subscription, managedLedgerStats); - } - - private static void metricType(SimpleTextOutputStream stream, String name) { - Map metricWithTypeDefinition = threadLocalMetricWithTypeDefinition.get(); - if (!metricWithTypeDefinition.containsKey(name)) { - metricWithTypeDefinition.put(name, "gauge"); - stream.write("# TYPE ").write(name).write(" gauge\n"); - } - - } - - private static void metric(SimpleTextOutputStream stream, String cluster, String name, - double value, long coordinatorId) { - metricType(stream, name); - stream.write(name) - .write("{cluster=\"").write(cluster) - .write("\",coordinator_id=\"").write(coordinatorId).write("\"} ") - .write(value).write(' ').write(System.currentTimeMillis()) - .write('\n'); + printManageLedgerStats(stream, cluster, namespace, topic, subscription, managedLedgerStats); } - private static void metrics(SimpleTextOutputStream stream, String cluster, String namespace, - String topic, String subscription, String name, long value) { - stream.write(name).write("{cluster=\"").write(cluster).write("\", namespace=\"").write(namespace) - .write("\",topic=\"").write(topic).write("\",subscription=\"").write(subscription).write("\"} "); - stream.write(value).write(' ').write(System.currentTimeMillis()).write('\n'); - } - - private static void metrics(SimpleTextOutputStream stream, String cluster, String namespace, - String topic, String subscription, String name, double value) { - stream.write(name).write("{cluster=\"").write(cluster).write("\", namespace=\"").write(namespace) - .write("\",topic=\"").write(topic).write("\",subscription=\"").write(subscription).write("\"} "); - stream.write(value).write(' ').write(System.currentTimeMillis()).write('\n'); - } - - private static void printManageLedgerStats(SimpleTextOutputStream stream, String cluster, String namespace, + private static void printManageLedgerStats(PrometheusMetricStreams stream, String cluster, String namespace, String topic, String subscription, ManagedLedgerStats stats) { - metrics(stream, cluster, namespace, topic, subscription, - "pulsar_storage_size", stats.storageSize); - metrics(stream, cluster, namespace, topic, subscription, - "pulsar_storage_logical_size", stats.storageLogicalSize); - metrics(stream, cluster, namespace, topic, subscription, - "pulsar_storage_backlog_size", stats.backlogSize); - metrics(stream, cluster, namespace, topic, subscription, - "pulsar_storage_offloaded_size", stats.offloadedStorageUsed); + writeMetric(stream, "pulsar_storage_size", stats.storageSize, cluster, namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_logical_size", stats.storageLogicalSize, cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_backlog_size", stats.backlogSize, cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_offloaded_size", stats.offloadedStorageUsed, cluster, namespace, topic, + subscription); - metrics(stream, cluster, namespace, topic, subscription, - "pulsar_storage_write_rate", stats.storageWriteRate); - metrics(stream, cluster, namespace, topic, subscription, - "pulsar_storage_read_rate", stats.storageReadRate); + writeMetric(stream, "pulsar_storage_write_rate", stats.storageWriteRate, cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_read_rate", stats.storageReadRate, cluster, namespace, topic, + subscription); stats.storageWriteLatencyBuckets.refresh(); long[] latencyBuckets = stats.storageWriteLatencyBuckets.getBuckets(); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_write_latency_le_0_5", latencyBuckets[0]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_write_latency_le_1", latencyBuckets[1]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_write_latency_le_5", latencyBuckets[2]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_write_latency_le_10", latencyBuckets[3]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_write_latency_le_20", latencyBuckets[4]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_write_latency_le_50", latencyBuckets[5]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_write_latency_le_100", latencyBuckets[6]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_write_latency_le_200", latencyBuckets[7]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_write_latency_le_1000", latencyBuckets[8]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_write_latency_overflow", latencyBuckets[9]); - metric(stream, cluster, namespace, topic, subscription, "pulsar_storage_write_latency_count", - stats.storageWriteLatencyBuckets.getCount()); - metric(stream, cluster, namespace, topic, subscription, "pulsar_storage_write_latency_sum", - stats.storageWriteLatencyBuckets.getSum()); + writeMetric(stream, "pulsar_storage_write_latency_le_0_5", latencyBuckets[0], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_write_latency_le_1", latencyBuckets[1], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_write_latency_le_5", latencyBuckets[2], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_write_latency_le_10", latencyBuckets[3], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_write_latency_le_20", latencyBuckets[4], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_write_latency_le_50", latencyBuckets[5], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_write_latency_le_100", latencyBuckets[6], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_write_latency_le_200", latencyBuckets[7], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_write_latency_le_1000", latencyBuckets[8], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_write_latency_overflow", latencyBuckets[9], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_storage_write_latency_count", stats.storageWriteLatencyBuckets.getCount(), + cluster, namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_write_latency_sum", stats.storageWriteLatencyBuckets.getSum(), cluster, + namespace, topic, subscription); stats.storageLedgerWriteLatencyBuckets.refresh(); - long[] ledgerWritelatencyBuckets = stats.storageLedgerWriteLatencyBuckets.getBuckets(); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_ledger_write_latency_le_0_5", ledgerWritelatencyBuckets[0]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_ledger_write_latency_le_1", ledgerWritelatencyBuckets[1]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_ledger_write_latency_le_5", ledgerWritelatencyBuckets[2]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_ledger_write_latency_le_10", ledgerWritelatencyBuckets[3]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_ledger_write_latency_le_20", ledgerWritelatencyBuckets[4]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_ledger_write_latency_le_50", ledgerWritelatencyBuckets[5]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_ledger_write_latency_le_100", ledgerWritelatencyBuckets[6]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_ledger_write_latency_le_200", ledgerWritelatencyBuckets[7]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_storage_ledger_write_latency_le_1000", ledgerWritelatencyBuckets[8]); - metric(stream, cluster, namespace, topic, subscription, "pulsar_storage_ledger_write_latency_overflow", - ledgerWritelatencyBuckets[9]); - metric(stream, cluster, namespace, topic, subscription, "pulsar_storage_ledger_write_latency_count", - stats.storageLedgerWriteLatencyBuckets.getCount()); - metric(stream, cluster, namespace, topic, subscription, "pulsar_storage_ledger_write_latency_sum", - stats.storageLedgerWriteLatencyBuckets.getSum()); + long[] ledgerWriteLatencyBuckets = stats.storageLedgerWriteLatencyBuckets.getBuckets(); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_0_5", ledgerWriteLatencyBuckets[0], cluster, + namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_1", ledgerWriteLatencyBuckets[1], cluster, + namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_5", ledgerWriteLatencyBuckets[2], cluster, + namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_10", ledgerWriteLatencyBuckets[3], cluster, + namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_20", ledgerWriteLatencyBuckets[4], cluster, + namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_50", ledgerWriteLatencyBuckets[5], cluster, + namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_100", ledgerWriteLatencyBuckets[6], cluster, + namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_200", ledgerWriteLatencyBuckets[7], cluster, + namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_1000", ledgerWriteLatencyBuckets[8], cluster, + namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_ledger_write_latency_overflow", ledgerWriteLatencyBuckets[9], cluster, + namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_ledger_write_latency_count", + stats.storageLedgerWriteLatencyBuckets.getCount(), cluster, namespace, topic, subscription); + writeMetric(stream, "pulsar_storage_ledger_write_latency_sum", + stats.storageLedgerWriteLatencyBuckets.getSum(), cluster, namespace, topic, subscription); stats.entrySizeBuckets.refresh(); long[] entrySizeBuckets = stats.entrySizeBuckets.getBuckets(); - metric(stream, cluster, namespace, topic, subscription, "pulsar_entry_size_le_128", entrySizeBuckets[0]); - metric(stream, cluster, namespace, topic, subscription, "pulsar_entry_size_le_512", entrySizeBuckets[1]); - metric(stream, cluster, namespace, topic, subscription, "pulsar_entry_size_le_1_kb", entrySizeBuckets[2]); - metric(stream, cluster, namespace, topic, subscription, "pulsar_entry_size_le_2_kb", entrySizeBuckets[3]); - metric(stream, cluster, namespace, topic, subscription, "pulsar_entry_size_le_4_kb", entrySizeBuckets[4]); - metric(stream, cluster, namespace, topic, subscription, "pulsar_entry_size_le_16_kb", entrySizeBuckets[5]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_entry_size_le_100_kb", entrySizeBuckets[6]); - metric(stream, cluster, namespace, topic, subscription, "pulsar_entry_size_le_1_mb", entrySizeBuckets[7]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_entry_size_le_overflow", entrySizeBuckets[8]); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_entry_size_count", stats.entrySizeBuckets.getCount()); - metric(stream, cluster, namespace, topic, subscription, - "pulsar_entry_size_sum", stats.entrySizeBuckets.getSum()); - } - - private static void metric(SimpleTextOutputStream stream, String cluster, - String namespace, String topic, String subscription, - String name, long value) { - stream.write(name).write("{cluster=\"").write(cluster).write("\",namespace=\"").write(namespace) - .write("\",topic=\"").write(topic).write("\",subscription=\"").write(subscription).write("\"} "); - stream.write(value).write(' ').write(System.currentTimeMillis()).write('\n'); + writeMetric(stream, "pulsar_entry_size_le_128", entrySizeBuckets[0], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_entry_size_le_512", entrySizeBuckets[1], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_entry_size_le_1_kb", entrySizeBuckets[2], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_entry_size_le_2_kb", entrySizeBuckets[3], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_entry_size_le_4_kb", entrySizeBuckets[4], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_entry_size_le_16_kb", entrySizeBuckets[5], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_entry_size_le_100_kb", entrySizeBuckets[6], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_entry_size_le_1_mb", entrySizeBuckets[7], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_entry_size_le_overflow", entrySizeBuckets[8], cluster, namespace, topic, + subscription); + writeMetric(stream, "pulsar_entry_size_count", stats.entrySizeBuckets.getCount(), cluster, namespace, + topic, subscription); + writeMetric(stream, "pulsar_entry_size_sum", stats.entrySizeBuckets.getSum(), cluster, namespace, topic, + subscription); } - static void printTransactionCoordinatorStats(SimpleTextOutputStream stream, String cluster, + static void printTransactionCoordinatorStats(PrometheusMetricStreams stream, String cluster, AggregatedTransactionCoordinatorStats stats, long coordinatorId) { - metric(stream, cluster, "pulsar_txn_active_count", - stats.actives, coordinatorId); - metric(stream, cluster, "pulsar_txn_committed_count", - stats.committedCount, coordinatorId); - metric(stream, cluster, "pulsar_txn_aborted_count", - stats.abortedCount, coordinatorId); - metric(stream, cluster, "pulsar_txn_created_count", - stats.createdCount, coordinatorId); - metric(stream, cluster, "pulsar_txn_timeout_count", - stats.timeoutCount, coordinatorId); - metric(stream, cluster, "pulsar_txn_append_log_count", - stats.appendLogCount, coordinatorId); + writeMetric(stream, "pulsar_txn_active_count", stats.actives, cluster, + coordinatorId); + writeMetric(stream, "pulsar_txn_committed_count", stats.committedCount, cluster, + coordinatorId); + writeMetric(stream, "pulsar_txn_aborted_count", stats.abortedCount, cluster, + coordinatorId); + writeMetric(stream, "pulsar_txn_created_count", stats.createdCount, cluster, + coordinatorId); + writeMetric(stream, "pulsar_txn_timeout_count", stats.timeoutCount, cluster, + coordinatorId); + writeMetric(stream, "pulsar_txn_append_log_count", stats.appendLogCount, cluster, + coordinatorId); long[] latencyBuckets = stats.executionLatency; - metric(stream, cluster, "pulsar_txn_execution_latency_le_10", latencyBuckets[0], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_20", latencyBuckets[1], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_50", latencyBuckets[2], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_100", latencyBuckets[3], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_500", latencyBuckets[4], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_1000", latencyBuckets[5], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_5000", latencyBuckets[6], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_15000", latencyBuckets[7], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_30000", latencyBuckets[8], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_60000", latencyBuckets[9], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_300000", - latencyBuckets[10], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_1500000", - latencyBuckets[11], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_3000000", - latencyBuckets[12], coordinatorId); - metric(stream, cluster, "pulsar_txn_execution_latency_le_overflow", - latencyBuckets[13], coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_10", latencyBuckets[0], cluster, coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_20", latencyBuckets[1], cluster, coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_50", latencyBuckets[2], cluster, coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_100", latencyBuckets[3], cluster, coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_500", latencyBuckets[4], cluster, coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_1000", latencyBuckets[5], cluster, coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_5000", latencyBuckets[6], cluster, coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_15000", latencyBuckets[7], cluster, coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_30000", latencyBuckets[8], cluster, coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_60000", latencyBuckets[9], cluster, coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_300000", latencyBuckets[10], cluster, + coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_1500000", latencyBuckets[11], cluster, + coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_3000000", latencyBuckets[12], cluster, + coordinatorId); + writeMetric(stream, "pulsar_txn_execution_latency_le_overflow", latencyBuckets[13], cluster, + coordinatorId); + } + + private static void writeMetric(PrometheusMetricStreams stream, String metricName, double value, String cluster, + long coordinatorId) { + stream.writeSample(metricName, value, "cluster", cluster, "coordinator_id", String.valueOf(coordinatorId)); + } + + private static void writeMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, + String namespace, String topic, String subscription) { + stream.writeSample(metricName, value, "cluster", cluster, "namespace", namespace, "topic", topic, + "subscription", subscription); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java index 5a991f7e0998b..8f704b11e764c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java @@ -18,13 +18,9 @@ */ package org.apache.pulsar.broker.stats.prometheus.metrics; -import io.prometheus.client.Collector; -import io.prometheus.client.Collector.MetricFamilySamples; import java.io.IOException; import java.io.Writer; -import java.util.Collection; import org.apache.bookkeeper.stats.Counter; -import org.apache.pulsar.common.util.SimpleTextOutputStream; /** * Logic to write metrics in Prometheus text format. @@ -139,34 +135,4 @@ private static void writeSum(Writer w, DataSketchesOpStatsLogger opStat, String .append(success.toString()).append("\"} ") .append(Double.toString(opStat.getSum(success))).append('\n'); } - - public static void writeMetrics(SimpleTextOutputStream stream, Collection familySamples) { - for (MetricFamilySamples familySample : familySamples) { - stream.write("# TYPE "); - stream.write(familySample.name); - stream.write(' '); - stream.write(familySample.type.name().toLowerCase()); - stream.write('\n'); - for (Collector.MetricFamilySamples.Sample sample : familySample.samples) { - stream.write(sample.name); - if (sample.labelNames.size() > 0) { - stream.write('{'); - for (int i = 0; i < sample.labelNames.size(); ++i) { - stream.write(sample.labelNames.get(i)); - stream.write("=\""); - stream.write(sample.labelValues.get(i)); - stream.write("\","); - } - stream.write('}'); - } - stream.write(' '); - stream.write(Collector.doubleToGoString(sample.value)); - if (sample.timestampMs != null) { - stream.write(' '); - stream.write(sample.timestampMs.toString()); - } - stream.write('\n'); - } - } - } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java index dffc065e985d3..39ecc4235888a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java @@ -20,6 +20,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; + import org.testng.annotations.Test; @Test(groups = "broker") diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NumberFormat.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NumberFormat.java index 0eae68314f809..502af8251e512 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NumberFormat.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NumberFormat.java @@ -25,7 +25,7 @@ */ public class NumberFormat { - public static void format(ByteBuf out, long num) { + static void format(ByteBuf out, long num) { if (num == 0) { out.writeByte('0'); return; diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java index f7e17eed6ed20..dd78b4cfe58b9 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java @@ -22,12 +22,11 @@ /** * Format strings and numbers into a ByteBuf without any memory allocation. - * */ public class SimpleTextOutputStream { private final ByteBuf buffer; - private static final char[] hexChars = { '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', - 'f' }; + private static final char[] hexChars = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', + 'f'}; public SimpleTextOutputStream(ByteBuf buffer) { this.buffer = buffer; @@ -134,6 +133,9 @@ public SimpleTextOutputStream write(double d) { public void write(ByteBuf byteBuf) { buffer.writeBytes(byteBuf); - byteBuf.release(); + } + + public ByteBuf getBuffer() { + return buffer; } } From 9b004c0a602a5b7ad28e10933cff6faf49debe9a Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Thu, 7 Jul 2022 13:37:19 +0100 Subject: [PATCH 13/19] [fix][broker] PR Comments --- .../stats/prometheus/PrometheusMetricStreams.java | 3 ++- .../stats/prometheus/PrometheusMetricsGenerator.java | 11 ++++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java index 7766031886b1c..1e699e86c7669 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java @@ -25,6 +25,8 @@ /** * Helper class to ensure that metrics of the same name are grouped together under the same TYPE header when written. + * Those are the requirements of the + * Prometheus Exposition Format. */ public class PrometheusMetricStreams { private final Map metricStreamMap = new HashMap<>(); @@ -53,7 +55,6 @@ void writeSample(String metricName, Number value, String... labelsAndValuesArray */ void flushAllToStream(SimpleTextOutputStream stream) { metricStreamMap.values().forEach(s -> stream.write(s.getBuffer())); - releaseAll(); } /** 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 1a8d9f6a10854..ee558911c616b 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 @@ -174,24 +174,24 @@ private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetri ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.compositeDirectBuffer(MAX_COMPONENTS); boolean exceptionHappens = false; //Used in namespace/topic and transaction aggregators as share metric names - PrometheusMetricStreams metricStream = new PrometheusMetricStreams(); + PrometheusMetricStreams metricStreams = new PrometheusMetricStreams(); try { SimpleTextOutputStream stream = new SimpleTextOutputStream(buf); generateSystemMetrics(stream, pulsar.getConfiguration().getClusterName()); NamespaceStatsAggregator.generate(pulsar, includeTopicMetrics, includeConsumerMetrics, - includeProducerMetrics, splitTopicAndPartitionIndexLabel, metricStream); + includeProducerMetrics, splitTopicAndPartitionIndexLabel, metricStreams); if (pulsar.getWorkerServiceOpt().isPresent()) { pulsar.getWorkerService().generateFunctionsStats(stream); } if (pulsar.getConfiguration().isTransactionCoordinatorEnabled()) { - TransactionAggregator.generate(pulsar, metricStream, includeTopicMetrics); + TransactionAggregator.generate(pulsar, metricStreams, includeTopicMetrics); } - metricStream.flushAllToStream(stream); + metricStreams.flushAllToStream(stream); generateBrokerBasicMetrics(pulsar, stream); @@ -204,10 +204,11 @@ private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetri } out.write(buf.array(), buf.arrayOffset(), buf.readableBytes()); } finally { + //release all the metrics buffers + metricStreams.releaseAll(); //if exception happens, release buffer if (exceptionHappens) { buf.release(); - metricStream.releaseAll(); } } } From ee944d3a85aeb9d938f9a03432b6e97dcf5514d9 Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Tue, 19 Jul 2022 16:42:38 +0100 Subject: [PATCH 14/19] [fix][broker] Fix failing test after merge --- .../org/apache/pulsar/broker/stats/PrometheusMetricsTest.java | 4 ++++ 1 file changed, 4 insertions(+) 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 1769e547f09ca..be10c8dc65e94 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 @@ -1444,6 +1444,10 @@ public void testMetricsGroupedByTypeDefinitions() throws Exception { metricName = metricName.substring(0, metricName.indexOf("_count")); } else if (metricName.endsWith("_sum") && !currentMetric.get().endsWith("_sum")) { metricName = metricName.substring(0, metricName.indexOf("_sum")); + } else if (metricName.endsWith("_total") && !currentMetric.get().endsWith("_total")) { + metricName = metricName.substring(0, metricName.indexOf("_total")); + } else if (metricName.endsWith("_created") && !currentMetric.get().endsWith("_created")) { + metricName = metricName.substring(0, metricName.indexOf("_created")); } if (!metricName.equals(currentMetric.get())) { From 6b9f7d27faa50742739ab15ea633babc2d688678 Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Wed, 3 Aug 2022 11:01:42 +0100 Subject: [PATCH 15/19] [fix][broker] Updates from PR comments and new test --- .../prometheus/NamespaceStatsAggregator.java | 104 ++++++++---------- .../prometheus/PrometheusMetricStreams.java | 2 +- .../broker/stats/prometheus/TopicStats.java | 1 - .../PrometheusMetricStreamsTest.java | 85 ++++++++++++++ 4 files changed, 130 insertions(+), 62 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreamsTest.java 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 17512c82178db..649b7f0d4f236 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 @@ -335,80 +335,64 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat writePulsarMsgBacklog(stream, stats.msgBacklog, cluster, namespace); stats.managedLedgerStats.storageWriteLatencyBuckets.refresh(); - writeMetric(stream, "pulsar_storage_write_latency_le_0_5", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[0], cluster, namespace); - writeMetric(stream, "pulsar_storage_write_latency_le_1", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[1], cluster, namespace); - writeMetric(stream, "pulsar_storage_write_latency_le_5", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[2], cluster, namespace); - writeMetric(stream, "pulsar_storage_write_latency_le_10", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[3], cluster, namespace); - writeMetric(stream, "pulsar_storage_write_latency_le_20", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[4], cluster, namespace); - writeMetric(stream, "pulsar_storage_write_latency_le_50", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[5], cluster, namespace); - writeMetric(stream, "pulsar_storage_write_latency_le_100", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[6], cluster, namespace); - writeMetric(stream, "pulsar_storage_write_latency_le_200", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[7], cluster, namespace); - writeMetric(stream, "pulsar_storage_write_latency_le_1000", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[8], cluster, namespace); - writeMetric(stream, "pulsar_storage_write_latency_overflow", - stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets()[9], cluster, namespace); + long[] latencyBuckets = stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets(); + writeMetric(stream, "pulsar_storage_write_latency_le_0_5", latencyBuckets[0], cluster, namespace); + writeMetric(stream, "pulsar_storage_write_latency_le_1", latencyBuckets[1], cluster, namespace); + writeMetric(stream, "pulsar_storage_write_latency_le_5", latencyBuckets[2], cluster, namespace); + writeMetric(stream, "pulsar_storage_write_latency_le_10", latencyBuckets[3], cluster, namespace); + writeMetric(stream, "pulsar_storage_write_latency_le_20", latencyBuckets[4], cluster, namespace); + writeMetric(stream, "pulsar_storage_write_latency_le_50", latencyBuckets[5], cluster, namespace); + writeMetric(stream, "pulsar_storage_write_latency_le_100", latencyBuckets[6], cluster, namespace); + writeMetric(stream, "pulsar_storage_write_latency_le_200", latencyBuckets[7], cluster, namespace); + writeMetric(stream, "pulsar_storage_write_latency_le_1000", latencyBuckets[8], cluster, namespace); + writeMetric(stream, "pulsar_storage_write_latency_overflow", latencyBuckets[9], cluster, namespace); writeMetric(stream, "pulsar_storage_write_latency_count", stats.managedLedgerStats.storageWriteLatencyBuckets.getCount(), cluster, namespace); writeMetric(stream, "pulsar_storage_write_latency_sum", stats.managedLedgerStats.storageWriteLatencyBuckets.getSum(), cluster, namespace); stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.refresh(); - writeMetric(stream, "pulsar_storage_ledger_write_latency_le_0_5", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[0], cluster, namespace); - writeMetric(stream, "pulsar_storage_ledger_write_latency_le_1", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[1], cluster, namespace); - writeMetric(stream, "pulsar_storage_ledger_write_latency_le_5", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[2], cluster, namespace); - writeMetric(stream, "pulsar_storage_ledger_write_latency_le_10", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[3], cluster, namespace); - writeMetric(stream, "pulsar_storage_ledger_write_latency_le_20", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[4], cluster, namespace); - writeMetric(stream, "pulsar_storage_ledger_write_latency_le_50", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[5], cluster, namespace); - writeMetric(stream, "pulsar_storage_ledger_write_latency_le_100", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[6], cluster, namespace); - writeMetric(stream, "pulsar_storage_ledger_write_latency_le_200", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[7], cluster, namespace); - writeMetric(stream, "pulsar_storage_ledger_write_latency_le_1000", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[8], cluster, namespace); - writeMetric(stream, "pulsar_storage_ledger_write_latency_overflow", - stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets()[9], cluster, namespace); + long[] ledgerWriteLatencyBuckets = stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getBuckets(); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_0_5", ledgerWriteLatencyBuckets[0], + cluster, namespace); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_1", ledgerWriteLatencyBuckets[1], + cluster, namespace); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_5", ledgerWriteLatencyBuckets[2], + cluster, namespace); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_10", ledgerWriteLatencyBuckets[3], + cluster, namespace); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_20", ledgerWriteLatencyBuckets[4], + cluster, namespace); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_50", ledgerWriteLatencyBuckets[5], + cluster, namespace); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_100", ledgerWriteLatencyBuckets[6], + cluster, namespace); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_200", ledgerWriteLatencyBuckets[7], + cluster, namespace); + writeMetric(stream, "pulsar_storage_ledger_write_latency_le_1000", ledgerWriteLatencyBuckets[8], + cluster, namespace); + writeMetric(stream, "pulsar_storage_ledger_write_latency_overflow", ledgerWriteLatencyBuckets[9], + cluster, namespace); writeMetric(stream, "pulsar_storage_ledger_write_latency_count", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getCount(), cluster, namespace); writeMetric(stream, "pulsar_storage_ledger_write_latency_sum", stats.managedLedgerStats.storageLedgerWriteLatencyBuckets.getSum(), cluster, namespace); stats.managedLedgerStats.entrySizeBuckets.refresh(); - writeMetric(stream, "pulsar_entry_size_le_128", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[0], - cluster, namespace); - writeMetric(stream, "pulsar_entry_size_le_512", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[1], - cluster, namespace); - writeMetric(stream, "pulsar_entry_size_le_1_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[2], - cluster, namespace); - writeMetric(stream, "pulsar_entry_size_le_2_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[3], - cluster, namespace); - writeMetric(stream, "pulsar_entry_size_le_4_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[4], - cluster, namespace); - writeMetric(stream, "pulsar_entry_size_le_16_kb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[5], + long[] entrySizeBuckets = stats.managedLedgerStats.entrySizeBuckets.getBuckets(); + writeMetric(stream, "pulsar_entry_size_le_128", entrySizeBuckets[0], cluster, namespace); + writeMetric(stream, "pulsar_entry_size_le_512", entrySizeBuckets[1], cluster, namespace); + writeMetric(stream, "pulsar_entry_size_le_1_kb", entrySizeBuckets[2], cluster, namespace); + writeMetric(stream, "pulsar_entry_size_le_2_kb", entrySizeBuckets[3], cluster, namespace); + writeMetric(stream, "pulsar_entry_size_le_4_kb", entrySizeBuckets[4], cluster, namespace); + writeMetric(stream, "pulsar_entry_size_le_16_kb", entrySizeBuckets[5], cluster, namespace); + writeMetric(stream, "pulsar_entry_size_le_100_kb", entrySizeBuckets[6], cluster, namespace); + writeMetric(stream, "pulsar_entry_size_le_1_mb", entrySizeBuckets[7], cluster, namespace); + writeMetric(stream, "pulsar_entry_size_le_overflow", entrySizeBuckets[8], cluster, namespace); + writeMetric(stream, "pulsar_entry_size_count", stats.managedLedgerStats.entrySizeBuckets.getCount(), cluster, namespace); - writeMetric(stream, "pulsar_entry_size_le_100_kb", - stats.managedLedgerStats.entrySizeBuckets.getBuckets()[6], cluster, namespace); - writeMetric(stream, "pulsar_entry_size_le_1_mb", stats.managedLedgerStats.entrySizeBuckets.getBuckets()[7], + writeMetric(stream, "pulsar_entry_size_sum", stats.managedLedgerStats.entrySizeBuckets.getSum(), cluster, namespace); - writeMetric(stream, "pulsar_entry_size_le_overflow", - stats.managedLedgerStats.entrySizeBuckets.getBuckets()[8], cluster, namespace); - writeMetric(stream, "pulsar_entry_size_count", - stats.managedLedgerStats.entrySizeBuckets.getCount(), cluster, namespace); - writeMetric(stream, "pulsar_entry_size_sum", - stats.managedLedgerStats.entrySizeBuckets.getSum(), cluster, namespace); writeReplicationStat(stream, "pulsar_replication_rate_in", stats, replStats -> replStats.msgRateIn, cluster, namespace); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java index 1e699e86c7669..42641b9be6710 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java @@ -46,7 +46,7 @@ void writeSample(String metricName, Number value, String... labelsAndValuesArray stream.write(','); } } - stream.write("\"} ").write(value).write(' ').write(System.currentTimeMillis()).write('\n'); + stream.write("} ").write(value).write(' ').write(System.currentTimeMillis()).write('\n'); } /** 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 b50a349bba381..4f38aaf43ba9a 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 @@ -43,7 +43,6 @@ class TopicStats { double averageMsgSize; public long msgBacklog; - long publishRateLimitedTimes; long backlogQuotaLimit; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreamsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreamsTest.java new file mode 100644 index 0000000000000..6e2bd529ffa1f --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreamsTest.java @@ -0,0 +1,85 @@ +/** + * 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; + +import static org.testng.Assert.assertTrue; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import java.io.ByteArrayOutputStream; +import java.nio.charset.StandardCharsets; +import org.apache.pulsar.common.util.SimpleTextOutputStream; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class PrometheusMetricStreamsTest { + + private PrometheusMetricStreams underTest; + + @BeforeMethod(alwaysRun = true) + protected void setup() throws Exception { + underTest = new PrometheusMetricStreams(); + } + + @AfterMethod(alwaysRun = true) + protected void cleanup() throws Exception { + underTest.releaseAll(); + } + + @Test + public void canWriteSampleWithoutLabels() { + underTest.writeSample("my-metric", 123); + + String actual = writeToString(); + + assertTrue(actual.startsWith("# TYPE my-metric gauge"), "Gauge type line missing"); + assertTrue(actual.contains("my-metric{} 123"), "Metric line missing"); + } + + @Test + public void canWriteSampleWithLabels() { + underTest.writeSample("my-other-metric", 123, "cluster", "local"); + underTest.writeSample("my-other-metric", 456, "cluster", "local", "namespace", "my-ns"); + + String actual = writeToString(); + + assertTrue(actual.startsWith("# TYPE my-other-metric gauge"), "Gauge type line missing"); + assertTrue(actual.contains("my-other-metric{cluster=\"local\"} 123"), "Cluster metric line missing"); + assertTrue(actual.contains("my-other-metric{cluster=\"local\",namespace=\"my-ns\"} 456"), + "Cluster and Namespace metric line missing"); + } + + private String writeToString() { + ByteBuf buffer = ByteBufAllocator.DEFAULT.directBuffer(); + try { + SimpleTextOutputStream stream = new SimpleTextOutputStream(buffer); + underTest.flushAllToStream(stream); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + int readIndex = buffer.readerIndex(); + int readableBytes = buffer.readableBytes(); + for (int i = 0; i < readableBytes; i++) { + out.write(buffer.getByte(readIndex + i)); + } + return out.toString(StandardCharsets.UTF_8); + } finally { + buffer.release(); + } + } +} \ No newline at end of file From 2eb407a220277589e3650be3cb77c2fecd644290 Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Mon, 8 Aug 2022 09:43:18 +0100 Subject: [PATCH 16/19] [fix][broker] Updates from PR comments --- .../broker/stats/prometheus/PrometheusMetricStreams.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java index 42641b9be6710..6b6b972c175f0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java @@ -18,9 +18,9 @@ */ package org.apache.pulsar.broker.stats.prometheus; -import io.netty.buffer.ByteBufAllocator; import java.util.HashMap; import java.util.Map; +import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.util.SimpleTextOutputStream; /** @@ -67,7 +67,7 @@ void releaseAll() { private SimpleTextOutputStream initGaugeType(String metricName) { return metricStreamMap.computeIfAbsent(metricName, s -> { - SimpleTextOutputStream stream = new SimpleTextOutputStream(ByteBufAllocator.DEFAULT.directBuffer()); + SimpleTextOutputStream stream = new SimpleTextOutputStream(PulsarByteBufAllocator.DEFAULT.directBuffer()); stream.write("# TYPE ").write(metricName).write(" gauge\n"); return stream; }); From 2454f1fcc31e4577d4e217a0c7f6c50ad849af2f Mon Sep 17 00:00:00 2001 From: Mark Silcox <63227862+marksilcox@users.noreply.github.com> Date: Sun, 21 Aug 2022 10:20:13 +0100 Subject: [PATCH 17/19] [fix][broker] Updates from PR comments --- .../prometheus/NamespaceStatsAggregator.java | 4 +- .../PrometheusMetricsGenerator.java | 102 +++--------------- .../prometheus/TransactionAggregator.java | 4 +- .../PrometheusMetricStreamsTest.java | 2 +- 4 files changed, 22 insertions(+), 90 deletions(-) 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 649b7f0d4f236..29915f071c099 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 @@ -43,14 +43,14 @@ public class NamespaceStatsAggregator { private static final FastThreadLocal localNamespaceStats = - new FastThreadLocal<>() { + new FastThreadLocal() { @Override protected AggregatedNamespaceStats initialValue() { return new AggregatedNamespaceStats(); } }; - private static final FastThreadLocal localTopicStats = new FastThreadLocal<>() { + private static final FastThreadLocal localTopicStats = new FastThreadLocal() { @Override protected TopicStats initialValue() { return new TopicStats(); 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 ee558911c616b..a993d1edf3a3d 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 @@ -56,8 +56,6 @@ * href="https://prometheus.io/docs/instrumenting/exposition_formats/">Exposition Formats */ public class PrometheusMetricsGenerator { - private static volatile TimeWindow timeWindow; - private static final int MAX_COMPONENTS = 64; static { DefaultExports.initialize(); @@ -100,78 +98,12 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b splitTopicAndPartitionIndexLabel, out, null); } - public static synchronized void generate(PulsarService pulsar, boolean includeTopicMetrics, - boolean includeConsumerMetrics, boolean includeProducerMetrics, - boolean splitTopicAndPartitionIndexLabel, OutputStream out, - List metricsProviders) throws IOException { - ByteBuf buffer; - boolean exposeBufferMetrics = pulsar.getConfiguration().isMetricsBufferResponse(); - - if (!exposeBufferMetrics) { - buffer = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, metricsProviders); - } else { - if (null == timeWindow) { - int period = pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds(); - timeWindow = new TimeWindow<>(1, (int) TimeUnit.SECONDS.toMillis(period)); - } - WindowWrap window = timeWindow.current(oldBuf -> { - // release expired buffer, in case of memory leak - if (oldBuf != null && oldBuf.refCnt() > 0) { - oldBuf.release(); - log.debug("Cached metrics buffer released"); - } - - try { - ByteBuf buf = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, metricsProviders); - log.debug("Generated metrics buffer size {}", buf.readableBytes()); - return buf; - } catch (IOException e) { - log.error("Generate metrics failed", e); - //return empty buffer if exception happens - return ByteBufAllocator.DEFAULT.heapBuffer(0); - } - }); - - if (null == window || null == window.value()) { - return; - } - buffer = window.value(); - log.debug("Current window start {}, current cached buf size {}", window.start(), buffer.readableBytes()); - } - - try { - if (out instanceof HttpOutput) { - HttpOutput output = (HttpOutput) out; - //no mem_copy and memory allocations here - ByteBuffer[] buffers = buffer.nioBuffers(); - for (ByteBuffer buffer0 : buffers) { - output.write(buffer0); - } - } else { - //read data from buffer and write it to output stream, with no more heap buffer(byte[]) allocation. - //not modify buffer readIndex/writeIndex here. - int readIndex = buffer.readerIndex(); - int readableBytes = buffer.readableBytes(); - for (int i = 0; i < readableBytes; i++) { - out.write(buffer.getByte(readIndex + i)); - } - } - } finally { - if (!exposeBufferMetrics && buffer.refCnt() > 0) { - buffer.release(); - log.debug("Metrics buffer released."); - } - } - } - - private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, - List metricsProviders) throws IOException { - //Use unpooled buffers here to avoid direct buffer usage increasing. - //when write out 200MB data, MAX_COMPONENTS = 64 needn't mem_copy. see: CompositeByteBuf#consolidateIfNeeded() - ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.compositeDirectBuffer(MAX_COMPONENTS); + public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, + boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, + OutputStream out, + List metricsProviders) + throws IOException { + ByteBuf buf = ByteBufAllocator.DEFAULT.heapBuffer(); boolean exceptionHappens = false; //Used in namespace/topic and transaction aggregators as share metric names PrometheusMetricStreams metricStreams = new PrometheusMetricStreams(); @@ -347,17 +279,17 @@ private static void generateSystemMetrics(SimpleTextOutputStream stream, String static String getTypeStr(Collector.Type type) { switch (type) { - case COUNTER: - return "counter"; - case GAUGE: - return "gauge"; - case SUMMARY : - return "summary"; - case HISTOGRAM: - return "histogram"; - case UNTYPED: - default: - return "untyped"; + case COUNTER: + return "counter"; + case GAUGE: + return "gauge"; + case SUMMARY: + return "summary"; + case HISTOGRAM: + return "histogram"; + case UNTYPED: + default: + return "untyped"; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java index 9dc4240c2aba5..8c58b516333f5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java @@ -36,7 +36,7 @@ public class TransactionAggregator { private static final FastThreadLocal localTransactionCoordinatorStats = - new FastThreadLocal<>() { + new FastThreadLocal() { @Override protected AggregatedTransactionCoordinatorStats initialValue() { return new AggregatedTransactionCoordinatorStats(); @@ -44,7 +44,7 @@ protected AggregatedTransactionCoordinatorStats initialValue() { }; private static final FastThreadLocal localManageLedgerStats = - new FastThreadLocal<>() { + new FastThreadLocal() { @Override protected ManagedLedgerStats initialValue() { return new ManagedLedgerStats(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreamsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreamsTest.java index 6e2bd529ffa1f..15c29a0dc66bb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreamsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreamsTest.java @@ -77,7 +77,7 @@ private String writeToString() { for (int i = 0; i < readableBytes; i++) { out.write(buffer.getByte(readIndex + i)); } - return out.toString(StandardCharsets.UTF_8); + return out.toString(); } finally { buffer.release(); } From 2f00fe31283af18137fd21bdb754077afe3826ff Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Tue, 13 Sep 2022 11:08:23 +0100 Subject: [PATCH 18/19] [fix][broker] revert unrelated change --- docker/pulsar/scripts/apply-config-from-env-with-prefix.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docker/pulsar/scripts/apply-config-from-env-with-prefix.py b/docker/pulsar/scripts/apply-config-from-env-with-prefix.py index b1c479343e01d..3f6bc2e4d3b85 100755 --- a/docker/pulsar/scripts/apply-config-from-env-with-prefix.py +++ b/docker/pulsar/scripts/apply-config-from-env-with-prefix.py @@ -73,10 +73,6 @@ lines[idx] = '%s=%s\n' % (k, v) - # Ensure we have a new-line at the end of the file, to avoid issue - # when appending more lines to the config - lines.append('\n') - # Add new keys from Env for k in sorted(os.environ.keys()): v = os.environ[k] From 312383738331abdad4deb03568f976d2c0604e1f Mon Sep 17 00:00:00 2001 From: Mark Silcox Date: Tue, 13 Sep 2022 11:42:10 +0100 Subject: [PATCH 19/19] [fix][broker] fix merge conflict --- .../org/apache/pulsar/broker/stats/prometheus/TopicStats.java | 4 ++++ 1 file changed, 4 insertions(+) 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 4f38aaf43ba9a..e91521aff5511 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 @@ -127,6 +127,10 @@ public static void printTopicStats(PrometheusMetricStreams stream, TopicStats st splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_msg_backlog", stats.msgBacklog, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_write_rate", stats.managedLedgerStats.storageWriteRate, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_read_rate", stats.managedLedgerStats.storageReadRate, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_publish_rate_limit_times", stats.publishRateLimitedTimes,