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/10/18 07:21:50 UTC

[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16931: [Improve][Txn][PIP-196]Segmented transaction buffer snapshot segment and index system topic

poorbarcode commented on code in PR #16931:
URL: https://github.com/apache/pulsar/pull/16931#discussion_r997736460


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java:
##########
@@ -420,7 +420,28 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) {
         });
     }
 
-
+    @Override
+    public CompletableFuture<ReadOnlyManagedLedgerImpl> asyncOpenReadOnlyManagedLedger(String managedLedgerName,
+                              ManagedLedgerConfig config, Object ctx) {
+        CompletableFuture<ReadOnlyManagedLedgerImpl> future = new CompletableFuture<>();
+        if (closed) {
+            return FutureUtil.failedFuture(new ManagedLedgerException.ManagedLedgerFactoryClosedException());
+        }
+        ReadOnlyManagedLedgerImpl roManagedLedger = new ReadOnlyManagedLedgerImpl(this,
+                bookkeeperFactory
+                        .get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(),
+                                config.getBookKeeperEnsemblePlacementPolicyProperties())),
+                store, config, scheduledExecutor, managedLedgerName);
+        roManagedLedger.initialize().thenRun(() -> {

Review Comment:
   `ReadOnlyManagedLedgerImpl.initialize()` has already return `CompletableFuture<ReadOnlyManagedLedgerImpl>` , can we write it like this: 
   
   ```java
   return new ReadOnlyManagedLedgerImpl(this,
                   bookkeeperFactory
                           .get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(),
                                   config.getBookKeeperEnsemblePlacementPolicyProperties())),
                   store, config, scheduledExecutor, managedLedgerName) initialize();
   ```
   
   Or the method `ReadOnlyManagedLedgerImpl.initialize()` can modify the return declaration to `CompletableFuture<Void>` ?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java:
##########
@@ -85,42 +85,48 @@ private TopicPolicyWriter(Producer<PulsarEvent> producer, SystemTopicClient<Puls
         }
 
         @Override
-        public MessageId write(PulsarEvent event) throws PulsarClientException {
-            TypedMessageBuilder<PulsarEvent> builder = producer.newMessage().key(getEventKey(event)).value(event);
+        public MessageId write(PulsarEvent event, String key) throws PulsarClientException {
+            TypedMessageBuilder<PulsarEvent> builder = producer.newMessage().key(key).value(event);
             setReplicateCluster(event, builder);
             return builder.send();
         }
 
         @Override
-        public CompletableFuture<MessageId> writeAsync(PulsarEvent event) {
-            TypedMessageBuilder<PulsarEvent> builder = producer.newMessage().key(getEventKey(event)).value(event);
+        public CompletableFuture<MessageId> writeAsync(PulsarEvent event, String key) {
+            TypedMessageBuilder<PulsarEvent> builder = producer.newMessage().key(key).value(event);
             setReplicateCluster(event, builder);
             return builder.sendAsync();
         }
 
         @Override
-        public MessageId delete(PulsarEvent event) throws PulsarClientException {
-            validateActionType(event);
-            TypedMessageBuilder<PulsarEvent> builder = producer.newMessage().key(getEventKey(event)).value(null);
+        public MessageId delete(PulsarEvent event, String key) throws PulsarClientException {
+            TypedMessageBuilder<PulsarEvent> builder = producer.newMessage().key(key).value(null);
             setReplicateCluster(event, builder);
             return builder.send();
         }
 
         @Override
-        public CompletableFuture<MessageId> deleteAsync(PulsarEvent event) {
+        public CompletableFuture<MessageId> deleteAsync(PulsarEvent event, String key) {
             validateActionType(event);
             TypedMessageBuilder<PulsarEvent> builder = producer.newMessage().key(getEventKey(event)).value(null);

Review Comment:
   should we use `@param key` instead of `getEventKey(event)` here?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java:
##########
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pulsar.broker.service;
+
+import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot;
+import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.common.events.EventType;
+
+public class TransactionBufferSnapshotServiceFactory {
+
+    private SystemTopicTxnBufferSnapshotService<TransactionBufferSnapshot> txnBufferSnapshotService;
+
+    private SystemTopicTxnBufferSnapshotService<TransactionBufferSnapshotIndexes.TransactionBufferSnapshot>
+            txnBufferSnapshotSegmentService;
+
+    private SystemTopicTxnBufferSnapshotService<TransactionBufferSnapshotIndexes> txnBufferSnapshotIndexService;
+
+    public TransactionBufferSnapshotServiceFactory(PulsarClient pulsarClient,
+                                                boolean transactionBufferSegmentedSnapshotEnabled) {

Review Comment:
   Now `@param transactionBufferSegmentedSnapshotEnabled ` is not unused, how do we turn the feature on or off?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java:
##########
@@ -23,64 +23,63 @@
 import java.util.concurrent.ConcurrentHashMap;
 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.broker.systopic.SystemTopicClientBase;
 import org.apache.pulsar.client.api.PulsarClient;
-import org.apache.pulsar.client.api.PulsarClientException.InvalidTopicNameException;
+import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.common.events.EventType;
 import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.util.FutureUtil;
 
-public class SystemTopicBaseTxnBufferSnapshotService implements TransactionBufferSnapshotService {
+public class SystemTopicTxnBufferSnapshotService<T> {
 
-    private final Map<TopicName, SystemTopicClient<TransactionBufferSnapshot>> clients;
+    protected final Map<TopicName, SystemTopicClient<T>> clients;
+    protected final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory;
 
-    private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory;
+    protected final Class<T> schemaType;
+    protected final EventType systemTopicType;
 
-    public SystemTopicBaseTxnBufferSnapshotService(PulsarClient client) {
+    public SystemTopicTxnBufferSnapshotService(PulsarClient client, EventType systemTopicType,
+                                               Class<T> schemaType) {
         this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client);
+        this.systemTopicType = systemTopicType;
+        this.schemaType = schemaType;
         this.clients = new ConcurrentHashMap<>();
     }
 
-    @Override
-    public CompletableFuture<Writer<TransactionBufferSnapshot>> createWriter(TopicName topicName) {
+    public CompletableFuture<SystemTopicClient.Writer<T>> 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) {
-            return FutureUtil.failedFuture(
-                    new InvalidTopicNameException("Can't create SystemTopicBaseTxnBufferSnapshotService, "
-                            + "because the topicName is null!"));
-        }
-        return CompletableFuture.completedFuture(clients.computeIfAbsent(systemTopicName,
-                (v) -> namespaceEventsSystemTopicFactory
-                        .createTransactionBufferSystemTopicClient(topicName.getNamespaceObject(), this)));
-    }
-
-    @Override
-    public CompletableFuture<Reader<TransactionBufferSnapshot>> createReader(TopicName topicName) {
+    public CompletableFuture<SystemTopicClient.Reader<T>> createReader(TopicName topicName) {
         return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newReaderAsync);
     }
 
-    @Override
-    public void removeClient(TopicName topicName,
-                                          TransactionBufferSystemTopicClient transactionBufferSystemTopicClient) {
+    public void removeClient(TopicName topicName, SystemTopicClientBase<T> transactionBufferSystemTopicClient) {
         if (transactionBufferSystemTopicClient.getReaders().size() == 0
                 && transactionBufferSystemTopicClient.getWriters().size() == 0) {
             clients.remove(topicName);

Review Comment:
   How do we prevent concurrency between `createReader | createWriter` and `removeClient`? Now that we have a fixed number of system topics, maybe we should not recycle `SystemTopicClient` objects



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java:
##########
@@ -117,6 +118,15 @@ ReadOnlyCursor openReadOnlyCursor(String managedLedgerName, Position startPositi
     void asyncOpenReadOnlyCursor(String managedLedgerName, Position startPosition, ManagedLedgerConfig config,
             OpenReadOnlyCursorCallback callback, Object ctx);
 
+    /**
+     * Asynchronous open a Read-only managedLedger.
+     * @param managedLedgerName the unique name that identifies the managed ledger
+     * @param config the managed ledegr configuratiion.
+     * @param ctx opaque context
+     */
+    CompletableFuture<ReadOnlyManagedLedgerImpl> asyncOpenReadOnlyManagedLedger(String managedLedgerName,

Review Comment:
   Why do we need this method when we can't seem to find anywhere else to use it but test?



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java:
##########
@@ -420,7 +420,28 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) {
         });
     }
 
-
+    @Override
+    public CompletableFuture<ReadOnlyManagedLedgerImpl> asyncOpenReadOnlyManagedLedger(String managedLedgerName,
+                              ManagedLedgerConfig config, Object ctx) {
+        CompletableFuture<ReadOnlyManagedLedgerImpl> future = new CompletableFuture<>();
+        if (closed) {
+            return FutureUtil.failedFuture(new ManagedLedgerException.ManagedLedgerFactoryClosedException());
+        }
+        ReadOnlyManagedLedgerImpl roManagedLedger = new ReadOnlyManagedLedgerImpl(this,
+                bookkeeperFactory
+                        .get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(),
+                                config.getBookKeeperEnsemblePlacementPolicyProperties())),
+                store, config, scheduledExecutor, managedLedgerName);
+        roManagedLedger.initialize().thenRun(() -> {
+            log.info("[{}] Successfully initialize Read-only managed ledger", managedLedgerName);
+            future.complete(roManagedLedger);
+        }).exceptionally(e -> {
+            log.error("[{}] Failed to initialize Read-only managed ledger", managedLedgerName, e);
+            future.completeExceptionally(new ManagedLedgerException.ManagedLedgerFactoryClosedException());

Review Comment:
   Why wrap it as a `ManagedLedgerFactoryClosedException `, and would it be better for the caller to handle it?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java:
##########
@@ -0,0 +1,218 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.systopic;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.common.naming.TopicName;
+
+@Slf4j
+public class  TransactionBufferSnapshotBaseSystemTopicClient<T> extends SystemTopicClientBase<T> {
+
+    protected final SystemTopicTxnBufferSnapshotService<T> systemTopicTxnBufferSnapshotService;
+    protected final Class<T> schemaType;
+
+    public TransactionBufferSnapshotBaseSystemTopicClient(PulsarClient client,
+                                                          TopicName topicName,
+                                                          SystemTopicTxnBufferSnapshotService<T>
+                                                                  systemTopicTxnBufferSnapshotService,
+                                                          Class<T> schemaType) {
+        super(client, topicName);
+        this.systemTopicTxnBufferSnapshotService = systemTopicTxnBufferSnapshotService;
+        this.schemaType = schemaType;
+    }
+
+    protected void removeWriter(Writer<T> writer) {
+        writers.remove(writer);
+        this.systemTopicTxnBufferSnapshotService.removeClient(topicName, this);
+    }
+
+    protected void removeReader(Reader<T> reader) {
+        readers.remove(reader);
+        this.systemTopicTxnBufferSnapshotService.removeClient(topicName, this);
+    }
+
+    protected static class TransactionBufferSnapshotWriter<T> implements Writer<T> {
+
+        protected final Producer<T> producer;
+        protected final TransactionBufferSnapshotBaseSystemTopicClient<T>
+                transactionBufferSnapshotBaseSystemTopicClient;
+
+        protected TransactionBufferSnapshotWriter(Producer<T> producer,
+                                                  TransactionBufferSnapshotBaseSystemTopicClient<T>
+                                                    transactionBufferSnapshotBaseSystemTopicClient) {
+            this.producer = producer;
+            this.transactionBufferSnapshotBaseSystemTopicClient = transactionBufferSnapshotBaseSystemTopicClient;
+        }
+
+        @Override
+        public MessageId write(T t, String key)
+                throws PulsarClientException {
+            return producer.newMessage().key(key)
+                    .value(t).send();
+        }
+
+        @Override
+        public CompletableFuture<MessageId> writeAsync(T t, String key) {
+            return producer.newMessage()
+                    .key(key)
+                    .value(t).sendAsync();
+        }
+
+        @Override
+        public MessageId delete(T t, String key)
+                throws PulsarClientException {
+            return producer.newMessage()
+                    .key(key)
+                    .value(null)
+                    .send();
+        }
+
+        @Override
+        public CompletableFuture<MessageId> deleteAsync(T t, String key) {
+            return producer.newMessage()
+                    .key(key)
+                    .value(null)
+                    .sendAsync();
+        }
+
+        @Override
+        public void close() throws IOException {
+            this.producer.close();
+            transactionBufferSnapshotBaseSystemTopicClient.removeWriter(this);

Review Comment:
   If `this.producer.close()` fails, should we delete `reader`? Maybe we can directly use `closeAsync().join`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java:
##########
@@ -0,0 +1,218 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.systopic;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.common.naming.TopicName;
+
+@Slf4j
+public class  TransactionBufferSnapshotBaseSystemTopicClient<T> extends SystemTopicClientBase<T> {
+
+    protected final SystemTopicTxnBufferSnapshotService<T> systemTopicTxnBufferSnapshotService;
+    protected final Class<T> schemaType;
+
+    public TransactionBufferSnapshotBaseSystemTopicClient(PulsarClient client,
+                                                          TopicName topicName,
+                                                          SystemTopicTxnBufferSnapshotService<T>
+                                                                  systemTopicTxnBufferSnapshotService,
+                                                          Class<T> schemaType) {
+        super(client, topicName);
+        this.systemTopicTxnBufferSnapshotService = systemTopicTxnBufferSnapshotService;
+        this.schemaType = schemaType;
+    }
+
+    protected void removeWriter(Writer<T> writer) {
+        writers.remove(writer);
+        this.systemTopicTxnBufferSnapshotService.removeClient(topicName, this);
+    }
+
+    protected void removeReader(Reader<T> reader) {
+        readers.remove(reader);
+        this.systemTopicTxnBufferSnapshotService.removeClient(topicName, this);
+    }
+
+    protected static class TransactionBufferSnapshotWriter<T> implements Writer<T> {
+
+        protected final Producer<T> producer;
+        protected final TransactionBufferSnapshotBaseSystemTopicClient<T>
+                transactionBufferSnapshotBaseSystemTopicClient;
+
+        protected TransactionBufferSnapshotWriter(Producer<T> producer,
+                                                  TransactionBufferSnapshotBaseSystemTopicClient<T>
+                                                    transactionBufferSnapshotBaseSystemTopicClient) {
+            this.producer = producer;
+            this.transactionBufferSnapshotBaseSystemTopicClient = transactionBufferSnapshotBaseSystemTopicClient;
+        }
+
+        @Override
+        public MessageId write(T t, String key)
+                throws PulsarClientException {
+            return producer.newMessage().key(key)
+                    .value(t).send();
+        }
+
+        @Override
+        public CompletableFuture<MessageId> writeAsync(T t, String key) {
+            return producer.newMessage()
+                    .key(key)
+                    .value(t).sendAsync();
+        }
+
+        @Override
+        public MessageId delete(T t, String key)
+                throws PulsarClientException {
+            return producer.newMessage()
+                    .key(key)
+                    .value(null)
+                    .send();
+        }
+
+        @Override
+        public CompletableFuture<MessageId> deleteAsync(T t, String key) {
+            return producer.newMessage()
+                    .key(key)
+                    .value(null)
+                    .sendAsync();
+        }
+
+        @Override
+        public void close() throws IOException {
+            this.producer.close();
+            transactionBufferSnapshotBaseSystemTopicClient.removeWriter(this);
+        }
+
+        @Override
+        public CompletableFuture<Void> closeAsync() {
+            CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+            producer.closeAsync().whenComplete((v, e) -> {
+                // if close fail, also need remove the producer
+                transactionBufferSnapshotBaseSystemTopicClient.removeWriter(this);
+                if (e != null) {
+                    completableFuture.completeExceptionally(e);
+                    return;
+                }
+                completableFuture.complete(null);
+            });
+            return completableFuture;
+        }
+
+        @Override
+        public SystemTopicClient<T> getSystemTopicClient() {
+            return transactionBufferSnapshotBaseSystemTopicClient;
+        }
+    }
+
+    protected static class TransactionBufferSnapshotReader<T> implements Reader<T> {
+
+        private final org.apache.pulsar.client.api.Reader<T> reader;
+        private final TransactionBufferSnapshotBaseSystemTopicClient<T> transactionBufferSnapshotBaseSystemTopicClient;
+
+        protected TransactionBufferSnapshotReader(
+                org.apache.pulsar.client.api.Reader<T> reader,
+                TransactionBufferSnapshotBaseSystemTopicClient<T> transactionBufferSnapshotBaseSystemTopicClient) {
+            this.reader = reader;
+            this.transactionBufferSnapshotBaseSystemTopicClient = transactionBufferSnapshotBaseSystemTopicClient;
+        }
+
+        @Override
+        public Message<T> readNext() throws PulsarClientException {
+            return reader.readNext();
+        }
+
+        @Override
+        public CompletableFuture<Message<T>> readNextAsync() {
+            return reader.readNextAsync();
+        }
+
+        @Override
+        public boolean hasMoreEvents() throws PulsarClientException {
+            return reader.hasMessageAvailable();
+        }
+
+        @Override
+        public CompletableFuture<Boolean> hasMoreEventsAsync() {
+            return reader.hasMessageAvailableAsync();
+        }
+
+        @Override
+        public void close() throws IOException {
+            this.reader.close();
+            transactionBufferSnapshotBaseSystemTopicClient.removeReader(this);
+        }
+
+        @Override
+        public CompletableFuture<Void> closeAsync() {
+            CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+            reader.closeAsync().whenComplete((v, e) -> {
+                // if close fail, also need remove the reader
+                transactionBufferSnapshotBaseSystemTopicClient.removeReader(this);
+                if (e != null) {
+                    completableFuture.completeExceptionally(e);
+                    return;
+                }
+                completableFuture.complete(null);
+            });
+            return completableFuture;
+        }
+
+        @Override
+        public SystemTopicClient<T> getSystemTopic() {
+            return transactionBufferSnapshotBaseSystemTopicClient;
+        }
+    }
+
+    @Override
+    protected CompletableFuture<Writer<T>> newWriterAsyncInternal() {
+        return client.newProducer(Schema.AVRO(schemaType))
+                .topic(topicName.toString())
+                .createAsync().thenCompose(producer -> {

Review Comment:
   Should we use `thenApply`, which would save a `CompletableFuture` object? 
   
   Same for `newReaderAsyncInternal`.



##########
pulsar-client-api/src/main/java/org/apache/pulsar/client/api/transaction/TxnID.java:
##########
@@ -39,14 +43,14 @@ public class TxnID implements Serializable {
      *
      * @serial
      */
-    private final long mostSigBits;
+    private long mostSigBits;

Review Comment:
   Why remove the final modifier?
   
   Same for `leastSigBits`



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