You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/04/04 01:32:31 UTC

[GitHub] [pulsar] Technoboy- commented on a diff in pull request #15015: [enh][transaction] Optimize to reuse transaction buffer snapshot writer

Technoboy- commented on code in PR #15015:
URL: https://github.com/apache/pulsar/pull/15015#discussion_r841313610


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotService.java:
##########
@@ -18,69 +18,168 @@
  */
 package org.apache.pulsar.broker.service;
 
+import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import io.netty.util.AbstractReferenceCounted;
+import io.netty.util.ReferenceCounted;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.util.SafeRun;
 import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory;
 import org.apache.pulsar.broker.systopic.SystemTopicClient;
 import org.apache.pulsar.broker.systopic.SystemTopicClient.Reader;
 import org.apache.pulsar.broker.systopic.SystemTopicClient.Writer;
 import org.apache.pulsar.broker.systopic.TransactionBufferSystemTopicClient;
 import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot;
 import org.apache.pulsar.client.api.PulsarClient;
-import org.apache.pulsar.client.api.PulsarClientException.InvalidTopicNameException;
-import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.impl.Backoff;
+import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.util.FutureUtil;
 
+@Slf4j
 public class SystemTopicBaseTxnBufferSnapshotService implements TransactionBufferSnapshotService {
 
-    private final Map<TopicName, SystemTopicClient<TransactionBufferSnapshot>> clients;
+    private final Map<NamespaceName, SystemTopicClient<TransactionBufferSnapshot>> clients;
 
     private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory;
 
-    public SystemTopicBaseTxnBufferSnapshotService(PulsarClient client) {
+    private final ScheduledExecutorService scheduledExecutorService;
+    private final ConcurrentHashMap<NamespaceName, ReferenceCountedWriter> writerFutureMap;
+    private final LinkedList<CompletableFuture<Writer<TransactionBufferSnapshot>>> pendingCloseWriterList;
+
+    // The class ReferenceCountedWriter will maintain the reference count,
+    // when the reference count decrement to 0, it will be removed from writerFutureMap, the writer will be closed.
+    public static class ReferenceCountedWriter extends AbstractReferenceCounted {
+
+        private final NamespaceName namespaceName;
+        private final SystemTopicBaseTxnBufferSnapshotService service;
+        private CompletableFuture<Writer<TransactionBufferSnapshot>> future;
+        private final Backoff backoff;
+
+        protected ReferenceCountedWriter(NamespaceName namespaceName,
+                                         SystemTopicBaseTxnBufferSnapshotService service) {
+            this.namespaceName = namespaceName;
+            this.service = service;
+            this.backoff = new Backoff(1, TimeUnit.SECONDS, 3, TimeUnit.SECONDS, 10, TimeUnit.SECONDS);
+            initWriterFuture();
+        }
+
+        private void initWriterFuture() {
+            this.future = service.getTransactionBufferSystemTopicClient(namespaceName).newWriterAsync();
+            this.future.thenRunAsync(this.backoff::reset).exceptionally(throwable -> {
+                long delay = backoff.next();
+                log.error("[{}] Failed to new transaction buffer system topic writer," +
+                                "try to re-create the writer in {} ms.", delay, namespaceName, throwable);
+                service.scheduledExecutorService.schedule(
+                        SafeRun.safeRun(this::initWriterFuture), delay, TimeUnit.MILLISECONDS);
+                return null;
+            });
+        }
+
+        public CompletableFuture<Writer<TransactionBufferSnapshot>> getFuture() {
+            if (future == null) {
+                initWriterFuture();
+            }
+            return future;
+        }
+
+        @Override
+        protected void deallocate() {
+            ReferenceCountedWriter referenceCountedWriter = service.writerFutureMap.remove(namespaceName);
+            if (referenceCountedWriter != null && referenceCountedWriter.getFuture() != null) {
+                service.pendingCloseWriterList.add(referenceCountedWriter.getFuture());
+                service.closePendingCloseWriter();
+            }
+        }
+
+        @Override
+        public ReferenceCounted touch(Object o) {
+            return this;
+        }
+
+    }
+
+    public SystemTopicBaseTxnBufferSnapshotService(PulsarClient client,
+                                                   ScheduledExecutorService scheduledExecutorService) {
         this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client);
         this.clients = new ConcurrentHashMap<>();
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.writerFutureMap = new ConcurrentHashMap<>();
+        this.pendingCloseWriterList = new LinkedList<>();
     }
 
     @Override
     public CompletableFuture<Writer<TransactionBufferSnapshot>> createWriter(TopicName topicName) {
-        return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync);
-    }
-
-    private CompletableFuture<SystemTopicClient<TransactionBufferSnapshot>> getTransactionBufferSystemTopicClient(
-            TopicName topicName) {
-        TopicName systemTopicName = NamespaceEventsSystemTopicFactory
-                .getSystemTopicName(topicName.getNamespaceObject(), EventType.TRANSACTION_BUFFER_SNAPSHOT);
-        if (systemTopicName == null) {
+        if (topicName == null) {
             return FutureUtil.failedFuture(
-                    new InvalidTopicNameException("Can't create SystemTopicBaseTxnBufferSnapshotService, "
-                            + "because the topicName is null!"));
+                    new PulsarClientException.InvalidTopicNameException(
+                            "Can't create SystemTopicBaseTxnBufferSnapshotService, because the topicName is null!"));
         }
-        return CompletableFuture.completedFuture(clients.computeIfAbsent(systemTopicName,
+        return getTransactionBufferSystemTopicClient(topicName.getNamespaceObject()).newWriterAsync();
+    }
+
+    @Override
+    public ReferenceCountedWriter createReferenceWriter(NamespaceName namespaceName) {
+        return writerFutureMap.compute(namespaceName, (ns, writerFuture) -> {
+            if (writerFuture == null) {
+                return new ReferenceCountedWriter(namespaceName, this);

Review Comment:
   Seems not safe here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org