You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by "dao-jun (via GitHub)" <gi...@apache.org> on 2023/02/27 14:49:38 UTC

[GitHub] [pulsar] dao-jun commented on a diff in pull request #19641: [feat][txn] Transaction buffer snapshot writer reuse

dao-jun commented on code in PR #19641:
URL: https://github.com/apache/pulsar/pull/19641#discussion_r1118850555


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java:
##########
@@ -18,66 +18,154 @@
  */
 package org.apache.pulsar.broker.service;
 
+import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
 import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory;
 import org.apache.pulsar.broker.systopic.SystemTopicClient;
 import org.apache.pulsar.broker.systopic.SystemTopicClientBase;
 import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.naming.TopicName;
-import org.apache.pulsar.common.util.FutureUtil;
 
+@Slf4j
 public class SystemTopicTxnBufferSnapshotService<T> {
 
-    protected final Map<TopicName, SystemTopicClient<T>> clients;
+    protected final ConcurrentHashMap<NamespaceName, SystemTopicClient<T>> clients;
     protected final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory;
 
     protected final Class<T> schemaType;
     protected final EventType systemTopicType;
 
+    private final HashMap<NamespaceName, ReferenceCountedWriter<T>> refCountedWriterMap;
+
+    // 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<T> {
+
+        private final AtomicLong referenceCount;
+        private final NamespaceName namespaceName;
+        private final CompletableFuture<SystemTopicClient.Writer<T>> future;
+        private final SystemTopicTxnBufferSnapshotService<T> snapshotService;
+
+        public ReferenceCountedWriter(NamespaceName namespaceName,
+                                      CompletableFuture<SystemTopicClient.Writer<T>> future,
+                                      SystemTopicTxnBufferSnapshotService<T> snapshotService) {
+            this.referenceCount = new AtomicLong(1);
+            this.namespaceName = namespaceName;
+            this.snapshotService = snapshotService;
+            this.future = future;
+            this.future.exceptionally(t -> {
+                        log.error("[{}] Failed to create transaction buffer snapshot writer.", namespaceName, t);
+                snapshotService.refCountedWriterMap.remove(namespaceName, this);
+                return null;
+            });
+        }
+
+        public CompletableFuture<SystemTopicClient.Writer<T>> getFuture() {
+            return future;
+        }
+
+        private void retain() {
+            operationValidate(true);
+            this.referenceCount.incrementAndGet();
+        }
+
+        private long release() {
+            operationValidate(false);
+            return this.referenceCount.decrementAndGet();
+        }
+
+        private void operationValidate(boolean isRetain) {
+            if (this.referenceCount.get() == 0) {
+                throw new RuntimeException(
+                        "[" + namespaceName + "] The reference counted transaction buffer snapshot writer couldn't "
+                                + "be " + (isRetain ? "retained" : "released") + ", refCnt is 0.");
+            }
+        }
+
+        public void close() {
+            if (release() == 0) {
+                snapshotService.refCountedWriterMap.remove(namespaceName, this);
+                future.thenAccept(writer -> {
+                    final String topicName = writer.getSystemTopicClient().getTopicName().toString();
+                    writer.closeAsync().exceptionally(t -> {
+                        if (t != null) {
+                            log.error("[{}] Failed to close writer.", topicName, t);
+                        } else {
+                            if (log.isDebugEnabled()) {
+                                log.debug("[{}] Success to close writer.", topicName);
+                            }
+                        }
+                        return null;
+                    });
+                });
+            }
+        }
+
+    }
+
     public SystemTopicTxnBufferSnapshotService(PulsarClient client, EventType systemTopicType,
                                                Class<T> schemaType) {
         this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client);
         this.systemTopicType = systemTopicType;
         this.schemaType = schemaType;
         this.clients = new ConcurrentHashMap<>();
+        this.refCountedWriterMap = new HashMap<>();

Review Comment:
   Maybe it's better to use ConcurrentHashMap?



-- 
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