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-unsubscr...@pulsar.apache.org

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

Reply via email to