Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Expose compaction metrics to Prometheus #11739

Merged
merged 10 commits into from Aug 31, 2021
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -1594,8 +1594,6 @@ public void forEachTopic(Consumer<Topic> consumer) {
});
}



public BacklogQuotaManager getBacklogQuotaManager() {
return this.backlogQuotaManager;
}
Expand Down
Expand Up @@ -151,6 +151,7 @@
import org.apache.pulsar.compaction.CompactedTopic;
import org.apache.pulsar.compaction.CompactedTopicContext;
import org.apache.pulsar.compaction.CompactedTopicImpl;
import org.apache.pulsar.compaction.CompactionRecord;
import org.apache.pulsar.compaction.Compactor;
import org.apache.pulsar.compaction.CompactorMXBean;
import org.apache.pulsar.metadata.api.MetadataStoreException;
Expand Down Expand Up @@ -1907,14 +1908,19 @@ public TopicStatsImpl getStats(boolean getPreciseBacklog, boolean subscriptionBa
stats.lastOffloadSuccessTimeStamp = ledger.getLastOffloadedSuccessTimestamp();
stats.lastOffloadFailureTimeStamp = ledger.getLastOffloadedFailureTimestamp();
Optional<CompactorMXBean> mxBean = getCompactorMXBean();
stats.compaction.lastCompactionRemovedEventCount = mxBean.map(stat ->
stat.getLastCompactionRemovedEventCount(topic)).orElse(0L);
stats.compaction.lastCompactionSucceedTimestamp = mxBean.map(stat ->
stat.getLastCompactionSucceedTimestamp(topic)).orElse(0L);
stats.compaction.lastCompactionFailedTimestamp = mxBean.map(stat ->
stat.getLastCompactionFailedTimestamp(topic)).orElse(0L);
stats.compaction.lastCompactionDurationTimeInMills = mxBean.map(stat ->
stat.getLastCompactionDurationTimeInMills(topic)).orElse(0L);

stats.compaction.lastCompactionRemovedEventCount = 0L;
stats.compaction.lastCompactionSucceedTimestamp = 0L;
stats.compaction.lastCompactionFailedTimestamp = 0L;
stats.compaction.lastCompactionDurationTimeInMills = 0L;
Technoboy- marked this conversation as resolved.
Show resolved Hide resolved
if (mxBean.isPresent()) {
CompactionRecord compactionRecord = mxBean.get().getCompactionRecordForTopic(topic);
stats.compaction.lastCompactionRemovedEventCount = compactionRecord.getLastCompactionRemovedEventCount();
stats.compaction.lastCompactionSucceedTimestamp = compactionRecord.getLastCompactionSucceedTimestamp();
stats.compaction.lastCompactionFailedTimestamp = compactionRecord.getLastCompactionFailedTimestamp();
stats.compaction.lastCompactionDurationTimeInMills =
compactionRecord.getLastCompactionDurationTimeInMills();
}

return stats;
}
Expand Down
Expand Up @@ -214,11 +214,7 @@ protected Metrics createMetricsByDimension(String namespace, String fromClusterN
}

protected void populateAggregationMap(Map<String, List<Double>> map, String mkey, double value) {
if (!map.containsKey(mkey)) {
map.put(mkey, Lists.newArrayList(value));
} else {
map.get(mkey).add(value);
}
map.computeIfAbsent(mkey, __ -> Lists.newArrayList(value)).add(value);
Technoboy- marked this conversation as resolved.
Show resolved Hide resolved
}

protected void populateAggregationMapWithSum(Map<String, Double> map, String mkey, double value) {
Expand All @@ -242,24 +238,11 @@ protected void populateMaxMap(Map<String, Long> map, String mkey, long value) {
*/
protected void populateDimensionMap(Map<Metrics, List<ManagedLedgerImpl>> ledgersByDimensionMap, Metrics metrics,
ManagedLedgerImpl ledger) {
if (!ledgersByDimensionMap.containsKey(metrics)) {
// create new list
ledgersByDimensionMap.put(metrics, Lists.newArrayList(ledger));
} else {
// add to collection
ledgersByDimensionMap.get(metrics).add(ledger);
}
ledgersByDimensionMap.computeIfAbsent(metrics, __ -> Lists.newArrayList(ledger)).add(ledger);
Technoboy- marked this conversation as resolved.
Show resolved Hide resolved
}

protected void populateDimensionMap(Map<Metrics, List<TopicStats>> topicsStatsByDimensionMap,
Metrics metrics, TopicStats destStats) {
if (!topicsStatsByDimensionMap.containsKey(metrics)) {
// create new list
topicsStatsByDimensionMap.put(metrics, Lists.newArrayList(destStats));
} else {
// add to collection
topicsStatsByDimensionMap.get(metrics).add(destStats);
}

topicsStatsByDimensionMap.computeIfAbsent(metrics, __ -> Lists.newArrayList(destStats)).add(destStats);
Technoboy- marked this conversation as resolved.
Show resolved Hide resolved
}
}
Expand Up @@ -35,6 +35,7 @@
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.stats.Metrics;
import org.apache.pulsar.compaction.CompactedTopicContext;
import org.apache.pulsar.compaction.CompactionRecord;
import org.apache.pulsar.compaction.Compactor;
import org.apache.pulsar.compaction.CompactorMXBean;
import org.apache.pulsar.compaction.CompactorMXBeanImpl;
Expand Down Expand Up @@ -82,26 +83,27 @@ private List<Metrics> aggregate(Map<Metrics, List<String>> topicsByDimension) {
List<String> topics = entry.getValue();
tempAggregatedMetricsMap.clear();
for (String topic : topics) {
CompactionRecord compactionRecord = compactorMXBean.getCompactionRecordForTopic(topic);
populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_compaction_removedEventCount",
compactorMXBean.getCompactionRemovedEventCount(topic));
compactionRecord.getCompactionRemovedEventCount());

populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_compaction_succeedCount",
compactorMXBean.getCompactionSucceedCount(topic));
compactionRecord.getCompactionSucceedCount());

populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_compaction_failedCount",
compactorMXBean.getCompactionFailedCount(topic));
compactionRecord.getCompactionFailedCount());

populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_compaction_durationTimeInMills",
compactorMXBean.getCompactionDurationTimeInMills(topic));
compactionRecord.getCompactionDurationTimeInMills());

populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_compaction_readThroughput",
compactorMXBean.getCompactionReadThroughput(topic));
compactionRecord.getCompactionReadThroughput());

populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_compaction_writeThroughput",
compactorMXBean.getCompactionWriteThroughput(topic));
compactionRecord.getCompactionWriteThroughput());

BRK_COMPACTION_LATENCY_BUCKETS.populateBucketEntries(tempAggregatedMetricsMap,
compactorMXBean.getCompactionLatencyBuckets(topic),
compactionRecord.getCompactionLatencyBuckets(),
statsPeriodSeconds);

CompletableFuture<Optional<Topic>> topicHandle = pulsar.getBrokerService().getTopicIfExists(topic);
Expand Down Expand Up @@ -159,10 +161,6 @@ private Optional<CompactorMXBean> getCompactorMXBean() {
}

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);
}
topicsByDimensionMap.computeIfAbsent(metrics, __ -> Lists.newArrayList(topic)).add(topic);
Technoboy- marked this conversation as resolved.
Show resolved Hide resolved
}
}
@@ -0,0 +1,126 @@
/**
* 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.compaction;

import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.LongAdder;
import lombok.Getter;
import org.apache.bookkeeper.mledger.util.StatsBuckets;
import org.apache.pulsar.common.stats.Rate;

public class CompactionRecord {

public static final long[] WRITE_LATENCY_BUCKETS_USEC = { 500, 1_000, 5_000, 10_000, 20_000, 50_000, 100_000,
200_000, 1000_000 };

@Getter
private long lastCompactionRemovedEventCount = 0L;
@Getter
private long lastCompactionSucceedTimestamp = 0L;
@Getter
private long lastCompactionFailedTimestamp = 0L;
@Getter
private long lastCompactionDurationTimeInMills = 0L;

private LongAdder lastCompactionRemovedEventCountOp = new LongAdder();
private long lastCompactionStartTimeOp;

private final LongAdder compactionRemovedEventCount = new LongAdder();
private final LongAdder compactionSucceedCount = new LongAdder();
private final LongAdder compactionFailedCount = new LongAdder();
private final LongAdder compactionDurationTimeInMills = new LongAdder();
public final StatsBuckets writeLatencyStats = new StatsBuckets(WRITE_LATENCY_BUCKETS_USEC);
public final Rate writeRate = new Rate();
public final Rate readRate = new Rate();

public void reset() {
compactionRemovedEventCount.reset();
compactionSucceedCount.reset();
compactionFailedCount.reset();
compactionDurationTimeInMills.reset();
writeLatencyStats.reset();
}

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

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

public void addCompactionEndOp(boolean succeed) {
lastCompactionDurationTimeInMills = System.currentTimeMillis()
- lastCompactionStartTimeOp;
compactionDurationTimeInMills.add(lastCompactionDurationTimeInMills);
lastCompactionRemovedEventCount = lastCompactionRemovedEventCountOp.longValue();
if (succeed) {
lastCompactionSucceedTimestamp = System.currentTimeMillis();
compactionSucceedCount.increment();
} else {
lastCompactionFailedTimestamp = System.currentTimeMillis();
compactionFailedCount.increment();
}
}

public void addCompactionReadOp(long readableBytes) {
readRate.recordEvent(readableBytes);
}

public void addCompactionWriteOp(long writeableBytes) {
writeRate.recordEvent(writeableBytes);
}

public void addCompactionLatencyOp(long latency, TimeUnit unit) {
writeLatencyStats.addValue(unit.toMicros(latency));
}

public long getCompactionRemovedEventCount() {
return compactionRemovedEventCount.longValue();
}

public long getCompactionSucceedCount() {
return compactionSucceedCount.longValue();
}

public long getCompactionFailedCount() {
return compactionFailedCount.longValue();
}

public long getCompactionDurationTimeInMills() {
return compactionDurationTimeInMills.longValue();
}

public long[] getCompactionLatencyBuckets() {
writeLatencyStats.refresh();
return writeLatencyStats.getBuckets();
}

public double getCompactionReadThroughput() {
readRate.calculateRate();
return readRate.getValueRate();
}

public double getCompactionWriteThroughput() {
writeRate.calculateRate();
return writeRate.getValueRate();
}
}
Expand Up @@ -28,64 +28,15 @@
@InterfaceStability.Stable
public interface CompactorMXBean {

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

/**
* @return the timestamp of last succeed compaction
*/
long getLastCompactionSucceedTimestamp(String topic);

/**
* @return the timestamp of last failed compaction
*/
long getLastCompactionFailedTimestamp(String topic);

/**
* @return the duration time of last compaction
*/
long getLastCompactionDurationTimeInMills(String topic);

/**
* Remove metrics about this topic.
* @param topic
*/
void removeTopic(String topic);

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

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

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

/**
* @return the duration time of compaction
*/
long getCompactionDurationTimeInMills(String topic);

/**
* @return the latency buckets
*/
long[] getCompactionLatencyBuckets(String topic);

/**
* @return the read throughput of compaction
*/
double getCompactionReadThroughput(String topic);

/**
* @return the write throughput of compaction
* Get the compaction record of the topic.
* @param topic
*/
double getCompactionWriteThroughput(String topic);
CompactionRecord getCompactionRecordForTopic(String topic);
}