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 23, 2022
1 parent 08df28a commit 4fd5fa1
Show file tree
Hide file tree
Showing 15 changed files with 509 additions and 83 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 @@ -168,6 +169,7 @@
import org.apache.pulsar.packages.management.core.impl.PackagesManagementImpl;
import org.apache.pulsar.policies.data.loadbalancer.AdvertisedListener;
import org.apache.pulsar.transaction.coordinator.TransactionMetadataStoreProvider;
import org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider;
import org.apache.pulsar.websocket.WebSocketConsumerServlet;
import org.apache.pulsar.websocket.WebSocketPingPongServlet;
import org.apache.pulsar.websocket.WebSocketProducerServlet;
Expand Down Expand Up @@ -822,6 +824,9 @@ public void start() throws PulsarServerException {

// Register pulsar system namespaces and start transaction meta store service
if (config.isTransactionCoordinatorEnabled()) {
MLTransactionMetadataStoreProvider.initBufferedWriterMetrics(getAdvertisedAddress());
MLPendingAckStoreProvider.initBufferedWriterMetrics(getAdvertisedAddress());

this.transactionBufferSnapshotService = new SystemTopicBaseTxnBufferSnapshotService(getClient());
this.transactionTimer =
new HashedWheelTimer(new DefaultThreadFactory("pulsar-transaction-timer"));
Expand Down
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 @@ -34,7 +35,9 @@
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 +46,21 @@
@Slf4j
public class MLPendingAckStoreProvider implements TransactionPendingAckStoreProvider {

private static volatile TxnLogBufferedWriterMetricsStats bufferedWriterMetrics =
DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS;

public static void initBufferedWriterMetrics(String brokerAdvertisedAddress){
if (bufferedWriterMetrics != DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS){
return;
}
synchronized (MLPendingAckStoreProvider.class){
if (bufferedWriterMetrics != DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS){
return;
}
bufferedWriterMetrics = new MLTxnPendingAckLogBufferedWriterMetrics(brokerAdvertisedAddress);
}
}

@Override
public CompletableFuture<PendingAckStore> newPendingAckStore(PersistentSubscription subscription) {
CompletableFuture<PendingAckStore> pendingAckStoreFuture = new CompletableFuture<>();
Expand Down Expand Up @@ -105,7 +123,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 +169,14 @@ public CompletableFuture<Boolean> checkInitializedBefore(PersistentSubscription
return originPersistentTopic.getBrokerService().getManagedLedgerFactory()
.asyncExists(TopicName.get(pendingAckTopicName).getPersistenceNamingEncoding());
}

private static class MLTxnPendingAckLogBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats{

private MLTxnPendingAckLogBufferedWriterMetrics(String brokerAdvertisedAddress) {
super("pulsar_txn_pending_ack_store",
new String[]{"broker"},
new String[]{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 4fd5fa1

Please sign in to comment.