Skip to content

Commit

Permalink
[improve][txn]PIP-160 make txn components supports buffered writer me…
Browse files Browse the repository at this point in the history
…trics
  • Loading branch information
poorbarcode committed Sep 17, 2022
1 parent fac14fb commit 2ca47e0
Show file tree
Hide file tree
Showing 19 changed files with 507 additions and 68 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,7 @@
import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider;
import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl;
import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider;
import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStoreProvider;
import org.apache.pulsar.broker.validator.MultipleListenerValidator;
import org.apache.pulsar.broker.validator.TransactionBatchedWriteValidator;
import org.apache.pulsar.broker.web.WebService;
Expand Down Expand Up @@ -551,6 +552,11 @@ public CompletableFuture<Void> closeAsync() {
if (transactionExecutorProvider != null) {
transactionExecutorProvider.shutdownNow();
}
if (transactionPendingAckStoreProvider != null){
if (transactionPendingAckStoreProvider instanceof MLPendingAckStoreProvider mlPendingAckStoreProvider){
mlPendingAckStoreProvider.closeBufferedWriterMetrics();
}
}
if (this.offloaderStats != null) {
this.offloaderStats.close();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import io.netty.util.HashedWheelTimer;
import io.netty.util.Timer;
import io.netty.util.concurrent.DefaultThreadFactory;
import io.prometheus.client.CollectorRegistry;
import java.util.Collections;
import java.util.Deque;
import java.util.List;
Expand Down Expand Up @@ -67,7 +68,9 @@
import org.apache.pulsar.transaction.coordinator.exceptions.CoordinatorException.CoordinatorNotFoundException;
import org.apache.pulsar.transaction.coordinator.exceptions.CoordinatorException.InvalidTxnStatusException;
import org.apache.pulsar.transaction.coordinator.exceptions.CoordinatorException.TransactionMetadataStoreStateException;
import org.apache.pulsar.transaction.coordinator.impl.DisabledTxnLogBufferedWriterMetricsStats;
import org.apache.pulsar.transaction.coordinator.impl.TxnLogBufferedWriterConfig;
import org.apache.pulsar.transaction.coordinator.impl.TxnLogBufferedWriterMetricsStats;
import org.apache.pulsar.transaction.coordinator.proto.TxnStatus;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -89,6 +92,7 @@ public class TransactionMetadataStoreService {
// finished the request will be poll and complete the future
private final ConcurrentLongHashMap<ConcurrentLinkedDeque<CompletableFuture<Void>>> pendingConnectRequests;
private final ExecutorService internalPinnedExecutor;
private final TxnLogBufferedWriterMetricsStats bufferedWriterMetrics;

private static final long HANDLE_PENDING_CONNECT_TIME_OUT = 30000L;

Expand All @@ -109,6 +113,12 @@ public TransactionMetadataStoreService(TransactionMetadataStoreProvider transact
this.pendingConnectRequests =
ConcurrentLongHashMap.<ConcurrentLinkedDeque<CompletableFuture<Void>>>newBuilder().build();
this.internalPinnedExecutor = Executors.newSingleThreadScheduledExecutor(threadFactory);
if (pulsarService.getConfiguration().isTransactionLogBatchedWriteEnabled()) {
this.bufferedWriterMetrics = new MLTransactionMetadataStoreBufferedWriterMetrics(
pulsarService.getConfiguration().getClusterName(), pulsarService.getAdvertisedAddress());
} else {
this.bufferedWriterMetrics = DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS;
}
}

public CompletableFuture<Void> handleTcClientConnect(TransactionCoordinatorID tcId) {
Expand Down Expand Up @@ -222,7 +232,7 @@ public CompletableFuture<TransactionMetadataStore> openTransactionMetadataStore(
.openStore(tcId, pulsarService.getManagedLedgerFactory(), v,
timeoutTracker, recoverTracker,
pulsarService.getConfig().getMaxActiveTransactionsPerCoordinator(),
txnLogBufferedWriterConfig, brokerClientSharedTimer);
txnLogBufferedWriterConfig, brokerClientSharedTimer, bufferedWriterMetrics);
});
}

Expand Down Expand Up @@ -488,5 +498,16 @@ public void close () {
});
});
stores.clear();
bufferedWriterMetrics.close();
}

private static class MLTransactionMetadataStoreBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats {

private MLTransactionMetadataStoreBufferedWriterMetrics(String cluster, String brokerAdvertisedAddress) {
super("pulsar_txn_tc",
new String[]{"cluster", "broker"},
new String[]{cluster, brokerAdvertisedAddress},
CollectorRegistry.defaultRegistry);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@
import org.apache.pulsar.transaction.coordinator.impl.TxnBatchedPositionImpl;
import org.apache.pulsar.transaction.coordinator.impl.TxnLogBufferedWriter;
import org.apache.pulsar.transaction.coordinator.impl.TxnLogBufferedWriterConfig;
import org.apache.pulsar.transaction.coordinator.impl.TxnLogBufferedWriterMetricsStats;
import org.jctools.queues.MessagePassingQueue;
import org.jctools.queues.SpscArrayQueue;
import org.slf4j.Logger;
Expand Down Expand Up @@ -119,7 +120,7 @@ public class MLPendingAckStore implements PendingAckStore {
public MLPendingAckStore(ManagedLedger managedLedger, ManagedCursor cursor,
ManagedCursor subManagedCursor, long transactionPendingAckLogIndexMinLag,
TxnLogBufferedWriterConfig bufferedWriterConfig,
Timer timer) {
Timer timer, TxnLogBufferedWriterMetricsStats bufferedWriterMetrics) {
this.managedLedger = managedLedger;
this.cursor = cursor;
this.currentLoadPosition = (PositionImpl) this.cursor.getMarkDeletedPosition();
Expand All @@ -132,7 +133,8 @@ public MLPendingAckStore(ManagedLedger managedLedger, ManagedCursor cursor,
this.bufferedWriter = new TxnLogBufferedWriter(managedLedger, ((ManagedLedgerImpl) managedLedger).getExecutor(),
timer, PendingAckLogSerializer.INSTANCE,
bufferedWriterConfig.getBatchedWriteMaxRecords(), bufferedWriterConfig.getBatchedWriteMaxSize(),
bufferedWriterConfig.getBatchedWriteMaxDelayInMillis(), bufferedWriterConfig.isBatchEnabled());
bufferedWriterConfig.getBatchedWriteMaxDelayInMillis(), bufferedWriterConfig.isBatchEnabled(),
bufferedWriterMetrics);
this.batchedPendingAckLogsWaitingForHandle = new ArrayList<>();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.pulsar.broker.transaction.pendingack.impl;

import io.netty.util.Timer;
import io.prometheus.client.CollectorRegistry;
import java.util.concurrent.CompletableFuture;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
Expand All @@ -27,14 +28,17 @@
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.TransactionMetadataStoreService;
import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.broker.transaction.exception.pendingack.TransactionPendingAckException;
import org.apache.pulsar.broker.transaction.pendingack.PendingAckStore;
import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider;
import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.transaction.coordinator.impl.DisabledTxnLogBufferedWriterMetricsStats;
import org.apache.pulsar.transaction.coordinator.impl.TxnLogBufferedWriterConfig;
import org.apache.pulsar.transaction.coordinator.impl.TxnLogBufferedWriterMetricsStats;


/**
Expand All @@ -43,6 +47,25 @@
@Slf4j
public class MLPendingAckStoreProvider implements TransactionPendingAckStoreProvider {

private volatile TxnLogBufferedWriterMetricsStats bufferedWriterMetrics;

private void ensureBufferedWriterMetricsInitialized(PulsarService pulsar){
if (bufferedWriterMetrics != null){
return;
}
synchronized (this){
if (bufferedWriterMetrics != null){
return;
}
if (pulsar.getConfiguration().isTransactionPendingAckBatchedWriteEnabled()) {
this.bufferedWriterMetrics = new MLTxnPendingAckLogBufferedWriterMetrics(
pulsar.getConfiguration().getClusterName(), pulsar.getAdvertisedAddress());
} else {
this.bufferedWriterMetrics = DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS;
}
}
}

@Override
public CompletableFuture<PendingAckStore> newPendingAckStore(PersistentSubscription subscription) {
CompletableFuture<PendingAckStore> pendingAckStoreFuture = new CompletableFuture<>();
Expand All @@ -57,6 +80,7 @@ public CompletableFuture<PendingAckStore> newPendingAckStore(PersistentSubscript
PersistentTopic originPersistentTopic = (PersistentTopic) subscription.getTopic();
PulsarService pulsarService = originPersistentTopic.getBrokerService().getPulsar();

ensureBufferedWriterMetricsInitialized(pulsarService);
final Timer brokerClientSharedTimer =
pulsarService.getBrokerClientSharedTimer();
final ServiceConfiguration serviceConfiguration = pulsarService.getConfiguration();
Expand Down Expand Up @@ -105,7 +129,7 @@ public void openCursorComplete(ManagedCursor cursor, Object ctx) {
.getConfiguration()
.getTransactionPendingAckLogIndexMinLag(),
txnLogBufferedWriterConfig,
brokerClientSharedTimer));
brokerClientSharedTimer, bufferedWriterMetrics));
if (log.isDebugEnabled()) {
log.debug("{},{} open MLPendingAckStore cursor success",
originPersistentTopic.getName(),
Expand Down Expand Up @@ -151,4 +175,20 @@ public CompletableFuture<Boolean> checkInitializedBefore(PersistentSubscription
return originPersistentTopic.getBrokerService().getManagedLedgerFactory()
.asyncExists(TopicName.get(pendingAckTopicName).getPersistenceNamingEncoding());
}

public void closeBufferedWriterMetrics() {
if (bufferedWriterMetrics != null) {
bufferedWriterMetrics.close();
}
}

private static class MLTxnPendingAckLogBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats{

private MLTxnPendingAckLogBufferedWriterMetrics(String cluster, String brokerAdvertisedAddress) {
super("pulsar_txn_pending_ack_store",
new String[]{"cluster", "broker"},
new String[]{cluster, brokerAdvertisedAddress},
CollectorRegistry.defaultRegistry);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.pulsar.broker.stats;

import static org.apache.pulsar.transaction.coordinator.impl.DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS;
import io.netty.util.HashedWheelTimer;
import io.netty.util.concurrent.DefaultThreadFactory;
import java.util.List;
Expand Down Expand Up @@ -109,7 +110,7 @@ public void testTransactionTopic() throws Exception {
managedLedgerConfig.setMaxEntriesPerLedger(2);
MLTransactionLogImpl mlTransactionLog = new MLTransactionLogImpl(TransactionCoordinatorID.get(0),
pulsar.getManagedLedgerFactory(), managedLedgerConfig, txnLogBufferedWriterConfig,
transactionTimer);
transactionTimer, DISABLED_BUFFERED_WRITER_METRICS);
mlTransactionLog.initialize().get(2, TimeUnit.SECONDS);
ManagedLedgerMetrics metrics = new ManagedLedgerMetrics(pulsar);
metrics.generate();
Expand Down
Loading

0 comments on commit 2ca47e0

Please sign in to comment.