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

Export Prometheus metric for messageTTL #8871

Merged
merged 7 commits into from Dec 11, 2020
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 @@ -20,6 +20,8 @@

import java.util.Optional;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import java.util.concurrent.atomic.LongAdder;

import org.apache.bookkeeper.mledger.AsyncCallbacks.FindEntryCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback;
import org.apache.bookkeeper.mledger.ManagedCursor;
Expand All @@ -39,6 +41,7 @@ public class PersistentMessageExpiryMonitor implements FindEntryCallback {
private final String subName;
private final String topicName;
private final Rate msgExpired;
private final LongAdder totalMsgExpired;
private final boolean autoSkipNonRecoverableData;
private final PersistentSubscription subscription;

Expand All @@ -55,6 +58,7 @@ public PersistentMessageExpiryMonitor(String topicName, String subscriptionName,
this.subName = subscriptionName;
this.subscription = subscription;
this.msgExpired = new Rate();
this.totalMsgExpired = new LongAdder();
// check to avoid test failures
this.autoSkipNonRecoverableData = this.cursor.getManagedLedger() != null
&& this.cursor.getManagedLedger().getConfig().isAutoSkipNonRecoverableData();
Expand Down Expand Up @@ -96,13 +100,18 @@ public double getMessageExpiryRate() {
return msgExpired.getRate();
}

public long getTotalMessageExpired() {
return totalMsgExpired.sum();
}

private static final Logger log = LoggerFactory.getLogger(PersistentMessageExpiryMonitor.class);

private final MarkDeleteCallback markDeleteCallback = new MarkDeleteCallback() {
@Override
public void markDeleteComplete(Object ctx) {
long numMessagesExpired = (long) ctx - cursor.getNumberOfEntriesInBacklog(false);
msgExpired.recordMultipleEvents(numMessagesExpired, 0 /* no value stats */);
totalMsgExpired.add(numMessagesExpired);
updateRates();
// If the subscription is a Key_Shared subscription, we should to trigger message dispatch.
if (subscription != null && subscription.getType() == PulsarApi.CommandSubscribe.SubType.Key_Shared) {
Expand Down
Expand Up @@ -928,6 +928,7 @@ public SubscriptionStats getStats(Boolean getPreciseBacklog) {
subStats.msgBacklog = getNumberOfEntriesInBacklog(getPreciseBacklog);
subStats.msgBacklogNoDelayed = subStats.msgBacklog - subStats.msgDelayed;
subStats.msgRateExpired = expiryMonitor.getMessageExpiryRate();
subStats.totalMsgExpired = expiryMonitor.getTotalMessageExpired();
subStats.isReplicated = isReplicated();
subStats.isDurable = cursor.isDurable();
if (getType() == SubType.Key_Shared && dispatcher instanceof PersistentStickyKeyDispatcherMultipleConsumers) {
Expand Down
Expand Up @@ -44,5 +44,11 @@ public class AggregatedSubscriptionStats {

long bytesOutCounter;

long lastExpireTimestamp;

double msgRateExpired;

long totalMsgExpired;

public Map<Consumer, AggregatedConsumerStats> consumerStat = new HashMap<>();
}
Expand Up @@ -137,6 +137,9 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include
.computeIfAbsent(subName, k -> new AggregatedSubscriptionStats());
subsStats.msgBacklog = subscriptionStats.msgBacklog;
subsStats.msgDelayed = subscriptionStats.msgDelayed;
subsStats.lastExpireTimestamp = subscriptionStats.lastExpireTimestamp;
subsStats.msgRateExpired = subscriptionStats.msgRateExpired;
subsStats.totalMsgExpired = subscriptionStats.totalMsgExpired;
subsStats.msgBacklogNoDelayed = subsStats.msgBacklog - subsStats.msgDelayed;
subscriptionStats.consumers.forEach(cStats -> {
stats.consumersCount++;
Expand Down
Expand Up @@ -157,6 +157,12 @@ static void printTopicStats(SimpleTextOutputStream stream, String cluster, Strin
metric(stream, cluster, namespace, topic, n, "pulsar_subscription_back_log", subsStats.msgBacklog);
metric(stream, cluster, namespace, topic, n, "pulsar_subscription_back_log_no_delayed", subsStats.msgBacklogNoDelayed);
metric(stream, cluster, namespace, topic, n, "pulsar_subscription_delayed", subsStats.msgDelayed);
metric(stream, cluster, namespace, topic, n, "pulsar_subscription_last_expire_timestamp",
subsStats.lastExpireTimestamp);
metric(stream, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_expired",
subsStats.msgRateExpired);
metric(stream, cluster, namespace, topic, n, "pulsar_subscription_total_msg_expired",
subsStats.totalMsgExpired);
metric(stream, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_redeliver", subsStats.msgRateRedeliver);
metric(stream, cluster, namespace, topic, n, "pulsar_subscription_unacked_messages", subsStats.unackedMessages);
metric(stream, cluster, namespace, topic, n, "pulsar_subscription_blocked_on_unacked_messages", subsStats.blockedSubscriptionOnUnackedMsgs ? 1 : 0);
Expand Down
Expand Up @@ -25,6 +25,10 @@
import static org.testng.Assert.fail;

import java.io.ByteArrayOutputStream;
import java.lang.reflect.Field;
import java.math.RoundingMode;
import java.text.NumberFormat;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
Expand All @@ -35,6 +39,9 @@
import java.util.regex.Pattern;

import org.apache.pulsar.broker.service.BrokerTestBase;
import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.broker.service.persistent.PersistentMessageExpiryMonitor;
import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Producer;
Expand Down Expand Up @@ -185,6 +192,95 @@ public void testPerTopicStats() throws Exception {
c2.close();
}

@Test
public void testPerTopicExpiredStat() throws Exception {
String ns = "prop/ns-abc1";
admin.namespaces().createNamespace(ns);
String topic1 = "persistent://" + ns + "/testPerTopicExpiredStat1";
String topic2 = "persistent://" + ns + "/testPerTopicExpiredStat2";
List<String> topicList = Arrays.asList(topic2,topic1);
Producer<byte[]> p1 = pulsarClient.newProducer().topic(topic1).create();
Producer<byte[]> p2 = pulsarClient.newProducer().topic(topic2).create();
admin.namespaces().setNamespaceMessageTTL(ns,1);
final String subName = "test";
for (String topic : topicList) {
pulsarClient.newConsumer()
.topic(topic)
.subscriptionName(subName)
.subscribe().close();
}

final int messages = 10;

for (int i = 0; i < messages; i++) {
String message = "my-message-" + i;
p1.send(message.getBytes());
p2.send(message.getBytes());
}

p1.close();
p2.close();
// make sure message is expired
Thread.sleep(2000);
315157973 marked this conversation as resolved.
Show resolved Hide resolved
pulsar.getBrokerService().forEachTopic(Topic::checkMessageExpiry);

ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
PrometheusMetricsGenerator.generate(pulsar, true, false, statsOut);
String metricsStr = new String(statsOut.toByteArray());
Multimap<String, Metric> metrics = parseMetrics(metricsStr);

// There should be 2 metrics with different tags for each topic
List<Metric> cm = (List<Metric>) metrics.get("pulsar_subscription_last_expire_timestamp");
assertEquals(cm.size(), 2);
assertEquals(cm.get(0).tags.get("topic"), topic2);
assertEquals(cm.get(0).tags.get("namespace"), ns);
assertEquals(cm.get(1).tags.get("topic"), topic1);
assertEquals(cm.get(1).tags.get("namespace"), ns);

//check value
Field field = PersistentSubscription.class.getDeclaredField("lastExpireTimestamp");
field.setAccessible(true);
for (int i = 0; i < topicList.size(); i++) {
PersistentSubscription subscription = (PersistentSubscription)
pulsar.getBrokerService().getTopicIfExists(topicList.get(i)).get().get().getSubscription(subName);
assertEquals((long) field.get(subscription), (long) cm.get(i).value);
}

cm = (List<Metric>) metrics.get("pulsar_subscription_msg_rate_expired");
assertEquals(cm.size(), 2);
assertEquals(cm.get(0).tags.get("topic"), topic2);
assertEquals(cm.get(0).tags.get("namespace"), ns);
assertEquals(cm.get(1).tags.get("topic"), topic1);
assertEquals(cm.get(1).tags.get("namespace"), ns);
//check value
field = PersistentSubscription.class.getDeclaredField("expiryMonitor");
field.setAccessible(true);
NumberFormat nf = NumberFormat.getNumberInstance();
nf.setMaximumFractionDigits(3);
nf.setRoundingMode(RoundingMode.DOWN);
for (int i = 0; i < topicList.size(); i++) {
PersistentSubscription subscription = (PersistentSubscription)
pulsar.getBrokerService().getTopicIfExists(topicList.get(i)).get().get().getSubscription(subName);
PersistentMessageExpiryMonitor monitor = (PersistentMessageExpiryMonitor) field.get(subscription);
assertEquals(Double.valueOf(nf.format(monitor.getMessageExpiryRate())).doubleValue(), cm.get(i).value);
}

cm = (List<Metric>) metrics.get("pulsar_subscription_total_msg_expired");
assertEquals(cm.size(), 2);
assertEquals(cm.get(0).tags.get("topic"), topic2);
assertEquals(cm.get(0).tags.get("namespace"), ns);
assertEquals(cm.get(1).tags.get("topic"), topic1);
assertEquals(cm.get(1).tags.get("namespace"), ns);
//check value
for (int i = 0; i < topicList.size(); i++) {
PersistentSubscription subscription = (PersistentSubscription)
pulsar.getBrokerService().getTopicIfExists(topicList.get(i)).get().get().getSubscription(subName);
PersistentMessageExpiryMonitor monitor = (PersistentMessageExpiryMonitor) field.get(subscription);
assertEquals(monitor.getTotalMessageExpired(), (long)cm.get(i).value);
}

}

@Test
public void testPerNamespaceStats() throws Exception {
Producer<byte[]> p1 = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic1").create();
Expand Down
Expand Up @@ -71,6 +71,9 @@ public class SubscriptionStats {
/** Total rate of messages expired on this subscription (msg/s). */
public double msgRateExpired;

/** Total messages expired on this subscription. */
public long totalMsgExpired;

/** Last message expire execution timestamp. */
public long lastExpireTimestamp;

Expand Down Expand Up @@ -110,6 +113,7 @@ public void reset() {
msgBacklogNoDelayed = 0;
unackedMessages = 0;
msgRateExpired = 0;
totalMsgExpired = 0;
lastExpireTimestamp = 0L;
consumers.clear();
}
Expand All @@ -127,6 +131,7 @@ public SubscriptionStats add(SubscriptionStats stats) {
this.msgBacklogNoDelayed += stats.msgBacklogNoDelayed;
this.unackedMessages += stats.unackedMessages;
this.msgRateExpired += stats.msgRateExpired;
this.totalMsgExpired += stats.totalMsgExpired;
this.isReplicated |= stats.isReplicated;
this.isDurable |= stats.isDurable;
if (this.consumers.size() != stats.consumers.size()) {
Expand Down