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

[monitor][txn] Add metrics for transaction #15140

Merged
merged 25 commits into from
Aug 12, 2022
Merged
Show file tree
Hide file tree
Changes from 5 commits
Commits
Show all changes
25 commits
Select commit Hold shift + click to select a range
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -753,7 +753,8 @@ public void start() throws PulsarServerException {
new HashedWheelTimer(new DefaultThreadFactory("pulsar-transaction-timer"));
transactionBufferClient = TransactionBufferClientImpl.create(this, transactionTimer,
config.getTransactionBufferClientMaxConcurrentRequests(),
config.getTransactionBufferClientOperationTimeoutInMills());
config.getTransactionBufferClientOperationTimeoutInMills(),
config.isExposeTopicLevelMetricsInPrometheus());

transactionMetadataStoreService = new TransactionMetadataStoreService(TransactionMetadataStoreProvider
.newProvider(config.getTransactionMetadataStoreProviderClassName()), this,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1888,6 +1888,8 @@ public CompletableFuture<TopicStatsImpl> asyncGetStats(boolean getPreciseBacklog
stats.bytesOutCounter = bytesOutFromRemovedSubscriptions.longValue();
stats.msgOutCounter = msgOutFromRemovedSubscriptions.longValue();
stats.publishRateLimitedTimes = publishRateLimitedTimes;
stats.ongoingTxnCount = getTransactionBuffer().getOngoingTxnCount();
tjiuming marked this conversation as resolved.
Show resolved Hide resolved
stats.abortTxnCount = getTransactionBuffer().getAbortTxnCount();

subscriptions.forEach((name, subscription) -> {
SubscriptionStatsImpl subStats =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,9 @@ public class AggregatedNamespaceStats {
public long msgBacklog;
public long msgDelayed;

public long ongoingTxnCount;
public long abortTxnCount;

long backlogQuotaLimit;
long backlogQuotaLimitTime;

Expand Down Expand Up @@ -76,6 +79,9 @@ void updateStats(TopicStats stats) {
bytesOutCounter += stats.bytesOutCounter;
msgOutCounter += stats.msgOutCounter;

this.ongoingTxnCount += stats.ongoingTxnCount;
this.abortTxnCount += stats.abortTxnCount;

managedLedgerStats.storageSize += stats.managedLedgerStats.storageSize;
managedLedgerStats.storageLogicalSize += stats.managedLedgerStats.storageLogicalSize;
managedLedgerStats.backlogSize += stats.managedLedgerStats.backlogSize;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,8 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include
stats.bytesOutCounter = tStatus.bytesOutCounter;
stats.averageMsgSize = tStatus.averageMsgSize;
stats.publishRateLimitedTimes = tStatus.publishRateLimitedTimes;
stats.abortTxnCount = tStatus.abortTxnCount;
stats.ongoingTxnCount = tStatus.ongoingTxnCount;

stats.producersCount = 0;
topic.getProducers().values().forEach(producer -> {
Expand Down Expand Up @@ -324,6 +326,8 @@ private static void printNamespaceStats(SimpleTextOutputStream stream, String cl
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_onging_transaction_count", stats.ongoingTxnCount);
metric(stream, cluster, namespace, "pulsar_abort_transaction_count", stats.abortTxnCount);

metric(stream, cluster, namespace, "pulsar_in_bytes_total", stats.bytesInCounter);
metric(stream, cluster, namespace, "pulsar_in_messages_total", stats.msgInCounter);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,9 @@ class TopicStats {
long bytesOutCounter;
double averageMsgSize;

long ongoingTxnCount;
tjiuming marked this conversation as resolved.
Show resolved Hide resolved
long abortTxnCount;

public long msgBacklog;
long publishRateLimitedTimes;

Expand Down Expand Up @@ -81,6 +84,9 @@ public void reset() {
bytesOutCounter = 0;
msgOutCounter = 0;

ongoingTxnCount = 0;
abortTxnCount = 0;

managedLedgerStats.reset();
msgBacklog = 0;
publishRateLimitedTimes = 0L;
Expand Down Expand Up @@ -127,6 +133,11 @@ static void printTopicStats(SimpleTextOutputStream stream, String cluster, Strin
metric(stream, cluster, namespace, topic, "pulsar_average_msg_size", stats.averageMsgSize,
splitTopicAndPartitionIndexLabel);

metric(stream, cluster, namespace, topic, "pulsar_onging_transaction_count", stats.ongoingTxnCount,
tjiuming marked this conversation as resolved.
Show resolved Hide resolved
splitTopicAndPartitionIndexLabel);
metric(stream, cluster, namespace, topic, "pulsar_abort_transaction_count", stats.abortTxnCount,
tjiuming marked this conversation as resolved.
Show resolved Hide resolved
splitTopicAndPartitionIndexLabel);

metric(stream, cluster, namespace, topic, "pulsar_storage_size", stats.managedLedgerStats.storageSize,
splitTopicAndPartitionIndexLabel);
metric(stream, cluster, namespace, topic, "pulsar_storage_logical_size",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -176,4 +176,10 @@ public interface TransactionBuffer {
* @return a future which has completely if isTxn = false. Or a future return by takeSnapshot.
*/
CompletableFuture<Void> checkIfTBRecoverCompletely(boolean isTxn);



long getOngoingTxnCount();

long getAbortTxnCount();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/**
* 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.transaction.buffer;

import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientStatsImpl;

public interface TransactionBufferClientStats {

void recordAbortFailed(String topic);
tjiuming marked this conversation as resolved.
Show resolved Hide resolved

void recordCommitFailed(String topic);

void recordAbortLatency(String topic, long cost);

void recordCommitLatency(String topic, long cost);

void close();
tjiuming marked this conversation as resolved.
Show resolved Hide resolved


static TransactionBufferClientStats create(boolean exposeTopicMetrics) {
return TransactionBufferClientStatsImpl.getInstance(exposeTopicMetrics);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -388,4 +388,21 @@ public TransactionBufferStats getStats() {
public CompletableFuture<Void> checkIfTBRecoverCompletely(boolean isTxn) {
return CompletableFuture.completedFuture(null);
}

@Override
public long getOngoingTxnCount() {
return this.buffers.values().stream()
.filter(txnBuffer -> txnBuffer.status.equals(TxnStatus.OPEN)
|| txnBuffer.status.equals(TxnStatus.COMMITTING)
|| txnBuffer.status.equals(TxnStatus.ABORTING)
)
.count();
}

@Override
public long getAbortTxnCount() {
return this.buffers.values().stream()
.filter(txnBuffer -> txnBuffer.status.equals(TxnStatus.ABORTED))
.count();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -236,6 +236,16 @@ public CompletableFuture<Void> checkIfTBRecoverCompletely(boolean isTxnEnabled)
}
}

@Override
public long getOngoingTxnCount() {
tjiuming marked this conversation as resolved.
Show resolved Hide resolved
return this.ongoingTxns.size();
}

@Override
public long getAbortTxnCount() {
return this.aborts.size();
}


@Override
public CompletableFuture<Position> appendBufferToTxn(TxnID txnId, long sequenceId, ByteBuf buffer) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import io.netty.util.HashedWheelTimer;
import java.util.concurrent.CompletableFuture;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.broker.transaction.buffer.TransactionBufferClientStats;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.client.api.transaction.TransactionBufferClient;
Expand All @@ -35,47 +36,84 @@
public class TransactionBufferClientImpl implements TransactionBufferClient {

private final TransactionBufferHandler tbHandler;
private final TransactionBufferClientStats stats;

private TransactionBufferClientImpl(TransactionBufferHandler tbHandler) {
private TransactionBufferClientImpl(TransactionBufferHandler tbHandler, boolean exposeTopicLevelMetrics) {
this.tbHandler = tbHandler;
this.stats = TransactionBufferClientStats.create(exposeTopicLevelMetrics);
}

public static TransactionBufferClient create(PulsarService pulsarService, HashedWheelTimer timer,
int maxConcurrentRequests, long operationTimeoutInMills) throws PulsarServerException {
int maxConcurrentRequests, long operationTimeoutInMills, boolean exposeTopicLevelMetrics)
throws PulsarServerException {
TransactionBufferHandler handler = new TransactionBufferHandlerImpl(pulsarService, timer,
maxConcurrentRequests, operationTimeoutInMills);
return new TransactionBufferClientImpl(handler);
return new TransactionBufferClientImpl(handler, exposeTopicLevelMetrics);
}

@Override
public CompletableFuture<TxnID> commitTxnOnTopic(String topic, long txnIdMostBits,
long txnIdLeastBits, long lowWaterMark) {
return tbHandler.endTxnOnTopic(topic, txnIdMostBits, txnIdLeastBits, TxnAction.COMMIT, lowWaterMark);
long start = System.currentTimeMillis();
tjiuming marked this conversation as resolved.
Show resolved Hide resolved
return tbHandler.endTxnOnTopic(topic, txnIdMostBits, txnIdLeastBits, TxnAction.COMMIT, lowWaterMark)
.whenComplete((__, t) -> {
if (null != t) {
this.stats.recordCommitFailed(topic);
} else {
this.stats.recordCommitLatency(topic, System.currentTimeMillis() - start);
}
});
}

@Override
public CompletableFuture<TxnID> abortTxnOnTopic(String topic, long txnIdMostBits,
long txnIdLeastBits, long lowWaterMark) {
return tbHandler.endTxnOnTopic(topic, txnIdMostBits, txnIdLeastBits, TxnAction.ABORT, lowWaterMark);
long start = System.currentTimeMillis();
return tbHandler.endTxnOnTopic(topic, txnIdMostBits, txnIdLeastBits, TxnAction.ABORT, lowWaterMark)
.whenComplete((__, t) -> {
if (null != t) {
this.stats.recordAbortFailed(topic);
} else {
this.stats.recordAbortLatency(topic, System.currentTimeMillis() - start);
}
});
}

@Override
public CompletableFuture<TxnID> commitTxnOnSubscription(String topic, String subscription, long txnIdMostBits,
long txnIdLeastBits, long lowWaterMark) {
long start = System.currentTimeMillis();
return tbHandler.endTxnOnSubscription(topic, subscription, txnIdMostBits, txnIdLeastBits,
TxnAction.COMMIT, lowWaterMark);
TxnAction.COMMIT, lowWaterMark)
.whenComplete((__, t) -> {
if (null != t) {
this.stats.recordCommitFailed(topic);
} else {
this.stats.recordCommitLatency(topic, System.currentTimeMillis() - start);
}
});
}

@Override
public CompletableFuture<TxnID> abortTxnOnSubscription(String topic, String subscription,
long txnIdMostBits, long txnIdLeastBits, long lowWaterMark) {
long start = System.currentTimeMillis();
return tbHandler.endTxnOnSubscription(topic, subscription, txnIdMostBits, txnIdLeastBits,
TxnAction.ABORT, lowWaterMark);
TxnAction.ABORT, lowWaterMark)
.whenComplete((__, t) -> {
if (null != t) {
this.stats.recordAbortFailed(topic);

} else {
this.stats.recordAbortLatency(topic, System.currentTimeMillis() - start);
}
});
}

@Override
public void close() {
tbHandler.close();
this.stats.close();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/**
* 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.transaction.buffer.impl;

import io.prometheus.client.CollectorRegistry;
import io.prometheus.client.Counter;
import io.prometheus.client.Summary;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.pulsar.broker.transaction.buffer.TransactionBufferClientStats;
import org.apache.pulsar.common.naming.TopicName;

public final class TransactionBufferClientStatsImpl implements TransactionBufferClientStats {
private static final double[] QUANTILES = {0.50, 0.75, 0.95, 0.99, 0.999, 0.9999, 1};
private final AtomicBoolean closed = new AtomicBoolean(false);

private final Counter abortFailed;
private final Counter commitFailed;
private final Summary abortLatency;
private final Summary commitLatency;

private final boolean exposeTopicLevelMetrics;

private static TransactionBufferClientStats instance;

private TransactionBufferClientStatsImpl(boolean exposeTopicLevelMetrics) {
this.exposeTopicLevelMetrics = exposeTopicLevelMetrics;
String[] labelNames = exposeTopicLevelMetrics ?
new String[]{"namespace", "topic"} : new String[]{"namespace"};

this.abortFailed = Counter.build("pulsar_transaction_buffer_client_abort_failed", "-")
.labelNames(labelNames)
.register();
tjiuming marked this conversation as resolved.
Show resolved Hide resolved
this.commitFailed = Counter.build("pulsar_transaction_buffer_client_commit_failed", "-")
.labelNames(labelNames)
.register();
this.abortLatency =
this.buildSummary("pulsar_transaction_buffer_client_abort_latency", "-", labelNames);
this.commitLatency =
this.buildSummary("pulsar_transaction_buffer_client_commit_latency", "-", labelNames);
}

private Summary buildSummary(String name, String help, String[] labelNames) {
Summary.Builder builder = Summary.build(name, help)
.labelNames(labelNames);
for (double quantile : QUANTILES) {
builder.quantile(quantile, 0.01D);
}
return builder.register();
}

public static synchronized TransactionBufferClientStats getInstance(boolean exposeTopicLevelMetrics) {
if (null == instance) {
instance = new TransactionBufferClientStatsImpl( exposeTopicLevelMetrics);
}

return instance;
}

@Override
public void recordAbortFailed(String topic) {
this.abortFailed.labels(labelValues(topic)).inc();
}

@Override
public void recordCommitFailed(String topic) {
this.commitFailed.labels(labelValues(topic)).inc();
}

@Override
public void recordAbortLatency(String topic, long cost) {
this.abortLatency.labels(labelValues(topic)).observe(cost);
}

@Override
public void recordCommitLatency(String topic, long cost) {
this.commitLatency.labels(labelValues(topic)).observe(cost);
}

private String[] labelValues(String topic) {
try {
TopicName topicName = TopicName.get(topic);
return exposeTopicLevelMetrics ?
new String[]{topicName.getNamespace(), topic} : new String[]{topicName.getNamespace()};
} catch (Throwable t) {
tjiuming marked this conversation as resolved.
Show resolved Hide resolved
return exposeTopicLevelMetrics ? new String[]{"unknown", "unknown"} : new String[]{"unknown"};
}
}

@Override
public void close() {
if (instance == this && this.closed.compareAndSet(false, true)) {
tjiuming marked this conversation as resolved.
Show resolved Hide resolved
instance = null;
CollectorRegistry.defaultRegistry.unregister(this.abortFailed);
CollectorRegistry.defaultRegistry.unregister(this.commitFailed);
CollectorRegistry.defaultRegistry.unregister(this.abortLatency);
CollectorRegistry.defaultRegistry.unregister(this.commitLatency);
}
}
}
Loading