Skip to content

Commit

Permalink
Expose compaction metrics to Prometheus
Browse files Browse the repository at this point in the history
  • Loading branch information
Technoboy- committed Aug 22, 2021
1 parent c0ef593 commit 88b3c8f
Show file tree
Hide file tree
Showing 5 changed files with 222 additions and 19 deletions.
Expand Up @@ -1999,6 +1999,11 @@ public class ServiceConfiguration implements PulsarConfiguration {
doc = "If true, export managed cursor metrics"
)
private boolean exposeManagedCursorMetricsInPrometheus = false;
@FieldContext(
category = CATEGORY_METRICS,
doc = "If true, export compaction metrics"
)
private boolean exposeCompactionMetricsInPrometheus = false;
@FieldContext(
category = CATEGORY_METRICS,
doc = "Classname of Pluggable JVM GC metrics logger that can log GC specific metrics")
Expand Down
@@ -0,0 +1,122 @@
/**
* 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.metrics;

import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.stats.Metrics;
import org.apache.pulsar.compaction.Compactor;
import org.apache.pulsar.compaction.CompactorMXBean;

@Slf4j
public class CompactionMetrics extends AbstractMetrics {

private List<Metrics> metricsCollection;
private Map<String, Double> tempAggregatedMetricsMap;
private Map<Metrics, List<String>> topicsByDimensionMap;
private Optional<CompactorMXBean> stats;

public CompactionMetrics(PulsarService pulsar) {
super(pulsar);
this.metricsCollection = Lists.newArrayList();
this.topicsByDimensionMap = Maps.newHashMap();
this.tempAggregatedMetricsMap = Maps.newHashMap();
this.stats = getCompactorMXBean();
}

@Override
public synchronized List<Metrics> generate() {
return aggregate(groupTopicsByDimension());
}


/**
* Aggregation by namespace.
*
* @return List<Metrics>
*/
private List<Metrics> aggregate(Map<Metrics, List<String>> topicsByDimension) {
metricsCollection.clear();
if (stats.isPresent()) {
CompactorMXBean compactorMXBean = stats.get();
for (Map.Entry<Metrics, List<String>> entry : topicsByDimension.entrySet()) {
Metrics metrics = entry.getKey();
List<String> topics = entry.getValue();
tempAggregatedMetricsMap.clear();
for (String topic : topics) {
populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_cp_totalCompactionCount",
compactorMXBean.getTotalCompactionCount(topic));

populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_cp_totalCompactionRemovedEventCount",
compactorMXBean.getTotalCompactionRemovedEventCount(topic));

populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_cp_totalCompactionSucceedCount",
compactorMXBean.getTotalCompactionSucceedCount(topic));

populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_cp_totalCompactionFailedCount",
compactorMXBean.getTotalCompactionFailedCount(topic));
}
for (Map.Entry<String, Double> ma : tempAggregatedMetricsMap.entrySet()) {
metrics.put(ma.getKey(), ma.getValue());
}
metricsCollection.add(metrics);
}
}

return metricsCollection;
}

private Map<Metrics, List<String>> groupTopicsByDimension() {
topicsByDimensionMap.clear();
if (stats.isPresent()) {
tempAggregatedMetricsMap.clear();
for (String topic : stats.get().getTopics()) {
String namespace = TopicName.get(topic).getNamespace();
Metrics metrics = super.createMetricsByDimension(namespace);
populateDimensionMap(topicsByDimensionMap, metrics, topic);
}
}
return topicsByDimensionMap;
}

private Optional<CompactorMXBean> getCompactorMXBean() {
Compactor compactor = null;
try {
compactor = pulsar.getCompactor(false);
} catch (PulsarServerException e) {
log.error("get compactor error", e);
}
return Optional.ofNullable(compactor).map(c -> c.getStats());
}

private void populateDimensionMap(Map<Metrics, List<String>> topicsByDimensionMap, Metrics metrics, String topic) {
if (!topicsByDimensionMap.containsKey(metrics)) {
topicsByDimensionMap.put(metrics, Lists.newArrayList(topic));
} else {
topicsByDimensionMap.get(metrics).add(topic);
}
}
}
Expand Up @@ -43,6 +43,7 @@
import org.apache.bookkeeper.stats.StatsProvider;
import org.apache.pulsar.PulsarVersion;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.stats.metrics.CompactionMetrics;
import org.apache.pulsar.broker.stats.metrics.ManagedCursorMetrics;
import org.apache.pulsar.broker.stats.metrics.ManagedLedgerCacheMetrics;
import org.apache.pulsar.broker.stats.metrics.ManagedLedgerMetrics;
Expand Down Expand Up @@ -143,6 +144,12 @@ private static void generateBrokerBasicMetrics(PulsarService pulsar, SimpleTextO
clusterName, Collector.Type.GAUGE, stream);
}

if (pulsar.getConfiguration().isExposeCompactionMetricsInPrometheus()) {
// generate compaction metrics
parseMetricsToPrometheusMetrics(new CompactionMetrics(pulsar).generate(),
clusterName, Collector.Type.GAUGE, stream);
}

parseMetricsToPrometheusMetrics(Collections.singletonList(pulsar.getBrokerService()
.getPulsarStats().getBrokerOperabilityMetrics().generateConnectionMetrics()),
clusterName, Collector.Type.GAUGE, stream);
Expand Down
Expand Up @@ -21,6 +21,8 @@
import org.apache.bookkeeper.common.annotation.InterfaceAudience;
import org.apache.bookkeeper.common.annotation.InterfaceStability;

import java.util.Set;

/**
* JMX Bean interface for Compactor stats.
*/
Expand Down Expand Up @@ -54,4 +56,29 @@ public interface CompactorMXBean {
*/
void removeTopic(String topic);

/**
* @return the count of total compaction
*/
long getTotalCompactionCount(String topic);

/**
* @return the removed event count of total compaction
*/
long getTotalCompactionRemovedEventCount(String topic);

/**
* @return the succeed count of total compaction
*/
long getTotalCompactionSucceedCount(String topic);

/**
* @return the failed count of total compaction
*/
long getTotalCompactionFailedCount(String topic);

/**
* @return the topic set
*/
Set<String> getTopics();

}
Expand Up @@ -18,59 +18,81 @@
*/
package org.apache.pulsar.compaction;

import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.LongAdder;

public class CompactorMXBeanImpl implements CompactorMXBean {

private final ConcurrentHashMap<String, CompactRecord> compactRecordOps = new ConcurrentHashMap<>();
private final ConcurrentHashMap<String, CompactionRecord> compactionRecordOps = new ConcurrentHashMap<>();

public void addCompactionRemovedEvent(String topic) {
compactRecordOps.computeIfAbsent(topic, k -> new CompactRecord()).addCompactionRemovedEvent();
compactionRecordOps.computeIfAbsent(topic, k -> new CompactionRecord()).addCompactionRemovedEvent();
}

public void addCompactionStartOp(String topic) {
compactRecordOps.computeIfAbsent(topic, k -> new CompactRecord()).reset();
compactionRecordOps.computeIfAbsent(topic, k -> new CompactionRecord()).addCompactionStartOp();
}

public void addCompactionEndOp(String topic, boolean succeed) {
CompactRecord compactRecord = compactRecordOps.computeIfAbsent(topic, k -> new CompactRecord());
compactRecord.lastCompactionDurationTimeInMills = System.currentTimeMillis()
- compactRecord.lastCompactionStartTimeOp;
compactRecord.lastCompactionRemovedEventCount = compactRecord.lastCompactionRemovedEventCountOp.longValue();
if (succeed) {
compactRecord.lastCompactionSucceedTimestamp = System.currentTimeMillis();
} else {
compactRecord.lastCompactionFailedTimestamp = System.currentTimeMillis();
}
compactionRecordOps.computeIfAbsent(topic, k -> new CompactionRecord()).addCompactionEndOp(succeed);
}

@Override
public long getLastCompactionRemovedEventCount(String topic) {
return compactRecordOps.getOrDefault(topic, new CompactRecord()).lastCompactionRemovedEventCount;
return compactionRecordOps.getOrDefault(topic, new CompactionRecord()).lastCompactionRemovedEventCount;
}

@Override
public long getLastCompactionSucceedTimestamp(String topic) {
return compactRecordOps.getOrDefault(topic, new CompactRecord()).lastCompactionSucceedTimestamp;
return compactionRecordOps.getOrDefault(topic, new CompactionRecord()).lastCompactionSucceedTimestamp;
}

@Override
public long getLastCompactionFailedTimestamp(String topic) {
return compactRecordOps.getOrDefault(topic, new CompactRecord()).lastCompactionFailedTimestamp;
return compactionRecordOps.getOrDefault(topic, new CompactionRecord()).lastCompactionFailedTimestamp;
}

@Override
public long getLastCompactionDurationTimeInMills(String topic) {
return compactRecordOps.getOrDefault(topic, new CompactRecord()).lastCompactionDurationTimeInMills;
return compactionRecordOps.getOrDefault(topic, new CompactionRecord()).lastCompactionDurationTimeInMills;
}

@Override
public void removeTopic(String topic) {
compactRecordOps.remove(topic);
compactionRecordOps.remove(topic);
}

@Override
public long getTotalCompactionCount(String topic) {
return compactionRecordOps.getOrDefault(topic, new CompactionRecord())
.totalCompactionCount.longValue();
}

@Override
public long getTotalCompactionRemovedEventCount(String topic) {
return compactionRecordOps.getOrDefault(topic, new CompactionRecord())
.totalCompactionRemovedEventCount.longValue();
}

static class CompactRecord {
@Override
public long getTotalCompactionSucceedCount(String topic) {
return compactionRecordOps.getOrDefault(topic, new CompactionRecord())
.totalCompactionSucceedCount.longValue();
}

@Override
public long getTotalCompactionFailedCount(String topic) {
return compactionRecordOps.getOrDefault(topic, new CompactionRecord())
.totalCompactionFailedCount.longValue();
}

@Override
public Set<String> getTopics() {
return compactionRecordOps.keySet();
}

static class CompactionRecord {

private long lastCompactionRemovedEventCount = 0L;
private long lastCompactionSucceedTimestamp = 0L;
Expand All @@ -80,13 +102,33 @@ static class CompactRecord {
private LongAdder lastCompactionRemovedEventCountOp = new LongAdder();
private long lastCompactionStartTimeOp;

private final LongAdder totalCompactionCount = new LongAdder();
private final LongAdder totalCompactionRemovedEventCount = new LongAdder();
private final LongAdder totalCompactionSucceedCount = new LongAdder();
private final LongAdder totalCompactionFailedCount = new LongAdder();

public void addCompactionRemovedEvent() {
lastCompactionRemovedEventCountOp.increment();
totalCompactionRemovedEventCount.increment();
}

public void reset() {
public void addCompactionStartOp() {
lastCompactionRemovedEventCountOp.reset();
lastCompactionStartTimeOp = System.currentTimeMillis();
}

public void addCompactionEndOp(boolean succeed) {
lastCompactionDurationTimeInMills = System.currentTimeMillis()
- lastCompactionStartTimeOp;
lastCompactionRemovedEventCount = lastCompactionRemovedEventCountOp.longValue();
if (succeed) {
lastCompactionSucceedTimestamp = System.currentTimeMillis();
totalCompactionSucceedCount.increment();
} else {
lastCompactionFailedTimestamp = System.currentTimeMillis();
totalCompactionFailedCount.increment();
}
totalCompactionCount.increment();
}
}
}

0 comments on commit 88b3c8f

Please sign in to comment.