You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/04/07 03:08:11 UTC

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

congbobo184 commented on code in PR #15015:
URL: https://github.com/apache/pulsar/pull/15015#discussion_r844594648


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotService.java:
##########
@@ -18,69 +18,176 @@
  */
 package org.apache.pulsar.broker.service;
 
+import io.netty.util.AbstractReferenceCounted;
+import io.netty.util.ReferenceCounted;
+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 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 {

Review Comment:
   I suggest don't extends AbstractReferenceCounted, deallocate seem not be used. and is complicated to implement



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotService.java:
##########
@@ -18,69 +18,176 @@
  */
 package org.apache.pulsar.broker.service;
 
+import io.netty.util.AbstractReferenceCounted;
+import io.netty.util.ReferenceCounted;
+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 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 synchronized void initWriterFuture() {
+            this.future = service.getTransactionBufferSystemTopicClient(namespaceName).newWriterAsync();
+            this.future.thenRunAsync(this.backoff::reset).exceptionally(throwable -> {
+                long delay = backoff.next();

Review Comment:
   seem we only use backoff.next() once and then reset



-- 
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: commits-unsubscribe@pulsar.apache.org

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