From 677f765bcfa47f0d5a94ecf4afde21a518ca3cbb Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Tue, 6 Sep 2022 20:52:26 +0800 Subject: [PATCH 01/27] Implement snapshot segment topic and snapshot index topic. --- .../apache/pulsar/broker/PulsarService.java | 10 + ...opicBaseTxnBufferSnapshotIndexService.java | 89 +++++++ ...icBaseTxnBufferSnapshotSegmentService.java | 88 +++++++ ...TransactionBufferSnapshotIndexService.java | 64 +++++ ...ansactionBufferSnapshotSegmentService.java | 61 +++++ .../NamespaceEventsSystemTopicFactory.java | 41 ++- .../broker/systopic/SystemTopicClient.java | 16 ++ .../TopicPoliciesSystemTopicClient.java | 18 ++ ...nBufferSnapshotIndexSystemTopicClient.java | 239 +++++++++++++++++ ...ufferSnapshotSegmentSystemTopicClient.java | 245 ++++++++++++++++++ .../TransactionBufferSystemTopicClient.java | 18 ++ .../v2/TransactionBufferSnapshot.java | 35 +++ .../v2/TransactionBufferSnapshotIndexes.java | 59 +++++ .../transaction/buffer/matadata/v2/TxnID.java | 51 ++++ .../buffer/matadata/v2/package-info.java | 19 ++ .../TopicTransactionBufferRecoverTest.java | 86 ++++++ .../pulsar/common/events/EventType.java | 12 +- .../common/naming/SystemTopicNames.java | 12 +- 18 files changed, 1151 insertions(+), 12 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotIndexService.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotIndexService.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshot.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TxnID.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/package-info.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index b5baa71709359..233b5381ac41d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -102,10 +102,14 @@ import org.apache.pulsar.broker.rest.Topics; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.PulsarMetadataEventSynchronizer; +import org.apache.pulsar.broker.service.SystemTopicBaseTxnBufferSnapshotIndexService; +import org.apache.pulsar.broker.service.SystemTopicBaseTxnBufferSnapshotSegmentService; import org.apache.pulsar.broker.service.SystemTopicBaseTxnBufferSnapshotService; import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotIndexService; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotSegmentService; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.stats.MetricsGenerator; @@ -260,6 +264,8 @@ public class PulsarService implements AutoCloseable, ShutdownService { private PulsarMetadataEventSynchronizer localMetadataSynchronizer; private CoordinationService coordinationService; private TransactionBufferSnapshotService transactionBufferSnapshotService; + private TransactionBufferSnapshotIndexService transactionBufferSnapshotIndexService; + private TransactionBufferSnapshotSegmentService transactionBufferSnapshotSegmentService; private MetadataStore configurationMetadataStore; private PulsarMetadataEventSynchronizer configMetadataSynchronizer; @@ -822,6 +828,10 @@ public void start() throws PulsarServerException { // Register pulsar system namespaces and start transaction meta store service if (config.isTransactionCoordinatorEnabled()) { this.transactionBufferSnapshotService = new SystemTopicBaseTxnBufferSnapshotService(getClient()); + this.transactionBufferSnapshotIndexService = + new SystemTopicBaseTxnBufferSnapshotIndexService(getClient()); + this.transactionBufferSnapshotSegmentService = + new SystemTopicBaseTxnBufferSnapshotSegmentService(getClient()); this.transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-transaction-timer")); transactionBufferClient = TransactionBufferClientImpl.create(this, transactionTimer, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotIndexService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotIndexService.java new file mode 100644 index 0000000000000..7120790fc1453 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotIndexService.java @@ -0,0 +1,89 @@ +/** + * 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 java.util.Map; +import java.util.concurrent.CompletableFuture; +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.TransactionBufferSnapshotIndexSystemTopicClient; +import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; +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.common.naming.TopicName; +import org.apache.pulsar.common.util.FutureUtil; + +public class SystemTopicBaseTxnBufferSnapshotIndexService implements TransactionBufferSnapshotIndexService { + + private final Map> clients; + + private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; + + public SystemTopicBaseTxnBufferSnapshotIndexService(PulsarClient client) { + this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); + this.clients = new ConcurrentHashMap<>(); + } + + @Override + public CompletableFuture> createWriter(TopicName topicName) { + return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync); + } + + private CompletableFuture> + getTransactionBufferSystemTopicClient( + TopicName topicName) { + TopicName systemTopicName = NamespaceEventsSystemTopicFactory + .getSystemTopicName(topicName.getNamespaceObject(), EventType.TRANSACTION_BUFFER_SNAPSHOT); + if (systemTopicName == null) { + return FutureUtil.failedFuture( + new InvalidTopicNameException("Can't create SystemTopicBaseTxnBufferSnapshotIndexService, " + + "because the topicName is null!")); + } + return CompletableFuture.completedFuture(clients.computeIfAbsent(systemTopicName, + (v) -> namespaceEventsSystemTopicFactory + .createTransactionBufferSnapshotIndexSystemTopicClient(topicName.getNamespaceObject(), + this))); + } + + @Override + public CompletableFuture> createReader(TopicName topicName) { + return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newReaderAsync); + } + + @Override + public void removeClient(TopicName topicName, + TransactionBufferSnapshotIndexSystemTopicClient + transactionBufferSnapshotIndexSystemTopicClient) { + if (transactionBufferSnapshotIndexSystemTopicClient.getReaders().size() == 0 + && transactionBufferSnapshotIndexSystemTopicClient.getWriters().size() == 0) { + clients.remove(topicName); + } + } + + @Override + public void close() throws Exception { + for (Map.Entry> entry : clients.entrySet()) { + entry.getValue().close(); + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java new file mode 100644 index 0000000000000..9d37b65afe3fc --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java @@ -0,0 +1,88 @@ +/** + * 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 java.util.Map; +import java.util.concurrent.CompletableFuture; +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.TransactionBufferSnapshotSegmentSystemTopicClient; +import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshot; +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.TopicName; +import org.apache.pulsar.common.util.FutureUtil; + +public class SystemTopicBaseTxnBufferSnapshotSegmentService implements TransactionBufferSnapshotSegmentService { + + private final Map> clients; + + private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; + + public SystemTopicBaseTxnBufferSnapshotSegmentService(PulsarClient client) { + this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); + this.clients = new ConcurrentHashMap<>(); + } + + @Override + public CompletableFuture> createWriter(TopicName topicName) { + return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync); + } + + private CompletableFuture> + getTransactionBufferSystemTopicClient( + TopicName topicName) { + TopicName systemTopicName = NamespaceEventsSystemTopicFactory + .getSystemTopicName(topicName.getNamespaceObject(), EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT); + if (systemTopicName == null) { + return FutureUtil.failedFuture( + new PulsarClientException.InvalidTopicNameException( + "Can't create SystemTopicBaseTxnBufferSnapshotSegmentService, " + + "because the topicName is null!")); + } + return CompletableFuture.completedFuture(clients.computeIfAbsent(systemTopicName, + (v) -> namespaceEventsSystemTopicFactory + .createTransactionBufferSnapshotSegmentSystemTopicClient(topicName.getNamespaceObject(), + this))); + } + + @Override + public CompletableFuture> createReader(TopicName topicName) { + return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newReaderAsync); + } + + @Override + public void removeClient(TopicName topicName, + TransactionBufferSnapshotSegmentSystemTopicClient + transactionBufferSnapshotSegmentSystemTopicClient) { + if (transactionBufferSnapshotSegmentSystemTopicClient.getReaders().size() == 0 + && transactionBufferSnapshotSegmentSystemTopicClient.getWriters().size() == 0) { + clients.remove(topicName); + } + } + + @Override + public void close() throws Exception { + for (Map.Entry> entry : clients.entrySet()) { + entry.getValue().close(); + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotIndexService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotIndexService.java new file mode 100644 index 0000000000000..ef913f36fa6f8 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotIndexService.java @@ -0,0 +1,64 @@ +/** + * 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 java.util.concurrent.CompletableFuture; +import org.apache.pulsar.broker.systopic.SystemTopicClient.Reader; +import org.apache.pulsar.broker.systopic.SystemTopicClient.Writer; +import org.apache.pulsar.broker.systopic.TransactionBufferSnapshotIndexSystemTopicClient; +import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; +import org.apache.pulsar.common.naming.TopicName; + +public interface TransactionBufferSnapshotIndexService { + + /** + * Create a transaction buffer snapshot index writer. + * + * @param topicName {@link TopicName} the topic name + * + * @return {@link CompletableFuture} return the future of writer + */ + CompletableFuture> createWriter(TopicName topicName); + + /** + * Create a transaction buffer snapshot index reader. + * + * @param topicName {@link TopicName} the topic name + * + * @return {@link CompletableFuture} return the future of reader + */ + CompletableFuture> createReader(TopicName topicName); + + /** + * Remove a topic client from cache. + * + * @param topicName {@link TopicName} the topic name + * @param transactionBufferSnapshotIndexSystemTopicClient + * {@link TransactionBufferSnapshotIndexSystemTopicClient} the topic client + * + */ + void removeClient(TopicName topicName, TransactionBufferSnapshotIndexSystemTopicClient + transactionBufferSnapshotIndexSystemTopicClient); + + /** + * Close transaction buffer snapshot service. + */ + void close() throws Exception; + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java new file mode 100644 index 0000000000000..0c6440404fae2 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java @@ -0,0 +1,61 @@ +/** + * 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 java.util.concurrent.CompletableFuture; +import org.apache.pulsar.broker.systopic.SystemTopicClient; +import org.apache.pulsar.broker.systopic.TransactionBufferSnapshotSegmentSystemTopicClient; +import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshot; +import org.apache.pulsar.common.naming.TopicName; + +public interface TransactionBufferSnapshotSegmentService { + /** + * Create a transaction buffer snapshot segment writer. + * + * @param topicName {@link TopicName} the topic name + * + * @return {@link CompletableFuture< SystemTopicClient.Writer >} return the future of writer + */ + CompletableFuture> createWriter(TopicName topicName); + + /** + * Create a transaction buffer snapshot segment reader. + * + * @param topicName {@link TopicName} the topic name + * + * @return {@link CompletableFuture< SystemTopicClient.Reader >} return the future of reader + */ + CompletableFuture> createReader(TopicName topicName); + + /** + * Remove a topic client from cache. + * + * @param topicName {@link TopicName} the topic name + * @param transactionBufferSnapshotSegmentSystemTopicClient + * {@link TransactionBufferSnapshotSegmentSystemTopicClient} the topic client + * + */ + void removeClient(TopicName topicName, TransactionBufferSnapshotSegmentSystemTopicClient + transactionBufferSnapshotSegmentSystemTopicClient); + + /** + * Close transaction buffer snapshot service. + */ + void close() throws Exception; +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java index 9e162f741b2c3..6faedf1c11307 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.systopic; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotIndexService; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotSegmentService; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.common.events.EventType; @@ -51,17 +53,36 @@ public TransactionBufferSystemTopicClient createTransactionBufferSystemTopicClie return new TransactionBufferSystemTopicClient(client, topicName, transactionBufferSnapshotService); } + public TransactionBufferSnapshotIndexSystemTopicClient createTransactionBufferSnapshotIndexSystemTopicClient( + NamespaceName namespaceName, TransactionBufferSnapshotIndexService transactionBufferSnapshotIndexService) { + TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES); + log.info("Create transaction buffer snapshot index client, topicName : {}", topicName.toString()); + return new TransactionBufferSnapshotIndexSystemTopicClient(client, topicName, + transactionBufferSnapshotIndexService); + } + + public TransactionBufferSnapshotSegmentSystemTopicClient createTransactionBufferSnapshotSegmentSystemTopicClient( + NamespaceName namespaceName, + TransactionBufferSnapshotSegmentService transactionBufferSnapshotSegmentService) { + TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT); + log.info("Create transaction buffer snapshot segment client, topicName : {}", topicName.toString()); + return new TransactionBufferSnapshotSegmentSystemTopicClient(client, topicName, + transactionBufferSnapshotSegmentService); + } + public static TopicName getSystemTopicName(NamespaceName namespaceName, EventType eventType) { - switch (eventType) { - case TOPIC_POLICY: - return TopicName.get(TopicDomain.persistent.value(), namespaceName, - SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME); - case TRANSACTION_BUFFER_SNAPSHOT: - return TopicName.get(TopicDomain.persistent.value(), namespaceName, - SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); - default: - return null; - } + return switch (eventType) { + case TOPIC_POLICY -> TopicName.get(TopicDomain.persistent.value(), namespaceName, + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME); + case TRANSACTION_BUFFER_SNAPSHOT -> TopicName.get(TopicDomain.persistent.value(), namespaceName, + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); + case TRANSACTION_BUFFER_SNAPSHOT_SEGMENT -> TopicName.get(TopicDomain.persistent.value(), namespaceName, + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT); + case TRANSACTION_BUFFER_SNAPSHOT_INDEXES -> TopicName.get(TopicDomain.persistent.value(), namespaceName, + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES); + }; } private static final Logger log = LoggerFactory.getLogger(NamespaceEventsSystemTopicFactory.class); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java index 2bc740a41d48b..052e474b0e747 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java @@ -156,6 +156,22 @@ interface Reader { */ CompletableFuture> readNextAsync(); + /** + * Read the specified event according to messageId. + * @param messageId the message id of the event that needs to be read. + * @return pulsar event + * @throws PulsarClientException + */ + + Message readByMessageId(MessageId messageId) throws PulsarClientException; + + /** + * Async read the specified event according to messageId. + * @param messageId the message id of the event that needs to be read. + * @return pulsar event future + */ + CompletableFuture> readByMessageIdAsync(MessageId messageId); + /** * Check has more events available for the reader. * @return true if has remaining events, otherwise false diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java index 84b8bd636bd3a..fc7d9885e54ca 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java @@ -28,6 +28,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TypedMessageBuilder; +import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.events.ActionType; import org.apache.pulsar.common.events.PulsarEvent; import org.apache.pulsar.common.naming.TopicName; @@ -167,6 +168,23 @@ public Message readNext() throws PulsarClientException { return reader.readNext(); } + @Override + public Message readByMessageId(MessageId messageId) throws PulsarClientException { + MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; + reader.seek(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, + messageIdImpl.getPartitionIndex())); + return reader.readNext(); + } + + @Override + public CompletableFuture> readByMessageIdAsync(MessageId messageId) { + MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; + return reader.seekAsync(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, + messageIdImpl.getPartitionIndex())).thenCompose((ignore) -> { + return reader.readNextAsync(); + }); + } + @Override public CompletableFuture> readNextAsync() { return reader.readNextAsync(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java new file mode 100644 index 0000000000000..aef58ee61b7b1 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java @@ -0,0 +1,239 @@ +/** + * 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.TransactionBufferSnapshotIndexService; +import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; +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.client.impl.MessageIdImpl; +import org.apache.pulsar.common.naming.TopicName; + +@Slf4j +public class TransactionBufferSnapshotIndexSystemTopicClient extends + SystemTopicClientBase { + + private final TransactionBufferSnapshotIndexService transactionBufferSnapshotIndexService; + + public TransactionBufferSnapshotIndexSystemTopicClient(PulsarClient client, TopicName topicName, + TransactionBufferSnapshotIndexService + transactionBufferSnapshotIndexService) { + super(client, topicName); + this.transactionBufferSnapshotIndexService = transactionBufferSnapshotIndexService; + } + + @Override + protected CompletableFuture> newWriterAsyncInternal() { + return client.newProducer(Schema.AVRO(TransactionBufferSnapshotIndexes.class)) + .topic(topicName.toString()) + .createAsync().thenCompose(producer -> { + if (log.isDebugEnabled()) { + log.debug("[{}] A new transactionBufferSnapshot writer is created", topicName); + } + return CompletableFuture.completedFuture( + new TransactionBufferSnapshotIndexWriter(producer, this)); + }); + } + + @Override + protected CompletableFuture> newReaderAsyncInternal() { + return client.newReader(Schema.AVRO(TransactionBufferSnapshotIndexes.class)) + .topic(topicName.toString()) + .startMessageId(MessageId.earliest) + .readCompacted(true) + .createAsync() + .thenCompose(reader -> { + if (log.isDebugEnabled()) { + log.debug("[{}] A new transactionBufferSnapshot buffer reader is created", topicName); + } + return CompletableFuture.completedFuture( + new TransactionBufferSnapshotIndexReader(reader, this)); + }); + } + + protected void removeWriter(TransactionBufferSnapshotIndexWriter writer) { + writers.remove(writer); + this.transactionBufferSnapshotIndexService.removeClient(topicName, this); + } + + protected void removeReader(TransactionBufferSnapshotIndexReader reader) { + readers.remove(reader); + this.transactionBufferSnapshotIndexService.removeClient(topicName, this); + } + + private static class TransactionBufferSnapshotIndexWriter implements Writer { + + private final Producer producer; + private final TransactionBufferSnapshotIndexSystemTopicClient + transactionBufferSnapshotIndexSystemTopicClient; + + private TransactionBufferSnapshotIndexWriter(Producer producer, + TransactionBufferSnapshotIndexSystemTopicClient + transactionBufferSnapshotIndexSystemTopicClient) { + this.producer = producer; + this.transactionBufferSnapshotIndexSystemTopicClient = transactionBufferSnapshotIndexSystemTopicClient; + } + + @Override + public MessageId write(TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes) + throws PulsarClientException { + return producer.newMessage().key(transactionBufferSnapshotIndexes.getTopicName()) + .value(transactionBufferSnapshotIndexes).send(); + } + + @Override + public CompletableFuture writeAsync(TransactionBufferSnapshotIndexes + transactionBufferSnapshotIndexes) { + return producer.newMessage() + .key(transactionBufferSnapshotIndexes.getTopicName()) + .value(transactionBufferSnapshotIndexes).sendAsync(); + } + + @Override + public MessageId delete(TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes) + throws PulsarClientException { + return producer.newMessage() + .key(transactionBufferSnapshotIndexes.getTopicName()) + .value(null) + .send(); + } + + @Override + public CompletableFuture deleteAsync(TransactionBufferSnapshotIndexes + transactionBufferSnapshotIndexes) { + return producer.newMessage() + .key(transactionBufferSnapshotIndexes.getTopicName()) + .value(null) + .sendAsync(); + } + + @Override + public void close() throws IOException { + this.producer.close(); + transactionBufferSnapshotIndexSystemTopicClient.removeWriter(this); + } + + @Override + public CompletableFuture closeAsync() { + CompletableFuture completableFuture = new CompletableFuture<>(); + producer.closeAsync().whenComplete((v, e) -> { + // if close fail, also need remove the producer + transactionBufferSnapshotIndexSystemTopicClient.removeWriter(this); + if (e != null) { + completableFuture.completeExceptionally(e); + return; + } + completableFuture.complete(null); + }); + return completableFuture; + } + + @Override + public SystemTopicClient getSystemTopicClient() { + return transactionBufferSnapshotIndexSystemTopicClient; + } + } + + private static class TransactionBufferSnapshotIndexReader implements Reader { + + private final org.apache.pulsar.client.api.Reader reader; + private final TransactionBufferSnapshotIndexSystemTopicClient + transactionBufferSnapshotIndexSystemTopicClient; + + private TransactionBufferSnapshotIndexReader( + org.apache.pulsar.client.api.Reader reader, + TransactionBufferSnapshotIndexSystemTopicClient transactionBufferSnapshotIndexSystemTopicClient) { + this.reader = reader; + this.transactionBufferSnapshotIndexSystemTopicClient = transactionBufferSnapshotIndexSystemTopicClient; + } + + @Override + public Message readNext() throws PulsarClientException { + return reader.readNext(); + } + + @Override + public CompletableFuture> readNextAsync() { + return reader.readNextAsync(); + } + + @Override + public Message readByMessageId(MessageId messageId) + throws PulsarClientException { + MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; + reader.seek(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, + messageIdImpl.getPartitionIndex())); + return reader.readNext(); + } + + @Override + public CompletableFuture> readByMessageIdAsync(MessageId messageId) { + MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; + return reader.seekAsync(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, + messageIdImpl.getPartitionIndex())).thenCompose((ignore) -> { + return reader.readNextAsync(); + }); + } + + @Override + public boolean hasMoreEvents() throws PulsarClientException { + return reader.hasMessageAvailable(); + } + + @Override + public CompletableFuture hasMoreEventsAsync() { + return reader.hasMessageAvailableAsync(); + } + + @Override + public void close() throws IOException { + this.reader.close(); + transactionBufferSnapshotIndexSystemTopicClient.removeReader(this); + } + + @Override + public CompletableFuture closeAsync() { + CompletableFuture completableFuture = new CompletableFuture<>(); + reader.closeAsync().whenComplete((v, e) -> { + // if close fail, also need remove the reader + transactionBufferSnapshotIndexSystemTopicClient.removeReader(this); + if (e != null) { + completableFuture.completeExceptionally(e); + return; + } + completableFuture.complete(null); + }); + return completableFuture; + } + + @Override + public SystemTopicClient getSystemTopic() { + return transactionBufferSnapshotIndexSystemTopicClient; + } + } +} + diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java new file mode 100644 index 0000000000000..6b4b5fb1c6a28 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java @@ -0,0 +1,245 @@ +/** + * 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.TransactionBufferSnapshotSegmentService; +import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshot; +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.client.impl.MessageIdImpl; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.FutureUtil; + +@Slf4j +public class TransactionBufferSnapshotSegmentSystemTopicClient extends + SystemTopicClientBase { + + private final TransactionBufferSnapshotSegmentService transactionBufferSnapshotSegmentService; + public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client, TopicName topicName, + TransactionBufferSnapshotSegmentService + transactionBufferSnapshotSegmentService) { + super(client, topicName); + this.transactionBufferSnapshotSegmentService = transactionBufferSnapshotSegmentService; + } + + @Override + protected CompletableFuture> newWriterAsyncInternal() { + return client.newProducer(Schema.AVRO(TransactionBufferSnapshot.class)) + .topic(topicName.toString()) + .createAsync().thenCompose(producer -> { + if (log.isDebugEnabled()) { + log.debug("[{}] A new transactionBufferSnapshot segment writer is created", topicName); + } + return CompletableFuture.completedFuture( + new TransactionBufferSnapshotSegmentWriter(producer, this)); + }); + } + + @Override + protected CompletableFuture> newReaderAsyncInternal() { + return client.newReader(Schema.AVRO(TransactionBufferSnapshot.class)) + .topic(topicName.toString()) + .startMessageId(MessageId.earliest) + .readCompacted(true) + .createAsync() + .thenCompose(reader -> { + if (log.isDebugEnabled()) { + log.debug("[{}] A new transactionBufferSnapshot buffer snapshot segment reader is created", + topicName); + } + return CompletableFuture.completedFuture( + new TransactionBufferSnapshotSegmentReader(reader, this)); + }); + } + + protected static String buildKey(TransactionBufferSnapshot snapshot) { + return "multiple-" + snapshot.getSequenceId() + "-" + snapshot.getTopicName(); + } + + protected void removeWriter(TransactionBufferSnapshotSegmentWriter writer) { + writers.remove(writer); + this.transactionBufferSnapshotSegmentService.removeClient(topicName, this); + } + + protected void removeReader(TransactionBufferSnapshotSegmentReader reader) { + readers.remove(reader); + this.transactionBufferSnapshotSegmentService.removeClient(topicName, this); + } + + + private static class TransactionBufferSnapshotSegmentWriter implements Writer { + + private final Producer producer; + private final TransactionBufferSnapshotSegmentSystemTopicClient + transactionBufferSnapshotSegmentSystemTopicClient; + + private TransactionBufferSnapshotSegmentWriter(Producer producer, + TransactionBufferSnapshotSegmentSystemTopicClient + transactionBufferSnapshotSegmentSystemTopicClient) { + this.producer = producer; + this.transactionBufferSnapshotSegmentSystemTopicClient = transactionBufferSnapshotSegmentSystemTopicClient; + } + + @Override + public MessageId write(TransactionBufferSnapshot transactionBufferSnapshot) + throws PulsarClientException { + return producer.newMessage() + .key(buildKey(transactionBufferSnapshot)) + .value(transactionBufferSnapshot).send(); + } + + @Override + public CompletableFuture writeAsync(TransactionBufferSnapshot + transactionBufferSnapshot) { + return producer.newMessage() + .key(buildKey(transactionBufferSnapshot)) + .value(transactionBufferSnapshot).sendAsync(); + } + + @Override + public MessageId delete(TransactionBufferSnapshot transactionBufferSnapshot) + throws PulsarClientException { + return producer.newMessage() + .key(buildKey(transactionBufferSnapshot)) + .value(null) + .send(); + } + + @Override + public CompletableFuture deleteAsync(TransactionBufferSnapshot + transactionBufferSnapshot) { + return producer.newMessage() + .key(buildKey(transactionBufferSnapshot)) + .value(null) + .sendAsync(); + } + + @Override + public void close() throws IOException { + this.producer.close(); + transactionBufferSnapshotSegmentSystemTopicClient.removeWriter(this); + } + + @Override + public CompletableFuture closeAsync() { + CompletableFuture completableFuture = new CompletableFuture<>(); + producer.closeAsync().whenComplete((v, e) -> { + // if close fail, also need remove the producer + transactionBufferSnapshotSegmentSystemTopicClient.removeWriter(this); + if (e != null) { + completableFuture.completeExceptionally(e); + return; + } + completableFuture.complete(null); + }); + return completableFuture; + } + + @Override + public SystemTopicClient getSystemTopicClient() { + return transactionBufferSnapshotSegmentSystemTopicClient; + } + } + + private static class TransactionBufferSnapshotSegmentReader implements Reader { + + private final org.apache.pulsar.client.api.Reader reader; + private final TransactionBufferSnapshotSegmentSystemTopicClient + transactionBufferSnapshotSegmentSystemTopicClient; + + private TransactionBufferSnapshotSegmentReader( + org.apache.pulsar.client.api.Reader reader, + TransactionBufferSnapshotSegmentSystemTopicClient transactionBufferSnapshotSegmentSystemTopicClient) { + this.reader = reader; + this.transactionBufferSnapshotSegmentSystemTopicClient = transactionBufferSnapshotSegmentSystemTopicClient; + } + + @Override + public Message readNext() throws PulsarClientException { + throw new UnsupportedOperationException( + "Transaction buffer snapshot segment does not support sequential reads."); + } + + @Override + public CompletableFuture> readNextAsync() { + return FutureUtil.failedFuture( + new UnsupportedOperationException( + "Transaction buffer snapshot segment does not support sequential reads.")); + } + + @Override + public Message readByMessageId(MessageId messageId) throws PulsarClientException { + MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; + reader.seek(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, + messageIdImpl.getPartitionIndex())); + return reader.readNext(); + } + + @Override + public CompletableFuture> readByMessageIdAsync(MessageId messageId) { + MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; + return reader.seekAsync(new MessageIdImpl(messageIdImpl.getLedgerId(), + messageIdImpl.getEntryId() - 1, messageIdImpl.getPartitionIndex())) + .thenCompose((ignore) -> reader.readNextAsync()); + } + + @Override + public boolean hasMoreEvents() throws PulsarClientException { + return reader.hasMessageAvailable(); + } + + @Override + public CompletableFuture hasMoreEventsAsync() { + return reader.hasMessageAvailableAsync(); + } + + @Override + public void close() throws IOException { + this.reader.close(); + transactionBufferSnapshotSegmentSystemTopicClient.removeReader(this); + } + + @Override + public CompletableFuture closeAsync() { + CompletableFuture completableFuture = new CompletableFuture<>(); + reader.closeAsync().whenComplete((v, e) -> { + // if close fail, also need remove the reader + transactionBufferSnapshotSegmentSystemTopicClient.removeReader(this); + if (e != null) { + completableFuture.completeExceptionally(e); + return; + } + completableFuture.complete(null); + }); + return completableFuture; + } + + @Override + public SystemTopicClient getSystemTopic() { + return transactionBufferSnapshotSegmentSystemTopicClient; + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSystemTopicClient.java index aaab858ab1ee2..087d98b618335 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSystemTopicClient.java @@ -30,6 +30,7 @@ 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.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicName; @Slf4j @@ -168,6 +169,23 @@ public CompletableFuture> readNextAsync() { return reader.readNextAsync(); } + @Override + public Message readByMessageId(MessageId messageId) throws PulsarClientException { + MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; + reader.seek(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, + messageIdImpl.getPartitionIndex())); + return reader.readNext(); + } + + @Override + public CompletableFuture> readByMessageIdAsync(MessageId messageId) { + MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; + return reader.seekAsync(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, + messageIdImpl.getPartitionIndex())).thenCompose((ignore) -> { + return reader.readNextAsync(); + }); + } + @Override public boolean hasMoreEvents() throws PulsarClientException { return reader.hasMessageAvailable(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshot.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshot.java new file mode 100644 index 0000000000000..484823c377584 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshot.java @@ -0,0 +1,35 @@ +/** + * 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.transaction.buffer.matadata.v2; + +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@AllArgsConstructor +@NoArgsConstructor +public class TransactionBufferSnapshot { + private String topicName; + private long sequenceId; + private long maxReadPositionLedgerId; + private long maxReadPositionEntryId; + private List aborts; +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java new file mode 100644 index 0000000000000..0c1c1cc5f0a92 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java @@ -0,0 +1,59 @@ +/** + * 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.transaction.buffer.matadata.v2; + +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; + +@AllArgsConstructor +@NoArgsConstructor +@Getter +@Setter +@Builder +public class TransactionBufferSnapshotIndexes { + public enum Type { + Indexes, + UnsealedSnapshot + } + + private String topicName; + + private Type type; + + private List indexList; + + private TransactionBufferSnapshot snapshot; + + @Builder + @Data + @AllArgsConstructor + @NoArgsConstructor + public static class TransactionBufferSnapshotIndex { + public long sequenceID; + public long maxReadPositionLedgerID; + public long maxReadPositionEntryID; + public long persistentPositionLedgerID; + public long persistentPositionEntryID; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TxnID.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TxnID.java new file mode 100644 index 0000000000000..6a74725c2cb3c --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TxnID.java @@ -0,0 +1,51 @@ +/** + * 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.transaction.buffer.matadata.v2; + +import java.util.Objects; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; + +@AllArgsConstructor +@NoArgsConstructor +@Getter +@Setter +public class TxnID { + long txnIdMostBits; + long txnIdLeastBits; + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TxnID txnID = (TxnID) o; + return txnIdMostBits == txnID.txnIdMostBits && txnIdLeastBits == txnID.txnIdLeastBits; + } + + @Override + public int hashCode() { + return Objects.hash(txnIdMostBits, txnIdLeastBits); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/package-info.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/package-info.java new file mode 100644 index 0000000000000..d0c7c97cfc4ae --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/package-info.java @@ -0,0 +1,19 @@ +/** + * 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.transaction.buffer.matadata.v2; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index c0afdbee48756..02e4c94572e48 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -31,6 +31,9 @@ import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Method; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; import java.util.NavigableMap; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -47,12 +50,15 @@ import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotIndexService; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotSegmentService; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBuffer; 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.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; @@ -82,6 +88,8 @@ public class TopicTransactionBufferRecoverTest extends TransactionTestBase { private static final String RECOVER_COMMIT = NAMESPACE1 + "/recover-commit"; private static final String RECOVER_ABORT = NAMESPACE1 + "/recover-abort"; + private static final String SNAPSHOT_INDEX = NAMESPACE1 + "/snapshot-index"; + private static final String SNAPSHOT_SEGMENT = NAMESPACE1 + "/snapshot-segment"; private static final String SUBSCRIPTION_NAME = "test-recover"; private static final String TAKE_SNAPSHOT = NAMESPACE1 + "/take-snapshot"; private static final String ABORT_DELETE = NAMESPACE1 + "/abort-delete"; @@ -565,4 +573,82 @@ public void testTransactionBufferNoSnapshotCloseReader() throws Exception{ assertTrue(stats.getSubscriptions().keySet().contains("__compaction")); } + @Test + public void testTransactionBufferIndexSystemTopic() throws Exception { + TransactionBufferSnapshotIndexService transactionBufferSnapshotIndexService = + getPulsarServiceList().get(0).getTransactionBufferSnapshotIndexService(); + + SystemTopicClient.Writer indexesWriter = + transactionBufferSnapshotIndexService.createWriter(TopicName.get(SNAPSHOT_INDEX)).get(); + + SystemTopicClient.Reader indexesReader = + transactionBufferSnapshotIndexService.createReader(TopicName.get(SNAPSHOT_INDEX)).get(); + + + List indexList = new LinkedList<>(); + + for (long i = 0; i < 5; i++) { + indexList.add(new TransactionBufferSnapshotIndexes.TransactionBufferSnapshotIndex(i, i, i, i, i)); + } + + TransactionBufferSnapshotIndexes transactionBufferTransactionBufferSnapshotIndexes = + new TransactionBufferSnapshotIndexes(SNAPSHOT_INDEX, TransactionBufferSnapshotIndexes.Type.Indexes, + indexList, null); + + indexesWriter.write(transactionBufferTransactionBufferSnapshotIndexes); + + assertTrue(indexesReader.hasMoreEvents()); + transactionBufferTransactionBufferSnapshotIndexes = indexesReader.readNext().getValue(); + assertEquals(transactionBufferTransactionBufferSnapshotIndexes.getTopicName(), SNAPSHOT_INDEX); + assertEquals( + transactionBufferTransactionBufferSnapshotIndexes.getType(), + TransactionBufferSnapshotIndexes.Type.Indexes); + assertEquals(transactionBufferTransactionBufferSnapshotIndexes.getIndexList().size(), 5); + assertNull(transactionBufferTransactionBufferSnapshotIndexes.getSnapshot()); + + TransactionBufferSnapshotIndexes.TransactionBufferSnapshotIndex transactionBufferSnapshotIndex = + transactionBufferTransactionBufferSnapshotIndexes.getIndexList().get(1); + assertEquals(transactionBufferSnapshotIndex.getMaxReadPositionLedgerID(), 1L); + assertEquals(transactionBufferSnapshotIndex.getMaxReadPositionEntryID(), 1L); + assertEquals(transactionBufferSnapshotIndex.getPersistentPositionLedgerID(), 1L); + assertEquals(transactionBufferSnapshotIndex.getPersistentPositionEntryID(), 1L); + assertEquals(transactionBufferSnapshotIndex.getSequenceID(), 1L); + } + + @Test + public void testTransactionBufferSegmentSystemTopic() throws Exception { + TransactionBufferSnapshotSegmentService transactionBufferSnapshotSegmentService = + getPulsarServiceList().get(0).getTransactionBufferSnapshotSegmentService(); + + SystemTopicClient.Writer + segmentWriter = + transactionBufferSnapshotSegmentService.createWriter(TopicName.get(SNAPSHOT_SEGMENT)).get(); + + SystemTopicClient.Reader + segmentReader = + transactionBufferSnapshotSegmentService.createReader(TopicName.get(SNAPSHOT_SEGMENT)).get(); + + org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshot snapshot = + new org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshot(); + + snapshot.setTopicName(SNAPSHOT_SEGMENT); + snapshot.setSequenceId(1L); + snapshot.setMaxReadPositionLedgerId(2L); + snapshot.setMaxReadPositionEntryId(3L); + snapshot.setAborts(Collections.singletonList( + new org.apache.pulsar.broker.transaction.buffer.matadata.v2.TxnID(1, 1))); + + MessageId messageId = segmentWriter.write(snapshot); + segmentWriter.write(snapshot); + + snapshot = segmentReader.readByMessageId(messageId).getValue(); + assertEquals(snapshot.getTopicName(), SNAPSHOT_SEGMENT); + assertEquals(snapshot.getSequenceId(), 1L); + assertEquals(snapshot.getMaxReadPositionLedgerId(), 2L); + assertEquals(snapshot.getMaxReadPositionEntryId(), 3L); + assertEquals(snapshot.getAborts().get(0), + new org.apache.pulsar.broker.transaction.buffer.matadata.v2.TxnID(1, 1)); + + } + } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/events/EventType.java b/pulsar-common/src/main/java/org/apache/pulsar/common/events/EventType.java index b60350e8e39a5..f5506318b9bd6 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/events/EventType.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/events/EventType.java @@ -31,5 +31,15 @@ public enum EventType { /** * Transaction buffer snapshot events. */ - TRANSACTION_BUFFER_SNAPSHOT + TRANSACTION_BUFFER_SNAPSHOT, + + /** + * Transaction buffer snapshot segment events. + */ + TRANSACTION_BUFFER_SNAPSHOT_SEGMENT, + + /** + * Transaction buffer snapshot indexes events. + */ + TRANSACTION_BUFFER_SNAPSHOT_INDEXES } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java index eaab826146000..f3a385b82e342 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java @@ -37,6 +37,15 @@ public class SystemTopicNames { */ public static final String TRANSACTION_BUFFER_SNAPSHOT = "__transaction_buffer_snapshot"; + /** + * Local topic name for the transaction buffer snapshot segment. + */ + public static final String TRANSACTION_BUFFER_SNAPSHOT_SEGMENT = "__transaction_buffer_snapshot_segment"; + + /** + * Local topic name for the transaction buffer snapshot indexes. + */ + public static final String TRANSACTION_BUFFER_SNAPSHOT_INDEXES = "__transaction_buffer_snapshot_indexes"; public static final String PENDING_ACK_STORE_SUFFIX = "__transaction_pending_ack"; @@ -46,7 +55,8 @@ public class SystemTopicNames { * The set of all local topic names declared above. */ public static final Set EVENTS_TOPIC_NAMES = - Collections.unmodifiableSet(Sets.newHashSet(NAMESPACE_EVENTS_LOCAL_NAME, TRANSACTION_BUFFER_SNAPSHOT)); + Collections.unmodifiableSet(Sets.newHashSet(NAMESPACE_EVENTS_LOCAL_NAME, TRANSACTION_BUFFER_SNAPSHOT, + TRANSACTION_BUFFER_SNAPSHOT_INDEXES, TRANSACTION_BUFFER_SNAPSHOT_SEGMENT)); public static final TopicName TRANSACTION_COORDINATOR_ASSIGN = TopicName.get(TopicDomain.persistent.value(), From f8fa9136211846cba3becccff80bc93861173d8a Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Thu, 8 Sep 2022 12:03:57 +0800 Subject: [PATCH 02/27] move snapshot to index --- ...icBaseTxnBufferSnapshotSegmentService.java | 12 ++--- ...ansactionBufferSnapshotSegmentService.java | 6 +-- ...ufferSnapshotSegmentSystemTopicClient.java | 46 +++++++++---------- .../v2/TransactionBufferSnapshot.java | 35 -------------- .../v2/TransactionBufferSnapshotIndexes.java | 11 +++++ .../TopicTransactionBufferRecoverTest.java | 8 ++-- 6 files changed, 47 insertions(+), 71 deletions(-) delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshot.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java index 9d37b65afe3fc..3b7d09f942aca 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java @@ -24,7 +24,7 @@ import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.systopic.TransactionBufferSnapshotSegmentSystemTopicClient; -import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshot; +import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.events.EventType; @@ -33,7 +33,7 @@ public class SystemTopicBaseTxnBufferSnapshotSegmentService implements TransactionBufferSnapshotSegmentService { - private final Map> clients; + private final Map> clients; private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; @@ -43,11 +43,11 @@ public SystemTopicBaseTxnBufferSnapshotSegmentService(PulsarClient client) { } @Override - public CompletableFuture> createWriter(TopicName topicName) { + public CompletableFuture> createWriter(TopicName topicName) { return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync); } - private CompletableFuture> + private CompletableFuture> getTransactionBufferSystemTopicClient( TopicName topicName) { TopicName systemTopicName = NamespaceEventsSystemTopicFactory @@ -65,7 +65,7 @@ public CompletableFuture> cr } @Override - public CompletableFuture> createReader(TopicName topicName) { + public CompletableFuture> createReader(TopicName topicName) { return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newReaderAsync); } @@ -81,7 +81,7 @@ public void removeClient(TopicName topicName, @Override public void close() throws Exception { - for (Map.Entry> entry : clients.entrySet()) { + for (Map.Entry> entry : clients.entrySet()) { entry.getValue().close(); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java index 0c6440404fae2..347b36f7d35e5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java @@ -21,7 +21,7 @@ import java.util.concurrent.CompletableFuture; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.systopic.TransactionBufferSnapshotSegmentSystemTopicClient; -import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshot; +import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.common.naming.TopicName; public interface TransactionBufferSnapshotSegmentService { @@ -32,7 +32,7 @@ public interface TransactionBufferSnapshotSegmentService { * * @return {@link CompletableFuture< SystemTopicClient.Writer >} return the future of writer */ - CompletableFuture> createWriter(TopicName topicName); + CompletableFuture> createWriter(TopicName topicName); /** * Create a transaction buffer snapshot segment reader. @@ -41,7 +41,7 @@ public interface TransactionBufferSnapshotSegmentService { * * @return {@link CompletableFuture< SystemTopicClient.Reader >} return the future of reader */ - CompletableFuture> createReader(TopicName topicName); + CompletableFuture> createReader(TopicName topicName); /** * Remove a topic client from cache. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java index 6b4b5fb1c6a28..892c33a11af57 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java @@ -22,7 +22,7 @@ import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.service.TransactionBufferSnapshotSegmentService; -import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshot; +import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -35,7 +35,7 @@ @Slf4j public class TransactionBufferSnapshotSegmentSystemTopicClient extends - SystemTopicClientBase { + SystemTopicClientBase { private final TransactionBufferSnapshotSegmentService transactionBufferSnapshotSegmentService; public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client, TopicName topicName, @@ -46,8 +46,8 @@ public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client, To } @Override - protected CompletableFuture> newWriterAsyncInternal() { - return client.newProducer(Schema.AVRO(TransactionBufferSnapshot.class)) + protected CompletableFuture> newWriterAsyncInternal() { + return client.newProducer(Schema.AVRO(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot.class)) .topic(topicName.toString()) .createAsync().thenCompose(producer -> { if (log.isDebugEnabled()) { @@ -59,8 +59,8 @@ protected CompletableFuture> newWriterAsyncInt } @Override - protected CompletableFuture> newReaderAsyncInternal() { - return client.newReader(Schema.AVRO(TransactionBufferSnapshot.class)) + protected CompletableFuture> newReaderAsyncInternal() { + return client.newReader(Schema.AVRO(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot.class)) .topic(topicName.toString()) .startMessageId(MessageId.earliest) .readCompacted(true) @@ -75,7 +75,7 @@ protected CompletableFuture> newReaderAsyncInt }); } - protected static String buildKey(TransactionBufferSnapshot snapshot) { + protected static String buildKey(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot snapshot) { return "multiple-" + snapshot.getSequenceId() + "-" + snapshot.getTopicName(); } @@ -90,13 +90,13 @@ protected void removeReader(TransactionBufferSnapshotSegmentReader reader) { } - private static class TransactionBufferSnapshotSegmentWriter implements Writer { + private static class TransactionBufferSnapshotSegmentWriter implements Writer { - private final Producer producer; + private final Producer producer; private final TransactionBufferSnapshotSegmentSystemTopicClient transactionBufferSnapshotSegmentSystemTopicClient; - private TransactionBufferSnapshotSegmentWriter(Producer producer, + private TransactionBufferSnapshotSegmentWriter(Producer producer, TransactionBufferSnapshotSegmentSystemTopicClient transactionBufferSnapshotSegmentSystemTopicClient) { this.producer = producer; @@ -104,7 +104,7 @@ private TransactionBufferSnapshotSegmentWriter(Producer writeAsync(TransactionBufferSnapshot + public CompletableFuture writeAsync(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot transactionBufferSnapshot) { return producer.newMessage() .key(buildKey(transactionBufferSnapshot)) @@ -120,7 +120,7 @@ public CompletableFuture writeAsync(TransactionBufferSnapshot } @Override - public MessageId delete(TransactionBufferSnapshot transactionBufferSnapshot) + public MessageId delete(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot transactionBufferSnapshot) throws PulsarClientException { return producer.newMessage() .key(buildKey(transactionBufferSnapshot)) @@ -129,7 +129,7 @@ public MessageId delete(TransactionBufferSnapshot transactionBufferSnapshot) } @Override - public CompletableFuture deleteAsync(TransactionBufferSnapshot + public CompletableFuture deleteAsync(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot transactionBufferSnapshot) { return producer.newMessage() .key(buildKey(transactionBufferSnapshot)) @@ -159,39 +159,39 @@ public CompletableFuture closeAsync() { } @Override - public SystemTopicClient getSystemTopicClient() { + public SystemTopicClient getSystemTopicClient() { return transactionBufferSnapshotSegmentSystemTopicClient; } } - private static class TransactionBufferSnapshotSegmentReader implements Reader { + private static class TransactionBufferSnapshotSegmentReader implements Reader { - private final org.apache.pulsar.client.api.Reader reader; + private final org.apache.pulsar.client.api.Reader reader; private final TransactionBufferSnapshotSegmentSystemTopicClient transactionBufferSnapshotSegmentSystemTopicClient; private TransactionBufferSnapshotSegmentReader( - org.apache.pulsar.client.api.Reader reader, + org.apache.pulsar.client.api.Reader reader, TransactionBufferSnapshotSegmentSystemTopicClient transactionBufferSnapshotSegmentSystemTopicClient) { this.reader = reader; this.transactionBufferSnapshotSegmentSystemTopicClient = transactionBufferSnapshotSegmentSystemTopicClient; } @Override - public Message readNext() throws PulsarClientException { + public Message readNext() throws PulsarClientException { throw new UnsupportedOperationException( "Transaction buffer snapshot segment does not support sequential reads."); } @Override - public CompletableFuture> readNextAsync() { + public CompletableFuture> readNextAsync() { return FutureUtil.failedFuture( new UnsupportedOperationException( "Transaction buffer snapshot segment does not support sequential reads.")); } @Override - public Message readByMessageId(MessageId messageId) throws PulsarClientException { + public Message readByMessageId(MessageId messageId) throws PulsarClientException { MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; reader.seek(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, messageIdImpl.getPartitionIndex())); @@ -199,7 +199,7 @@ public Message readByMessageId(MessageId messageId) t } @Override - public CompletableFuture> readByMessageIdAsync(MessageId messageId) { + public CompletableFuture> readByMessageIdAsync(MessageId messageId) { MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; return reader.seekAsync(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, messageIdImpl.getPartitionIndex())) @@ -238,7 +238,7 @@ public CompletableFuture closeAsync() { } @Override - public SystemTopicClient getSystemTopic() { + public SystemTopicClient getSystemTopic() { return transactionBufferSnapshotSegmentSystemTopicClient; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshot.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshot.java deleted file mode 100644 index 484823c377584..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshot.java +++ /dev/null @@ -1,35 +0,0 @@ -/** - * 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.transaction.buffer.matadata.v2; - -import java.util.List; -import lombok.AllArgsConstructor; -import lombok.Data; -import lombok.NoArgsConstructor; - -@Data -@AllArgsConstructor -@NoArgsConstructor -public class TransactionBufferSnapshot { - private String topicName; - private long sequenceId; - private long maxReadPositionLedgerId; - private long maxReadPositionEntryId; - private List aborts; -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java index 0c1c1cc5f0a92..df9c179642dc8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java @@ -56,4 +56,15 @@ public static class TransactionBufferSnapshotIndex { public long persistentPositionLedgerID; public long persistentPositionEntryID; } + + @Data + @AllArgsConstructor + @NoArgsConstructor + public static class TransactionBufferSnapshot { + private String topicName; + private long sequenceId; + private long maxReadPositionLedgerId; + private long maxReadPositionEntryId; + private List aborts; + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 02e4c94572e48..db9349d31f994 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -620,16 +620,16 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { TransactionBufferSnapshotSegmentService transactionBufferSnapshotSegmentService = getPulsarServiceList().get(0).getTransactionBufferSnapshotSegmentService(); - SystemTopicClient.Writer + SystemTopicClient.Writer segmentWriter = transactionBufferSnapshotSegmentService.createWriter(TopicName.get(SNAPSHOT_SEGMENT)).get(); - SystemTopicClient.Reader + SystemTopicClient.Reader segmentReader = transactionBufferSnapshotSegmentService.createReader(TopicName.get(SNAPSHOT_SEGMENT)).get(); - org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshot snapshot = - new org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshot(); + TransactionBufferSnapshotIndexes.TransactionBufferSnapshot snapshot = + new TransactionBufferSnapshotIndexes.TransactionBufferSnapshot(); snapshot.setTopicName(SNAPSHOT_SEGMENT); snapshot.setSequenceId(1L); From d98e006c6a936690c66d2ae8719e75824306f3a5 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Thu, 15 Sep 2022 17:57:49 +0800 Subject: [PATCH 03/27] abstract --- .../apache/pulsar/broker/PulsarService.java | 11 +- ...TopicBaseTxnBufferSnapshotBaseService.java | 65 +++++++ ...opicBaseTxnBufferSnapshotIndexService.java | 38 +--- ...icBaseTxnBufferSnapshotSegmentService.java | 36 +--- ...stemTopicBaseTxnBufferSnapshotService.java | 38 +--- ...TransactionBufferSnapshotIndexService.java | 64 ------- ...ansactionBufferSnapshotSegmentService.java | 61 ------ .../TransactionBufferSnapshotService.java | 10 +- .../NamespaceEventsSystemTopicFactory.java | 13 +- ...onBufferSnapshotBaseSystemTopicClient.java | 173 ++++++++++++++++++ ...nBufferSnapshotIndexSystemTopicClient.java | 135 ++------------ ...ufferSnapshotSegmentSystemTopicClient.java | 147 ++------------- .../TransactionBufferSystemTopicClient.java | 130 ++----------- .../TopicTransactionBufferRecoverTest.java | 7 +- 14 files changed, 312 insertions(+), 616 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotBaseService.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotIndexService.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 2a3335e4ffef2..4d850db471724 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -107,8 +107,6 @@ import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotIndexService; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotSegmentService; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.SchemaStorageFactory; @@ -118,6 +116,8 @@ import org.apache.pulsar.broker.storage.ManagedLedgerStorage; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; +import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; +import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; import org.apache.pulsar.broker.validator.MultipleListenerValidator; import org.apache.pulsar.broker.web.WebService; @@ -264,9 +264,10 @@ public class PulsarService implements AutoCloseable, ShutdownService { private MetadataStoreExtended localMetadataStore; private PulsarMetadataEventSynchronizer localMetadataSynchronizer; private CoordinationService coordinationService; - private TransactionBufferSnapshotService transactionBufferSnapshotService; - private TransactionBufferSnapshotIndexService transactionBufferSnapshotIndexService; - private TransactionBufferSnapshotSegmentService transactionBufferSnapshotSegmentService; + private TransactionBufferSnapshotService transactionBufferSnapshotService; + private TransactionBufferSnapshotService transactionBufferSnapshotIndexService; + private TransactionBufferSnapshotService + transactionBufferSnapshotSegmentService; private MetadataStore configurationMetadataStore; private PulsarMetadataEventSynchronizer configMetadataSynchronizer; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotBaseService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotBaseService.java new file mode 100644 index 0000000000000..dcae1c27fe761 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotBaseService.java @@ -0,0 +1,65 @@ +/** + * 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 java.util.Map; +import java.util.concurrent.CompletableFuture; +import org.apache.pulsar.broker.systopic.SystemTopicClient; +import org.apache.pulsar.broker.systopic.SystemTopicClientBase; +import org.apache.pulsar.common.naming.TopicName; + +public abstract class SystemTopicBaseTxnBufferSnapshotBaseService implements TransactionBufferSnapshotService { + + protected final Map> clients; + + public SystemTopicBaseTxnBufferSnapshotBaseService( + Map> clients) { + this.clients = clients; + } + + @Override + public CompletableFuture> createWriter(TopicName topicName) { + return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync); + + } + + @Override + public CompletableFuture> createReader(TopicName topicName) { + return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newReaderAsync); + } + + @Override + public void removeClient(TopicName topicName, SystemTopicClientBase transactionBufferSystemTopicClient) { + if (transactionBufferSystemTopicClient.getReaders().size() == 0 + && transactionBufferSystemTopicClient.getWriters().size() == 0) { + clients.remove(topicName); + } + } + + @Override + public void close() throws Exception { + for (Map.Entry> entry : clients.entrySet()) { + entry.getValue().close(); + } + } + + protected abstract CompletableFuture> getTransactionBufferSystemTopicClient( + TopicName topicName); + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotIndexService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotIndexService.java index 7120790fc1453..a60b7b3083ef7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotIndexService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotIndexService.java @@ -18,14 +18,10 @@ */ package org.apache.pulsar.broker.service; -import java.util.Map; import java.util.concurrent.CompletableFuture; 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.TransactionBufferSnapshotIndexSystemTopicClient; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException.InvalidTopicNameException; @@ -33,23 +29,18 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.FutureUtil; -public class SystemTopicBaseTxnBufferSnapshotIndexService implements TransactionBufferSnapshotIndexService { - - private final Map> clients; +public class SystemTopicBaseTxnBufferSnapshotIndexService extends + SystemTopicBaseTxnBufferSnapshotBaseService { private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; public SystemTopicBaseTxnBufferSnapshotIndexService(PulsarClient client) { + super(new ConcurrentHashMap<>()); this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); - this.clients = new ConcurrentHashMap<>(); } @Override - public CompletableFuture> createWriter(TopicName topicName) { - return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync); - } - - private CompletableFuture> + protected CompletableFuture> getTransactionBufferSystemTopicClient( TopicName topicName) { TopicName systemTopicName = NamespaceEventsSystemTopicFactory @@ -65,25 +56,4 @@ public CompletableFuture> createWriter( this))); } - @Override - public CompletableFuture> createReader(TopicName topicName) { - return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newReaderAsync); - } - - @Override - public void removeClient(TopicName topicName, - TransactionBufferSnapshotIndexSystemTopicClient - transactionBufferSnapshotIndexSystemTopicClient) { - if (transactionBufferSnapshotIndexSystemTopicClient.getReaders().size() == 0 - && transactionBufferSnapshotIndexSystemTopicClient.getWriters().size() == 0) { - clients.remove(topicName); - } - } - - @Override - public void close() throws Exception { - for (Map.Entry> entry : clients.entrySet()) { - entry.getValue().close(); - } - } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java index 3b7d09f942aca..58a323f7f825f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java @@ -18,12 +18,10 @@ */ package org.apache.pulsar.broker.service; -import java.util.Map; import java.util.concurrent.CompletableFuture; 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.TransactionBufferSnapshotSegmentSystemTopicClient; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -31,23 +29,17 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.FutureUtil; -public class SystemTopicBaseTxnBufferSnapshotSegmentService implements TransactionBufferSnapshotSegmentService { - - private final Map> clients; - +public class SystemTopicBaseTxnBufferSnapshotSegmentService extends + SystemTopicBaseTxnBufferSnapshotBaseService { private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; public SystemTopicBaseTxnBufferSnapshotSegmentService(PulsarClient client) { + super(new ConcurrentHashMap<>()); this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); - this.clients = new ConcurrentHashMap<>(); } @Override - public CompletableFuture> createWriter(TopicName topicName) { - return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync); - } - - private CompletableFuture> + protected CompletableFuture> getTransactionBufferSystemTopicClient( TopicName topicName) { TopicName systemTopicName = NamespaceEventsSystemTopicFactory @@ -64,25 +56,5 @@ public CompletableFuture> createReader(TopicName topicName) { - return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newReaderAsync); - } - @Override - public void removeClient(TopicName topicName, - TransactionBufferSnapshotSegmentSystemTopicClient - transactionBufferSnapshotSegmentSystemTopicClient) { - if (transactionBufferSnapshotSegmentSystemTopicClient.getReaders().size() == 0 - && transactionBufferSnapshotSegmentSystemTopicClient.getWriters().size() == 0) { - clients.remove(topicName); - } - } - - @Override - public void close() throws Exception { - for (Map.Entry> entry : clients.entrySet()) { - entry.getValue().close(); - } - } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotService.java index 719d492a52427..69867580e6a0a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotService.java @@ -18,14 +18,10 @@ */ package org.apache.pulsar.broker.service; -import java.util.Map; import java.util.concurrent.CompletableFuture; 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.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException.InvalidTopicNameException; @@ -33,23 +29,18 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.FutureUtil; -public class SystemTopicBaseTxnBufferSnapshotService implements TransactionBufferSnapshotService { - - private final Map> clients; +public class SystemTopicBaseTxnBufferSnapshotService + extends SystemTopicBaseTxnBufferSnapshotBaseService { private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; public SystemTopicBaseTxnBufferSnapshotService(PulsarClient client) { + super(new ConcurrentHashMap<>()); this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); - this.clients = new ConcurrentHashMap<>(); } @Override - public CompletableFuture> createWriter(TopicName topicName) { - return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync); - } - - private CompletableFuture> getTransactionBufferSystemTopicClient( + protected CompletableFuture> getTransactionBufferSystemTopicClient( TopicName topicName) { TopicName systemTopicName = NamespaceEventsSystemTopicFactory .getSystemTopicName(topicName.getNamespaceObject(), EventType.TRANSACTION_BUFFER_SNAPSHOT); @@ -62,25 +53,4 @@ private CompletableFuture> getTrans (v) -> namespaceEventsSystemTopicFactory .createTransactionBufferSystemTopicClient(topicName.getNamespaceObject(), this))); } - - @Override - public CompletableFuture> createReader(TopicName topicName) { - return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newReaderAsync); - } - - @Override - public void removeClient(TopicName topicName, - TransactionBufferSystemTopicClient transactionBufferSystemTopicClient) { - if (transactionBufferSystemTopicClient.getReaders().size() == 0 - && transactionBufferSystemTopicClient.getWriters().size() == 0) { - clients.remove(topicName); - } - } - - @Override - public void close() throws Exception { - for (Map.Entry> entry : clients.entrySet()) { - entry.getValue().close(); - } - } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotIndexService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotIndexService.java deleted file mode 100644 index ef913f36fa6f8..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotIndexService.java +++ /dev/null @@ -1,64 +0,0 @@ -/** - * 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 java.util.concurrent.CompletableFuture; -import org.apache.pulsar.broker.systopic.SystemTopicClient.Reader; -import org.apache.pulsar.broker.systopic.SystemTopicClient.Writer; -import org.apache.pulsar.broker.systopic.TransactionBufferSnapshotIndexSystemTopicClient; -import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; -import org.apache.pulsar.common.naming.TopicName; - -public interface TransactionBufferSnapshotIndexService { - - /** - * Create a transaction buffer snapshot index writer. - * - * @param topicName {@link TopicName} the topic name - * - * @return {@link CompletableFuture} return the future of writer - */ - CompletableFuture> createWriter(TopicName topicName); - - /** - * Create a transaction buffer snapshot index reader. - * - * @param topicName {@link TopicName} the topic name - * - * @return {@link CompletableFuture} return the future of reader - */ - CompletableFuture> createReader(TopicName topicName); - - /** - * Remove a topic client from cache. - * - * @param topicName {@link TopicName} the topic name - * @param transactionBufferSnapshotIndexSystemTopicClient - * {@link TransactionBufferSnapshotIndexSystemTopicClient} the topic client - * - */ - void removeClient(TopicName topicName, TransactionBufferSnapshotIndexSystemTopicClient - transactionBufferSnapshotIndexSystemTopicClient); - - /** - * Close transaction buffer snapshot service. - */ - void close() throws Exception; - -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java deleted file mode 100644 index 347b36f7d35e5..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentService.java +++ /dev/null @@ -1,61 +0,0 @@ -/** - * 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 java.util.concurrent.CompletableFuture; -import org.apache.pulsar.broker.systopic.SystemTopicClient; -import org.apache.pulsar.broker.systopic.TransactionBufferSnapshotSegmentSystemTopicClient; -import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; -import org.apache.pulsar.common.naming.TopicName; - -public interface TransactionBufferSnapshotSegmentService { - /** - * Create a transaction buffer snapshot segment writer. - * - * @param topicName {@link TopicName} the topic name - * - * @return {@link CompletableFuture< SystemTopicClient.Writer >} return the future of writer - */ - CompletableFuture> createWriter(TopicName topicName); - - /** - * Create a transaction buffer snapshot segment reader. - * - * @param topicName {@link TopicName} the topic name - * - * @return {@link CompletableFuture< SystemTopicClient.Reader >} return the future of reader - */ - CompletableFuture> createReader(TopicName topicName); - - /** - * Remove a topic client from cache. - * - * @param topicName {@link TopicName} the topic name - * @param transactionBufferSnapshotSegmentSystemTopicClient - * {@link TransactionBufferSnapshotSegmentSystemTopicClient} the topic client - * - */ - void removeClient(TopicName topicName, TransactionBufferSnapshotSegmentSystemTopicClient - transactionBufferSnapshotSegmentSystemTopicClient); - - /** - * Close transaction buffer snapshot service. - */ - void close() throws Exception; -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java index b090e8fe46a4f..841874b27801e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java @@ -21,11 +21,11 @@ import java.util.concurrent.CompletableFuture; import org.apache.pulsar.broker.systopic.SystemTopicClient.Reader; import org.apache.pulsar.broker.systopic.SystemTopicClient.Writer; +import org.apache.pulsar.broker.systopic.SystemTopicClientBase; import org.apache.pulsar.broker.systopic.TransactionBufferSystemTopicClient; -import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; import org.apache.pulsar.common.naming.TopicName; -public interface TransactionBufferSnapshotService { +public interface TransactionBufferSnapshotService { /** * Create a transaction buffer snapshot writer. @@ -34,7 +34,7 @@ public interface TransactionBufferSnapshotService { * * @return {@link CompletableFuture} return the future of writer */ - CompletableFuture> createWriter(TopicName topicName); + CompletableFuture> createWriter(TopicName topicName); /** * Create a transaction buffer snapshot reader. @@ -43,7 +43,7 @@ public interface TransactionBufferSnapshotService { * * @return {@link CompletableFuture} return the future of reader */ - CompletableFuture> createReader(TopicName topicName); + CompletableFuture> createReader(TopicName topicName); /** * Remove a topic client from cache. @@ -52,7 +52,7 @@ public interface TransactionBufferSnapshotService { * @param transactionBufferSystemTopicClient {@link TransactionBufferSystemTopicClient} the topic client * */ - void removeClient(TopicName topicName, TransactionBufferSystemTopicClient transactionBufferSystemTopicClient); + void removeClient(TopicName topicName, SystemTopicClientBase transactionBufferSystemTopicClient); /** * Close transaction buffer snapshot service. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java index 6faedf1c11307..3cf73a1300c29 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java @@ -18,9 +18,9 @@ */ package org.apache.pulsar.broker.systopic; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotIndexService; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotSegmentService; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; +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; import org.apache.pulsar.common.naming.NamespaceName; @@ -46,7 +46,8 @@ public TopicPoliciesSystemTopicClient createTopicPoliciesSystemTopicClient(Names } public TransactionBufferSystemTopicClient createTransactionBufferSystemTopicClient(NamespaceName namespaceName, - TransactionBufferSnapshotService transactionBufferSnapshotService) { + TransactionBufferSnapshotService + transactionBufferSnapshotService) { TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); log.info("Create transaction buffer snapshot client, topicName : {}", topicName.toString()); @@ -54,7 +55,8 @@ public TransactionBufferSystemTopicClient createTransactionBufferSystemTopicClie } public TransactionBufferSnapshotIndexSystemTopicClient createTransactionBufferSnapshotIndexSystemTopicClient( - NamespaceName namespaceName, TransactionBufferSnapshotIndexService transactionBufferSnapshotIndexService) { + NamespaceName namespaceName, TransactionBufferSnapshotService + transactionBufferSnapshotIndexService) { TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES); log.info("Create transaction buffer snapshot index client, topicName : {}", topicName.toString()); @@ -64,7 +66,8 @@ public TransactionBufferSnapshotIndexSystemTopicClient createTransactionBufferSn public TransactionBufferSnapshotSegmentSystemTopicClient createTransactionBufferSnapshotSegmentSystemTopicClient( NamespaceName namespaceName, - TransactionBufferSnapshotSegmentService transactionBufferSnapshotSegmentService) { + TransactionBufferSnapshotService + transactionBufferSnapshotSegmentService) { TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT); log.info("Create transaction buffer snapshot segment client, topicName : {}", topicName.toString()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java new file mode 100644 index 0000000000000..4769b28376fa3 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -0,0 +1,173 @@ +/** + * 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 org.apache.pulsar.broker.service.TransactionBufferSnapshotService; +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.impl.MessageIdImpl; +import org.apache.pulsar.common.naming.TopicName; + +public abstract class TransactionBufferSnapshotBaseSystemTopicClient extends + SystemTopicClientBase { + + protected final TransactionBufferSnapshotService transactionBufferSnapshotService; + public TransactionBufferSnapshotBaseSystemTopicClient(PulsarClient client, + TopicName topicName, + TransactionBufferSnapshotService + transactionBufferSnapshotService) { + super(client, topicName); + this.transactionBufferSnapshotService = transactionBufferSnapshotService; + } + + + protected void removeWriter(Writer writer) { + writers.remove(writer); + this.transactionBufferSnapshotService.removeClient(topicName, this); + } + + protected void removeReader(Reader reader) { + readers.remove(reader); + this.transactionBufferSnapshotService.removeClient(topicName, this); + } + + protected abstract static class TransactionBufferSnapshotBaseWriter implements Writer { + + protected final Producer producer; + protected final TransactionBufferSnapshotBaseSystemTopicClient + transactionBufferSnapshotBaseSystemTopicClient; + + protected TransactionBufferSnapshotBaseWriter(Producer producer, + TransactionBufferSnapshotBaseSystemTopicClient + transactionBufferSnapshotBaseSystemTopicClient) { + this.producer = producer; + this.transactionBufferSnapshotBaseSystemTopicClient = transactionBufferSnapshotBaseSystemTopicClient; + } + + @Override + public void close() throws IOException { + this.producer.close(); + transactionBufferSnapshotBaseSystemTopicClient.removeWriter(this); + } + + @Override + public CompletableFuture closeAsync() { + CompletableFuture 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 getSystemTopicClient() { + return transactionBufferSnapshotBaseSystemTopicClient; + } + } + + protected static class TransactionBufferSnapshotBaseReader implements Reader { + + private final org.apache.pulsar.client.api.Reader reader; + private final TransactionBufferSnapshotBaseSystemTopicClient transactionBufferSnapshotBaseSystemTopicClient; + + protected TransactionBufferSnapshotBaseReader( + org.apache.pulsar.client.api.Reader reader, + TransactionBufferSnapshotBaseSystemTopicClient transactionBufferSnapshotBaseSystemTopicClient) { + this.reader = reader; + this.transactionBufferSnapshotBaseSystemTopicClient = transactionBufferSnapshotBaseSystemTopicClient; + } + + @Override + public Message readNext() throws PulsarClientException { + return reader.readNext(); + } + + @Override + public CompletableFuture> readNextAsync() { + return reader.readNextAsync(); + } + + @Override + public Message readByMessageId(MessageId messageId) + throws PulsarClientException { + MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; + reader.seek(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, + messageIdImpl.getPartitionIndex())); + return reader.readNext(); + } + + @Override + public CompletableFuture> readByMessageIdAsync(MessageId messageId) { + MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; + return reader.seekAsync(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, + messageIdImpl.getPartitionIndex())).thenCompose((ignore) -> { + return reader.readNextAsync(); + }); + } + + @Override + public boolean hasMoreEvents() throws PulsarClientException { + return reader.hasMessageAvailable(); + } + + @Override + public CompletableFuture hasMoreEventsAsync() { + return reader.hasMessageAvailableAsync(); + } + + @Override + public void close() throws IOException { + this.reader.close(); + transactionBufferSnapshotBaseSystemTopicClient.removeReader(this); + } + + @Override + public CompletableFuture closeAsync() { + CompletableFuture 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 getSystemTopic() { + return transactionBufferSnapshotBaseSystemTopicClient; + } + } + + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java index aef58ee61b7b1..3a71f21990a20 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java @@ -19,30 +19,27 @@ 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.TransactionBufferSnapshotIndexService; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; -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.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicName; @Slf4j public class TransactionBufferSnapshotIndexSystemTopicClient extends - SystemTopicClientBase { + TransactionBufferSnapshotBaseSystemTopicClient { - private final TransactionBufferSnapshotIndexService transactionBufferSnapshotIndexService; + private final TransactionBufferSnapshotService + transactionBufferSnapshotIndexService; public TransactionBufferSnapshotIndexSystemTopicClient(PulsarClient client, TopicName topicName, - TransactionBufferSnapshotIndexService - transactionBufferSnapshotIndexService) { - super(client, topicName); + TransactionBufferSnapshotService transactionBufferSnapshotIndexService) { + super(client, topicName, transactionBufferSnapshotIndexService); this.transactionBufferSnapshotIndexService = transactionBufferSnapshotIndexService; } @@ -75,27 +72,14 @@ protected CompletableFuture> newReaderA }); } - protected void removeWriter(TransactionBufferSnapshotIndexWriter writer) { - writers.remove(writer); - this.transactionBufferSnapshotIndexService.removeClient(topicName, this); - } - - protected void removeReader(TransactionBufferSnapshotIndexReader reader) { - readers.remove(reader); - this.transactionBufferSnapshotIndexService.removeClient(topicName, this); - } + private static class TransactionBufferSnapshotIndexWriter extends + TransactionBufferSnapshotBaseWriter { - private static class TransactionBufferSnapshotIndexWriter implements Writer { - - private final Producer producer; - private final TransactionBufferSnapshotIndexSystemTopicClient - transactionBufferSnapshotIndexSystemTopicClient; private TransactionBufferSnapshotIndexWriter(Producer producer, TransactionBufferSnapshotIndexSystemTopicClient transactionBufferSnapshotIndexSystemTopicClient) { - this.producer = producer; - this.transactionBufferSnapshotIndexSystemTopicClient = transactionBufferSnapshotIndexSystemTopicClient; + super(producer, transactionBufferSnapshotIndexSystemTopicClient); } @Override @@ -130,109 +114,14 @@ public CompletableFuture deleteAsync(TransactionBufferSnapshotIndexes .value(null) .sendAsync(); } - - @Override - public void close() throws IOException { - this.producer.close(); - transactionBufferSnapshotIndexSystemTopicClient.removeWriter(this); - } - - @Override - public CompletableFuture closeAsync() { - CompletableFuture completableFuture = new CompletableFuture<>(); - producer.closeAsync().whenComplete((v, e) -> { - // if close fail, also need remove the producer - transactionBufferSnapshotIndexSystemTopicClient.removeWriter(this); - if (e != null) { - completableFuture.completeExceptionally(e); - return; - } - completableFuture.complete(null); - }); - return completableFuture; - } - - @Override - public SystemTopicClient getSystemTopicClient() { - return transactionBufferSnapshotIndexSystemTopicClient; - } } - private static class TransactionBufferSnapshotIndexReader implements Reader { - - private final org.apache.pulsar.client.api.Reader reader; - private final TransactionBufferSnapshotIndexSystemTopicClient - transactionBufferSnapshotIndexSystemTopicClient; - + private static class TransactionBufferSnapshotIndexReader extends TransactionBufferSnapshotBaseSystemTopicClient + .TransactionBufferSnapshotBaseReader { private TransactionBufferSnapshotIndexReader( org.apache.pulsar.client.api.Reader reader, TransactionBufferSnapshotIndexSystemTopicClient transactionBufferSnapshotIndexSystemTopicClient) { - this.reader = reader; - this.transactionBufferSnapshotIndexSystemTopicClient = transactionBufferSnapshotIndexSystemTopicClient; - } - - @Override - public Message readNext() throws PulsarClientException { - return reader.readNext(); - } - - @Override - public CompletableFuture> readNextAsync() { - return reader.readNextAsync(); - } - - @Override - public Message readByMessageId(MessageId messageId) - throws PulsarClientException { - MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; - reader.seek(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, - messageIdImpl.getPartitionIndex())); - return reader.readNext(); - } - - @Override - public CompletableFuture> readByMessageIdAsync(MessageId messageId) { - MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; - return reader.seekAsync(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, - messageIdImpl.getPartitionIndex())).thenCompose((ignore) -> { - return reader.readNextAsync(); - }); - } - - @Override - public boolean hasMoreEvents() throws PulsarClientException { - return reader.hasMessageAvailable(); - } - - @Override - public CompletableFuture hasMoreEventsAsync() { - return reader.hasMessageAvailableAsync(); - } - - @Override - public void close() throws IOException { - this.reader.close(); - transactionBufferSnapshotIndexSystemTopicClient.removeReader(this); - } - - @Override - public CompletableFuture closeAsync() { - CompletableFuture completableFuture = new CompletableFuture<>(); - reader.closeAsync().whenComplete((v, e) -> { - // if close fail, also need remove the reader - transactionBufferSnapshotIndexSystemTopicClient.removeReader(this); - if (e != null) { - completableFuture.completeExceptionally(e); - return; - } - completableFuture.complete(null); - }); - return completableFuture; - } - - @Override - public SystemTopicClient getSystemTopic() { - return transactionBufferSnapshotIndexSystemTopicClient; + super(reader, transactionBufferSnapshotIndexSystemTopicClient); } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java index 892c33a11af57..8b6dba75cf621 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java @@ -18,35 +18,30 @@ */ 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.TransactionBufferSnapshotSegmentService; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; -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.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.util.FutureUtil; @Slf4j public class TransactionBufferSnapshotSegmentSystemTopicClient extends - SystemTopicClientBase { + TransactionBufferSnapshotBaseSystemTopicClient { - private final TransactionBufferSnapshotSegmentService transactionBufferSnapshotSegmentService; - public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client, TopicName topicName, - TransactionBufferSnapshotSegmentService + public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client, TopicName topicName, + TransactionBufferSnapshotService transactionBufferSnapshotSegmentService) { - super(client, topicName); - this.transactionBufferSnapshotSegmentService = transactionBufferSnapshotSegmentService; + super(client, topicName, transactionBufferSnapshotSegmentService); } @Override - protected CompletableFuture> newWriterAsyncInternal() { + protected CompletableFuture> + newWriterAsyncInternal() { return client.newProducer(Schema.AVRO(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot.class)) .topic(topicName.toString()) .createAsync().thenCompose(producer -> { @@ -59,7 +54,8 @@ protected CompletableFuture> newReaderAsyncInternal() { + protected CompletableFuture> + newReaderAsyncInternal() { return client.newReader(Schema.AVRO(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot.class)) .topic(topicName.toString()) .startMessageId(MessageId.earliest) @@ -79,28 +75,14 @@ protected static String buildKey(TransactionBufferSnapshotIndexes.TransactionBuf return "multiple-" + snapshot.getSequenceId() + "-" + snapshot.getTopicName(); } - protected void removeWriter(TransactionBufferSnapshotSegmentWriter writer) { - writers.remove(writer); - this.transactionBufferSnapshotSegmentService.removeClient(topicName, this); - } - - protected void removeReader(TransactionBufferSnapshotSegmentReader reader) { - readers.remove(reader); - this.transactionBufferSnapshotSegmentService.removeClient(topicName, this); - } + private static class TransactionBufferSnapshotSegmentWriter extends TransactionBufferSnapshotBaseSystemTopicClient + .TransactionBufferSnapshotBaseWriter { - private static class TransactionBufferSnapshotSegmentWriter implements Writer { - - private final Producer producer; - private final TransactionBufferSnapshotSegmentSystemTopicClient - transactionBufferSnapshotSegmentSystemTopicClient; - - private TransactionBufferSnapshotSegmentWriter(Producer producer, - TransactionBufferSnapshotSegmentSystemTopicClient + private TransactionBufferSnapshotSegmentWriter(Producer producer, TransactionBufferSnapshotSegmentSystemTopicClient transactionBufferSnapshotSegmentSystemTopicClient) { - this.producer = producer; - this.transactionBufferSnapshotSegmentSystemTopicClient = transactionBufferSnapshotSegmentSystemTopicClient; + super(producer, transactionBufferSnapshotSegmentSystemTopicClient); } @Override @@ -136,110 +118,15 @@ public CompletableFuture deleteAsync(TransactionBufferSnapshotIndexes .value(null) .sendAsync(); } - - @Override - public void close() throws IOException { - this.producer.close(); - transactionBufferSnapshotSegmentSystemTopicClient.removeWriter(this); - } - - @Override - public CompletableFuture closeAsync() { - CompletableFuture completableFuture = new CompletableFuture<>(); - producer.closeAsync().whenComplete((v, e) -> { - // if close fail, also need remove the producer - transactionBufferSnapshotSegmentSystemTopicClient.removeWriter(this); - if (e != null) { - completableFuture.completeExceptionally(e); - return; - } - completableFuture.complete(null); - }); - return completableFuture; - } - - @Override - public SystemTopicClient getSystemTopicClient() { - return transactionBufferSnapshotSegmentSystemTopicClient; - } } - private static class TransactionBufferSnapshotSegmentReader implements Reader { - - private final org.apache.pulsar.client.api.Reader reader; - private final TransactionBufferSnapshotSegmentSystemTopicClient - transactionBufferSnapshotSegmentSystemTopicClient; + private static class TransactionBufferSnapshotSegmentReader extends TransactionBufferSnapshotBaseSystemTopicClient + .TransactionBufferSnapshotBaseReader { private TransactionBufferSnapshotSegmentReader( org.apache.pulsar.client.api.Reader reader, TransactionBufferSnapshotSegmentSystemTopicClient transactionBufferSnapshotSegmentSystemTopicClient) { - this.reader = reader; - this.transactionBufferSnapshotSegmentSystemTopicClient = transactionBufferSnapshotSegmentSystemTopicClient; - } - - @Override - public Message readNext() throws PulsarClientException { - throw new UnsupportedOperationException( - "Transaction buffer snapshot segment does not support sequential reads."); - } - - @Override - public CompletableFuture> readNextAsync() { - return FutureUtil.failedFuture( - new UnsupportedOperationException( - "Transaction buffer snapshot segment does not support sequential reads.")); - } - - @Override - public Message readByMessageId(MessageId messageId) throws PulsarClientException { - MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; - reader.seek(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, - messageIdImpl.getPartitionIndex())); - return reader.readNext(); - } - - @Override - public CompletableFuture> readByMessageIdAsync(MessageId messageId) { - MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; - return reader.seekAsync(new MessageIdImpl(messageIdImpl.getLedgerId(), - messageIdImpl.getEntryId() - 1, messageIdImpl.getPartitionIndex())) - .thenCompose((ignore) -> reader.readNextAsync()); - } - - @Override - public boolean hasMoreEvents() throws PulsarClientException { - return reader.hasMessageAvailable(); - } - - @Override - public CompletableFuture hasMoreEventsAsync() { - return reader.hasMessageAvailableAsync(); - } - - @Override - public void close() throws IOException { - this.reader.close(); - transactionBufferSnapshotSegmentSystemTopicClient.removeReader(this); - } - - @Override - public CompletableFuture closeAsync() { - CompletableFuture completableFuture = new CompletableFuture<>(); - reader.closeAsync().whenComplete((v, e) -> { - // if close fail, also need remove the reader - transactionBufferSnapshotSegmentSystemTopicClient.removeReader(this); - if (e != null) { - completableFuture.completeExceptionally(e); - return; - } - completableFuture.complete(null); - }); - return completableFuture; - } - - @Override - public SystemTopicClient getSystemTopic() { - return transactionBufferSnapshotSegmentSystemTopicClient; + super(reader, transactionBufferSnapshotSegmentSystemTopicClient); } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSystemTopicClient.java index 087d98b618335..47e2c83b807da 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSystemTopicClient.java @@ -19,28 +19,26 @@ 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.TransactionBufferSnapshotService; import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; -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.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicName; @Slf4j -public class TransactionBufferSystemTopicClient extends SystemTopicClientBase { +public class TransactionBufferSystemTopicClient extends + TransactionBufferSnapshotBaseSystemTopicClient { private TransactionBufferSnapshotService transactionBufferSnapshotService; public TransactionBufferSystemTopicClient(PulsarClient client, TopicName topicName, - TransactionBufferSnapshotService transactionBufferSnapshotService) { - super(client, topicName); - this.transactionBufferSnapshotService = transactionBufferSnapshotService; + TransactionBufferSnapshotService + transactionBufferSnapshotService) { + super(client, topicName, transactionBufferSnapshotService); } @Override @@ -72,25 +70,12 @@ protected CompletableFuture> newReaderAsyncInt }); } - protected void removeWriter(TransactionBufferSnapshotWriter writer) { - writers.remove(writer); - this.transactionBufferSnapshotService.removeClient(topicName, this); - } - - protected void removeReader(TransactionBufferSnapshotReader reader) { - readers.remove(reader); - this.transactionBufferSnapshotService.removeClient(topicName, this); - } - - private static class TransactionBufferSnapshotWriter implements Writer { - - private final Producer producer; - private final TransactionBufferSystemTopicClient transactionBufferSystemTopicClient; + private static class TransactionBufferSnapshotWriter extends + TransactionBufferSnapshotBaseWriter { private TransactionBufferSnapshotWriter(Producer producer, TransactionBufferSystemTopicClient transactionBufferSystemTopicClient) { - this.producer = producer; - this.transactionBufferSystemTopicClient = transactionBufferSystemTopicClient; + super(producer, transactionBufferSystemTopicClient); } @Override @@ -120,106 +105,13 @@ public CompletableFuture deleteAsync(TransactionBufferSnapshot transa .value(null) .sendAsync(); } - - @Override - public void close() throws IOException { - this.producer.close(); - transactionBufferSystemTopicClient.removeWriter(this); - } - - @Override - public CompletableFuture closeAsync() { - CompletableFuture completableFuture = new CompletableFuture<>(); - producer.closeAsync().whenComplete((v, e) -> { - // if close fail, also need remove the producer - transactionBufferSystemTopicClient.removeWriter(this); - if (e != null) { - completableFuture.completeExceptionally(e); - return; - } - completableFuture.complete(null); - }); - return completableFuture; - } - - @Override - public SystemTopicClient getSystemTopicClient() { - return transactionBufferSystemTopicClient; - } } - private static class TransactionBufferSnapshotReader implements Reader { - - private final org.apache.pulsar.client.api.Reader reader; - private final TransactionBufferSystemTopicClient transactionBufferSystemTopicClient; - + private static class TransactionBufferSnapshotReader extends + TransactionBufferSnapshotBaseReader { private TransactionBufferSnapshotReader(org.apache.pulsar.client.api.Reader reader, TransactionBufferSystemTopicClient transactionBufferSystemTopicClient) { - this.reader = reader; - this.transactionBufferSystemTopicClient = transactionBufferSystemTopicClient; - } - - @Override - public Message readNext() throws PulsarClientException { - return reader.readNext(); - } - - @Override - public CompletableFuture> readNextAsync() { - return reader.readNextAsync(); - } - - @Override - public Message readByMessageId(MessageId messageId) throws PulsarClientException { - MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; - reader.seek(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, - messageIdImpl.getPartitionIndex())); - return reader.readNext(); - } - - @Override - public CompletableFuture> readByMessageIdAsync(MessageId messageId) { - MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; - return reader.seekAsync(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, - messageIdImpl.getPartitionIndex())).thenCompose((ignore) -> { - return reader.readNextAsync(); - }); - } - - @Override - public boolean hasMoreEvents() throws PulsarClientException { - return reader.hasMessageAvailable(); - } - - @Override - public CompletableFuture hasMoreEventsAsync() { - return reader.hasMessageAvailableAsync(); - } - - @Override - public void close() throws IOException { - this.reader.close(); - transactionBufferSystemTopicClient.removeReader(this); - } - - @Override - public CompletableFuture closeAsync() { - CompletableFuture completableFuture = new CompletableFuture<>(); - reader.closeAsync().whenComplete((v, e) -> { - // if close fail, also need remove the reader - transactionBufferSystemTopicClient.removeReader(this); - if (e != null) { - completableFuture.completeExceptionally(e); - return; - } - completableFuture.complete(null); - }); - return completableFuture; - } - - @Override - public SystemTopicClient getSystemTopic() { - return transactionBufferSystemTopicClient; + super(reader, transactionBufferSystemTopicClient); } } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index db9349d31f994..b336f20bf6b2e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -50,8 +50,6 @@ import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotIndexService; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotSegmentService; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; @@ -575,7 +573,7 @@ public void testTransactionBufferNoSnapshotCloseReader() throws Exception{ @Test public void testTransactionBufferIndexSystemTopic() throws Exception { - TransactionBufferSnapshotIndexService transactionBufferSnapshotIndexService = + TransactionBufferSnapshotService transactionBufferSnapshotIndexService = getPulsarServiceList().get(0).getTransactionBufferSnapshotIndexService(); SystemTopicClient.Writer indexesWriter = @@ -617,7 +615,8 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { @Test public void testTransactionBufferSegmentSystemTopic() throws Exception { - TransactionBufferSnapshotSegmentService transactionBufferSnapshotSegmentService = + TransactionBufferSnapshotService + transactionBufferSnapshotSegmentService = getPulsarServiceList().get(0).getTransactionBufferSnapshotSegmentService(); SystemTopicClient.Writer From e8dda12f2ec3433dabab90d546bbfa267ece39b7 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Sun, 18 Sep 2022 20:40:44 +0800 Subject: [PATCH 04/27] change reader to cursor for segment snapshot. --- .../bookkeeper/mledger/ReadOnlyCursor.java | 16 +++++ .../mledger/impl/ReadOnlyCursorImpl.java | 40 ++++++++++++ .../broker/systopic/SystemTopicClient.java | 16 ----- .../TopicPoliciesSystemTopicClient.java | 18 ------ ...onBufferSnapshotBaseSystemTopicClient.java | 20 ------ ...ufferSnapshotSegmentSystemTopicClient.java | 26 ++------ .../TopicTransactionBufferRecoverTest.java | 62 ++++++++++++++----- 7 files changed, 107 insertions(+), 91 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java index 28454d647b008..9018b8d180ac3 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java @@ -62,6 +62,22 @@ void asyncReadEntries(int numberOfEntriesToRead, ReadEntriesCallback callback, void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesCallback callback, Object ctx, PositionImpl maxPosition); + /** + * Asynchronously read entry by position from the ManagedLedger. + * @param position the position of the entry to read. + * @param callback callback object + * @param ctx opaque context + */ + void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallback callback, Object ctx); + + /** + * Read entry by position from the ManagedLedger. + * @param position the position of the entry to read. + * @param ctx opaque context. + * @return the entry. + * @throws ManagedLedgerException + */ + Entry readEntry(PositionImpl position, Object ctx) throws ManagedLedgerException, InterruptedException; /** * Get the read position. This points to the next message to be read from the cursor. * diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java index 8c20e0d560ee7..d5093e3eb5313 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java @@ -19,10 +19,13 @@ package org.apache.bookkeeper.mledger.impl; import com.google.common.collect.Range; +import java.util.concurrent.CountDownLatch; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound; import org.apache.bookkeeper.mledger.proto.MLDataFormats; @@ -49,6 +52,43 @@ public ReadOnlyCursorImpl(BookKeeper bookkeeper, ManagedLedgerConfig config, Man this.state = State.NoLedger; } + @Override + public void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { + ledger.asyncReadEntry(position, callback, ctx); + } + + @Override + public Entry readEntry(PositionImpl position, Object ctx) throws ManagedLedgerException, InterruptedException { + class Result { + Entry entry; + ManagedLedgerException exception; + } + + Result result = new Result(); + CountDownLatch wait = new CountDownLatch(1); + + AsyncCallbacks.ReadEntryCallback callback = new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + result.entry = entry; + wait.countDown(); + } + + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + result.exception = exception; + wait.countDown(); + } + }; + asyncReadEntry(position, callback, null); + wait.await(); + if (result.exception != null) { + throw result.exception; + } else { + return result.entry; + } + } + @Override public void skipEntries(int numEntriesToSkip) { log.info("[{}] Skipping {} entries on read-only cursor {}", ledger.getName(), numEntriesToSkip, getName()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java index 052e474b0e747..2bc740a41d48b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java @@ -156,22 +156,6 @@ interface Reader { */ CompletableFuture> readNextAsync(); - /** - * Read the specified event according to messageId. - * @param messageId the message id of the event that needs to be read. - * @return pulsar event - * @throws PulsarClientException - */ - - Message readByMessageId(MessageId messageId) throws PulsarClientException; - - /** - * Async read the specified event according to messageId. - * @param messageId the message id of the event that needs to be read. - * @return pulsar event future - */ - CompletableFuture> readByMessageIdAsync(MessageId messageId); - /** * Check has more events available for the reader. * @return true if has remaining events, otherwise false diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java index fc7d9885e54ca..84b8bd636bd3a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java @@ -28,7 +28,6 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TypedMessageBuilder; -import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.events.ActionType; import org.apache.pulsar.common.events.PulsarEvent; import org.apache.pulsar.common.naming.TopicName; @@ -168,23 +167,6 @@ public Message readNext() throws PulsarClientException { return reader.readNext(); } - @Override - public Message readByMessageId(MessageId messageId) throws PulsarClientException { - MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; - reader.seek(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, - messageIdImpl.getPartitionIndex())); - return reader.readNext(); - } - - @Override - public CompletableFuture> readByMessageIdAsync(MessageId messageId) { - MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; - return reader.seekAsync(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, - messageIdImpl.getPartitionIndex())).thenCompose((ignore) -> { - return reader.readNextAsync(); - }); - } - @Override public CompletableFuture> readNextAsync() { return reader.readNextAsync(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index 4769b28376fa3..9697b8d76364b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -22,11 +22,9 @@ import java.util.concurrent.CompletableFuture; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; 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.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicName; public abstract class TransactionBufferSnapshotBaseSystemTopicClient extends @@ -114,24 +112,6 @@ public CompletableFuture> readNextAsync() { return reader.readNextAsync(); } - @Override - public Message readByMessageId(MessageId messageId) - throws PulsarClientException { - MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; - reader.seek(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, - messageIdImpl.getPartitionIndex())); - return reader.readNext(); - } - - @Override - public CompletableFuture> readByMessageIdAsync(MessageId messageId) { - MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; - return reader.seekAsync(new MessageIdImpl(messageIdImpl.getLedgerId(), messageIdImpl.getEntryId() - 1, - messageIdImpl.getPartitionIndex())).thenCompose((ignore) -> { - return reader.readNextAsync(); - }); - } - @Override public boolean hasMoreEvents() throws PulsarClientException { return reader.hasMessageAvailable(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java index 8b6dba75cf621..508518966cdce 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java @@ -20,6 +20,7 @@ import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.client.api.MessageId; @@ -28,6 +29,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.FutureUtil; @Slf4j public class TransactionBufferSnapshotSegmentSystemTopicClient extends @@ -56,19 +58,8 @@ public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client @Override protected CompletableFuture> newReaderAsyncInternal() { - return client.newReader(Schema.AVRO(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot.class)) - .topic(topicName.toString()) - .startMessageId(MessageId.earliest) - .readCompacted(true) - .createAsync() - .thenCompose(reader -> { - if (log.isDebugEnabled()) { - log.debug("[{}] A new transactionBufferSnapshot buffer snapshot segment reader is created", - topicName); - } - return CompletableFuture.completedFuture( - new TransactionBufferSnapshotSegmentReader(reader, this)); - }); + return FutureUtil.failedFuture(new BrokerServiceException + .NotAllowedException("Do not allow to get reader for segment topic reader")); } protected static String buildKey(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot snapshot) { @@ -120,13 +111,4 @@ public CompletableFuture deleteAsync(TransactionBufferSnapshotIndexes } } - private static class TransactionBufferSnapshotSegmentReader extends TransactionBufferSnapshotBaseSystemTopicClient - .TransactionBufferSnapshotBaseReader { - - private TransactionBufferSnapshotSegmentReader( - org.apache.pulsar.client.api.Reader reader, - TransactionBufferSnapshotSegmentSystemTopicClient transactionBufferSnapshotSegmentSystemTopicClient) { - super(reader, transactionBufferSnapshotSegmentSystemTopicClient); - } - } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index b336f20bf6b2e..19dcda52a68d1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -28,6 +28,8 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -41,6 +43,8 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats; @@ -70,9 +74,11 @@ import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.transaction.TransactionImpl; +import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.events.EventType; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; @@ -615,39 +621,65 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { @Test public void testTransactionBufferSegmentSystemTopic() throws Exception { + // init topic and topicName + String snapshotTopic = NAMESPACE1 + "/" + EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT; + TopicName snapshotSegmentTopicName = TopicName.getPartitionedTopicName(snapshotTopic); + + //send message to create manager ledger + Producer producer = + pulsarClient.newProducer(Schema.AVRO(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot.class)) + .topic(snapshotTopic) + .create(); + + // get brokerService and pulsarService + PulsarService pulsarService = getPulsarServiceList().get(0); + BrokerService brokerService = pulsarService.getBrokerService(); + + // create snapshot sgement writer TransactionBufferSnapshotService - transactionBufferSnapshotSegmentService = - getPulsarServiceList().get(0).getTransactionBufferSnapshotSegmentService(); + transactionBufferSnapshotSegmentService = pulsarService.getTransactionBufferSnapshotSegmentService(); SystemTopicClient.Writer - segmentWriter = - transactionBufferSnapshotSegmentService.createWriter(TopicName.get(SNAPSHOT_SEGMENT)).get(); - - SystemTopicClient.Reader - segmentReader = - transactionBufferSnapshotSegmentService.createReader(TopicName.get(SNAPSHOT_SEGMENT)).get(); + segmentWriter = transactionBufferSnapshotSegmentService.createWriter(snapshotSegmentTopicName).get(); + // write two snapshot to snapshot segment topic TransactionBufferSnapshotIndexes.TransactionBufferSnapshot snapshot = new TransactionBufferSnapshotIndexes.TransactionBufferSnapshot(); - snapshot.setTopicName(SNAPSHOT_SEGMENT); + //build and send sanpshot + snapshot.setTopicName(snapshotTopic); snapshot.setSequenceId(1L); snapshot.setMaxReadPositionLedgerId(2L); snapshot.setMaxReadPositionEntryId(3L); snapshot.setAborts(Collections.singletonList( new org.apache.pulsar.broker.transaction.buffer.matadata.v2.TxnID(1, 1))); - MessageId messageId = segmentWriter.write(snapshot); segmentWriter.write(snapshot); - - snapshot = segmentReader.readByMessageId(messageId).getValue(); - assertEquals(snapshot.getTopicName(), SNAPSHOT_SEGMENT); - assertEquals(snapshot.getSequenceId(), 1L); + snapshot.setSequenceId(2L); + + MessageIdImpl messageId = (MessageIdImpl) segmentWriter.write(snapshot); + + //create cursor + ReadOnlyCursor readOnlyCursor = pulsarService.getManagedLedgerFactory() + .openReadOnlyCursor(snapshotSegmentTopicName.getPersistenceNamingEncoding(), + PositionImpl.EARLIEST, brokerService.getManagedLedgerConfig(snapshotSegmentTopicName).get()); + //read the entry and decode entry to snapshot + Entry entry = readOnlyCursor.readEntry(new PositionImpl(messageId.getLedgerId(), messageId.getEntryId()), null); + + //decode snapshot from entry + ByteBuf headersAndPayload = entry.getDataBuffer(); + //skip metadata + MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload); + snapshot = Schema.AVRO(TransactionBufferSnapshotIndexes + .TransactionBufferSnapshot.class).decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer()); + + //verify snapshot + assertEquals(snapshot.getTopicName(), snapshotTopic); + assertEquals(snapshot.getSequenceId(), 2L); assertEquals(snapshot.getMaxReadPositionLedgerId(), 2L); assertEquals(snapshot.getMaxReadPositionEntryId(), 3L); assertEquals(snapshot.getAborts().get(0), new org.apache.pulsar.broker.transaction.buffer.matadata.v2.TxnID(1, 1)); - } } From bba847e8ef69747ad4c554db7cb983067f299bf4 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Sun, 18 Sep 2022 21:18:55 +0800 Subject: [PATCH 05/27] optimize name --- .../service/TransactionBufferSnapshotService.java | 8 ++++---- .../NamespaceEventsSystemTopicFactory.java | 8 ++++---- ...ransactionBufferSnapshotSystemTopicClient.java} | 14 +++++++------- 3 files changed, 15 insertions(+), 15 deletions(-) rename pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/{TransactionBufferSystemTopicClient.java => TransactionBufferSnapshotSystemTopicClient.java} (88%) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java index 841874b27801e..85e0f04188856 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java @@ -22,7 +22,7 @@ import org.apache.pulsar.broker.systopic.SystemTopicClient.Reader; import org.apache.pulsar.broker.systopic.SystemTopicClient.Writer; import org.apache.pulsar.broker.systopic.SystemTopicClientBase; -import org.apache.pulsar.broker.systopic.TransactionBufferSystemTopicClient; +import org.apache.pulsar.broker.systopic.TransactionBufferSnapshotBaseSystemTopicClient; import org.apache.pulsar.common.naming.TopicName; public interface TransactionBufferSnapshotService { @@ -49,10 +49,10 @@ public interface TransactionBufferSnapshotService { * Remove a topic client from cache. * * @param topicName {@link TopicName} the topic name - * @param transactionBufferSystemTopicClient {@link TransactionBufferSystemTopicClient} the topic client - * + * @param TransactionBufferSnapshotBaseSystemTopicClient {@link TransactionBufferSnapshotBaseSystemTopicClient} + * the topic client */ - void removeClient(TopicName topicName, SystemTopicClientBase transactionBufferSystemTopicClient); + void removeClient(TopicName topicName, SystemTopicClientBase TransactionBufferSnapshotBaseSystemTopicClient); /** * Close transaction buffer snapshot service. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java index 3cf73a1300c29..8af6b27d62910 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java @@ -45,13 +45,13 @@ public TopicPoliciesSystemTopicClient createTopicPoliciesSystemTopicClient(Names return new TopicPoliciesSystemTopicClient(client, topicName); } - public TransactionBufferSystemTopicClient createTransactionBufferSystemTopicClient(NamespaceName namespaceName, - TransactionBufferSnapshotService - transactionBufferSnapshotService) { + public TransactionBufferSnapshotSystemTopicClient createTransactionBufferSystemTopicClient( + NamespaceName namespaceName, TransactionBufferSnapshotService + transactionBufferSnapshotService) { TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); log.info("Create transaction buffer snapshot client, topicName : {}", topicName.toString()); - return new TransactionBufferSystemTopicClient(client, topicName, transactionBufferSnapshotService); + return new TransactionBufferSnapshotSystemTopicClient(client, topicName, transactionBufferSnapshotService); } public TransactionBufferSnapshotIndexSystemTopicClient createTransactionBufferSnapshotIndexSystemTopicClient( diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java similarity index 88% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSystemTopicClient.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java index 47e2c83b807da..cbdcd654443f8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java @@ -31,12 +31,12 @@ import org.apache.pulsar.common.naming.TopicName; @Slf4j -public class TransactionBufferSystemTopicClient extends +public class TransactionBufferSnapshotSystemTopicClient extends TransactionBufferSnapshotBaseSystemTopicClient { private TransactionBufferSnapshotService transactionBufferSnapshotService; - public TransactionBufferSystemTopicClient(PulsarClient client, TopicName topicName, - TransactionBufferSnapshotService + public TransactionBufferSnapshotSystemTopicClient(PulsarClient client, TopicName topicName, + TransactionBufferSnapshotService transactionBufferSnapshotService) { super(client, topicName, transactionBufferSnapshotService); } @@ -74,8 +74,8 @@ private static class TransactionBufferSnapshotWriter extends TransactionBufferSnapshotBaseWriter { private TransactionBufferSnapshotWriter(Producer producer, - TransactionBufferSystemTopicClient transactionBufferSystemTopicClient) { - super(producer, transactionBufferSystemTopicClient); + TransactionBufferSnapshotSystemTopicClient transactionBufferSnapshotSystemTopicClient) { + super(producer, transactionBufferSnapshotSystemTopicClient); } @Override @@ -110,8 +110,8 @@ public CompletableFuture deleteAsync(TransactionBufferSnapshot transa private static class TransactionBufferSnapshotReader extends TransactionBufferSnapshotBaseReader { private TransactionBufferSnapshotReader(org.apache.pulsar.client.api.Reader reader, - TransactionBufferSystemTopicClient transactionBufferSystemTopicClient) { - super(reader, transactionBufferSystemTopicClient); + TransactionBufferSnapshotSystemTopicClient transactionBufferSnapshotSystemTopicClient) { + super(reader, transactionBufferSnapshotSystemTopicClient); } } } From 63dec027397a6414e631d4a29825835271f8efb4 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Sun, 18 Sep 2022 21:44:02 +0800 Subject: [PATCH 06/27] optimize --- .../broker/service/TransactionBufferSnapshotService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java index 85e0f04188856..933bbf577fe98 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java @@ -49,10 +49,10 @@ public interface TransactionBufferSnapshotService { * Remove a topic client from cache. * * @param topicName {@link TopicName} the topic name - * @param TransactionBufferSnapshotBaseSystemTopicClient {@link TransactionBufferSnapshotBaseSystemTopicClient} + * @param transactionBufferSnapshotBaseSystemTopicClient {@link TransactionBufferSnapshotBaseSystemTopicClient} * the topic client */ - void removeClient(TopicName topicName, SystemTopicClientBase TransactionBufferSnapshotBaseSystemTopicClient); + void removeClient(TopicName topicName, SystemTopicClientBase transactionBufferSnapshotBaseSystemTopicClient); /** * Close transaction buffer snapshot service. From 73a4731ac9a921e15c759d3add4b3c752afd35b1 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Mon, 19 Sep 2022 17:41:13 +0800 Subject: [PATCH 07/27] change read-only cursor to read-only managedledger. --- .../mledger/ManagedLedgerFactory.java | 10 +++++ .../bookkeeper/mledger/ReadOnlyCursor.java | 16 -------- .../impl/ManagedLedgerFactoryImpl.java | 26 ++++++++++++ .../mledger/impl/ReadOnlyCursorImpl.java | 40 ------------------- 4 files changed, 36 insertions(+), 56 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java index e42c2581ba101..c94e3555de18a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java @@ -116,6 +116,16 @@ 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 callback callback object + * @param ctx opaque context + */ + void asyncOpenReadOnlyManagedLedger(String managedLedgerName, ManagedLedgerConfig config, + OpenLedgerCallback callback, Object ctx); + /** * Get the current metadata info for a managed ledger. * diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java index 9018b8d180ac3..28454d647b008 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java @@ -62,22 +62,6 @@ void asyncReadEntries(int numberOfEntriesToRead, ReadEntriesCallback callback, void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesCallback callback, Object ctx, PositionImpl maxPosition); - /** - * Asynchronously read entry by position from the ManagedLedger. - * @param position the position of the entry to read. - * @param callback callback object - * @param ctx opaque context - */ - void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallback callback, Object ctx); - - /** - * Read entry by position from the ManagedLedger. - * @param position the position of the entry to read. - * @param ctx opaque context. - * @return the entry. - * @throws ManagedLedgerException - */ - Entry readEntry(PositionImpl position, Object ctx) throws ManagedLedgerException, InterruptedException; /** * Get the read position. This points to the next message to be read from the cursor. * diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index d7596a7468a40..9bb4de5500332 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -418,7 +418,33 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { }); } + @Override + public void asyncOpenReadOnlyManagedLedger(String managedLedgerName, ManagedLedgerConfig config, + OpenLedgerCallback callback, Object ctx) { + if (closed) { + callback.openLedgerFailed(new ManagedLedgerException.ManagedLedgerFactoryClosedException(), ctx); + return; + } + ReadOnlyManagedLedgerImpl roManagedLedger = new ReadOnlyManagedLedgerImpl(this, + bookkeeperFactory + .get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), + config.getBookKeeperEnsemblePlacementPolicyProperties())), + store, config, scheduledExecutor, managedLedgerName); + + roManagedLedger.initialize(new ManagedLedgerInitializeLedgerCallback() { + @Override + public void initializeComplete() { + log.info("[{}] Successfully initialize Read-only managed ledger", managedLedgerName); + callback.openLedgerComplete(roManagedLedger, ctx); + } + @Override + public void initializeFailed(ManagedLedgerException e) { + log.error("[{}] Failed to initialize Read-only managed ledger", managedLedgerName, e); + callback.openLedgerFailed(e, ctx); + } + }, ctx); + } @Override public ReadOnlyCursor openReadOnlyCursor(String managedLedgerName, Position startPosition, diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java index d5093e3eb5313..8c20e0d560ee7 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java @@ -19,13 +19,10 @@ package org.apache.bookkeeper.mledger.impl; import com.google.common.collect.Range; -import java.util.concurrent.CountDownLatch; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.mledger.AsyncCallbacks; -import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; -import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound; import org.apache.bookkeeper.mledger.proto.MLDataFormats; @@ -52,43 +49,6 @@ public ReadOnlyCursorImpl(BookKeeper bookkeeper, ManagedLedgerConfig config, Man this.state = State.NoLedger; } - @Override - public void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { - ledger.asyncReadEntry(position, callback, ctx); - } - - @Override - public Entry readEntry(PositionImpl position, Object ctx) throws ManagedLedgerException, InterruptedException { - class Result { - Entry entry; - ManagedLedgerException exception; - } - - Result result = new Result(); - CountDownLatch wait = new CountDownLatch(1); - - AsyncCallbacks.ReadEntryCallback callback = new AsyncCallbacks.ReadEntryCallback() { - @Override - public void readEntryComplete(Entry entry, Object ctx) { - result.entry = entry; - wait.countDown(); - } - - @Override - public void readEntryFailed(ManagedLedgerException exception, Object ctx) { - result.exception = exception; - wait.countDown(); - } - }; - asyncReadEntry(position, callback, null); - wait.await(); - if (result.exception != null) { - throw result.exception; - } else { - return result.entry; - } - } - @Override public void skipEntries(int numEntriesToSkip) { log.info("[{}] Skipping {} entries on read-only cursor {}", ledger.getName(), numEntriesToSkip, getName()); From e8b0c4f4ba1f49616164865c6a8e7a87b58b3ed4 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Mon, 19 Sep 2022 21:54:55 +0800 Subject: [PATCH 08/27] abstract transactionBufferSnapshotService --- .../apache/pulsar/broker/PulsarService.java | 23 ++--- ...TopicBaseTxnBufferSnapshotBaseService.java | 3 +- .../SystemTopicTxnBufferSnapshotService.java | 62 +++++++++++++ ...ctionBufferSnapshotSegmentServiceImpl.java | 55 ++++++++++++ .../TransactionBufferSnapshotService.java | 40 +++------ .../TransactionBufferSnapshotServiceImpl.java | 51 +++++++++++ .../NamespaceEventsSystemTopicFactory.java | 12 +-- ...onBufferSnapshotBaseSystemTopicClient.java | 14 +-- ...nBufferSnapshotIndexSystemTopicClient.java | 7 +- ...ufferSnapshotSegmentSystemTopicClient.java | 4 +- ...actionBufferSnapshotSystemTopicClient.java | 10 +-- .../buffer/impl/TopicTransactionBuffer.java | 5 +- .../TopicTransactionBufferRecoverTest.java | 89 ++++++++++++++----- .../broker/transaction/TransactionTest.java | 12 ++- 14 files changed, 293 insertions(+), 94 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentServiceImpl.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 4d850db471724..5766e09b57404 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -101,13 +101,12 @@ import org.apache.pulsar.broker.rest.Topics; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.PulsarMetadataEventSynchronizer; -import org.apache.pulsar.broker.service.SystemTopicBaseTxnBufferSnapshotIndexService; -import org.apache.pulsar.broker.service.SystemTopicBaseTxnBufferSnapshotSegmentService; -import org.apache.pulsar.broker.service.SystemTopicBaseTxnBufferSnapshotService; import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotSegmentServiceImpl; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotServiceImpl; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.SchemaStorageFactory; import org.apache.pulsar.broker.stats.MetricsGenerator; @@ -116,8 +115,6 @@ import org.apache.pulsar.broker.storage.ManagedLedgerStorage; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; -import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; -import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; import org.apache.pulsar.broker.validator.MultipleListenerValidator; import org.apache.pulsar.broker.web.WebService; @@ -264,11 +261,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private MetadataStoreExtended localMetadataStore; private PulsarMetadataEventSynchronizer localMetadataSynchronizer; private CoordinationService coordinationService; - private TransactionBufferSnapshotService transactionBufferSnapshotService; - private TransactionBufferSnapshotService transactionBufferSnapshotIndexService; - private TransactionBufferSnapshotService - transactionBufferSnapshotSegmentService; - + private TransactionBufferSnapshotService transactionBufferSnapshotService; private MetadataStore configurationMetadataStore; private PulsarMetadataEventSynchronizer configMetadataSynchronizer; private boolean shouldShutdownConfigurationMetadataStore; @@ -826,11 +819,11 @@ public void start() throws PulsarServerException { // Register pulsar system namespaces and start transaction meta store service if (config.isTransactionCoordinatorEnabled()) { - this.transactionBufferSnapshotService = new SystemTopicBaseTxnBufferSnapshotService(getClient()); - this.transactionBufferSnapshotIndexService = - new SystemTopicBaseTxnBufferSnapshotIndexService(getClient()); - this.transactionBufferSnapshotSegmentService = - new SystemTopicBaseTxnBufferSnapshotSegmentService(getClient()); +// if (transactionBufferSegmentedSnapshotEnabled) + this.transactionBufferSnapshotService = new TransactionBufferSnapshotSegmentServiceImpl(getClient()); +// else + this.transactionBufferSnapshotService = new TransactionBufferSnapshotServiceImpl(getClient()); + this.transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-transaction-timer")); transactionBufferClient = TransactionBufferClientImpl.create(this, transactionTimer, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotBaseService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotBaseService.java index dcae1c27fe761..36990cb5e65a6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotBaseService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotBaseService.java @@ -24,7 +24,8 @@ import org.apache.pulsar.broker.systopic.SystemTopicClientBase; import org.apache.pulsar.common.naming.TopicName; -public abstract class SystemTopicBaseTxnBufferSnapshotBaseService implements TransactionBufferSnapshotService { +public abstract class SystemTopicBaseTxnBufferSnapshotBaseService implements + SystemTopicTxnBufferSnapshotService { protected final Map> clients; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java new file mode 100644 index 0000000000000..ad2a7dbf36a2c --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java @@ -0,0 +1,62 @@ +/** + * 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 java.util.concurrent.CompletableFuture; +import org.apache.pulsar.broker.systopic.SystemTopicClient.Reader; +import org.apache.pulsar.broker.systopic.SystemTopicClient.Writer; +import org.apache.pulsar.broker.systopic.SystemTopicClientBase; +import org.apache.pulsar.broker.systopic.TransactionBufferSnapshotBaseSystemTopicClient; +import org.apache.pulsar.common.naming.TopicName; + +public interface SystemTopicTxnBufferSnapshotService { + + /** + * Create a transaction buffer snapshot writer. + * + * @param topicName {@link TopicName} the topic name + * + * @return {@link CompletableFuture} return the future of writer + */ + CompletableFuture> createWriter(TopicName topicName); + + /** + * Create a transaction buffer snapshot reader. + * + * @param topicName {@link TopicName} the topic name + * + * @return {@link CompletableFuture} return the future of reader + */ + CompletableFuture> createReader(TopicName topicName); + + /** + * Remove a topic client from cache. + * + * @param topicName {@link TopicName} the topic name + * @param transactionBufferSnapshotBaseSystemTopicClient {@link TransactionBufferSnapshotBaseSystemTopicClient} + * the topic client + */ + void removeClient(TopicName topicName, SystemTopicClientBase transactionBufferSnapshotBaseSystemTopicClient); + + /** + * Close transaction buffer snapshot service. + */ + void close() throws Exception; + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentServiceImpl.java new file mode 100644 index 0000000000000..77cccf32eeb5e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentServiceImpl.java @@ -0,0 +1,55 @@ +/** + * 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.client.api.PulsarClient; + +public class TransactionBufferSnapshotSegmentServiceImpl implements TransactionBufferSnapshotService { + + private final SystemTopicBaseTxnBufferSnapshotSegmentService txnBufferSnapshotSegmentService; + + private final SystemTopicBaseTxnBufferSnapshotIndexService txnBufferSnapshotIndexService; + + public TransactionBufferSnapshotSegmentServiceImpl(PulsarClient pulsarClient) { + this.txnBufferSnapshotSegmentService = new SystemTopicBaseTxnBufferSnapshotSegmentService(pulsarClient); + this.txnBufferSnapshotIndexService = new SystemTopicBaseTxnBufferSnapshotIndexService(pulsarClient); + } + + @Override + public SystemTopicBaseTxnBufferSnapshotIndexService getTxnBufferSnapshotIndexService() { + return this.txnBufferSnapshotIndexService; + } + + @Override + public SystemTopicBaseTxnBufferSnapshotSegmentService getTxnBufferSnapshotSegmentService() { + return this.txnBufferSnapshotSegmentService; + } + + @Override + public SystemTopicBaseTxnBufferSnapshotService getTxnBufferSnapshotService() { + return null; + } + + @Override + public void close() throws Exception { + this.txnBufferSnapshotSegmentService.close(); + this.txnBufferSnapshotIndexService.close(); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java index 933bbf577fe98..23e934ac698c8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java @@ -16,46 +16,32 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.service; -import java.util.concurrent.CompletableFuture; -import org.apache.pulsar.broker.systopic.SystemTopicClient.Reader; -import org.apache.pulsar.broker.systopic.SystemTopicClient.Writer; -import org.apache.pulsar.broker.systopic.SystemTopicClientBase; -import org.apache.pulsar.broker.systopic.TransactionBufferSnapshotBaseSystemTopicClient; -import org.apache.pulsar.common.naming.TopicName; +package org.apache.pulsar.broker.service; -public interface TransactionBufferSnapshotService { +public interface TransactionBufferSnapshotService { /** - * Create a transaction buffer snapshot writer. - * - * @param topicName {@link TopicName} the topic name - * - * @return {@link CompletableFuture} return the future of writer + * Get the systemTopicBaseTxnBufferSnapshotIndexService. + * @return systemTopicBaseTxnBufferSnapshotIndexService */ - CompletableFuture> createWriter(TopicName topicName); + SystemTopicBaseTxnBufferSnapshotIndexService getTxnBufferSnapshotIndexService (); /** - * Create a transaction buffer snapshot reader. - * - * @param topicName {@link TopicName} the topic name - * - * @return {@link CompletableFuture} return the future of reader + * Get the systemTopicBaseTxnBufferSnapshotSegmentService. + * @return systemTopicBaseTxnBufferSnapshotSegmentService */ - CompletableFuture> createReader(TopicName topicName); + SystemTopicBaseTxnBufferSnapshotSegmentService getTxnBufferSnapshotSegmentService(); /** - * Remove a topic client from cache. - * - * @param topicName {@link TopicName} the topic name - * @param transactionBufferSnapshotBaseSystemTopicClient {@link TransactionBufferSnapshotBaseSystemTopicClient} - * the topic client + * Get the systemTopicBaseTxnBufferSnapshotService. + * @return */ - void removeClient(TopicName topicName, SystemTopicClientBase transactionBufferSnapshotBaseSystemTopicClient); + SystemTopicBaseTxnBufferSnapshotService getTxnBufferSnapshotService(); /** - * Close transaction buffer snapshot service. + * Close the systemTopicBaseTxnBufferSnapshotxxService in TransactionBufferSnapshotService . + * @throws Exception */ void close() throws Exception; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java new file mode 100644 index 0000000000000..a48ae59f402b9 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java @@ -0,0 +1,51 @@ +/** + * 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.client.api.PulsarClient; + +public class TransactionBufferSnapshotServiceImpl implements TransactionBufferSnapshotService{ + + private final SystemTopicBaseTxnBufferSnapshotService txnBufferSnapshotService; + + public TransactionBufferSnapshotServiceImpl(PulsarClient pulsarClient) { + this.txnBufferSnapshotService = new SystemTopicBaseTxnBufferSnapshotService(pulsarClient); + } + + @Override + public SystemTopicBaseTxnBufferSnapshotIndexService getTxnBufferSnapshotIndexService() { + return null; + } + + @Override + public SystemTopicBaseTxnBufferSnapshotSegmentService getTxnBufferSnapshotSegmentService() { + return null; + } + + @Override + public SystemTopicBaseTxnBufferSnapshotService getTxnBufferSnapshotService() { + return this.txnBufferSnapshotService; + } + + @Override + public void close() throws Exception { + this.txnBufferSnapshotService.close(); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java index 8af6b27d62910..fe82fddead720 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.broker.systopic; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; +import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; 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; @@ -46,16 +46,16 @@ public TopicPoliciesSystemTopicClient createTopicPoliciesSystemTopicClient(Names } public TransactionBufferSnapshotSystemTopicClient createTransactionBufferSystemTopicClient( - NamespaceName namespaceName, TransactionBufferSnapshotService - transactionBufferSnapshotService) { + NamespaceName namespaceName, SystemTopicTxnBufferSnapshotService + systemTopicTxnBufferSnapshotService) { TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); log.info("Create transaction buffer snapshot client, topicName : {}", topicName.toString()); - return new TransactionBufferSnapshotSystemTopicClient(client, topicName, transactionBufferSnapshotService); + return new TransactionBufferSnapshotSystemTopicClient(client, topicName, systemTopicTxnBufferSnapshotService); } public TransactionBufferSnapshotIndexSystemTopicClient createTransactionBufferSnapshotIndexSystemTopicClient( - NamespaceName namespaceName, TransactionBufferSnapshotService + NamespaceName namespaceName, SystemTopicTxnBufferSnapshotService transactionBufferSnapshotIndexService) { TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES); @@ -66,7 +66,7 @@ public TransactionBufferSnapshotIndexSystemTopicClient createTransactionBufferSn public TransactionBufferSnapshotSegmentSystemTopicClient createTransactionBufferSnapshotSegmentSystemTopicClient( NamespaceName namespaceName, - TransactionBufferSnapshotService + SystemTopicTxnBufferSnapshotService transactionBufferSnapshotSegmentService) { TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index 9697b8d76364b..6d054fe6c331e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.util.concurrent.CompletableFuture; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; +import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; @@ -30,24 +30,24 @@ public abstract class TransactionBufferSnapshotBaseSystemTopicClient extends SystemTopicClientBase { - protected final TransactionBufferSnapshotService transactionBufferSnapshotService; + protected final SystemTopicTxnBufferSnapshotService systemTopicTxnBufferSnapshotService; public TransactionBufferSnapshotBaseSystemTopicClient(PulsarClient client, TopicName topicName, - TransactionBufferSnapshotService - transactionBufferSnapshotService) { + SystemTopicTxnBufferSnapshotService + systemTopicTxnBufferSnapshotService) { super(client, topicName); - this.transactionBufferSnapshotService = transactionBufferSnapshotService; + this.systemTopicTxnBufferSnapshotService = systemTopicTxnBufferSnapshotService; } protected void removeWriter(Writer writer) { writers.remove(writer); - this.transactionBufferSnapshotService.removeClient(topicName, this); + this.systemTopicTxnBufferSnapshotService.removeClient(topicName, this); } protected void removeReader(Reader reader) { readers.remove(reader); - this.transactionBufferSnapshotService.removeClient(topicName, this); + this.systemTopicTxnBufferSnapshotService.removeClient(topicName, this); } protected abstract static class TransactionBufferSnapshotBaseWriter implements Writer { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java index 3a71f21990a20..bbaf8c854d24a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java @@ -21,7 +21,7 @@ import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; +import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -34,11 +34,12 @@ public class TransactionBufferSnapshotIndexSystemTopicClient extends TransactionBufferSnapshotBaseSystemTopicClient { - private final TransactionBufferSnapshotService + private final SystemTopicTxnBufferSnapshotService transactionBufferSnapshotIndexService; public TransactionBufferSnapshotIndexSystemTopicClient(PulsarClient client, TopicName topicName, - TransactionBufferSnapshotService transactionBufferSnapshotIndexService) { + SystemTopicTxnBufferSnapshotService + transactionBufferSnapshotIndexService) { super(client, topicName, transactionBufferSnapshotIndexService); this.transactionBufferSnapshotIndexService = transactionBufferSnapshotIndexService; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java index 508518966cdce..d6e8f95e3b60c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java @@ -21,7 +21,7 @@ import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.service.BrokerServiceException; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; +import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -36,7 +36,7 @@ public class TransactionBufferSnapshotSegmentSystemTopicClient extends TransactionBufferSnapshotBaseSystemTopicClient { public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client, TopicName topicName, - TransactionBufferSnapshotService + SystemTopicTxnBufferSnapshotService transactionBufferSnapshotSegmentService) { super(client, topicName, transactionBufferSnapshotSegmentService); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java index cbdcd654443f8..855b291900df3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java @@ -21,7 +21,7 @@ import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; +import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -33,12 +33,12 @@ @Slf4j public class TransactionBufferSnapshotSystemTopicClient extends TransactionBufferSnapshotBaseSystemTopicClient { - private TransactionBufferSnapshotService transactionBufferSnapshotService; + private SystemTopicTxnBufferSnapshotService systemTopicTxnBufferSnapshotService; public TransactionBufferSnapshotSystemTopicClient(PulsarClient client, TopicName topicName, - TransactionBufferSnapshotService - transactionBufferSnapshotService) { - super(client, topicName, transactionBufferSnapshotService); + SystemTopicTxnBufferSnapshotService + systemTopicTxnBufferSnapshotService) { + super(client, topicName, systemTopicTxnBufferSnapshotService); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index ad778137001e3..6113437281e51 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -117,7 +117,8 @@ public TopicTransactionBuffer(PersistentTopic topic) { super(State.None); this.topic = topic; this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar() - .getTransactionBufferSnapshotService().createWriter(TopicName.get(topic.getName())); + .getTransactionBufferSnapshotService() + .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName())); this.timer = topic.getBrokerService().getPulsar().getTransactionTimer(); this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar() .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount(); @@ -645,7 +646,7 @@ public void run() { this, topic.getName()); return; } - topic.getBrokerService().getPulsar().getTransactionBufferSnapshotService() + topic.getBrokerService().getPulsar().getTransactionBufferSnapshotService().getTxnBufferSnapshotService() .createReader(TopicName.get(topic.getName())).thenAcceptAsync(reader -> { try { boolean hasSnapshot = false; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 19dcda52a68d1..16ac60da7c46e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -43,17 +43,23 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.ReadOnlyCursor; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.RandomUtils; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.service.SystemTopicBaseTxnBufferSnapshotService; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotSegmentServiceImpl; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; @@ -484,50 +490,56 @@ public void testTransactionBufferRecoverThrowException() throws Exception { PersistentTopic originalTopic = (PersistentTopic) getPulsarServiceList().get(0) .getBrokerService().getTopic(TopicName.get(topic).toString(), false).get().get(); - TransactionBufferSnapshotService transactionBufferSnapshotService = - mock(TransactionBufferSnapshotService.class); + SystemTopicBaseTxnBufferSnapshotService systemTopicTxnBufferSnapshotService = + mock(SystemTopicBaseTxnBufferSnapshotService.class); SystemTopicClient.Reader reader = mock(SystemTopicClient.Reader.class); SystemTopicClient.Writer writer = mock(SystemTopicClient.Writer.class); - doReturn(CompletableFuture.completedFuture(reader)).when(transactionBufferSnapshotService).createReader(any()); - doReturn(CompletableFuture.completedFuture(writer)).when(transactionBufferSnapshotService).createWriter(any()); + doReturn(CompletableFuture.completedFuture(reader)) + .when(systemTopicTxnBufferSnapshotService).createReader(any()); + doReturn(CompletableFuture.completedFuture(writer)) + .when(systemTopicTxnBufferSnapshotService).createWriter(any()); + TransactionBufferSnapshotService transactionBufferSnapshotService = + mock(TransactionBufferSnapshotService.class); + doReturn(systemTopicTxnBufferSnapshotService) + .when(transactionBufferSnapshotService).getTxnBufferSnapshotService(); doReturn(CompletableFuture.completedFuture(null)).when(reader).closeAsync(); doReturn(CompletableFuture.completedFuture(null)).when(writer).closeAsync(); Field field = PulsarService.class.getDeclaredField("transactionBufferSnapshotService"); field.setAccessible(true); - TransactionBufferSnapshotService transactionBufferSnapshotServiceOriginal = - (TransactionBufferSnapshotService) field.get(getPulsarServiceList().get(0)); + TransactionBufferSnapshotService systemTopicTxnBufferSnapshotServiceOriginal = + ((TransactionBufferSnapshotService)field.get(getPulsarServiceList().get(0))); // mock reader can't read snapshot fail throw RuntimeException doThrow(new RuntimeException("test")).when(reader).hasMoreEvents(); // check reader close topic - checkCloseTopic(pulsarClient, transactionBufferSnapshotServiceOriginal, + checkCloseTopic(pulsarClient, systemTopicTxnBufferSnapshotServiceOriginal, transactionBufferSnapshotService, originalTopic, field, producer); doReturn(true).when(reader).hasMoreEvents(); // mock reader can't read snapshot fail throw PulsarClientException doThrow(new PulsarClientException("test")).when(reader).hasMoreEvents(); // check reader close topic - checkCloseTopic(pulsarClient, transactionBufferSnapshotServiceOriginal, + checkCloseTopic(pulsarClient, systemTopicTxnBufferSnapshotServiceOriginal, transactionBufferSnapshotService, originalTopic, field, producer); doReturn(true).when(reader).hasMoreEvents(); // mock create reader fail doReturn(FutureUtil.failedFuture(new PulsarClientException("test"))) - .when(transactionBufferSnapshotService).createReader(any()); + .when(systemTopicTxnBufferSnapshotService).createReader(any()); // check create reader fail close topic originalTopic = (PersistentTopic) getPulsarServiceList().get(0) .getBrokerService().getTopic(TopicName.get(topic).toString(), false).get().get(); - checkCloseTopic(pulsarClient, transactionBufferSnapshotServiceOriginal, + checkCloseTopic(pulsarClient, systemTopicTxnBufferSnapshotServiceOriginal, transactionBufferSnapshotService, originalTopic, field, producer); - doReturn(CompletableFuture.completedFuture(reader)).when(transactionBufferSnapshotService).createReader(any()); + doReturn(CompletableFuture.completedFuture(reader)).when(systemTopicTxnBufferSnapshotService).createReader(any()); // check create writer fail close topic originalTopic = (PersistentTopic) getPulsarServiceList().get(0) .getBrokerService().getTopic(TopicName.get(topic).toString(), false).get().get(); // mock create writer fail doReturn(FutureUtil.failedFuture(new PulsarClientException("test"))) - .when(transactionBufferSnapshotService).createWriter(any()); - checkCloseTopic(pulsarClient, transactionBufferSnapshotServiceOriginal, + .when(systemTopicTxnBufferSnapshotService).createWriter(any()); + checkCloseTopic(pulsarClient, systemTopicTxnBufferSnapshotServiceOriginal, transactionBufferSnapshotService, originalTopic, field, producer); } @@ -579,8 +591,8 @@ public void testTransactionBufferNoSnapshotCloseReader() throws Exception{ @Test public void testTransactionBufferIndexSystemTopic() throws Exception { - TransactionBufferSnapshotService transactionBufferSnapshotIndexService = - getPulsarServiceList().get(0).getTransactionBufferSnapshotIndexService(); + SystemTopicTxnBufferSnapshotService transactionBufferSnapshotIndexService = + new TransactionBufferSnapshotSegmentServiceImpl(pulsarClient).getTxnBufferSnapshotIndexService(); SystemTopicClient.Writer indexesWriter = transactionBufferSnapshotIndexService.createWriter(TopicName.get(SNAPSHOT_INDEX)).get(); @@ -636,8 +648,9 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { BrokerService brokerService = pulsarService.getBrokerService(); // create snapshot sgement writer - TransactionBufferSnapshotService - transactionBufferSnapshotSegmentService = pulsarService.getTransactionBufferSnapshotSegmentService(); + SystemTopicTxnBufferSnapshotService + transactionBufferSnapshotSegmentService = + new TransactionBufferSnapshotSegmentServiceImpl(pulsarClient).getTxnBufferSnapshotSegmentService(); SystemTopicClient.Writer segmentWriter = transactionBufferSnapshotSegmentService.createWriter(snapshotSegmentTopicName).get(); @@ -659,12 +672,42 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { MessageIdImpl messageId = (MessageIdImpl) segmentWriter.write(snapshot); - //create cursor - ReadOnlyCursor readOnlyCursor = pulsarService.getManagedLedgerFactory() - .openReadOnlyCursor(snapshotSegmentTopicName.getPersistenceNamingEncoding(), - PositionImpl.EARLIEST, brokerService.getManagedLedgerConfig(snapshotSegmentTopicName).get()); + //create read-only managed ledger + CompletableFuture readOnlyManagedLedgerCompletableFuture = new CompletableFuture<>(); + pulsarService.getManagedLedgerFactory() + .asyncOpenReadOnlyManagedLedger(snapshotSegmentTopicName.getPersistenceNamingEncoding(), + brokerService.getManagedLedgerConfig(snapshotSegmentTopicName).get(), + new AsyncCallbacks.OpenLedgerCallback() { + @Override + public void openLedgerComplete(ManagedLedger ledger, Object ctx) { + readOnlyManagedLedgerCompletableFuture.complete((ReadOnlyManagedLedgerImpl) ledger); + } + + @Override + public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { + readOnlyManagedLedgerCompletableFuture.completeExceptionally(exception); + } + }, null); + + + ReadOnlyManagedLedgerImpl readOnlyManagedLedger = readOnlyManagedLedgerCompletableFuture.get(); + //read the entry and decode entry to snapshot - Entry entry = readOnlyCursor.readEntry(new PositionImpl(messageId.getLedgerId(), messageId.getEntryId()), null); + CompletableFuture entryCompletableFuture = new CompletableFuture<>(); + readOnlyManagedLedger.asyncReadEntry( + new PositionImpl(messageId.getLedgerId(), messageId.getEntryId()), + new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + entryCompletableFuture.complete(entry); + } + + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + entryCompletableFuture.completeExceptionally(exception); + } + }, null); + Entry entry = entryCompletableFuture.get(); //decode snapshot from entry ByteBuf headersAndPayload = entry.getDataBuffer(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index d0674721c00be..d3993c5a9471e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -77,6 +77,7 @@ import org.apache.pulsar.broker.service.BacklogQuotaManager; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; +import org.apache.pulsar.broker.service.SystemTopicBaseTxnBufferSnapshotService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -1431,12 +1432,17 @@ public Object answer(InvocationOnMock invocation) throws Throwable { when(pendingAckStoreProvider.newPendingAckStore(any())) .thenReturn(CompletableFuture.completedFuture(pendingAckStore)); // Mock TransactionBufferSnapshotService - TransactionBufferSnapshotService transactionBufferSnapshotService - = mock(TransactionBufferSnapshotService.class); + SystemTopicBaseTxnBufferSnapshotService systemTopicTxnBufferSnapshotService + = mock(SystemTopicBaseTxnBufferSnapshotService.class); SystemTopicClient.Writer writer = mock(SystemTopicClient.Writer.class); when(writer.closeAsync()).thenReturn(CompletableFuture.completedFuture(null)); - when(transactionBufferSnapshotService.createWriter(any())) + when(systemTopicTxnBufferSnapshotService.createWriter(any())) .thenReturn(CompletableFuture.completedFuture(writer)); + TransactionBufferSnapshotService transactionBufferSnapshotService = + mock(TransactionBufferSnapshotService.class); + when(transactionBufferSnapshotService.getTxnBufferSnapshotService()) + .thenReturn(systemTopicTxnBufferSnapshotService); + // Mock pulsar. PulsarService pulsar = mock(PulsarService.class); when(pulsar.getConfiguration()).thenReturn(serviceConfiguration); From bc459175abeb9f626bd26b04ce0ef8d11cc093b2 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Tue, 20 Sep 2022 18:41:03 +0800 Subject: [PATCH 09/27] abstract writer and read, and fix read-only managedLedger init. --- .../impl/ManagedLedgerFactoryImpl.java | 22 +++----- .../impl/ReadOnlyManagedLedgerImpl.java | 14 +++-- .../apache/pulsar/broker/PulsarService.java | 8 +-- .../SystemTopicBasedTopicPoliciesService.java | 7 ++- ...temTopicTxnBufferSnapshotBaseService.java} | 9 ++- ...picTxnBufferSnapshotIndexServiceImpl.java} | 9 ++- ...cTxnBufferSnapshotSegmentServiceImpl.java} | 9 ++- ...temTopicTxnBufferSnapshotServiceImpl.java} | 9 ++- ...ctionBufferSnapshotSegmentServiceImpl.java | 55 ------------------- .../TransactionBufferSnapshotService.java | 6 +- .../TransactionBufferSnapshotServiceImpl.java | 41 ++++++++++---- .../broker/systopic/SystemTopicClient.java | 8 +-- .../TopicPoliciesSystemTopicClient.java | 26 +++++---- ...onBufferSnapshotBaseSystemTopicClient.java | 32 +++++++++++ ...nBufferSnapshotIndexSystemTopicClient.java | 34 ------------ ...ufferSnapshotSegmentSystemTopicClient.java | 41 +------------- ...actionBufferSnapshotSystemTopicClient.java | 29 ---------- .../buffer/impl/TopicTransactionBuffer.java | 4 +- .../v2/TransactionBufferSnapshotIndexes.java | 1 + .../transaction/buffer/matadata/v2/TxnID.java | 51 ----------------- ...NamespaceEventsSystemTopicServiceTest.java | 3 +- .../TopicTransactionBufferRecoverTest.java | 25 +++++---- .../broker/transaction/TransactionTest.java | 8 ++- 23 files changed, 153 insertions(+), 298 deletions(-) rename pulsar-broker/src/main/java/org/apache/pulsar/broker/service/{SystemTopicBaseTxnBufferSnapshotBaseService.java => SystemTopicTxnBufferSnapshotBaseService.java} (89%) rename pulsar-broker/src/main/java/org/apache/pulsar/broker/service/{SystemTopicBaseTxnBufferSnapshotIndexService.java => SystemTopicTxnBufferSnapshotIndexServiceImpl.java} (89%) rename pulsar-broker/src/main/java/org/apache/pulsar/broker/service/{SystemTopicBaseTxnBufferSnapshotSegmentService.java => SystemTopicTxnBufferSnapshotSegmentServiceImpl.java} (88%) rename pulsar-broker/src/main/java/org/apache/pulsar/broker/service/{SystemTopicBaseTxnBufferSnapshotService.java => SystemTopicTxnBufferSnapshotServiceImpl.java} (89%) delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentServiceImpl.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TxnID.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 9bb4de5500332..630877e8e132c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -430,20 +430,14 @@ public void asyncOpenReadOnlyManagedLedger(String managedLedgerName, ManagedLedg .get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), config.getBookKeeperEnsemblePlacementPolicyProperties())), store, config, scheduledExecutor, managedLedgerName); - - roManagedLedger.initialize(new ManagedLedgerInitializeLedgerCallback() { - @Override - public void initializeComplete() { - log.info("[{}] Successfully initialize Read-only managed ledger", managedLedgerName); - callback.openLedgerComplete(roManagedLedger, ctx); - } - - @Override - public void initializeFailed(ManagedLedgerException e) { - log.error("[{}] Failed to initialize Read-only managed ledger", managedLedgerName, e); - callback.openLedgerFailed(e, ctx); - } - }, ctx); + roManagedLedger.initialize().thenRun(() -> { + log.info("[{}] Successfully initialize Read-only managed ledger", managedLedgerName); + callback.openLedgerComplete(roManagedLedger, ctx); + }).exceptionally(e -> { + log.error("[{}] Failed to initialize Read-only managed ledger", managedLedgerName, e); + callback.openLedgerFailed((ManagedLedgerException) e, ctx); + return null; + }); } @Override diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index 214c3afe1bcdb..0be5b26829560 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -47,6 +47,12 @@ public ReadOnlyManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bo CompletableFuture initializeAndCreateCursor(PositionImpl startPosition) { CompletableFuture future = new CompletableFuture<>(); + initialize().thenRun(() -> future.complete(createReadOnlyCursor(startPosition))); + return future; + } + + CompletableFuture initialize() { + CompletableFuture future = new CompletableFuture<>(); // Fetch the list of existing ledgers in the managed ledger store.getManagedLedgerInfo(name, false, new MetaStoreCallback() { @@ -72,7 +78,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { .setTimestamp(clock.millis()).build(); ledgers.put(lastLedgerId, info); - future.complete(createReadOnlyCursor(startPosition)); + future.complete(null); }).exceptionally(ex -> { if (ex instanceof CompletionException && ex.getCause() instanceof IllegalArgumentException) { @@ -80,7 +86,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(lastLedgerId) .setEntries(0).setSize(0).setTimestamp(clock.millis()).build(); ledgers.put(lastLedgerId, info); - future.complete(createReadOnlyCursor(startPosition)); + future.complete(null); } else { future.completeExceptionally(new ManagedLedgerException(ex)); } @@ -93,7 +99,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(lastLedgerId).setEntries(0) .setSize(0).setTimestamp(clock.millis()).build(); ledgers.put(lastLedgerId, info); - future.complete(createReadOnlyCursor(startPosition)); + future.complete(null); } else { future.completeExceptionally(new ManagedLedgerException(ex)); } @@ -101,7 +107,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { }); } else { // The read-only managed ledger is ready to use - future.complete(createReadOnlyCursor(startPosition)); + future.complete(null); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 5766e09b57404..fd8b3d7994b09 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -104,9 +104,8 @@ import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotSegmentServiceImpl; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.service.TransactionBufferSnapshotServiceImpl; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.SchemaStorageFactory; import org.apache.pulsar.broker.stats.MetricsGenerator; @@ -819,10 +818,7 @@ public void start() throws PulsarServerException { // Register pulsar system namespaces and start transaction meta store service if (config.isTransactionCoordinatorEnabled()) { -// if (transactionBufferSegmentedSnapshotEnabled) - this.transactionBufferSnapshotService = new TransactionBufferSnapshotSegmentServiceImpl(getClient()); -// else - this.transactionBufferSnapshotService = new TransactionBufferSnapshotServiceImpl(getClient()); + this.transactionBufferSnapshotService = new TransactionBufferSnapshotServiceImpl(getClient(), false); this.transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-transaction-timer")); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index ed69428386adb..15c332dfc30d0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.broker.systopic.TopicPoliciesSystemTopicClient.TopicPolicyWriter.getEventKey; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -118,7 +119,8 @@ private CompletableFuture sendTopicPolicyEvent(TopicName topicName, Action } else { PulsarEvent event = getPulsarEvent(topicName, actionType, policies); CompletableFuture actionFuture = - ActionType.DELETE.equals(actionType) ? writer.deleteAsync(event) : writer.writeAsync(event); + ActionType.DELETE.equals(actionType) ? writer.deleteAsync(event, getEventKey(event)) + : writer.writeAsync(event, getEventKey(event)); actionFuture.whenComplete(((messageId, e) -> { if (e != null) { result.completeExceptionally(e); @@ -444,7 +446,8 @@ private void refreshTopicPoliciesCache(Message msg) { SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory .createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject()); systemTopicClient.newWriterAsync().thenAccept(writer - -> writer.deleteAsync(getPulsarEvent(topicName, ActionType.DELETE, null)) + -> writer.deleteAsync(getPulsarEvent(topicName, ActionType.DELETE, null), + getEventKey(topicName)) .whenComplete((result, e) -> writer.closeAsync().whenComplete((res, ex) -> { if (ex != null) { log.error("close writer failed ", ex); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotBaseService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotBaseService.java similarity index 89% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotBaseService.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotBaseService.java index 36990cb5e65a6..9a628736aeab9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotBaseService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotBaseService.java @@ -20,18 +20,17 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.systopic.SystemTopicClientBase; import org.apache.pulsar.common.naming.TopicName; -public abstract class SystemTopicBaseTxnBufferSnapshotBaseService implements - SystemTopicTxnBufferSnapshotService { +public abstract class SystemTopicTxnBufferSnapshotBaseService implements SystemTopicTxnBufferSnapshotService { protected final Map> clients; - public SystemTopicBaseTxnBufferSnapshotBaseService( - Map> clients) { - this.clients = clients; + public SystemTopicTxnBufferSnapshotBaseService() { + this.clients = new ConcurrentHashMap<>(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotIndexService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotIndexServiceImpl.java similarity index 89% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotIndexService.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotIndexServiceImpl.java index a60b7b3083ef7..667ca43dfd79c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotIndexService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotIndexServiceImpl.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.service; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; @@ -29,13 +28,13 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.FutureUtil; -public class SystemTopicBaseTxnBufferSnapshotIndexService extends - SystemTopicBaseTxnBufferSnapshotBaseService { +public class SystemTopicTxnBufferSnapshotIndexServiceImpl extends + SystemTopicTxnBufferSnapshotBaseService { private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; - public SystemTopicBaseTxnBufferSnapshotIndexService(PulsarClient client) { - super(new ConcurrentHashMap<>()); + public SystemTopicTxnBufferSnapshotIndexServiceImpl(PulsarClient client) { + super(); this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotSegmentServiceImpl.java similarity index 88% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotSegmentServiceImpl.java index 58a323f7f825f..9a710df65c1fc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotSegmentService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotSegmentServiceImpl.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.service; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; @@ -29,12 +28,12 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.FutureUtil; -public class SystemTopicBaseTxnBufferSnapshotSegmentService extends - SystemTopicBaseTxnBufferSnapshotBaseService { +public class SystemTopicTxnBufferSnapshotSegmentServiceImpl extends + SystemTopicTxnBufferSnapshotBaseService { private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; - public SystemTopicBaseTxnBufferSnapshotSegmentService(PulsarClient client) { - super(new ConcurrentHashMap<>()); + public SystemTopicTxnBufferSnapshotSegmentServiceImpl(PulsarClient client) { + super(); this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotServiceImpl.java similarity index 89% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotService.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotServiceImpl.java index 69867580e6a0a..0bd24543c0aa9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotServiceImpl.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.service; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; @@ -29,13 +28,13 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.FutureUtil; -public class SystemTopicBaseTxnBufferSnapshotService - extends SystemTopicBaseTxnBufferSnapshotBaseService { +public class SystemTopicTxnBufferSnapshotServiceImpl + extends SystemTopicTxnBufferSnapshotBaseService { private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; - public SystemTopicBaseTxnBufferSnapshotService(PulsarClient client) { - super(new ConcurrentHashMap<>()); + public SystemTopicTxnBufferSnapshotServiceImpl(PulsarClient client) { + super(); this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentServiceImpl.java deleted file mode 100644 index 77cccf32eeb5e..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotSegmentServiceImpl.java +++ /dev/null @@ -1,55 +0,0 @@ -/** - * 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.client.api.PulsarClient; - -public class TransactionBufferSnapshotSegmentServiceImpl implements TransactionBufferSnapshotService { - - private final SystemTopicBaseTxnBufferSnapshotSegmentService txnBufferSnapshotSegmentService; - - private final SystemTopicBaseTxnBufferSnapshotIndexService txnBufferSnapshotIndexService; - - public TransactionBufferSnapshotSegmentServiceImpl(PulsarClient pulsarClient) { - this.txnBufferSnapshotSegmentService = new SystemTopicBaseTxnBufferSnapshotSegmentService(pulsarClient); - this.txnBufferSnapshotIndexService = new SystemTopicBaseTxnBufferSnapshotIndexService(pulsarClient); - } - - @Override - public SystemTopicBaseTxnBufferSnapshotIndexService getTxnBufferSnapshotIndexService() { - return this.txnBufferSnapshotIndexService; - } - - @Override - public SystemTopicBaseTxnBufferSnapshotSegmentService getTxnBufferSnapshotSegmentService() { - return this.txnBufferSnapshotSegmentService; - } - - @Override - public SystemTopicBaseTxnBufferSnapshotService getTxnBufferSnapshotService() { - return null; - } - - @Override - public void close() throws Exception { - this.txnBufferSnapshotSegmentService.close(); - this.txnBufferSnapshotIndexService.close(); - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java index 23e934ac698c8..70fb6067c16e0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java @@ -25,19 +25,19 @@ public interface TransactionBufferSnapshotService { * Get the systemTopicBaseTxnBufferSnapshotIndexService. * @return systemTopicBaseTxnBufferSnapshotIndexService */ - SystemTopicBaseTxnBufferSnapshotIndexService getTxnBufferSnapshotIndexService (); + SystemTopicTxnBufferSnapshotIndexServiceImpl getTxnBufferSnapshotIndexService (); /** * Get the systemTopicBaseTxnBufferSnapshotSegmentService. * @return systemTopicBaseTxnBufferSnapshotSegmentService */ - SystemTopicBaseTxnBufferSnapshotSegmentService getTxnBufferSnapshotSegmentService(); + SystemTopicTxnBufferSnapshotSegmentServiceImpl getTxnBufferSnapshotSegmentService(); /** * Get the systemTopicBaseTxnBufferSnapshotService. * @return */ - SystemTopicBaseTxnBufferSnapshotService getTxnBufferSnapshotService(); + SystemTopicTxnBufferSnapshotServiceImpl getTxnBufferSnapshotService(); /** * Close the systemTopicBaseTxnBufferSnapshotxxService in TransactionBufferSnapshotService . diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java index a48ae59f402b9..f01d29b094cfb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java @@ -21,31 +21,52 @@ import org.apache.pulsar.client.api.PulsarClient; -public class TransactionBufferSnapshotServiceImpl implements TransactionBufferSnapshotService{ +public class TransactionBufferSnapshotServiceImpl implements TransactionBufferSnapshotService { - private final SystemTopicBaseTxnBufferSnapshotService txnBufferSnapshotService; + private SystemTopicTxnBufferSnapshotServiceImpl txnBufferSnapshotService; - public TransactionBufferSnapshotServiceImpl(PulsarClient pulsarClient) { - this.txnBufferSnapshotService = new SystemTopicBaseTxnBufferSnapshotService(pulsarClient); + private SystemTopicTxnBufferSnapshotSegmentServiceImpl txnBufferSnapshotSegmentService; + + private SystemTopicTxnBufferSnapshotIndexServiceImpl txnBufferSnapshotIndexService; + + public TransactionBufferSnapshotServiceImpl(PulsarClient pulsarClient, + boolean transactionBufferSegmentedSnapshotEnabled) { + if (transactionBufferSegmentedSnapshotEnabled) { + this.txnBufferSnapshotSegmentService = new SystemTopicTxnBufferSnapshotSegmentServiceImpl(pulsarClient); + this.txnBufferSnapshotIndexService = new SystemTopicTxnBufferSnapshotIndexServiceImpl(pulsarClient); + } else { + this.txnBufferSnapshotService = new SystemTopicTxnBufferSnapshotServiceImpl(pulsarClient); + } } @Override - public SystemTopicBaseTxnBufferSnapshotIndexService getTxnBufferSnapshotIndexService() { - return null; + public SystemTopicTxnBufferSnapshotIndexServiceImpl getTxnBufferSnapshotIndexService() { + return this.txnBufferSnapshotIndexService; } @Override - public SystemTopicBaseTxnBufferSnapshotSegmentService getTxnBufferSnapshotSegmentService() { - return null; + public SystemTopicTxnBufferSnapshotSegmentServiceImpl getTxnBufferSnapshotSegmentService() { + return this.txnBufferSnapshotSegmentService; } @Override - public SystemTopicBaseTxnBufferSnapshotService getTxnBufferSnapshotService() { + public SystemTopicTxnBufferSnapshotServiceImpl getTxnBufferSnapshotService() { return this.txnBufferSnapshotService; } @Override public void close() throws Exception { - this.txnBufferSnapshotService.close(); + if (this.txnBufferSnapshotIndexService != null) { + this.txnBufferSnapshotIndexService.close(); + this.txnBufferSnapshotIndexService = null; + } + if (this.txnBufferSnapshotSegmentService != null) { + this.txnBufferSnapshotSegmentService.close(); + this.txnBufferSnapshotSegmentService = null; + } + if (this.txnBufferSnapshotService != null) { + this.txnBufferSnapshotService.close(); + this.txnBufferSnapshotService = null; + } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java index 2bc740a41d48b..246c5a8c194e9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java @@ -93,14 +93,14 @@ interface Writer { * @return message id * @throws PulsarClientException exception while write event cause */ - MessageId write(T t) throws PulsarClientException; + MessageId write(T t, String key) throws PulsarClientException; /** * Async write event to the system topic. * @param t pulsar event * @return message id future */ - CompletableFuture writeAsync(T t); + CompletableFuture writeAsync(T t, String key); /** * Delete event in the system topic. @@ -108,7 +108,7 @@ interface Writer { * @return message id * @throws PulsarClientException exception while write event cause */ - default MessageId delete(T t) throws PulsarClientException { + default MessageId delete(T t, String key) throws PulsarClientException { throw new UnsupportedOperationException("Unsupported operation"); } @@ -117,7 +117,7 @@ default MessageId delete(T t) throws PulsarClientException { * @param t pulsar event * @return message id future */ - default CompletableFuture deleteAsync(T t) { + default CompletableFuture deleteAsync(T t, String key) { throw new UnsupportedOperationException("Unsupported operation"); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java index 84b8bd636bd3a..c7b7860c1e7dc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java @@ -72,7 +72,7 @@ protected CompletableFuture> newReaderAsyncInternal() { }); } - private static class TopicPolicyWriter implements Writer { + public static class TopicPolicyWriter implements Writer { private final Producer producer; private final SystemTopicClient systemTopicClient; @@ -83,42 +83,48 @@ private TopicPolicyWriter(Producer producer, SystemTopicClient builder = producer.newMessage().key(getEventKey(event)).value(event); + public MessageId write(PulsarEvent event, String key) throws PulsarClientException { + TypedMessageBuilder builder = producer.newMessage().key(key).value(event); setReplicateCluster(event, builder); return builder.send(); } @Override - public CompletableFuture writeAsync(PulsarEvent event) { - TypedMessageBuilder builder = producer.newMessage().key(getEventKey(event)).value(event); + public CompletableFuture writeAsync(PulsarEvent event, String key) { + TypedMessageBuilder builder = producer.newMessage().key(key).value(event); setReplicateCluster(event, builder); return builder.sendAsync(); } @Override - public MessageId delete(PulsarEvent event) throws PulsarClientException { - validateActionType(event); - TypedMessageBuilder builder = producer.newMessage().key(getEventKey(event)).value(null); + public MessageId delete(PulsarEvent event, String key) throws PulsarClientException { + TypedMessageBuilder builder = producer.newMessage().key(key).value(null); setReplicateCluster(event, builder); return builder.send(); } @Override - public CompletableFuture deleteAsync(PulsarEvent event) { + public CompletableFuture deleteAsync(PulsarEvent event, String key) { validateActionType(event); TypedMessageBuilder builder = producer.newMessage().key(getEventKey(event)).value(null); setReplicateCluster(event, builder); return builder.sendAsync(); } - private String getEventKey(PulsarEvent event) { + public static String getEventKey(PulsarEvent event) { return TopicName.get(event.getTopicPoliciesEvent().getDomain(), event.getTopicPoliciesEvent().getTenant(), event.getTopicPoliciesEvent().getNamespace(), event.getTopicPoliciesEvent().getTopic()).toString(); } + public static String getEventKey(TopicName topicName) { + return TopicName.get(topicName.getDomain().toString(), + topicName.getTenant(), + topicName.getNamespace(), + TopicName.get(topicName.getPartitionedTopicName()).getLocalName()).toString(); + } + @Override public void close() throws IOException { this.producer.close(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index 6d054fe6c331e..12f64e21d82e2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -22,6 +22,7 @@ import java.util.concurrent.CompletableFuture; 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; @@ -63,6 +64,37 @@ protected TransactionBufferSnapshotBaseWriter(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 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 deleteAsync(T t, String key) { + return producer.newMessage() + .key(key) + .value(null) + .sendAsync(); + } + @Override public void close() throws IOException { this.producer.close(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java index bbaf8c854d24a..bc61bff074c1c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java @@ -26,7 +26,6 @@ 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; @@ -82,39 +81,6 @@ private TransactionBufferSnapshotIndexWriter(Producer writeAsync(TransactionBufferSnapshotIndexes - transactionBufferSnapshotIndexes) { - return producer.newMessage() - .key(transactionBufferSnapshotIndexes.getTopicName()) - .value(transactionBufferSnapshotIndexes).sendAsync(); - } - - @Override - public MessageId delete(TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes) - throws PulsarClientException { - return producer.newMessage() - .key(transactionBufferSnapshotIndexes.getTopicName()) - .value(null) - .send(); - } - - @Override - public CompletableFuture deleteAsync(TransactionBufferSnapshotIndexes - transactionBufferSnapshotIndexes) { - return producer.newMessage() - .key(transactionBufferSnapshotIndexes.getTopicName()) - .value(null) - .sendAsync(); - } } private static class TransactionBufferSnapshotIndexReader extends TransactionBufferSnapshotBaseSystemTopicClient diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java index d6e8f95e3b60c..a0cf81e64d5b9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java @@ -23,10 +23,8 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; -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; import org.apache.pulsar.common.util.FutureUtil; @@ -35,6 +33,7 @@ public class TransactionBufferSnapshotSegmentSystemTopicClient extends TransactionBufferSnapshotBaseSystemTopicClient { + private static final String prefix = "multiple-"; public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client, TopicName topicName, SystemTopicTxnBufferSnapshotService transactionBufferSnapshotSegmentService) { @@ -62,8 +61,8 @@ public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client .NotAllowedException("Do not allow to get reader for segment topic reader")); } - protected static String buildKey(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot snapshot) { - return "multiple-" + snapshot.getSequenceId() + "-" + snapshot.getTopicName(); + public static String buildKey(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot snapshot) { + return prefix + snapshot.getSequenceId() + "-" + snapshot.getTopicName(); } @@ -75,40 +74,6 @@ private TransactionBufferSnapshotSegmentWriter(Producer writeAsync(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot - transactionBufferSnapshot) { - return producer.newMessage() - .key(buildKey(transactionBufferSnapshot)) - .value(transactionBufferSnapshot).sendAsync(); - } - - @Override - public MessageId delete(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot transactionBufferSnapshot) - throws PulsarClientException { - return producer.newMessage() - .key(buildKey(transactionBufferSnapshot)) - .value(null) - .send(); - } - - @Override - public CompletableFuture deleteAsync(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot - transactionBufferSnapshot) { - return producer.newMessage() - .key(buildKey(transactionBufferSnapshot)) - .value(null) - .sendAsync(); - } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java index 855b291900df3..ad0fff51c7cd7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java @@ -26,7 +26,6 @@ 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; @@ -77,34 +76,6 @@ private TransactionBufferSnapshotWriter(Producer prod TransactionBufferSnapshotSystemTopicClient transactionBufferSnapshotSystemTopicClient) { super(producer, transactionBufferSnapshotSystemTopicClient); } - - @Override - public MessageId write(TransactionBufferSnapshot transactionBufferSnapshot) throws PulsarClientException { - return producer.newMessage().key(transactionBufferSnapshot.getTopicName()) - .value(transactionBufferSnapshot).send(); - } - - @Override - public CompletableFuture writeAsync(TransactionBufferSnapshot transactionBufferSnapshot) { - return producer.newMessage().key(transactionBufferSnapshot.getTopicName()) - .value(transactionBufferSnapshot).sendAsync(); - } - - @Override - public MessageId delete(TransactionBufferSnapshot transactionBufferSnapshot) throws PulsarClientException { - return producer.newMessage() - .key(transactionBufferSnapshot.getTopicName()) - .value(null) - .send(); - } - - @Override - public CompletableFuture deleteAsync(TransactionBufferSnapshot transactionBufferSnapshot) { - return producer.newMessage() - .key(transactionBufferSnapshot.getTopicName()) - .value(null) - .sendAsync(); - } } private static class TransactionBufferSnapshotReader extends diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index 6113437281e51..d325d5b4129c1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -485,7 +485,7 @@ private CompletableFuture takeSnapshot() { }); snapshot.setAborts(list); } - return writer.writeAsync(snapshot).thenAccept(messageId-> { + return writer.writeAsync(snapshot, snapshot.getTopicName()).thenAccept(messageId-> { this.lastSnapshotTimestamps = System.currentTimeMillis(); if (log.isDebugEnabled()) { log.debug("[{}]Transaction buffer take snapshot success! " @@ -533,7 +533,7 @@ public CompletableFuture clearSnapshot() { return this.takeSnapshotWriter.thenCompose(writer -> { TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot(); snapshot.setTopicName(topic.getName()); - return writer.deleteAsync(snapshot); + return writer.deleteAsync(snapshot, snapshot.getTopicName()); }).thenCompose(__ -> CompletableFuture.completedFuture(null)); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java index df9c179642dc8..97cb64c20404a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java @@ -25,6 +25,7 @@ import lombok.Getter; import lombok.NoArgsConstructor; import lombok.Setter; +import org.apache.pulsar.client.api.transaction.TxnID; @AllArgsConstructor @NoArgsConstructor diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TxnID.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TxnID.java deleted file mode 100644 index 6a74725c2cb3c..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TxnID.java +++ /dev/null @@ -1,51 +0,0 @@ -/** - * 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.transaction.buffer.matadata.v2; - -import java.util.Objects; -import lombok.AllArgsConstructor; -import lombok.Getter; -import lombok.NoArgsConstructor; -import lombok.Setter; - -@AllArgsConstructor -@NoArgsConstructor -@Getter -@Setter -public class TxnID { - long txnIdMostBits; - long txnIdLeastBits; - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - TxnID txnID = (TxnID) o; - return txnIdMostBits == txnID.txnIdMostBits && txnIdLeastBits == txnID.txnIdLeastBits; - } - - @Override - public int hashCode() { - return Objects.hash(txnIdMostBits, txnIdLeastBits); - } -} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java index 244d7587a42e1..bba9a0422b430 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.systopic; +import static org.apache.pulsar.broker.systopic.TopicPoliciesSystemTopicClient.TopicPolicyWriter.getEventKey; import com.google.common.collect.Sets; import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -107,7 +108,7 @@ public void testSendAndReceiveNamespaceEvents() throws Exception { .policies(policies) .build()) .build(); - systemTopicClientForNamespace1.newWriter().write(event); + systemTopicClientForNamespace1.newWriter().write(event, getEventKey(event)); SystemTopicClient.Reader reader = systemTopicClientForNamespace1.newReader(); Message received = reader.readNext(); log.info("Receive pulsar event from system topic : {}", received.getValue()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 16ac60da7c46e..2cd562529712c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.transaction; +import static org.apache.pulsar.broker.systopic.TransactionBufferSnapshotSegmentSystemTopicClient.buildKey; import static org.apache.pulsar.common.naming.SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; @@ -56,10 +57,10 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; -import org.apache.pulsar.broker.service.SystemTopicBaseTxnBufferSnapshotService; +import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotServiceImpl; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotSegmentServiceImpl; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotServiceImpl; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; @@ -490,8 +491,8 @@ public void testTransactionBufferRecoverThrowException() throws Exception { PersistentTopic originalTopic = (PersistentTopic) getPulsarServiceList().get(0) .getBrokerService().getTopic(TopicName.get(topic).toString(), false).get().get(); - SystemTopicBaseTxnBufferSnapshotService systemTopicTxnBufferSnapshotService = - mock(SystemTopicBaseTxnBufferSnapshotService.class); + SystemTopicTxnBufferSnapshotServiceImpl systemTopicTxnBufferSnapshotService = + mock(SystemTopicTxnBufferSnapshotServiceImpl.class); SystemTopicClient.Reader reader = mock(SystemTopicClient.Reader.class); SystemTopicClient.Writer writer = mock(SystemTopicClient.Writer.class); @@ -592,7 +593,7 @@ public void testTransactionBufferNoSnapshotCloseReader() throws Exception{ @Test public void testTransactionBufferIndexSystemTopic() throws Exception { SystemTopicTxnBufferSnapshotService transactionBufferSnapshotIndexService = - new TransactionBufferSnapshotSegmentServiceImpl(pulsarClient).getTxnBufferSnapshotIndexService(); + new TransactionBufferSnapshotServiceImpl(pulsarClient, true).getTxnBufferSnapshotIndexService(); SystemTopicClient.Writer indexesWriter = transactionBufferSnapshotIndexService.createWriter(TopicName.get(SNAPSHOT_INDEX)).get(); @@ -611,7 +612,7 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { new TransactionBufferSnapshotIndexes(SNAPSHOT_INDEX, TransactionBufferSnapshotIndexes.Type.Indexes, indexList, null); - indexesWriter.write(transactionBufferTransactionBufferSnapshotIndexes); + indexesWriter.write(transactionBufferTransactionBufferSnapshotIndexes, SNAPSHOT_INDEX); assertTrue(indexesReader.hasMoreEvents()); transactionBufferTransactionBufferSnapshotIndexes = indexesReader.readNext().getValue(); @@ -650,7 +651,8 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { // create snapshot sgement writer SystemTopicTxnBufferSnapshotService transactionBufferSnapshotSegmentService = - new TransactionBufferSnapshotSegmentServiceImpl(pulsarClient).getTxnBufferSnapshotSegmentService(); + new TransactionBufferSnapshotServiceImpl(pulsarClient, true) + .getTxnBufferSnapshotSegmentService(); SystemTopicClient.Writer segmentWriter = transactionBufferSnapshotSegmentService.createWriter(snapshotSegmentTopicName).get(); @@ -665,12 +667,12 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { snapshot.setMaxReadPositionLedgerId(2L); snapshot.setMaxReadPositionEntryId(3L); snapshot.setAborts(Collections.singletonList( - new org.apache.pulsar.broker.transaction.buffer.matadata.v2.TxnID(1, 1))); + new TxnID(1, 1))); - segmentWriter.write(snapshot); + segmentWriter.write(snapshot, buildKey(snapshot)); snapshot.setSequenceId(2L); - MessageIdImpl messageId = (MessageIdImpl) segmentWriter.write(snapshot); + MessageIdImpl messageId = (MessageIdImpl) segmentWriter.write(snapshot, buildKey(snapshot)); //create read-only managed ledger CompletableFuture readOnlyManagedLedgerCompletableFuture = new CompletableFuture<>(); @@ -721,8 +723,7 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { assertEquals(snapshot.getSequenceId(), 2L); assertEquals(snapshot.getMaxReadPositionLedgerId(), 2L); assertEquals(snapshot.getMaxReadPositionEntryId(), 3L); - assertEquals(snapshot.getAborts().get(0), - new org.apache.pulsar.broker.transaction.buffer.matadata.v2.TxnID(1, 1)); + assertEquals(snapshot.getAborts().get(0), new TxnID(1, 1)); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index d3993c5a9471e..8bbb57c9a0acf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -77,7 +77,7 @@ import org.apache.pulsar.broker.service.BacklogQuotaManager; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; -import org.apache.pulsar.broker.service.SystemTopicBaseTxnBufferSnapshotService; +import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotServiceImpl; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -142,7 +142,9 @@ import org.powermock.reflect.Whitebox; import org.testng.Assert; import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; /** @@ -1432,8 +1434,8 @@ public Object answer(InvocationOnMock invocation) throws Throwable { when(pendingAckStoreProvider.newPendingAckStore(any())) .thenReturn(CompletableFuture.completedFuture(pendingAckStore)); // Mock TransactionBufferSnapshotService - SystemTopicBaseTxnBufferSnapshotService systemTopicTxnBufferSnapshotService - = mock(SystemTopicBaseTxnBufferSnapshotService.class); + SystemTopicTxnBufferSnapshotServiceImpl systemTopicTxnBufferSnapshotService + = mock(SystemTopicTxnBufferSnapshotServiceImpl.class); SystemTopicClient.Writer writer = mock(SystemTopicClient.Writer.class); when(writer.closeAsync()).thenReturn(CompletableFuture.completedFuture(null)); when(systemTopicTxnBufferSnapshotService.createWriter(any())) From 0626685107248fa021745f2e7d8cbd0c29d5d554 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Tue, 20 Sep 2022 18:44:08 +0800 Subject: [PATCH 10/27] merge config --- .../src/main/java/org/apache/pulsar/broker/PulsarService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 6857ffb1c8a91..1c3701b6ddd6f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -820,7 +820,8 @@ public void start() throws PulsarServerException { // Register pulsar system namespaces and start transaction meta store service if (config.isTransactionCoordinatorEnabled()) { - this.transactionBufferSnapshotService = new TransactionBufferSnapshotServiceImpl(getClient(), false); + this.transactionBufferSnapshotService = new TransactionBufferSnapshotServiceImpl(getClient(), + getConfig().isTransactionBufferSegmentedSnapshotEnabled()); this.transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-transaction-timer")); From b64343470be842a930e5d1e063355f50c8c25d7b Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Wed, 21 Sep 2022 13:19:42 +0800 Subject: [PATCH 11/27] optimize --- .../mledger/ManagedLedgerFactory.java | 6 +-- .../impl/ManagedLedgerFactoryImpl.java | 46 +++++++++---------- .../impl/ReadOnlyManagedLedgerImpl.java | 20 +++----- ...stemTopicTxnBufferSnapshotBaseService.java | 7 ++- ...opicTxnBufferSnapshotIndexServiceImpl.java | 5 +- ...icTxnBufferSnapshotSegmentServiceImpl.java | 4 +- .../SystemTopicTxnBufferSnapshotService.java | 2 +- ...stemTopicTxnBufferSnapshotServiceImpl.java | 5 +- .../TransactionBufferSnapshotServiceImpl.java | 9 ++-- .../broker/systopic/SystemTopicClient.java | 3 ++ ...onBufferSnapshotBaseSystemTopicClient.java | 3 +- .../TopicTransactionBufferRecoverTest.java | 19 +------- 12 files changed, 49 insertions(+), 80 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java index c94e3555de18a..9532a4cb85444 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java @@ -26,6 +26,7 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.ManagedLedgerInfoCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenLedgerCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenReadOnlyCursorCallback; +import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; /** @@ -120,11 +121,10 @@ void asyncOpenReadOnlyCursor(String managedLedgerName, Position startPosition, M * Asynchronous open a Read-only managedLedger. * @param managedLedgerName the unique name that identifies the managed ledger * @param config the managed ledegr configuratiion. - * @param callback callback object * @param ctx opaque context */ - void asyncOpenReadOnlyManagedLedger(String managedLedgerName, ManagedLedgerConfig config, - OpenLedgerCallback callback, Object ctx); + CompletableFuture asyncOpenReadOnlyManagedLedger(String managedLedgerName, + ManagedLedgerConfig config, Object ctx); /** * Get the current metadata info for a managed ledger. diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 630877e8e132c..55921154d16d7 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -419,11 +419,11 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { } @Override - public void asyncOpenReadOnlyManagedLedger(String managedLedgerName, ManagedLedgerConfig config, - OpenLedgerCallback callback, Object ctx) { + public CompletableFuture asyncOpenReadOnlyManagedLedger(String managedLedgerName, + ManagedLedgerConfig config, Object ctx) { + CompletableFuture future = new CompletableFuture<>(); if (closed) { - callback.openLedgerFailed(new ManagedLedgerException.ManagedLedgerFactoryClosedException(), ctx); - return; + return FutureUtil.failedFuture(new ManagedLedgerException.ManagedLedgerFactoryClosedException()); } ReadOnlyManagedLedgerImpl roManagedLedger = new ReadOnlyManagedLedgerImpl(this, bookkeeperFactory @@ -432,12 +432,13 @@ public void asyncOpenReadOnlyManagedLedger(String managedLedgerName, ManagedLedg store, config, scheduledExecutor, managedLedgerName); roManagedLedger.initialize().thenRun(() -> { log.info("[{}] Successfully initialize Read-only managed ledger", managedLedgerName); - callback.openLedgerComplete(roManagedLedger, ctx); + future.complete(roManagedLedger); }).exceptionally(e -> { log.error("[{}] Failed to initialize Read-only managed ledger", managedLedgerName, e); - callback.openLedgerFailed((ManagedLedgerException) e, ctx); + future.completeExceptionally(new ManagedLedgerException.ManagedLedgerFactoryClosedException()); return null; }); + return future; } @Override @@ -480,28 +481,23 @@ public void asyncOpenReadOnlyCursor(String managedLedgerName, Position startPosi return; } checkArgument(startPosition instanceof PositionImpl); - ReadOnlyManagedLedgerImpl roManagedLedger = new ReadOnlyManagedLedgerImpl(this, - bookkeeperFactory - .get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), - config.getBookKeeperEnsemblePlacementPolicyProperties())), - store, config, scheduledExecutor, managedLedgerName); - - roManagedLedger.initializeAndCreateCursor((PositionImpl) startPosition) - .thenAccept(roCursor -> callback.openReadOnlyCursorComplete(roCursor, ctx)) + asyncOpenReadOnlyManagedLedger(managedLedgerName, config, null).thenAccept(readOnlyManagedLedger -> + callback.openReadOnlyCursorComplete(readOnlyManagedLedger. + createReadOnlyCursor((PositionImpl) startPosition), ctx)) .exceptionally(ex -> { - Throwable t = ex; - if (t instanceof CompletionException) { - t = ex.getCause(); - } + Throwable t = ex; + if (t instanceof CompletionException) { + t = ex.getCause(); + } - if (t instanceof ManagedLedgerException) { - callback.openReadOnlyCursorFailed((ManagedLedgerException) t, ctx); - } else { - callback.openReadOnlyCursorFailed(new ManagedLedgerException(t), ctx); - } + if (t instanceof ManagedLedgerException) { + callback.openReadOnlyCursorFailed((ManagedLedgerException) t, ctx); + } else { + callback.openReadOnlyCursorFailed(new ManagedLedgerException(t), ctx); + } - return null; - }); + return null; + }); } void close(ManagedLedger ledger) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index 0be5b26829560..03379dfd82244 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -45,14 +45,8 @@ public ReadOnlyManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bo super(factory, bookKeeper, store, config, scheduledExecutor, name); } - CompletableFuture initializeAndCreateCursor(PositionImpl startPosition) { - CompletableFuture future = new CompletableFuture<>(); - initialize().thenRun(() -> future.complete(createReadOnlyCursor(startPosition))); - return future; - } - - CompletableFuture initialize() { - CompletableFuture future = new CompletableFuture<>(); + CompletableFuture initialize() { + CompletableFuture future = new CompletableFuture<>(); // Fetch the list of existing ledgers in the managed ledger store.getManagedLedgerInfo(name, false, new MetaStoreCallback() { @@ -78,7 +72,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { .setTimestamp(clock.millis()).build(); ledgers.put(lastLedgerId, info); - future.complete(null); + future.complete(ReadOnlyManagedLedgerImpl.this); }).exceptionally(ex -> { if (ex instanceof CompletionException && ex.getCause() instanceof IllegalArgumentException) { @@ -86,7 +80,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(lastLedgerId) .setEntries(0).setSize(0).setTimestamp(clock.millis()).build(); ledgers.put(lastLedgerId, info); - future.complete(null); + future.complete(ReadOnlyManagedLedgerImpl.this); } else { future.completeExceptionally(new ManagedLedgerException(ex)); } @@ -99,7 +93,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(lastLedgerId).setEntries(0) .setSize(0).setTimestamp(clock.millis()).build(); ledgers.put(lastLedgerId, info); - future.complete(null); + future.complete(ReadOnlyManagedLedgerImpl.this); } else { future.completeExceptionally(new ManagedLedgerException(ex)); } @@ -107,7 +101,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { }); } else { // The read-only managed ledger is ready to use - future.complete(null); + future.complete(ReadOnlyManagedLedgerImpl.this); } } @@ -124,7 +118,7 @@ public void operationFailed(MetaStoreException e) { return future; } - private ReadOnlyCursor createReadOnlyCursor(PositionImpl startPosition) { + ReadOnlyCursor createReadOnlyCursor(PositionImpl startPosition) { if (ledgers.isEmpty()) { lastConfirmedEntry = PositionImpl.EARLIEST; } else if (ledgers.lastEntry().getValue().getEntries() > 0) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotBaseService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotBaseService.java index 9a628736aeab9..d7e0715296032 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotBaseService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotBaseService.java @@ -21,22 +21,25 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; 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.SystemTopicClientBase; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.common.naming.TopicName; public abstract class SystemTopicTxnBufferSnapshotBaseService implements SystemTopicTxnBufferSnapshotService { protected final Map> clients; + protected final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; - public SystemTopicTxnBufferSnapshotBaseService() { + public SystemTopicTxnBufferSnapshotBaseService(PulsarClient client) { + this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); this.clients = new ConcurrentHashMap<>(); } @Override public CompletableFuture> createWriter(TopicName topicName) { return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync); - } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotIndexServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotIndexServiceImpl.java index 667ca43dfd79c..7e2015324ea45 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotIndexServiceImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotIndexServiceImpl.java @@ -31,11 +31,8 @@ public class SystemTopicTxnBufferSnapshotIndexServiceImpl extends SystemTopicTxnBufferSnapshotBaseService { - private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; - public SystemTopicTxnBufferSnapshotIndexServiceImpl(PulsarClient client) { - super(); - this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); + super(client); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotSegmentServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotSegmentServiceImpl.java index 9a710df65c1fc..fb22da69aea8e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotSegmentServiceImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotSegmentServiceImpl.java @@ -30,11 +30,9 @@ public class SystemTopicTxnBufferSnapshotSegmentServiceImpl extends SystemTopicTxnBufferSnapshotBaseService { - private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; public SystemTopicTxnBufferSnapshotSegmentServiceImpl(PulsarClient client) { - super(); - this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); + super(client); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java index ad2a7dbf36a2c..f62314b15ac07 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java @@ -25,7 +25,7 @@ import org.apache.pulsar.broker.systopic.TransactionBufferSnapshotBaseSystemTopicClient; import org.apache.pulsar.common.naming.TopicName; -public interface SystemTopicTxnBufferSnapshotService { +public interface SystemTopicTxnBufferSnapshotService { /** * Create a transaction buffer snapshot writer. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotServiceImpl.java index 0bd24543c0aa9..adfba8edace47 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotServiceImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotServiceImpl.java @@ -31,11 +31,8 @@ public class SystemTopicTxnBufferSnapshotServiceImpl extends SystemTopicTxnBufferSnapshotBaseService { - private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; - public SystemTopicTxnBufferSnapshotServiceImpl(PulsarClient client) { - super(); - this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); + super(client); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java index f01d29b094cfb..e93e5d341af2d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java @@ -31,12 +31,9 @@ public class TransactionBufferSnapshotServiceImpl implements TransactionBufferSn public TransactionBufferSnapshotServiceImpl(PulsarClient pulsarClient, boolean transactionBufferSegmentedSnapshotEnabled) { - if (transactionBufferSegmentedSnapshotEnabled) { - this.txnBufferSnapshotSegmentService = new SystemTopicTxnBufferSnapshotSegmentServiceImpl(pulsarClient); - this.txnBufferSnapshotIndexService = new SystemTopicTxnBufferSnapshotIndexServiceImpl(pulsarClient); - } else { - this.txnBufferSnapshotService = new SystemTopicTxnBufferSnapshotServiceImpl(pulsarClient); - } + this.txnBufferSnapshotSegmentService = new SystemTopicTxnBufferSnapshotSegmentServiceImpl(pulsarClient); + this.txnBufferSnapshotIndexService = new SystemTopicTxnBufferSnapshotIndexServiceImpl(pulsarClient); + this.txnBufferSnapshotService = new SystemTopicTxnBufferSnapshotServiceImpl(pulsarClient); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java index 246c5a8c194e9..be494a4299902 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java @@ -87,9 +87,11 @@ public interface SystemTopicClient { * Writer for system topic. */ interface Writer { + /** * Write event to the system topic. * @param t pulsar event + * @param key the key of the event * @return message id * @throws PulsarClientException exception while write event cause */ @@ -98,6 +100,7 @@ interface Writer { /** * Async write event to the system topic. * @param t pulsar event + * @param key the key of the event * @return message id future */ CompletableFuture writeAsync(T t, String key); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index 12f64e21d82e2..84b565ad32f4b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -28,8 +28,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.naming.TopicName; -public abstract class TransactionBufferSnapshotBaseSystemTopicClient extends - SystemTopicClientBase { +public abstract class TransactionBufferSnapshotBaseSystemTopicClient extends SystemTopicClientBase { protected final SystemTopicTxnBufferSnapshotService systemTopicTxnBufferSnapshotService; public TransactionBufferSnapshotBaseSystemTopicClient(PulsarClient client, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 2cd562529712c..a9ac712a6bc47 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -675,24 +675,9 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { MessageIdImpl messageId = (MessageIdImpl) segmentWriter.write(snapshot, buildKey(snapshot)); //create read-only managed ledger - CompletableFuture readOnlyManagedLedgerCompletableFuture = new CompletableFuture<>(); - pulsarService.getManagedLedgerFactory() + ReadOnlyManagedLedgerImpl readOnlyManagedLedger = pulsarService.getManagedLedgerFactory() .asyncOpenReadOnlyManagedLedger(snapshotSegmentTopicName.getPersistenceNamingEncoding(), - brokerService.getManagedLedgerConfig(snapshotSegmentTopicName).get(), - new AsyncCallbacks.OpenLedgerCallback() { - @Override - public void openLedgerComplete(ManagedLedger ledger, Object ctx) { - readOnlyManagedLedgerCompletableFuture.complete((ReadOnlyManagedLedgerImpl) ledger); - } - - @Override - public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { - readOnlyManagedLedgerCompletableFuture.completeExceptionally(exception); - } - }, null); - - - ReadOnlyManagedLedgerImpl readOnlyManagedLedger = readOnlyManagedLedgerCompletableFuture.get(); + brokerService.getManagedLedgerConfig(snapshotSegmentTopicName).get(),null).get(); //read the entry and decode entry to snapshot CompletableFuture entryCompletableFuture = new CompletableFuture<>(); From 8cf7ee0cb14ff1f5ef4ef96d3e57ff72724096dd Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Thu, 22 Sep 2022 17:39:01 +0800 Subject: [PATCH 12/27] fix --- ...onBufferSnapshotBaseSystemTopicClient.java | 2 +- ...nBufferSnapshotIndexSystemTopicClient.java | 28 ++----------------- ...ufferSnapshotSegmentSystemTopicClient.java | 13 +-------- ...actionBufferSnapshotSystemTopicClient.java | 23 ++------------- 4 files changed, 6 insertions(+), 60 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index 84b565ad32f4b..f79f173dd7519 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -50,7 +50,7 @@ protected void removeReader(Reader reader) { this.systemTopicTxnBufferSnapshotService.removeClient(topicName, this); } - protected abstract static class TransactionBufferSnapshotBaseWriter implements Writer { + protected static class TransactionBufferSnapshotBaseWriter implements Writer { protected final Producer producer; protected final TransactionBufferSnapshotBaseSystemTopicClient diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java index bc61bff074c1c..4e7337c1f14f1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java @@ -24,7 +24,6 @@ import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; 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.Schema; import org.apache.pulsar.common.naming.TopicName; @@ -33,14 +32,10 @@ public class TransactionBufferSnapshotIndexSystemTopicClient extends TransactionBufferSnapshotBaseSystemTopicClient { - private final SystemTopicTxnBufferSnapshotService - transactionBufferSnapshotIndexService; - public TransactionBufferSnapshotIndexSystemTopicClient(PulsarClient client, TopicName topicName, SystemTopicTxnBufferSnapshotService transactionBufferSnapshotIndexService) { super(client, topicName, transactionBufferSnapshotIndexService); - this.transactionBufferSnapshotIndexService = transactionBufferSnapshotIndexService; } @Override @@ -52,7 +47,7 @@ protected CompletableFuture> newWriterA log.debug("[{}] A new transactionBufferSnapshot writer is created", topicName); } return CompletableFuture.completedFuture( - new TransactionBufferSnapshotIndexWriter(producer, this)); + new TransactionBufferSnapshotBaseWriter<>(producer, this)); }); } @@ -68,28 +63,9 @@ protected CompletableFuture> newReaderA log.debug("[{}] A new transactionBufferSnapshot buffer reader is created", topicName); } return CompletableFuture.completedFuture( - new TransactionBufferSnapshotIndexReader(reader, this)); + new TransactionBufferSnapshotBaseReader<>(reader, this)); }); } - private static class TransactionBufferSnapshotIndexWriter extends - TransactionBufferSnapshotBaseWriter { - - - private TransactionBufferSnapshotIndexWriter(Producer producer, - TransactionBufferSnapshotIndexSystemTopicClient - transactionBufferSnapshotIndexSystemTopicClient) { - super(producer, transactionBufferSnapshotIndexSystemTopicClient); - } - } - - private static class TransactionBufferSnapshotIndexReader extends TransactionBufferSnapshotBaseSystemTopicClient - .TransactionBufferSnapshotBaseReader { - private TransactionBufferSnapshotIndexReader( - org.apache.pulsar.client.api.Reader reader, - TransactionBufferSnapshotIndexSystemTopicClient transactionBufferSnapshotIndexSystemTopicClient) { - super(reader, transactionBufferSnapshotIndexSystemTopicClient); - } - } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java index a0cf81e64d5b9..a7bcb580d3dd2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java @@ -23,7 +23,6 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; -import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.naming.TopicName; @@ -50,7 +49,7 @@ public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client log.debug("[{}] A new transactionBufferSnapshot segment writer is created", topicName); } return CompletableFuture.completedFuture( - new TransactionBufferSnapshotSegmentWriter(producer, this)); + new TransactionBufferSnapshotBaseWriter<>(producer, this)); }); } @@ -66,14 +65,4 @@ public static String buildKey(TransactionBufferSnapshotIndexes.TransactionBuffer } - private static class TransactionBufferSnapshotSegmentWriter extends TransactionBufferSnapshotBaseSystemTopicClient - .TransactionBufferSnapshotBaseWriter { - - private TransactionBufferSnapshotSegmentWriter(Producer producer, TransactionBufferSnapshotSegmentSystemTopicClient - transactionBufferSnapshotSegmentSystemTopicClient) { - super(producer, transactionBufferSnapshotSegmentSystemTopicClient); - } - } - } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java index ad0fff51c7cd7..f21a1bd91da7f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java @@ -24,7 +24,6 @@ import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; 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.Schema; import org.apache.pulsar.common.naming.TopicName; @@ -32,7 +31,6 @@ @Slf4j public class TransactionBufferSnapshotSystemTopicClient extends TransactionBufferSnapshotBaseSystemTopicClient { - private SystemTopicTxnBufferSnapshotService systemTopicTxnBufferSnapshotService; public TransactionBufferSnapshotSystemTopicClient(PulsarClient client, TopicName topicName, SystemTopicTxnBufferSnapshotService @@ -49,7 +47,7 @@ protected CompletableFuture> newWriterAsyncInt log.debug("[{}] A new transactionBufferSnapshot writer is created", topicName); } return CompletableFuture.completedFuture( - new TransactionBufferSnapshotWriter(producer, this)); + new TransactionBufferSnapshotBaseWriter(producer, this)); }); } @@ -65,25 +63,8 @@ protected CompletableFuture> newReaderAsyncInt log.debug("[{}] A new transactionBufferSnapshot buffer reader is created", topicName); } return CompletableFuture.completedFuture( - new TransactionBufferSnapshotReader(reader, this)); + new TransactionBufferSnapshotBaseReader<>(reader, this)); }); } - - private static class TransactionBufferSnapshotWriter extends - TransactionBufferSnapshotBaseWriter { - - private TransactionBufferSnapshotWriter(Producer producer, - TransactionBufferSnapshotSystemTopicClient transactionBufferSnapshotSystemTopicClient) { - super(producer, transactionBufferSnapshotSystemTopicClient); - } - } - - private static class TransactionBufferSnapshotReader extends - TransactionBufferSnapshotBaseReader { - private TransactionBufferSnapshotReader(org.apache.pulsar.client.api.Reader reader, - TransactionBufferSnapshotSystemTopicClient transactionBufferSnapshotSystemTopicClient) { - super(reader, transactionBufferSnapshotSystemTopicClient); - } - } } From 78492062e8c0dd05b6c9e257cd0545af7d21dd30 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Thu, 22 Sep 2022 17:44:08 +0800 Subject: [PATCH 13/27] fix --- .../broker/service/TransactionBufferSnapshotService.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java index 70fb6067c16e0..d2f79211fb494 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java @@ -23,19 +23,19 @@ public interface TransactionBufferSnapshotService { /** * Get the systemTopicBaseTxnBufferSnapshotIndexService. - * @return systemTopicBaseTxnBufferSnapshotIndexService + * @return the systemTopicBaseTxnBufferSnapshotIndexService */ SystemTopicTxnBufferSnapshotIndexServiceImpl getTxnBufferSnapshotIndexService (); /** * Get the systemTopicBaseTxnBufferSnapshotSegmentService. - * @return systemTopicBaseTxnBufferSnapshotSegmentService + * @return the systemTopicBaseTxnBufferSnapshotSegmentService */ SystemTopicTxnBufferSnapshotSegmentServiceImpl getTxnBufferSnapshotSegmentService(); /** * Get the systemTopicBaseTxnBufferSnapshotService. - * @return + * @return the systemTopicTxnBufferSnapshotServiceImpl */ SystemTopicTxnBufferSnapshotServiceImpl getTxnBufferSnapshotService(); From 2737664673a452f1f3f3bbb00102a1ae2da67da6 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Thu, 22 Sep 2022 17:52:29 +0800 Subject: [PATCH 14/27] fix --- ...ransactionBufferSnapshotBaseSystemTopicClient.java | 11 +++++------ ...ansactionBufferSnapshotIndexSystemTopicClient.java | 4 ++-- ...sactionBufferSnapshotSegmentSystemTopicClient.java | 2 +- .../TransactionBufferSnapshotSystemTopicClient.java | 4 ++-- .../matadata/v2/TransactionBufferSnapshotIndexes.java | 7 ------- 5 files changed, 10 insertions(+), 18 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index f79f173dd7519..4f8dfeede2670 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -39,7 +39,6 @@ public TransactionBufferSnapshotBaseSystemTopicClient(PulsarClient client, this.systemTopicTxnBufferSnapshotService = systemTopicTxnBufferSnapshotService; } - protected void removeWriter(Writer writer) { writers.remove(writer); this.systemTopicTxnBufferSnapshotService.removeClient(topicName, this); @@ -50,14 +49,14 @@ protected void removeReader(Reader reader) { this.systemTopicTxnBufferSnapshotService.removeClient(topicName, this); } - protected static class TransactionBufferSnapshotBaseWriter implements Writer { + protected static class TransactionBufferSnapshotWriter implements Writer { protected final Producer producer; protected final TransactionBufferSnapshotBaseSystemTopicClient transactionBufferSnapshotBaseSystemTopicClient; - protected TransactionBufferSnapshotBaseWriter(Producer producer, - TransactionBufferSnapshotBaseSystemTopicClient + protected TransactionBufferSnapshotWriter(Producer producer, + TransactionBufferSnapshotBaseSystemTopicClient transactionBufferSnapshotBaseSystemTopicClient) { this.producer = producer; this.transactionBufferSnapshotBaseSystemTopicClient = transactionBufferSnapshotBaseSystemTopicClient; @@ -121,12 +120,12 @@ public SystemTopicClient getSystemTopicClient() { } } - protected static class TransactionBufferSnapshotBaseReader implements Reader { + protected static class TransactionBufferSnapshotReader implements Reader { private final org.apache.pulsar.client.api.Reader reader; private final TransactionBufferSnapshotBaseSystemTopicClient transactionBufferSnapshotBaseSystemTopicClient; - protected TransactionBufferSnapshotBaseReader( + protected TransactionBufferSnapshotReader( org.apache.pulsar.client.api.Reader reader, TransactionBufferSnapshotBaseSystemTopicClient transactionBufferSnapshotBaseSystemTopicClient) { this.reader = reader; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java index 4e7337c1f14f1..17862b942ed2d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java @@ -47,7 +47,7 @@ protected CompletableFuture> newWriterA log.debug("[{}] A new transactionBufferSnapshot writer is created", topicName); } return CompletableFuture.completedFuture( - new TransactionBufferSnapshotBaseWriter<>(producer, this)); + new TransactionBufferSnapshotWriter<>(producer, this)); }); } @@ -63,7 +63,7 @@ protected CompletableFuture> newReaderA log.debug("[{}] A new transactionBufferSnapshot buffer reader is created", topicName); } return CompletableFuture.completedFuture( - new TransactionBufferSnapshotBaseReader<>(reader, this)); + new TransactionBufferSnapshotReader<>(reader, this)); }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java index a7bcb580d3dd2..b83685186dccc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java @@ -49,7 +49,7 @@ public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client log.debug("[{}] A new transactionBufferSnapshot segment writer is created", topicName); } return CompletableFuture.completedFuture( - new TransactionBufferSnapshotBaseWriter<>(producer, this)); + new TransactionBufferSnapshotWriter<>(producer, this)); }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java index f21a1bd91da7f..cc099410f2378 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java @@ -47,7 +47,7 @@ protected CompletableFuture> newWriterAsyncInt log.debug("[{}] A new transactionBufferSnapshot writer is created", topicName); } return CompletableFuture.completedFuture( - new TransactionBufferSnapshotBaseWriter(producer, this)); + new TransactionBufferSnapshotWriter(producer, this)); }); } @@ -63,7 +63,7 @@ protected CompletableFuture> newReaderAsyncInt log.debug("[{}] A new transactionBufferSnapshot buffer reader is created", topicName); } return CompletableFuture.completedFuture( - new TransactionBufferSnapshotBaseReader<>(reader, this)); + new TransactionBufferSnapshotReader<>(reader, this)); }); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java index 97cb64c20404a..9b4bd4ba10cf4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java @@ -33,15 +33,8 @@ @Setter @Builder public class TransactionBufferSnapshotIndexes { - public enum Type { - Indexes, - UnsealedSnapshot - } - private String topicName; - private Type type; - private List indexList; private TransactionBufferSnapshot snapshot; From f9dc0ae2e68b34f619a02abb48294e1dfe18e270 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Mon, 17 Oct 2022 15:16:08 +0800 Subject: [PATCH 15/27] optimize by class --- .../apache/pulsar/broker/PulsarService.java | 13 ++- ...stemTopicTxnBufferSnapshotBaseService.java | 68 ------------- ...opicTxnBufferSnapshotIndexServiceImpl.java | 55 ----------- ...icTxnBufferSnapshotSegmentServiceImpl.java | 57 ----------- .../SystemTopicTxnBufferSnapshotService.java | 95 ++++++++++++------- ...stemTopicTxnBufferSnapshotServiceImpl.java | 52 ---------- .../TransactionBufferSnapshotService.java | 48 ---------- ...nsactionBufferSnapshotServiceFactory.java} | 35 ++++--- .../NamespaceEventsSystemTopicFactory.java | 32 +------ ...onBufferSnapshotBaseSystemTopicClient.java | 45 ++++++++- ...nBufferSnapshotIndexSystemTopicClient.java | 71 -------------- ...ufferSnapshotSegmentSystemTopicClient.java | 68 ------------- ...actionBufferSnapshotSystemTopicClient.java | 70 -------------- .../TopicTransactionBufferRecoverTest.java | 66 +++++++------ .../broker/transaction/TransactionTest.java | 20 ++-- 15 files changed, 171 insertions(+), 624 deletions(-) delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotBaseService.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotIndexServiceImpl.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotSegmentServiceImpl.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotServiceImpl.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java rename pulsar-broker/src/main/java/org/apache/pulsar/broker/service/{TransactionBufferSnapshotServiceImpl.java => TransactionBufferSnapshotServiceFactory.java} (55%) delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 1c3701b6ddd6f..c1757ec46b402 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -101,8 +101,7 @@ import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotServiceImpl; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotServiceFactory; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.SchemaStorageFactory; import org.apache.pulsar.broker.stats.MetricsGenerator; @@ -258,7 +257,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private MetadataStoreExtended localMetadataStore; private PulsarMetadataEventSynchronizer localMetadataSynchronizer; private CoordinationService coordinationService; - private TransactionBufferSnapshotService transactionBufferSnapshotService; + private TransactionBufferSnapshotServiceFactory transactionBufferSnapshotServiceFactory; private MetadataStore configurationMetadataStore; private PulsarMetadataEventSynchronizer configMetadataSynchronizer; private boolean shouldShutdownConfigurationMetadataStore; @@ -498,9 +497,9 @@ public CompletableFuture closeAsync() { adminClient = null; } - if (transactionBufferSnapshotService != null) { - transactionBufferSnapshotService.close(); - transactionBufferSnapshotService = null; + if (transactionBufferSnapshotServiceFactory != null) { + transactionBufferSnapshotServiceFactory.close(); + transactionBufferSnapshotServiceFactory = null; } if (transactionBufferClient != null) { @@ -820,7 +819,7 @@ public void start() throws PulsarServerException { // Register pulsar system namespaces and start transaction meta store service if (config.isTransactionCoordinatorEnabled()) { - this.transactionBufferSnapshotService = new TransactionBufferSnapshotServiceImpl(getClient(), + this.transactionBufferSnapshotServiceFactory = new TransactionBufferSnapshotServiceFactory(getClient(), getConfig().isTransactionBufferSegmentedSnapshotEnabled()); this.transactionTimer = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotBaseService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotBaseService.java deleted file mode 100644 index d7e0715296032..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotBaseService.java +++ /dev/null @@ -1,68 +0,0 @@ -/** - * 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 java.util.Map; -import java.util.concurrent.CompletableFuture; -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.SystemTopicClientBase; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.common.naming.TopicName; - -public abstract class SystemTopicTxnBufferSnapshotBaseService implements SystemTopicTxnBufferSnapshotService { - - protected final Map> clients; - protected final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; - - public SystemTopicTxnBufferSnapshotBaseService(PulsarClient client) { - this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); - this.clients = new ConcurrentHashMap<>(); - } - - @Override - public CompletableFuture> createWriter(TopicName topicName) { - return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync); - } - - @Override - public CompletableFuture> createReader(TopicName topicName) { - return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newReaderAsync); - } - - @Override - public void removeClient(TopicName topicName, SystemTopicClientBase transactionBufferSystemTopicClient) { - if (transactionBufferSystemTopicClient.getReaders().size() == 0 - && transactionBufferSystemTopicClient.getWriters().size() == 0) { - clients.remove(topicName); - } - } - - @Override - public void close() throws Exception { - for (Map.Entry> entry : clients.entrySet()) { - entry.getValue().close(); - } - } - - protected abstract CompletableFuture> getTransactionBufferSystemTopicClient( - TopicName topicName); - -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotIndexServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotIndexServiceImpl.java deleted file mode 100644 index 7e2015324ea45..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotIndexServiceImpl.java +++ /dev/null @@ -1,55 +0,0 @@ -/** - * 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 java.util.concurrent.CompletableFuture; -import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; -import org.apache.pulsar.broker.systopic.SystemTopicClient; -import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; -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.common.naming.TopicName; -import org.apache.pulsar.common.util.FutureUtil; - -public class SystemTopicTxnBufferSnapshotIndexServiceImpl extends - SystemTopicTxnBufferSnapshotBaseService { - - public SystemTopicTxnBufferSnapshotIndexServiceImpl(PulsarClient client) { - super(client); - } - - @Override - protected CompletableFuture> - getTransactionBufferSystemTopicClient( - TopicName topicName) { - TopicName systemTopicName = NamespaceEventsSystemTopicFactory - .getSystemTopicName(topicName.getNamespaceObject(), EventType.TRANSACTION_BUFFER_SNAPSHOT); - if (systemTopicName == null) { - return FutureUtil.failedFuture( - new InvalidTopicNameException("Can't create SystemTopicBaseTxnBufferSnapshotIndexService, " - + "because the topicName is null!")); - } - return CompletableFuture.completedFuture(clients.computeIfAbsent(systemTopicName, - (v) -> namespaceEventsSystemTopicFactory - .createTransactionBufferSnapshotIndexSystemTopicClient(topicName.getNamespaceObject(), - this))); - } - -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotSegmentServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotSegmentServiceImpl.java deleted file mode 100644 index fb22da69aea8e..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotSegmentServiceImpl.java +++ /dev/null @@ -1,57 +0,0 @@ -/** - * 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 java.util.concurrent.CompletableFuture; -import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; -import org.apache.pulsar.broker.systopic.SystemTopicClient; -import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; -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.TopicName; -import org.apache.pulsar.common.util.FutureUtil; - -public class SystemTopicTxnBufferSnapshotSegmentServiceImpl extends - SystemTopicTxnBufferSnapshotBaseService { - - public SystemTopicTxnBufferSnapshotSegmentServiceImpl(PulsarClient client) { - super(client); - } - - @Override - protected CompletableFuture> - getTransactionBufferSystemTopicClient( - TopicName topicName) { - TopicName systemTopicName = NamespaceEventsSystemTopicFactory - .getSystemTopicName(topicName.getNamespaceObject(), EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT); - if (systemTopicName == null) { - return FutureUtil.failedFuture( - new PulsarClientException.InvalidTopicNameException( - "Can't create SystemTopicBaseTxnBufferSnapshotSegmentService, " - + "because the topicName is null!")); - } - return CompletableFuture.completedFuture(clients.computeIfAbsent(systemTopicName, - (v) -> namespaceEventsSystemTopicFactory - .createTransactionBufferSnapshotSegmentSystemTopicClient(topicName.getNamespaceObject(), - this))); - } - - -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java index f62314b15ac07..9dc1fa1b8a325 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java @@ -18,45 +18,68 @@ */ package org.apache.pulsar.broker.service; +import java.util.Map; import java.util.concurrent.CompletableFuture; -import org.apache.pulsar.broker.systopic.SystemTopicClient.Reader; -import org.apache.pulsar.broker.systopic.SystemTopicClient.Writer; +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.SystemTopicClientBase; -import org.apache.pulsar.broker.systopic.TransactionBufferSnapshotBaseSystemTopicClient; +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.TopicName; +import org.apache.pulsar.common.util.FutureUtil; -public interface SystemTopicTxnBufferSnapshotService { - - /** - * Create a transaction buffer snapshot writer. - * - * @param topicName {@link TopicName} the topic name - * - * @return {@link CompletableFuture} return the future of writer - */ - CompletableFuture> createWriter(TopicName topicName); - - /** - * Create a transaction buffer snapshot reader. - * - * @param topicName {@link TopicName} the topic name - * - * @return {@link CompletableFuture} return the future of reader - */ - CompletableFuture> createReader(TopicName topicName); - - /** - * Remove a topic client from cache. - * - * @param topicName {@link TopicName} the topic name - * @param transactionBufferSnapshotBaseSystemTopicClient {@link TransactionBufferSnapshotBaseSystemTopicClient} - * the topic client - */ - void removeClient(TopicName topicName, SystemTopicClientBase transactionBufferSnapshotBaseSystemTopicClient); - - /** - * Close transaction buffer snapshot service. - */ - void close() throws Exception; +public class SystemTopicTxnBufferSnapshotService { + + protected final Map> clients; + protected final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; + + protected final Class schemaType; + protected final EventType systemTopicType; + + public SystemTopicTxnBufferSnapshotService(PulsarClient client, EventType systemTopicType, + Class schemaType) { + this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); + this.systemTopicType = systemTopicType; + this.schemaType = schemaType; + this.clients = new ConcurrentHashMap<>(); + } + + public CompletableFuture> createWriter(TopicName topicName) { + return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync); + } + + public CompletableFuture> createReader(TopicName topicName) { + return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newReaderAsync); + } + + public void removeClient(TopicName topicName, SystemTopicClientBase transactionBufferSystemTopicClient) { + if (transactionBufferSystemTopicClient.getReaders().size() == 0 + && transactionBufferSystemTopicClient.getWriters().size() == 0) { + clients.remove(topicName); + } + } + + protected CompletableFuture> getTransactionBufferSystemTopicClient(TopicName topicName) { + TopicName systemTopicName = NamespaceEventsSystemTopicFactory + .getSystemTopicName(topicName.getNamespaceObject(), systemTopicType); + if (systemTopicName == null) { + return FutureUtil.failedFuture( + new PulsarClientException + .InvalidTopicNameException("Can't create SystemTopicBaseTxnBufferSnapshotIndexService, " + + "because the topicName is null!")); + } + return CompletableFuture.completedFuture(clients.computeIfAbsent(systemTopicName, + (v) -> namespaceEventsSystemTopicFactory + .createTransactionBufferSystemTopicClient(topicName.getNamespaceObject(), + this, schemaType))); + } + + public void close() throws Exception { + for (Map.Entry> entry : clients.entrySet()) { + entry.getValue().close(); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotServiceImpl.java deleted file mode 100644 index adfba8edace47..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotServiceImpl.java +++ /dev/null @@ -1,52 +0,0 @@ -/** - * 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 java.util.concurrent.CompletableFuture; -import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; -import org.apache.pulsar.broker.systopic.SystemTopicClient; -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.common.naming.TopicName; -import org.apache.pulsar.common.util.FutureUtil; - -public class SystemTopicTxnBufferSnapshotServiceImpl - extends SystemTopicTxnBufferSnapshotBaseService { - - public SystemTopicTxnBufferSnapshotServiceImpl(PulsarClient client) { - super(client); - } - - @Override - protected CompletableFuture> 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))); - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java deleted file mode 100644 index d2f79211fb494..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java +++ /dev/null @@ -1,48 +0,0 @@ -/** - * 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; - -public interface TransactionBufferSnapshotService { - - /** - * Get the systemTopicBaseTxnBufferSnapshotIndexService. - * @return the systemTopicBaseTxnBufferSnapshotIndexService - */ - SystemTopicTxnBufferSnapshotIndexServiceImpl getTxnBufferSnapshotIndexService (); - - /** - * Get the systemTopicBaseTxnBufferSnapshotSegmentService. - * @return the systemTopicBaseTxnBufferSnapshotSegmentService - */ - SystemTopicTxnBufferSnapshotSegmentServiceImpl getTxnBufferSnapshotSegmentService(); - - /** - * Get the systemTopicBaseTxnBufferSnapshotService. - * @return the systemTopicTxnBufferSnapshotServiceImpl - */ - SystemTopicTxnBufferSnapshotServiceImpl getTxnBufferSnapshotService(); - - /** - * Close the systemTopicBaseTxnBufferSnapshotxxService in TransactionBufferSnapshotService . - * @throws Exception - */ - void close() throws Exception; - -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java similarity index 55% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java index e93e5d341af2d..e32a13de5bf4b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java @@ -19,39 +19,44 @@ 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 TransactionBufferSnapshotServiceImpl implements TransactionBufferSnapshotService { +public class TransactionBufferSnapshotServiceFactory { - private SystemTopicTxnBufferSnapshotServiceImpl txnBufferSnapshotService; + private SystemTopicTxnBufferSnapshotService txnBufferSnapshotService; - private SystemTopicTxnBufferSnapshotSegmentServiceImpl txnBufferSnapshotSegmentService; + private SystemTopicTxnBufferSnapshotService + txnBufferSnapshotSegmentService; - private SystemTopicTxnBufferSnapshotIndexServiceImpl txnBufferSnapshotIndexService; + private SystemTopicTxnBufferSnapshotService txnBufferSnapshotIndexService; - public TransactionBufferSnapshotServiceImpl(PulsarClient pulsarClient, + public TransactionBufferSnapshotServiceFactory(PulsarClient pulsarClient, boolean transactionBufferSegmentedSnapshotEnabled) { - this.txnBufferSnapshotSegmentService = new SystemTopicTxnBufferSnapshotSegmentServiceImpl(pulsarClient); - this.txnBufferSnapshotIndexService = new SystemTopicTxnBufferSnapshotIndexServiceImpl(pulsarClient); - this.txnBufferSnapshotService = new SystemTopicTxnBufferSnapshotServiceImpl(pulsarClient); + this.txnBufferSnapshotSegmentService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, + EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT, + TransactionBufferSnapshotIndexes.TransactionBufferSnapshot.class); + this.txnBufferSnapshotIndexService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, + EventType.TRANSACTION_BUFFER_SNAPSHOT_INDEXES, TransactionBufferSnapshotIndexes.class); + this.txnBufferSnapshotService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, + EventType.TRANSACTION_BUFFER_SNAPSHOT, TransactionBufferSnapshot.class); } - @Override - public SystemTopicTxnBufferSnapshotIndexServiceImpl getTxnBufferSnapshotIndexService() { + public SystemTopicTxnBufferSnapshotService getTxnBufferSnapshotIndexService() { return this.txnBufferSnapshotIndexService; } - @Override - public SystemTopicTxnBufferSnapshotSegmentServiceImpl getTxnBufferSnapshotSegmentService() { + public SystemTopicTxnBufferSnapshotService + getTxnBufferSnapshotSegmentService() { return this.txnBufferSnapshotSegmentService; } - @Override - public SystemTopicTxnBufferSnapshotServiceImpl getTxnBufferSnapshotService() { + public SystemTopicTxnBufferSnapshotService getTxnBufferSnapshotService() { return this.txnBufferSnapshotService; } - @Override public void close() throws Exception { if (this.txnBufferSnapshotIndexService != null) { this.txnBufferSnapshotIndexService.close(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java index fe82fddead720..db51fb2763442 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java @@ -19,8 +19,6 @@ package org.apache.pulsar.broker.systopic; import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; -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; import org.apache.pulsar.common.naming.NamespaceName; @@ -45,34 +43,14 @@ public TopicPoliciesSystemTopicClient createTopicPoliciesSystemTopicClient(Names return new TopicPoliciesSystemTopicClient(client, topicName); } - public TransactionBufferSnapshotSystemTopicClient createTransactionBufferSystemTopicClient( - NamespaceName namespaceName, SystemTopicTxnBufferSnapshotService - systemTopicTxnBufferSnapshotService) { + public TransactionBufferSnapshotBaseSystemTopicClient createTransactionBufferSystemTopicClient( + NamespaceName namespaceName, SystemTopicTxnBufferSnapshotService + systemTopicTxnBufferSnapshotService, Class schemaType) { TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); log.info("Create transaction buffer snapshot client, topicName : {}", topicName.toString()); - return new TransactionBufferSnapshotSystemTopicClient(client, topicName, systemTopicTxnBufferSnapshotService); - } - - public TransactionBufferSnapshotIndexSystemTopicClient createTransactionBufferSnapshotIndexSystemTopicClient( - NamespaceName namespaceName, SystemTopicTxnBufferSnapshotService - transactionBufferSnapshotIndexService) { - TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, - SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES); - log.info("Create transaction buffer snapshot index client, topicName : {}", topicName.toString()); - return new TransactionBufferSnapshotIndexSystemTopicClient(client, topicName, - transactionBufferSnapshotIndexService); - } - - public TransactionBufferSnapshotSegmentSystemTopicClient createTransactionBufferSnapshotSegmentSystemTopicClient( - NamespaceName namespaceName, - SystemTopicTxnBufferSnapshotService - transactionBufferSnapshotSegmentService) { - TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, - SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT); - log.info("Create transaction buffer snapshot segment client, topicName : {}", topicName.toString()); - return new TransactionBufferSnapshotSegmentSystemTopicClient(client, topicName, - transactionBufferSnapshotSegmentService); + return new TransactionBufferSnapshotBaseSystemTopicClient(client, topicName, + systemTopicTxnBufferSnapshotService, schemaType); } public static TopicName getSystemTopicName(NamespaceName namespaceName, EventType eventType) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index 4f8dfeede2670..d75b12ed32a8c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -20,23 +20,30 @@ 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; -public abstract class TransactionBufferSnapshotBaseSystemTopicClient extends SystemTopicClientBase { +@Slf4j +public class TransactionBufferSnapshotBaseSystemTopicClient extends SystemTopicClientBase { protected final SystemTopicTxnBufferSnapshotService systemTopicTxnBufferSnapshotService; + protected final Class schemaType; + public TransactionBufferSnapshotBaseSystemTopicClient(PulsarClient client, TopicName topicName, SystemTopicTxnBufferSnapshotService - systemTopicTxnBufferSnapshotService) { + systemTopicTxnBufferSnapshotService, + Class schemaType) { super(client, topicName); this.systemTopicTxnBufferSnapshotService = systemTopicTxnBufferSnapshotService; + this.schemaType = schemaType; } protected void removeWriter(Writer writer) { @@ -52,11 +59,11 @@ protected void removeReader(Reader reader) { protected static class TransactionBufferSnapshotWriter implements Writer { protected final Producer producer; - protected final TransactionBufferSnapshotBaseSystemTopicClient + protected final TransactionBufferSnapshotBaseSystemTopicClient transactionBufferSnapshotBaseSystemTopicClient; protected TransactionBufferSnapshotWriter(Producer producer, - TransactionBufferSnapshotBaseSystemTopicClient + TransactionBufferSnapshotBaseSystemTopicClient transactionBufferSnapshotBaseSystemTopicClient) { this.producer = producer; this.transactionBufferSnapshotBaseSystemTopicClient = transactionBufferSnapshotBaseSystemTopicClient; @@ -123,7 +130,7 @@ public SystemTopicClient getSystemTopicClient() { protected static class TransactionBufferSnapshotReader implements Reader { private final org.apache.pulsar.client.api.Reader reader; - private final TransactionBufferSnapshotBaseSystemTopicClient transactionBufferSnapshotBaseSystemTopicClient; + private final TransactionBufferSnapshotBaseSystemTopicClient transactionBufferSnapshotBaseSystemTopicClient; protected TransactionBufferSnapshotReader( org.apache.pulsar.client.api.Reader reader, @@ -179,5 +186,33 @@ public SystemTopicClient getSystemTopic() { } } + @Override + protected CompletableFuture> newWriterAsyncInternal() { + return client.newProducer(Schema.AVRO(schemaType)) + .topic(topicName.toString()) + .createAsync().thenCompose(producer -> { + if (log.isDebugEnabled()) { + log.debug("[{}] A new {} writer is created", topicName, schemaType.getName()); + } + return CompletableFuture.completedFuture( + new TransactionBufferSnapshotWriter<>(producer, this)); + }); + } + + @Override + protected CompletableFuture> newReaderAsyncInternal() { + return client.newReader(Schema.AVRO(schemaType)) + .topic(topicName.toString()) + .startMessageId(MessageId.earliest) + .readCompacted(true) + .createAsync() + .thenCompose(reader -> { + if (log.isDebugEnabled()) { + log.debug("[{}] A new {} reader is created", topicName, schemaType); + } + return CompletableFuture.completedFuture( + new TransactionBufferSnapshotReader<>(reader, this)); + }); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java deleted file mode 100644 index 17862b942ed2d..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotIndexSystemTopicClient.java +++ /dev/null @@ -1,71 +0,0 @@ -/** - * 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.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; -import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.common.naming.TopicName; - -@Slf4j -public class TransactionBufferSnapshotIndexSystemTopicClient extends - TransactionBufferSnapshotBaseSystemTopicClient { - - public TransactionBufferSnapshotIndexSystemTopicClient(PulsarClient client, TopicName topicName, - SystemTopicTxnBufferSnapshotService - transactionBufferSnapshotIndexService) { - super(client, topicName, transactionBufferSnapshotIndexService); - } - - @Override - protected CompletableFuture> newWriterAsyncInternal() { - return client.newProducer(Schema.AVRO(TransactionBufferSnapshotIndexes.class)) - .topic(topicName.toString()) - .createAsync().thenCompose(producer -> { - if (log.isDebugEnabled()) { - log.debug("[{}] A new transactionBufferSnapshot writer is created", topicName); - } - return CompletableFuture.completedFuture( - new TransactionBufferSnapshotWriter<>(producer, this)); - }); - } - - @Override - protected CompletableFuture> newReaderAsyncInternal() { - return client.newReader(Schema.AVRO(TransactionBufferSnapshotIndexes.class)) - .topic(topicName.toString()) - .startMessageId(MessageId.earliest) - .readCompacted(true) - .createAsync() - .thenCompose(reader -> { - if (log.isDebugEnabled()) { - log.debug("[{}] A new transactionBufferSnapshot buffer reader is created", topicName); - } - return CompletableFuture.completedFuture( - new TransactionBufferSnapshotReader<>(reader, this)); - }); - } - -} - diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java deleted file mode 100644 index b83685186dccc..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSegmentSystemTopicClient.java +++ /dev/null @@ -1,68 +0,0 @@ -/** - * 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.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.service.BrokerServiceException; -import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; -import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.util.FutureUtil; - -@Slf4j -public class TransactionBufferSnapshotSegmentSystemTopicClient extends - TransactionBufferSnapshotBaseSystemTopicClient { - - private static final String prefix = "multiple-"; - public TransactionBufferSnapshotSegmentSystemTopicClient(PulsarClient client, TopicName topicName, - SystemTopicTxnBufferSnapshotService - transactionBufferSnapshotSegmentService) { - super(client, topicName, transactionBufferSnapshotSegmentService); - } - - @Override - protected CompletableFuture> - newWriterAsyncInternal() { - return client.newProducer(Schema.AVRO(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot.class)) - .topic(topicName.toString()) - .createAsync().thenCompose(producer -> { - if (log.isDebugEnabled()) { - log.debug("[{}] A new transactionBufferSnapshot segment writer is created", topicName); - } - return CompletableFuture.completedFuture( - new TransactionBufferSnapshotWriter<>(producer, this)); - }); - } - - @Override - protected CompletableFuture> - newReaderAsyncInternal() { - return FutureUtil.failedFuture(new BrokerServiceException - .NotAllowedException("Do not allow to get reader for segment topic reader")); - } - - public static String buildKey(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot snapshot) { - return prefix + snapshot.getSequenceId() + "-" + snapshot.getTopicName(); - } - - -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java deleted file mode 100644 index cc099410f2378..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotSystemTopicClient.java +++ /dev/null @@ -1,70 +0,0 @@ -/** - * 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.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; -import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.common.naming.TopicName; - -@Slf4j -public class TransactionBufferSnapshotSystemTopicClient extends - TransactionBufferSnapshotBaseSystemTopicClient { - - public TransactionBufferSnapshotSystemTopicClient(PulsarClient client, TopicName topicName, - SystemTopicTxnBufferSnapshotService - systemTopicTxnBufferSnapshotService) { - super(client, topicName, systemTopicTxnBufferSnapshotService); - } - - @Override - protected CompletableFuture> newWriterAsyncInternal() { - return client.newProducer(Schema.AVRO(TransactionBufferSnapshot.class)) - .topic(topicName.toString()) - .createAsync().thenCompose(producer -> { - if (log.isDebugEnabled()) { - log.debug("[{}] A new transactionBufferSnapshot writer is created", topicName); - } - return CompletableFuture.completedFuture( - new TransactionBufferSnapshotWriter(producer, this)); - }); - } - - @Override - protected CompletableFuture> newReaderAsyncInternal() { - return client.newReader(Schema.AVRO(TransactionBufferSnapshot.class)) - .topic(topicName.toString()) - .startMessageId(MessageId.earliest) - .readCompacted(true) - .createAsync() - .thenCompose(reader -> { - if (log.isDebugEnabled()) { - log.debug("[{}] A new transactionBufferSnapshot buffer reader is created", topicName); - } - return CompletableFuture.completedFuture( - new TransactionBufferSnapshotReader<>(reader, this)); - }); - } -} - diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index a9ac712a6bc47..3cb45ac27a57a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.transaction; -import static org.apache.pulsar.broker.systopic.TransactionBufferSnapshotSegmentSystemTopicClient.buildKey; import static org.apache.pulsar.common.naming.SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; @@ -46,7 +45,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -57,11 +55,9 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; -import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotServiceImpl; -import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotServiceImpl; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotServiceFactory; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; @@ -491,8 +487,8 @@ public void testTransactionBufferRecoverThrowException() throws Exception { PersistentTopic originalTopic = (PersistentTopic) getPulsarServiceList().get(0) .getBrokerService().getTopic(TopicName.get(topic).toString(), false).get().get(); - SystemTopicTxnBufferSnapshotServiceImpl systemTopicTxnBufferSnapshotService = - mock(SystemTopicTxnBufferSnapshotServiceImpl.class); + SystemTopicTxnBufferSnapshotService systemTopicTxnBufferSnapshotService = + mock(SystemTopicTxnBufferSnapshotService.class); SystemTopicClient.Reader reader = mock(SystemTopicClient.Reader.class); SystemTopicClient.Writer writer = mock(SystemTopicClient.Writer.class); @@ -500,28 +496,28 @@ public void testTransactionBufferRecoverThrowException() throws Exception { .when(systemTopicTxnBufferSnapshotService).createReader(any()); doReturn(CompletableFuture.completedFuture(writer)) .when(systemTopicTxnBufferSnapshotService).createWriter(any()); - TransactionBufferSnapshotService transactionBufferSnapshotService = - mock(TransactionBufferSnapshotService.class); + TransactionBufferSnapshotServiceFactory transactionBufferSnapshotServiceFactory = + mock(TransactionBufferSnapshotServiceFactory.class); doReturn(systemTopicTxnBufferSnapshotService) - .when(transactionBufferSnapshotService).getTxnBufferSnapshotService(); + .when(transactionBufferSnapshotServiceFactory).getTxnBufferSnapshotService(); doReturn(CompletableFuture.completedFuture(null)).when(reader).closeAsync(); doReturn(CompletableFuture.completedFuture(null)).when(writer).closeAsync(); - Field field = PulsarService.class.getDeclaredField("transactionBufferSnapshotService"); + Field field = PulsarService.class.getDeclaredField("transactionBufferSnapshotServiceFactory"); field.setAccessible(true); - TransactionBufferSnapshotService systemTopicTxnBufferSnapshotServiceOriginal = - ((TransactionBufferSnapshotService)field.get(getPulsarServiceList().get(0))); + TransactionBufferSnapshotServiceFactory transactionBufferSnapshotServiceFactoryOriginal = + ((TransactionBufferSnapshotServiceFactory)field.get(getPulsarServiceList().get(0))); // mock reader can't read snapshot fail throw RuntimeException doThrow(new RuntimeException("test")).when(reader).hasMoreEvents(); // check reader close topic - checkCloseTopic(pulsarClient, systemTopicTxnBufferSnapshotServiceOriginal, - transactionBufferSnapshotService, originalTopic, field, producer); + checkCloseTopic(pulsarClient, transactionBufferSnapshotServiceFactoryOriginal, + transactionBufferSnapshotServiceFactory, originalTopic, field, producer); doReturn(true).when(reader).hasMoreEvents(); // mock reader can't read snapshot fail throw PulsarClientException doThrow(new PulsarClientException("test")).when(reader).hasMoreEvents(); // check reader close topic - checkCloseTopic(pulsarClient, systemTopicTxnBufferSnapshotServiceOriginal, - transactionBufferSnapshotService, originalTopic, field, producer); + checkCloseTopic(pulsarClient, transactionBufferSnapshotServiceFactoryOriginal, + transactionBufferSnapshotServiceFactory, originalTopic, field, producer); doReturn(true).when(reader).hasMoreEvents(); // mock create reader fail @@ -530,8 +526,8 @@ public void testTransactionBufferRecoverThrowException() throws Exception { // check create reader fail close topic originalTopic = (PersistentTopic) getPulsarServiceList().get(0) .getBrokerService().getTopic(TopicName.get(topic).toString(), false).get().get(); - checkCloseTopic(pulsarClient, systemTopicTxnBufferSnapshotServiceOriginal, - transactionBufferSnapshotService, originalTopic, field, producer); + checkCloseTopic(pulsarClient, transactionBufferSnapshotServiceFactoryOriginal, + transactionBufferSnapshotServiceFactory, originalTopic, field, producer); doReturn(CompletableFuture.completedFuture(reader)).when(systemTopicTxnBufferSnapshotService).createReader(any()); // check create writer fail close topic @@ -540,17 +536,17 @@ public void testTransactionBufferRecoverThrowException() throws Exception { // mock create writer fail doReturn(FutureUtil.failedFuture(new PulsarClientException("test"))) .when(systemTopicTxnBufferSnapshotService).createWriter(any()); - checkCloseTopic(pulsarClient, systemTopicTxnBufferSnapshotServiceOriginal, - transactionBufferSnapshotService, originalTopic, field, producer); + checkCloseTopic(pulsarClient, transactionBufferSnapshotServiceFactoryOriginal, + transactionBufferSnapshotServiceFactory, originalTopic, field, producer); } private void checkCloseTopic(PulsarClient pulsarClient, - TransactionBufferSnapshotService transactionBufferSnapshotServiceOriginal, - TransactionBufferSnapshotService transactionBufferSnapshotService, + TransactionBufferSnapshotServiceFactory transactionBufferSnapshotServiceFactoryOriginal, + TransactionBufferSnapshotServiceFactory transactionBufferSnapshotServiceFactory, PersistentTopic originalTopic, Field field, Producer producer) throws Exception { - field.set(getPulsarServiceList().get(0), transactionBufferSnapshotService); + field.set(getPulsarServiceList().get(0), transactionBufferSnapshotServiceFactory); // recover again will throw then close topic new TopicTransactionBuffer(originalTopic); @@ -561,7 +557,7 @@ private void checkCloseTopic(PulsarClient pulsarClient, assertTrue((boolean) close.get(originalTopic)); }); - field.set(getPulsarServiceList().get(0), transactionBufferSnapshotServiceOriginal); + field.set(getPulsarServiceList().get(0), transactionBufferSnapshotServiceFactoryOriginal); Transaction txn = pulsarClient.newTransaction() .withTransactionTimeout(5, TimeUnit.SECONDS) @@ -593,7 +589,8 @@ public void testTransactionBufferNoSnapshotCloseReader() throws Exception{ @Test public void testTransactionBufferIndexSystemTopic() throws Exception { SystemTopicTxnBufferSnapshotService transactionBufferSnapshotIndexService = - new TransactionBufferSnapshotServiceImpl(pulsarClient, true).getTxnBufferSnapshotIndexService(); + new TransactionBufferSnapshotServiceFactory(pulsarClient, true) + .getTxnBufferSnapshotIndexService(); SystemTopicClient.Writer indexesWriter = transactionBufferSnapshotIndexService.createWriter(TopicName.get(SNAPSHOT_INDEX)).get(); @@ -609,7 +606,7 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { } TransactionBufferSnapshotIndexes transactionBufferTransactionBufferSnapshotIndexes = - new TransactionBufferSnapshotIndexes(SNAPSHOT_INDEX, TransactionBufferSnapshotIndexes.Type.Indexes, + new TransactionBufferSnapshotIndexes(SNAPSHOT_INDEX, indexList, null); indexesWriter.write(transactionBufferTransactionBufferSnapshotIndexes, SNAPSHOT_INDEX); @@ -617,9 +614,6 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { assertTrue(indexesReader.hasMoreEvents()); transactionBufferTransactionBufferSnapshotIndexes = indexesReader.readNext().getValue(); assertEquals(transactionBufferTransactionBufferSnapshotIndexes.getTopicName(), SNAPSHOT_INDEX); - assertEquals( - transactionBufferTransactionBufferSnapshotIndexes.getType(), - TransactionBufferSnapshotIndexes.Type.Indexes); assertEquals(transactionBufferTransactionBufferSnapshotIndexes.getIndexList().size(), 5); assertNull(transactionBufferTransactionBufferSnapshotIndexes.getSnapshot()); @@ -632,6 +626,10 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { assertEquals(transactionBufferSnapshotIndex.getSequenceID(), 1L); } + public static String buildKey(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot snapshot) { + return "multiple-" + snapshot.getSequenceId() + "-" + snapshot.getTopicName(); + } + @Test public void testTransactionBufferSegmentSystemTopic() throws Exception { // init topic and topicName @@ -648,10 +646,10 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { PulsarService pulsarService = getPulsarServiceList().get(0); BrokerService brokerService = pulsarService.getBrokerService(); - // create snapshot sgement writer + // create snapshot segment writer SystemTopicTxnBufferSnapshotService transactionBufferSnapshotSegmentService = - new TransactionBufferSnapshotServiceImpl(pulsarClient, true) + new TransactionBufferSnapshotServiceFactory(pulsarClient, true) .getTxnBufferSnapshotSegmentService(); SystemTopicClient.Writer @@ -661,7 +659,7 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { TransactionBufferSnapshotIndexes.TransactionBufferSnapshot snapshot = new TransactionBufferSnapshotIndexes.TransactionBufferSnapshot(); - //build and send sanpshot + //build and send snapshot snapshot.setTopicName(snapshotTopic); snapshot.setSequenceId(1L); snapshot.setMaxReadPositionLedgerId(2L); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index aab765a905fd3..75d0ebc69d244 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -80,9 +80,9 @@ import org.apache.pulsar.broker.service.BacklogQuotaManager; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; -import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotServiceImpl; +import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; import org.apache.pulsar.broker.service.Topic; -import org.apache.pulsar.broker.service.TransactionBufferSnapshotService; +import org.apache.pulsar.broker.service.TransactionBufferSnapshotServiceFactory; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; @@ -144,9 +144,7 @@ import org.mockito.stubbing.Answer; import org.testng.Assert; import org.testng.annotations.AfterClass; -import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; /** @@ -1434,15 +1432,15 @@ public void testTBRecoverChangeStateError() throws InterruptedException, Timeout when(pendingAckStoreProvider.newPendingAckStore(any())) .thenReturn(CompletableFuture.completedFuture(pendingAckStore)); // Mock TransactionBufferSnapshotService - SystemTopicTxnBufferSnapshotServiceImpl systemTopicTxnBufferSnapshotService - = mock(SystemTopicTxnBufferSnapshotServiceImpl.class); - SystemTopicClient.Writer writer = mock(SystemTopicClient.Writer.class); + SystemTopicTxnBufferSnapshotService systemTopicTxnBufferSnapshotService + = mock(SystemTopicTxnBufferSnapshotService.class); + SystemTopicClient.Writer writer = mock(SystemTopicClient.Writer.class); when(writer.closeAsync()).thenReturn(CompletableFuture.completedFuture(null)); when(systemTopicTxnBufferSnapshotService.createWriter(any())) .thenReturn(CompletableFuture.completedFuture(writer)); - TransactionBufferSnapshotService transactionBufferSnapshotService = - mock(TransactionBufferSnapshotService.class); - when(transactionBufferSnapshotService.getTxnBufferSnapshotService()) + TransactionBufferSnapshotServiceFactory transactionBufferSnapshotServiceFactory = + mock(TransactionBufferSnapshotServiceFactory.class); + when(transactionBufferSnapshotServiceFactory.getTxnBufferSnapshotService()) .thenReturn(systemTopicTxnBufferSnapshotService); // Mock pulsar. @@ -1450,7 +1448,7 @@ public void testTBRecoverChangeStateError() throws InterruptedException, Timeout when(pulsar.getConfiguration()).thenReturn(serviceConfiguration); when(pulsar.getConfig()).thenReturn(serviceConfiguration); when(pulsar.getTransactionExecutorProvider()).thenReturn(executorProvider); - when(pulsar.getTransactionBufferSnapshotService()).thenReturn(transactionBufferSnapshotService); + when(pulsar.getTransactionBufferSnapshotServiceFactory()).thenReturn(transactionBufferSnapshotServiceFactory); TopicTransactionBufferProvider topicTransactionBufferProvider = new TopicTransactionBufferProvider(); when(pulsar.getTransactionBufferProvider()).thenReturn(topicTransactionBufferProvider); // Mock BacklogQuotaManager From 189fe408d3836588d4b850806276dbb6f7a222e3 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Mon, 17 Oct 2022 15:29:33 +0800 Subject: [PATCH 16/27] optimize by class --- .../transaction/buffer/impl/TopicTransactionBuffer.java | 7 ++++--- .../org/apache/pulsar/client/api/transaction/TxnID.java | 8 ++++++-- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index d325d5b4129c1..9e885e5edc525 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -117,7 +117,7 @@ public TopicTransactionBuffer(PersistentTopic topic) { super(State.None); this.topic = topic; this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar() - .getTransactionBufferSnapshotService() + .getTransactionBufferSnapshotServiceFactory() .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName())); this.timer = topic.getBrokerService().getPulsar().getTransactionTimer(); this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar() @@ -646,8 +646,9 @@ public void run() { this, topic.getName()); return; } - topic.getBrokerService().getPulsar().getTransactionBufferSnapshotService().getTxnBufferSnapshotService() - .createReader(TopicName.get(topic.getName())).thenAcceptAsync(reader -> { + topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory() + .getTxnBufferSnapshotService().createReader(TopicName.get(topic.getName())) + .thenAcceptAsync(reader -> { try { boolean hasSnapshot = false; while (reader.hasMoreEvents()) { diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/transaction/TxnID.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/transaction/TxnID.java index d503abeafc239..891a653d69e39 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/transaction/TxnID.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/transaction/TxnID.java @@ -20,7 +20,9 @@ import java.io.Serializable; import java.util.Objects; +import lombok.AllArgsConstructor; import lombok.Data; +import lombok.NoArgsConstructor; import org.apache.pulsar.common.classification.InterfaceAudience; import org.apache.pulsar.common.classification.InterfaceStability; @@ -30,6 +32,8 @@ @InterfaceAudience.Public @InterfaceStability.Evolving @Data +@AllArgsConstructor +@NoArgsConstructor public class TxnID implements Serializable { private static final long serialVersionUID = 0L; @@ -39,14 +43,14 @@ public class TxnID implements Serializable { * * @serial */ - private final long mostSigBits; + private long mostSigBits; /* * The least significant 64 bits of this TxnID. * * @serial */ - private final long leastSigBits; + private long leastSigBits; @Override public String toString() { From 6cba2824101aa8ff3bbfadb829ff989b31310470 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Tue, 18 Oct 2022 22:01:14 +0800 Subject: [PATCH 17/27] fix some comments --- .../bookkeeper/mledger/AsyncCallbacks.java | 7 +++ .../mledger/ManagedLedgerFactory.java | 9 ++-- .../impl/ManagedLedgerFactoryImpl.java | 39 ++++++++------- .../TopicPoliciesSystemTopicClient.java | 2 +- ...onBufferSnapshotBaseSystemTopicClient.java | 14 +++--- .../TopicTransactionBufferRecoverTest.java | 48 +++++++++++-------- 6 files changed, 67 insertions(+), 52 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/AsyncCallbacks.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/AsyncCallbacks.java index 395da52b2aff9..78cca061c7898 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/AsyncCallbacks.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/AsyncCallbacks.java @@ -24,6 +24,7 @@ import java.util.Optional; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; +import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; /** * Definition of all the callbacks used for the ManagedLedger asynchronous API. @@ -46,6 +47,12 @@ interface OpenReadOnlyCursorCallback { void openReadOnlyCursorFailed(ManagedLedgerException exception, Object ctx); } + interface OpenReadOnlyManagedLedgerCallback { + void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl managedLedger, Object ctx); + + void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx); + } + interface DeleteLedgerCallback { void deleteLedgerComplete(Object ctx); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java index 6e0a3223d8ffd..d62df23d96755 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java @@ -27,7 +27,6 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.ManagedLedgerInfoCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenLedgerCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenReadOnlyCursorCallback; -import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; /** @@ -121,11 +120,13 @@ void asyncOpenReadOnlyCursor(String managedLedgerName, Position startPosition, M /** * Asynchronous open a Read-only managedLedger. * @param managedLedgerName the unique name that identifies the managed ledger - * @param config the managed ledegr configuratiion. + * @param callback + * @param config the managed ledger configuration. * @param ctx opaque context */ - CompletableFuture asyncOpenReadOnlyManagedLedger(String managedLedgerName, - ManagedLedgerConfig config, Object ctx); + void asyncOpenReadOnlyManagedLedger(String managedLedgerName, + AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback, + ManagedLedgerConfig config, Object ctx); /** * Get the current metadata info for a managed ledger. diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 8742481269dfb..73b0d28d9a7e2 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -421,11 +421,13 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { } @Override - public CompletableFuture asyncOpenReadOnlyManagedLedger(String managedLedgerName, + public void asyncOpenReadOnlyManagedLedger(String managedLedgerName, + AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback, ManagedLedgerConfig config, Object ctx) { CompletableFuture future = new CompletableFuture<>(); if (closed) { - return FutureUtil.failedFuture(new ManagedLedgerException.ManagedLedgerFactoryClosedException()); + callback.openReadOnlyManagedLedgerFailed( + new ManagedLedgerException.ManagedLedgerFactoryClosedException(), ctx); } ReadOnlyManagedLedgerImpl roManagedLedger = new ReadOnlyManagedLedgerImpl(this, bookkeeperFactory @@ -434,13 +436,13 @@ public CompletableFuture asyncOpenReadOnlyManagedLedg store, config, scheduledExecutor, managedLedgerName); roManagedLedger.initialize().thenRun(() -> { log.info("[{}] Successfully initialize Read-only managed ledger", managedLedgerName); - future.complete(roManagedLedger); + callback.openReadOnlyManagedLedgerComplete(roManagedLedger, ctx); + }).exceptionally(e -> { log.error("[{}] Failed to initialize Read-only managed ledger", managedLedgerName, e); - future.completeExceptionally(new ManagedLedgerException.ManagedLedgerFactoryClosedException()); + callback.openReadOnlyManagedLedgerFailed((ManagedLedgerException) e.getCause(), ctx); return null; }); - return future; } @Override @@ -483,23 +485,20 @@ public void asyncOpenReadOnlyCursor(String managedLedgerName, Position startPosi return; } checkArgument(startPosition instanceof PositionImpl); - asyncOpenReadOnlyManagedLedger(managedLedgerName, config, null).thenAccept(readOnlyManagedLedger -> + AsyncCallbacks.OpenReadOnlyManagedLedgerCallback openReadOnlyManagedLedgerCallback = + new AsyncCallbacks.OpenReadOnlyManagedLedgerCallback() { + @Override + public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnlyManagedLedger, Object ctx) { callback.openReadOnlyCursorComplete(readOnlyManagedLedger. - createReadOnlyCursor((PositionImpl) startPosition), ctx)) - .exceptionally(ex -> { - Throwable t = ex; - if (t instanceof CompletionException) { - t = ex.getCause(); - } - - if (t instanceof ManagedLedgerException) { - callback.openReadOnlyCursorFailed((ManagedLedgerException) t, ctx); - } else { - callback.openReadOnlyCursorFailed(new ManagedLedgerException(t), ctx); - } + createReadOnlyCursor((PositionImpl) startPosition), ctx); + } - return null; - }); + @Override + public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx) { + callback.openReadOnlyCursorFailed(exception, ctx); + } + }; + asyncOpenReadOnlyManagedLedger(managedLedgerName, openReadOnlyManagedLedgerCallback, config, null); } void close(ManagedLedger ledger) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java index 7611f074c5012..173f4f7c4d66e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java @@ -108,7 +108,7 @@ public MessageId delete(PulsarEvent event, String key) throws PulsarClientExcept @Override public CompletableFuture deleteAsync(PulsarEvent event, String key) { validateActionType(event); - TypedMessageBuilder builder = producer.newMessage().key(getEventKey(event)).value(null); + TypedMessageBuilder builder = producer.newMessage().key(key).value(null); setReplicateCluster(event, builder); return builder.sendAsync(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index d75b12ed32a8c..be91ee6061574 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -102,7 +102,7 @@ public CompletableFuture deleteAsync(T t, String key) { @Override public void close() throws IOException { - this.producer.close(); + this.producer.closeAsync().join(); transactionBufferSnapshotBaseSystemTopicClient.removeWriter(this); } @@ -161,7 +161,7 @@ public CompletableFuture hasMoreEventsAsync() { @Override public void close() throws IOException { - this.reader.close(); + this.reader.closeAsync().join(); transactionBufferSnapshotBaseSystemTopicClient.removeReader(this); } @@ -190,12 +190,11 @@ public SystemTopicClient getSystemTopic() { protected CompletableFuture> newWriterAsyncInternal() { return client.newProducer(Schema.AVRO(schemaType)) .topic(topicName.toString()) - .createAsync().thenCompose(producer -> { + .createAsync().thenApply(producer -> { if (log.isDebugEnabled()) { log.debug("[{}] A new {} writer is created", topicName, schemaType.getName()); } - return CompletableFuture.completedFuture( - new TransactionBufferSnapshotWriter<>(producer, this)); + return new TransactionBufferSnapshotWriter<>(producer, this); }); } @@ -206,12 +205,11 @@ protected CompletableFuture> newReaderAsyncInternal() { .startMessageId(MessageId.earliest) .readCompacted(true) .createAsync() - .thenCompose(reader -> { + .thenApply(reader -> { if (log.isDebugEnabled()) { log.debug("[{}] A new {} reader is created", topicName, schemaType); } - return CompletableFuture.completedFuture( - new TransactionBufferSnapshotReader<>(reader, this)); + return new TransactionBufferSnapshotReader<>(reader, this); }); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 3cb45ac27a57a..0c6fb753ebf36 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -672,28 +672,38 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { MessageIdImpl messageId = (MessageIdImpl) segmentWriter.write(snapshot, buildKey(snapshot)); - //create read-only managed ledger - ReadOnlyManagedLedgerImpl readOnlyManagedLedger = pulsarService.getManagedLedgerFactory() - .asyncOpenReadOnlyManagedLedger(snapshotSegmentTopicName.getPersistenceNamingEncoding(), - brokerService.getManagedLedgerConfig(snapshotSegmentTopicName).get(),null).get(); - - //read the entry and decode entry to snapshot + //Create read-only managed ledger + //And read the entry and decode entry to snapshot CompletableFuture entryCompletableFuture = new CompletableFuture<>(); - readOnlyManagedLedger.asyncReadEntry( - new PositionImpl(messageId.getLedgerId(), messageId.getEntryId()), - new AsyncCallbacks.ReadEntryCallback() { - @Override - public void readEntryComplete(Entry entry, Object ctx) { - entryCompletableFuture.complete(entry); - } + AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback = new AsyncCallbacks + .OpenReadOnlyManagedLedgerCallback() { + @Override + public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnlyManagedLedger, Object ctx) { + readOnlyManagedLedger.asyncReadEntry( + new PositionImpl(messageId.getLedgerId(), messageId.getEntryId()), + new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + entryCompletableFuture.complete(entry); + } + + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + entryCompletableFuture.completeExceptionally(exception); + } + }, null); + } - @Override - public void readEntryFailed(ManagedLedgerException exception, Object ctx) { - entryCompletableFuture.completeExceptionally(exception); - } - }, null); - Entry entry = entryCompletableFuture.get(); + @Override + public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx) { + // + } + }; + pulsarService.getManagedLedgerFactory() + .asyncOpenReadOnlyManagedLedger(snapshotSegmentTopicName.getPersistenceNamingEncoding(), callback, + brokerService.getManagedLedgerConfig(snapshotSegmentTopicName).get(),null); + Entry entry = entryCompletableFuture.get(); //decode snapshot from entry ByteBuf headersAndPayload = entry.getDataBuffer(); //skip metadata From 8da3749f072ae3235f30dd2652fb37d7992f215d Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Wed, 19 Oct 2022 13:57:37 +0800 Subject: [PATCH 18/27] fix some comments --- .../main/java/org/apache/pulsar/broker/PulsarService.java | 3 +-- .../service/TransactionBufferSnapshotServiceFactory.java | 3 +-- .../transaction/TopicTransactionBufferRecoverTest.java | 6 ++---- 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index aae8db0d233ee..7489e073c391a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -833,8 +833,7 @@ public void start() throws PulsarServerException { MLTransactionMetadataStoreProvider.initBufferedWriterMetrics(getAdvertisedAddress()); MLPendingAckStoreProvider.initBufferedWriterMetrics(getAdvertisedAddress()); - this.transactionBufferSnapshotServiceFactory = new TransactionBufferSnapshotServiceFactory(getClient(), - getConfig().isTransactionBufferSegmentedSnapshotEnabled()); + this.transactionBufferSnapshotServiceFactory = new TransactionBufferSnapshotServiceFactory(getClient()); this.transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-transaction-timer")); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java index e32a13de5bf4b..ae90d7bccadf2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java @@ -33,8 +33,7 @@ public class TransactionBufferSnapshotServiceFactory { private SystemTopicTxnBufferSnapshotService txnBufferSnapshotIndexService; - public TransactionBufferSnapshotServiceFactory(PulsarClient pulsarClient, - boolean transactionBufferSegmentedSnapshotEnabled) { + public TransactionBufferSnapshotServiceFactory(PulsarClient pulsarClient) { this.txnBufferSnapshotSegmentService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT, TransactionBufferSnapshotIndexes.TransactionBufferSnapshot.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 0c6fb753ebf36..c78cabe7927d0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -589,8 +589,7 @@ public void testTransactionBufferNoSnapshotCloseReader() throws Exception{ @Test public void testTransactionBufferIndexSystemTopic() throws Exception { SystemTopicTxnBufferSnapshotService transactionBufferSnapshotIndexService = - new TransactionBufferSnapshotServiceFactory(pulsarClient, true) - .getTxnBufferSnapshotIndexService(); + new TransactionBufferSnapshotServiceFactory(pulsarClient).getTxnBufferSnapshotIndexService(); SystemTopicClient.Writer indexesWriter = transactionBufferSnapshotIndexService.createWriter(TopicName.get(SNAPSHOT_INDEX)).get(); @@ -649,8 +648,7 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { // create snapshot segment writer SystemTopicTxnBufferSnapshotService transactionBufferSnapshotSegmentService = - new TransactionBufferSnapshotServiceFactory(pulsarClient, true) - .getTxnBufferSnapshotSegmentService(); + new TransactionBufferSnapshotServiceFactory(pulsarClient).getTxnBufferSnapshotSegmentService(); SystemTopicClient.Writer segmentWriter = transactionBufferSnapshotSegmentService.createWriter(snapshotSegmentTopicName).get(); From cd6c9099c4f64ab28bf4260d05f2f70087ae35b3 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Thu, 20 Oct 2022 12:42:41 +0800 Subject: [PATCH 19/27] (T t, String key) to (String key, T t) --- .../SystemTopicBasedTopicPoliciesService.java | 8 ++++---- .../pulsar/broker/systopic/SystemTopicClient.java | 14 ++++++++------ .../systopic/TopicPoliciesSystemTopicClient.java | 8 ++++---- ...sactionBufferSnapshotBaseSystemTopicClient.java | 8 ++++---- .../buffer/impl/TopicTransactionBuffer.java | 4 ++-- .../TopicTransactionBufferRecoverTest.java | 6 +++--- 6 files changed, 25 insertions(+), 23 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 03ad629962fa9..ffdbfc34234d5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -124,8 +124,8 @@ private CompletableFuture sendTopicPolicyEvent(TopicName topicName, Action } else { PulsarEvent event = getPulsarEvent(topicName, actionType, policies); CompletableFuture actionFuture = - ActionType.DELETE.equals(actionType) ? writer.deleteAsync(event, getEventKey(event)) - : writer.writeAsync(event, getEventKey(event)); + ActionType.DELETE.equals(actionType) ? writer.deleteAsync(getEventKey(event), event) + : writer.writeAsync(getEventKey(event), event); actionFuture.whenComplete(((messageId, e) -> { if (e != null) { result.completeExceptionally(e); @@ -457,8 +457,8 @@ private void refreshTopicPoliciesCache(Message msg) { SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory .createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject()); systemTopicClient.newWriterAsync().thenAccept(writer - -> writer.deleteAsync(getPulsarEvent(topicName, ActionType.DELETE, null), - getEventKey(topicName)) + -> writer.deleteAsync(getEventKey(topicName), + getPulsarEvent(topicName, ActionType.DELETE, null)) .whenComplete((result, e) -> writer.closeAsync().whenComplete((res, ex) -> { if (ex != null) { log.error("close writer failed ", ex); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java index be494a4299902..11dfeff200896 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/SystemTopicClient.java @@ -90,37 +90,39 @@ interface Writer { /** * Write event to the system topic. - * @param t pulsar event * @param key the key of the event + * @param t pulsar event * @return message id * @throws PulsarClientException exception while write event cause */ - MessageId write(T t, String key) throws PulsarClientException; + MessageId write(String key, T t) throws PulsarClientException; /** * Async write event to the system topic. - * @param t pulsar event * @param key the key of the event + * @param t pulsar event * @return message id future */ - CompletableFuture writeAsync(T t, String key); + CompletableFuture writeAsync(String key, T t); /** * Delete event in the system topic. + * @param key the key of the event * @param t pulsar event * @return message id * @throws PulsarClientException exception while write event cause */ - default MessageId delete(T t, String key) throws PulsarClientException { + default MessageId delete(String key, T t) throws PulsarClientException { throw new UnsupportedOperationException("Unsupported operation"); } /** * Async delete event in the system topic. + * @param key the key of the event * @param t pulsar event * @return message id future */ - default CompletableFuture deleteAsync(T t, String key) { + default CompletableFuture deleteAsync(String key, T t) { throw new UnsupportedOperationException("Unsupported operation"); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java index 173f4f7c4d66e..fa724c1994ea9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java @@ -85,28 +85,28 @@ private TopicPolicyWriter(Producer producer, SystemTopicClient builder = producer.newMessage().key(key).value(event); setReplicateCluster(event, builder); return builder.send(); } @Override - public CompletableFuture writeAsync(PulsarEvent event, String key) { + public CompletableFuture writeAsync(String key, PulsarEvent event) { TypedMessageBuilder builder = producer.newMessage().key(key).value(event); setReplicateCluster(event, builder); return builder.sendAsync(); } @Override - public MessageId delete(PulsarEvent event, String key) throws PulsarClientException { + public MessageId delete(String key, PulsarEvent event) throws PulsarClientException { TypedMessageBuilder builder = producer.newMessage().key(key).value(null); setReplicateCluster(event, builder); return builder.send(); } @Override - public CompletableFuture deleteAsync(PulsarEvent event, String key) { + public CompletableFuture deleteAsync(String key, PulsarEvent event) { validateActionType(event); TypedMessageBuilder builder = producer.newMessage().key(key).value(null); setReplicateCluster(event, builder); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index be91ee6061574..e4798af80491f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -70,21 +70,21 @@ protected TransactionBufferSnapshotWriter(Producer producer, } @Override - public MessageId write(T t, String key) + public MessageId write(String key, T t) throws PulsarClientException { return producer.newMessage().key(key) .value(t).send(); } @Override - public CompletableFuture writeAsync(T t, String key) { + public CompletableFuture writeAsync(String key, T t) { return producer.newMessage() .key(key) .value(t).sendAsync(); } @Override - public MessageId delete(T t, String key) + public MessageId delete(String key, T t) throws PulsarClientException { return producer.newMessage() .key(key) @@ -93,7 +93,7 @@ public MessageId delete(T t, String key) } @Override - public CompletableFuture deleteAsync(T t, String key) { + public CompletableFuture deleteAsync(String key, T t) { return producer.newMessage() .key(key) .value(null) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index 9e885e5edc525..405058bb2f02b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -485,7 +485,7 @@ private CompletableFuture takeSnapshot() { }); snapshot.setAborts(list); } - return writer.writeAsync(snapshot, snapshot.getTopicName()).thenAccept(messageId-> { + return writer.writeAsync(snapshot.getTopicName(), snapshot).thenAccept(messageId-> { this.lastSnapshotTimestamps = System.currentTimeMillis(); if (log.isDebugEnabled()) { log.debug("[{}]Transaction buffer take snapshot success! " @@ -533,7 +533,7 @@ public CompletableFuture clearSnapshot() { return this.takeSnapshotWriter.thenCompose(writer -> { TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot(); snapshot.setTopicName(topic.getName()); - return writer.deleteAsync(snapshot, snapshot.getTopicName()); + return writer.deleteAsync(snapshot.getTopicName(), snapshot); }).thenCompose(__ -> CompletableFuture.completedFuture(null)); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index c78cabe7927d0..15f13b6fa3c44 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -608,7 +608,7 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { new TransactionBufferSnapshotIndexes(SNAPSHOT_INDEX, indexList, null); - indexesWriter.write(transactionBufferTransactionBufferSnapshotIndexes, SNAPSHOT_INDEX); + indexesWriter.write(SNAPSHOT_INDEX, transactionBufferTransactionBufferSnapshotIndexes); assertTrue(indexesReader.hasMoreEvents()); transactionBufferTransactionBufferSnapshotIndexes = indexesReader.readNext().getValue(); @@ -665,10 +665,10 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { snapshot.setAborts(Collections.singletonList( new TxnID(1, 1))); - segmentWriter.write(snapshot, buildKey(snapshot)); + segmentWriter.write(buildKey(snapshot), snapshot); snapshot.setSequenceId(2L); - MessageIdImpl messageId = (MessageIdImpl) segmentWriter.write(snapshot, buildKey(snapshot)); + MessageIdImpl messageId = (MessageIdImpl) segmentWriter.write(buildKey(snapshot), snapshot); //Create read-only managed ledger //And read the entry and decode entry to snapshot From d2f388943207c523c793bae7c2170df07eb983ef Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Thu, 20 Oct 2022 12:54:55 +0800 Subject: [PATCH 20/27] optimize inner class and TxnIDData --- ...ansactionBufferSnapshotServiceFactory.java | 10 ++--- .../buffer/impl/TopicTransactionBuffer.java | 4 +- ...TopicTransactionBufferRecoverCallBack.java | 2 +- .../AbortTxnMetadata.java | 2 +- .../TransactionBufferSnapshot.java | 2 +- .../{matadata => metadata}/package-info.java | 2 +- .../v2/TransactionBufferSnapshot.java | 17 +++++++ .../v2/TransactionBufferSnapshotIndex.java | 18 ++++++++ .../v2/TransactionBufferSnapshotIndexes.java | 26 +---------- .../buffer/metadata/v2/TxnIDData.java | 45 +++++++++++++++++++ .../v2/package-info.java | 2 +- .../TopicTransactionBufferRecoverTest.java | 33 +++++++------- .../broker/transaction/TransactionTest.java | 2 +- .../common/naming/SystemTopicNames.java | 2 +- 14 files changed, 113 insertions(+), 54 deletions(-) rename pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/{matadata => metadata}/AbortTxnMetadata.java (94%) rename pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/{matadata => metadata}/TransactionBufferSnapshot.java (95%) rename pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/{matadata => metadata}/package-info.java (93%) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshot.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndex.java rename pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/{matadata => metadata}/v2/TransactionBufferSnapshotIndexes.java (60%) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TxnIDData.java rename pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/{matadata => metadata}/v2/package-info.java (92%) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java index ae90d7bccadf2..07e029de054bd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java @@ -19,8 +19,8 @@ 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.broker.transaction.buffer.metadata.TransactionBufferSnapshot; +import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.common.events.EventType; @@ -28,7 +28,7 @@ public class TransactionBufferSnapshotServiceFactory { private SystemTopicTxnBufferSnapshotService txnBufferSnapshotService; - private SystemTopicTxnBufferSnapshotService + private SystemTopicTxnBufferSnapshotService txnBufferSnapshotSegmentService; private SystemTopicTxnBufferSnapshotService txnBufferSnapshotIndexService; @@ -36,7 +36,7 @@ public class TransactionBufferSnapshotServiceFactory { public TransactionBufferSnapshotServiceFactory(PulsarClient pulsarClient) { this.txnBufferSnapshotSegmentService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT, - TransactionBufferSnapshotIndexes.TransactionBufferSnapshot.class); + org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot.class); this.txnBufferSnapshotIndexService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, EventType.TRANSACTION_BUFFER_SNAPSHOT_INDEXES, TransactionBufferSnapshotIndexes.class); this.txnBufferSnapshotService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, @@ -47,7 +47,7 @@ public SystemTopicTxnBufferSnapshotService get return this.txnBufferSnapshotIndexService; } - public SystemTopicTxnBufferSnapshotService + public SystemTopicTxnBufferSnapshotService getTxnBufferSnapshotSegmentService() { return this.txnBufferSnapshotSegmentService; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index 405058bb2f02b..1245c7d8129aa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -48,8 +48,8 @@ import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferReader; import org.apache.pulsar.broker.transaction.buffer.TransactionMeta; -import org.apache.pulsar.broker.transaction.buffer.matadata.AbortTxnMetadata; -import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; +import org.apache.pulsar.broker.transaction.buffer.metadata.AbortTxnMetadata; +import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TxnID; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBufferRecoverCallBack.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBufferRecoverCallBack.java index 87b8e930a2792..d229fbb8f5dd3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBufferRecoverCallBack.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBufferRecoverCallBack.java @@ -19,7 +19,7 @@ package org.apache.pulsar.broker.transaction.buffer.impl; import org.apache.bookkeeper.mledger.Entry; -import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; +import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot; public interface TopicTransactionBufferRecoverCallBack { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/AbortTxnMetadata.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/AbortTxnMetadata.java similarity index 94% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/AbortTxnMetadata.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/AbortTxnMetadata.java index 5e532d6bff76f..7e997111a041b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/AbortTxnMetadata.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/AbortTxnMetadata.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.transaction.buffer.matadata; +package org.apache.pulsar.broker.transaction.buffer.metadata; import lombok.AllArgsConstructor; import lombok.Getter; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/TransactionBufferSnapshot.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/TransactionBufferSnapshot.java similarity index 95% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/TransactionBufferSnapshot.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/TransactionBufferSnapshot.java index 59b851e73970e..dbe15e8185408 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/TransactionBufferSnapshot.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/TransactionBufferSnapshot.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.transaction.buffer.matadata; +package org.apache.pulsar.broker.transaction.buffer.metadata; import java.util.List; import lombok.AllArgsConstructor; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/package-info.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/package-info.java similarity index 93% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/package-info.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/package-info.java index f32853a7d86a7..74688c857c91b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/package-info.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/package-info.java @@ -19,4 +19,4 @@ /** * The transaction buffer snapshot metadata. */ -package org.apache.pulsar.broker.transaction.buffer.matadata; +package org.apache.pulsar.broker.transaction.buffer.metadata; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshot.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshot.java new file mode 100644 index 0000000000000..cfbd25bc8ba3f --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshot.java @@ -0,0 +1,17 @@ +package org.apache.pulsar.broker.transaction.buffer.metadata.v2; + +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@AllArgsConstructor +@NoArgsConstructor +public class TransactionBufferSnapshot { + private String topicName; + private long sequenceId; + private long maxReadPositionLedgerId; + private long maxReadPositionEntryId; + private List aborts; +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndex.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndex.java new file mode 100644 index 0000000000000..857ab61aaa769 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndex.java @@ -0,0 +1,18 @@ +package org.apache.pulsar.broker.transaction.buffer.metadata.v2; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Builder +@Data +@AllArgsConstructor +@NoArgsConstructor +public class TransactionBufferSnapshotIndex { + public long sequenceID; + public long maxReadPositionLedgerID; + public long maxReadPositionEntryID; + public long persistentPositionLedgerID; + public long persistentPositionEntryID; +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndexes.java similarity index 60% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndexes.java index 9b4bd4ba10cf4..2bbc01460e41a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/TransactionBufferSnapshotIndexes.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndexes.java @@ -16,16 +16,14 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.transaction.buffer.matadata.v2; +package org.apache.pulsar.broker.transaction.buffer.metadata.v2; import java.util.List; import lombok.AllArgsConstructor; import lombok.Builder; -import lombok.Data; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.Setter; -import org.apache.pulsar.client.api.transaction.TxnID; @AllArgsConstructor @NoArgsConstructor @@ -39,26 +37,4 @@ public class TransactionBufferSnapshotIndexes { private TransactionBufferSnapshot snapshot; - @Builder - @Data - @AllArgsConstructor - @NoArgsConstructor - public static class TransactionBufferSnapshotIndex { - public long sequenceID; - public long maxReadPositionLedgerID; - public long maxReadPositionEntryID; - public long persistentPositionLedgerID; - public long persistentPositionEntryID; - } - - @Data - @AllArgsConstructor - @NoArgsConstructor - public static class TransactionBufferSnapshot { - private String topicName; - private long sequenceId; - private long maxReadPositionLedgerId; - private long maxReadPositionEntryId; - private List aborts; - } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TxnIDData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TxnIDData.java new file mode 100644 index 0000000000000..ec98a9b04ba4c --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TxnIDData.java @@ -0,0 +1,45 @@ +package org.apache.pulsar.broker.transaction.buffer.metadata.v2; + +import java.util.Objects; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@NoArgsConstructor +@AllArgsConstructor +public class TxnIDData { + /* + * The most significant 64 bits of this TxnID. + * + * @serial + */ + private long mostSigBits; + + /* + * The least significant 64 bits of this TxnID. + * + * @serial + */ + private long leastSigBits; + + @Override + public String toString() { + return "(" + mostSigBits + "," + leastSigBits + ")"; + } + + @Override + public int hashCode() { + return Objects.hash(mostSigBits, leastSigBits); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof TxnIDData other) { + return Objects.equals(mostSigBits, other.mostSigBits) + && Objects.equals(leastSigBits, other.leastSigBits); + } + + return false; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/package-info.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/package-info.java similarity index 92% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/package-info.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/package-info.java index d0c7c97cfc4ae..02d6e7a716d4f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/matadata/v2/package-info.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/package-info.java @@ -16,4 +16,4 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.transaction.buffer.matadata.v2; +package org.apache.pulsar.broker.transaction.buffer.metadata.v2; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 15f13b6fa3c44..6543752c902bf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -62,8 +62,10 @@ import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBuffer; -import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; -import org.apache.pulsar.broker.transaction.buffer.matadata.v2.TransactionBufferSnapshotIndexes; +import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot; +import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex; +import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes; +import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; @@ -598,10 +600,10 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { transactionBufferSnapshotIndexService.createReader(TopicName.get(SNAPSHOT_INDEX)).get(); - List indexList = new LinkedList<>(); + List indexList = new LinkedList<>(); for (long i = 0; i < 5; i++) { - indexList.add(new TransactionBufferSnapshotIndexes.TransactionBufferSnapshotIndex(i, i, i, i, i)); + indexList.add(new TransactionBufferSnapshotIndex(i, i, i, i, i)); } TransactionBufferSnapshotIndexes transactionBufferTransactionBufferSnapshotIndexes = @@ -616,7 +618,7 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { assertEquals(transactionBufferTransactionBufferSnapshotIndexes.getIndexList().size(), 5); assertNull(transactionBufferTransactionBufferSnapshotIndexes.getSnapshot()); - TransactionBufferSnapshotIndexes.TransactionBufferSnapshotIndex transactionBufferSnapshotIndex = + TransactionBufferSnapshotIndex transactionBufferSnapshotIndex = transactionBufferTransactionBufferSnapshotIndexes.getIndexList().get(1); assertEquals(transactionBufferSnapshotIndex.getMaxReadPositionLedgerID(), 1L); assertEquals(transactionBufferSnapshotIndex.getMaxReadPositionEntryID(), 1L); @@ -625,7 +627,8 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { assertEquals(transactionBufferSnapshotIndex.getSequenceID(), 1L); } - public static String buildKey(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot snapshot) { + public static String buildKey( + org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot snapshot) { return "multiple-" + snapshot.getSequenceId() + "-" + snapshot.getTopicName(); } @@ -636,8 +639,9 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { TopicName snapshotSegmentTopicName = TopicName.getPartitionedTopicName(snapshotTopic); //send message to create manager ledger - Producer producer = - pulsarClient.newProducer(Schema.AVRO(TransactionBufferSnapshotIndexes.TransactionBufferSnapshot.class)) + Producer producer = + pulsarClient.newProducer(Schema.AVRO( + org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot.class)) .topic(snapshotTopic) .create(); @@ -646,16 +650,16 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { BrokerService brokerService = pulsarService.getBrokerService(); // create snapshot segment writer - SystemTopicTxnBufferSnapshotService + SystemTopicTxnBufferSnapshotService transactionBufferSnapshotSegmentService = new TransactionBufferSnapshotServiceFactory(pulsarClient).getTxnBufferSnapshotSegmentService(); - SystemTopicClient.Writer + SystemTopicClient.Writer segmentWriter = transactionBufferSnapshotSegmentService.createWriter(snapshotSegmentTopicName).get(); // write two snapshot to snapshot segment topic - TransactionBufferSnapshotIndexes.TransactionBufferSnapshot snapshot = - new TransactionBufferSnapshotIndexes.TransactionBufferSnapshot(); + org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot snapshot = + new org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot(); //build and send snapshot snapshot.setTopicName(snapshotTopic); @@ -663,7 +667,7 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { snapshot.setMaxReadPositionLedgerId(2L); snapshot.setMaxReadPositionEntryId(3L); snapshot.setAborts(Collections.singletonList( - new TxnID(1, 1))); + new TxnIDData(1, 1))); segmentWriter.write(buildKey(snapshot), snapshot); snapshot.setSequenceId(2L); @@ -706,8 +710,7 @@ public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Ob ByteBuf headersAndPayload = entry.getDataBuffer(); //skip metadata MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload); - snapshot = Schema.AVRO(TransactionBufferSnapshotIndexes - .TransactionBufferSnapshot.class).decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer()); + snapshot = Schema.AVRO(org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot.class).decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer()); //verify snapshot assertEquals(snapshot.getTopicName(), snapshotTopic); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index f49bc63e1d599..67e7fe268a644 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -97,7 +97,7 @@ import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBufferRecoverCallBack; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBufferState; import org.apache.pulsar.broker.transaction.pendingack.PendingAckStore; -import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; +import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot; import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckReplyCallBack; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java index f3a385b82e342..313660ef3c5df 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java @@ -40,7 +40,7 @@ public class SystemTopicNames { /** * Local topic name for the transaction buffer snapshot segment. */ - public static final String TRANSACTION_BUFFER_SNAPSHOT_SEGMENT = "__transaction_buffer_snapshot_segment"; + public static final String TRANSACTION_BUFFER_SNAPSHOT_SEGMENT = "__transaction_buffer_snapshot_segments"; /** * Local topic name for the transaction buffer snapshot indexes. From 2dec0a554e099e79d05abfad07b30156c07fc58e Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Thu, 20 Oct 2022 13:29:27 +0800 Subject: [PATCH 21/27] CompletableFuture initialize, and getEventKey --- .../mledger/impl/ManagedLedgerFactoryImpl.java | 1 - .../mledger/impl/ReadOnlyManagedLedgerImpl.java | 12 ++++++------ .../SystemTopicBasedTopicPoliciesService.java | 15 ++++++++++++++- .../systopic/TopicPoliciesSystemTopicClient.java | 12 ------------ .../NamespaceEventsSystemTopicServiceTest.java | 4 ++-- 5 files changed, 22 insertions(+), 22 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 73b0d28d9a7e2..4fe98fd1d1039 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -424,7 +424,6 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { public void asyncOpenReadOnlyManagedLedger(String managedLedgerName, AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback, ManagedLedgerConfig config, Object ctx) { - CompletableFuture future = new CompletableFuture<>(); if (closed) { callback.openReadOnlyManagedLedgerFailed( new ManagedLedgerException.ManagedLedgerFactoryClosedException(), ctx); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index 03379dfd82244..8878b2aece0df 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -45,8 +45,8 @@ public ReadOnlyManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bo super(factory, bookKeeper, store, config, scheduledExecutor, name); } - CompletableFuture initialize() { - CompletableFuture future = new CompletableFuture<>(); + CompletableFuture initialize() { + CompletableFuture future = new CompletableFuture<>(); // Fetch the list of existing ledgers in the managed ledger store.getManagedLedgerInfo(name, false, new MetaStoreCallback() { @@ -72,7 +72,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { .setTimestamp(clock.millis()).build(); ledgers.put(lastLedgerId, info); - future.complete(ReadOnlyManagedLedgerImpl.this); + future.complete(null); }).exceptionally(ex -> { if (ex instanceof CompletionException && ex.getCause() instanceof IllegalArgumentException) { @@ -80,7 +80,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(lastLedgerId) .setEntries(0).setSize(0).setTimestamp(clock.millis()).build(); ledgers.put(lastLedgerId, info); - future.complete(ReadOnlyManagedLedgerImpl.this); + future.complete(null); } else { future.completeExceptionally(new ManagedLedgerException(ex)); } @@ -93,7 +93,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(lastLedgerId).setEntries(0) .setSize(0).setTimestamp(clock.millis()).build(); ledgers.put(lastLedgerId, info); - future.complete(ReadOnlyManagedLedgerImpl.this); + future.complete(null); } else { future.completeExceptionally(new ManagedLedgerException(ex)); } @@ -101,7 +101,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { }); } else { // The read-only managed ledger is ready to use - future.complete(ReadOnlyManagedLedgerImpl.this); + future.complete(null); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index ffdbfc34234d5..90c58f5910ff0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service; -import static org.apache.pulsar.broker.systopic.TopicPoliciesSystemTopicClient.TopicPolicyWriter.getEventKey; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; import java.util.HashSet; @@ -542,6 +541,20 @@ private void fetchTopicPoliciesAsyncAndCloseReader(SystemTopicClient.Reader deleteAsync(String key, PulsarEvent event) { return builder.sendAsync(); } - public static String getEventKey(PulsarEvent event) { - return TopicName.get(event.getTopicPoliciesEvent().getDomain(), - event.getTopicPoliciesEvent().getTenant(), - event.getTopicPoliciesEvent().getNamespace(), - event.getTopicPoliciesEvent().getTopic()).toString(); - } - public static String getEventKey(TopicName topicName) { - return TopicName.get(topicName.getDomain().toString(), - topicName.getTenant(), - topicName.getNamespace(), - TopicName.get(topicName.getPartitionedTopicName()).getLocalName()).toString(); - } @Override public void close() throws IOException { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java index 98788869a8ab4..a0d7801f2fd48 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.broker.systopic; -import static org.apache.pulsar.broker.systopic.TopicPoliciesSystemTopicClient.TopicPolicyWriter.getEventKey; +import static org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService.getEventKey; import static org.mockito.Mockito.mock; import com.google.common.collect.Sets; import java.util.HashSet; @@ -122,7 +122,7 @@ public void testSendAndReceiveNamespaceEvents() throws Exception { .policies(policies) .build()) .build(); - systemTopicClientForNamespace1.newWriter().write(event, getEventKey(event)); + systemTopicClientForNamespace1.newWriter().write(getEventKey(event), event); SystemTopicClient.Reader reader = systemTopicClientForNamespace1.newReader(); Message received = reader.readNext(); log.info("Receive pulsar event from system topic : {}", received.getValue()); From 108edc9b2e01e8eb4b455ea12f21463b8f6072e7 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Thu, 20 Oct 2022 13:44:59 +0800 Subject: [PATCH 22/27] rename TransactionBufferSnapshot to TransactionBufferSnapshotSegment --- ...TransactionBufferSnapshotServiceFactory.java | 7 ++++--- .../v2/TransactionBufferSnapshotIndexes.java | 2 +- ...va => TransactionBufferSnapshotSegment.java} | 2 +- .../TopicTransactionBufferRecoverTest.java | 17 +++++++++-------- 4 files changed, 15 insertions(+), 13 deletions(-) rename pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/{TransactionBufferSnapshot.java => TransactionBufferSnapshotSegment.java} (89%) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java index 07e029de054bd..3e7bf51299691 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java @@ -21,6 +21,7 @@ import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot; import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes; +import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.common.events.EventType; @@ -28,7 +29,7 @@ public class TransactionBufferSnapshotServiceFactory { private SystemTopicTxnBufferSnapshotService txnBufferSnapshotService; - private SystemTopicTxnBufferSnapshotService + private SystemTopicTxnBufferSnapshotService txnBufferSnapshotSegmentService; private SystemTopicTxnBufferSnapshotService txnBufferSnapshotIndexService; @@ -36,7 +37,7 @@ public class TransactionBufferSnapshotServiceFactory { public TransactionBufferSnapshotServiceFactory(PulsarClient pulsarClient) { this.txnBufferSnapshotSegmentService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT, - org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot.class); + TransactionBufferSnapshotSegment.class); this.txnBufferSnapshotIndexService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, EventType.TRANSACTION_BUFFER_SNAPSHOT_INDEXES, TransactionBufferSnapshotIndexes.class); this.txnBufferSnapshotService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, @@ -47,7 +48,7 @@ public SystemTopicTxnBufferSnapshotService get return this.txnBufferSnapshotIndexService; } - public SystemTopicTxnBufferSnapshotService + public SystemTopicTxnBufferSnapshotService getTxnBufferSnapshotSegmentService() { return this.txnBufferSnapshotSegmentService; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndexes.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndexes.java index 2bbc01460e41a..28b2b05a4969e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndexes.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndexes.java @@ -35,6 +35,6 @@ public class TransactionBufferSnapshotIndexes { private List indexList; - private TransactionBufferSnapshot snapshot; + private TransactionBufferSnapshotSegment snapshot; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshot.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotSegment.java similarity index 89% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshot.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotSegment.java index cfbd25bc8ba3f..924430d64e964 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshot.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotSegment.java @@ -8,7 +8,7 @@ @Data @AllArgsConstructor @NoArgsConstructor -public class TransactionBufferSnapshot { +public class TransactionBufferSnapshotSegment { private String topicName; private long sequenceId; private long maxReadPositionLedgerId; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 6543752c902bf..f258647a11298 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -65,6 +65,7 @@ import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot; import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex; import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes; +import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment; import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -628,7 +629,7 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { } public static String buildKey( - org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot snapshot) { + TransactionBufferSnapshotSegment snapshot) { return "multiple-" + snapshot.getSequenceId() + "-" + snapshot.getTopicName(); } @@ -639,9 +640,9 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { TopicName snapshotSegmentTopicName = TopicName.getPartitionedTopicName(snapshotTopic); //send message to create manager ledger - Producer producer = + Producer producer = pulsarClient.newProducer(Schema.AVRO( - org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot.class)) + TransactionBufferSnapshotSegment.class)) .topic(snapshotTopic) .create(); @@ -650,16 +651,16 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { BrokerService brokerService = pulsarService.getBrokerService(); // create snapshot segment writer - SystemTopicTxnBufferSnapshotService + SystemTopicTxnBufferSnapshotService transactionBufferSnapshotSegmentService = new TransactionBufferSnapshotServiceFactory(pulsarClient).getTxnBufferSnapshotSegmentService(); - SystemTopicClient.Writer + SystemTopicClient.Writer segmentWriter = transactionBufferSnapshotSegmentService.createWriter(snapshotSegmentTopicName).get(); // write two snapshot to snapshot segment topic - org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot snapshot = - new org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot(); + TransactionBufferSnapshotSegment snapshot = + new TransactionBufferSnapshotSegment(); //build and send snapshot snapshot.setTopicName(snapshotTopic); @@ -710,7 +711,7 @@ public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Ob ByteBuf headersAndPayload = entry.getDataBuffer(); //skip metadata MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload); - snapshot = Schema.AVRO(org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshot.class).decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer()); + snapshot = Schema.AVRO(TransactionBufferSnapshotSegment.class).decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer()); //verify snapshot assertEquals(snapshot.getTopicName(), snapshotTopic); From 83c380fd83f8e4839260ff536c5ed3db583283f3 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Thu, 20 Oct 2022 13:50:25 +0800 Subject: [PATCH 23/27] license header --- .../v2/TransactionBufferSnapshotIndex.java | 18 ++++++++++++++++++ .../v2/TransactionBufferSnapshotSegment.java | 18 ++++++++++++++++++ .../buffer/metadata/v2/TxnIDData.java | 18 ++++++++++++++++++ 3 files changed, 54 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndex.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndex.java index 857ab61aaa769..bc4101e1c92ad 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndex.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotIndex.java @@ -1,3 +1,21 @@ +/** + * 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.transaction.buffer.metadata.v2; import lombok.AllArgsConstructor; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotSegment.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotSegment.java index 924430d64e964..478ec53ba293a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotSegment.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TransactionBufferSnapshotSegment.java @@ -1,3 +1,21 @@ +/** + * 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.transaction.buffer.metadata.v2; import java.util.List; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TxnIDData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TxnIDData.java index ec98a9b04ba4c..8e565c017eaff 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TxnIDData.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/metadata/v2/TxnIDData.java @@ -1,3 +1,21 @@ +/** + * 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.transaction.buffer.metadata.v2; import java.util.Objects; From 6a41ee87ed76ac54943a0d4973d39835b399608e Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Fri, 21 Oct 2022 10:53:32 +0800 Subject: [PATCH 24/27] rollback txnID and optimize close --- .../TransactionBufferSnapshotBaseSystemTopicClient.java | 3 +-- .../org/apache/pulsar/client/api/transaction/TxnID.java | 8 ++------ 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index e4798af80491f..fe168437399d7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -102,8 +102,7 @@ public CompletableFuture deleteAsync(String key, T t) { @Override public void close() throws IOException { - this.producer.closeAsync().join(); - transactionBufferSnapshotBaseSystemTopicClient.removeWriter(this); + this.closeAsync().join(); } @Override diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/transaction/TxnID.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/transaction/TxnID.java index 891a653d69e39..d503abeafc239 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/transaction/TxnID.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/transaction/TxnID.java @@ -20,9 +20,7 @@ import java.io.Serializable; import java.util.Objects; -import lombok.AllArgsConstructor; import lombok.Data; -import lombok.NoArgsConstructor; import org.apache.pulsar.common.classification.InterfaceAudience; import org.apache.pulsar.common.classification.InterfaceStability; @@ -32,8 +30,6 @@ @InterfaceAudience.Public @InterfaceStability.Evolving @Data -@AllArgsConstructor -@NoArgsConstructor public class TxnID implements Serializable { private static final long serialVersionUID = 0L; @@ -43,14 +39,14 @@ public class TxnID implements Serializable { * * @serial */ - private long mostSigBits; + private final long mostSigBits; /* * The least significant 64 bits of this TxnID. * * @serial */ - private long leastSigBits; + private final long leastSigBits; @Override public String toString() { From fe6ec1b9f2eaffa5f5508bb14c6451379a5b09ba Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Fri, 21 Oct 2022 14:46:40 +0800 Subject: [PATCH 25/27] fix --- .../service/TransactionBufferSnapshotServiceFactory.java | 2 +- .../broker/systopic/NamespaceEventsSystemTopicFactory.java | 4 ++-- .../broker/systopic/TopicPoliciesSystemTopicClient.java | 2 +- .../TransactionBufferSnapshotBaseSystemTopicClient.java | 2 +- .../transaction/TopicTransactionBufferRecoverTest.java | 2 +- .../java/org/apache/pulsar/common/events/EventType.java | 2 +- .../org/apache/pulsar/common/naming/SystemTopicNames.java | 6 +++--- 7 files changed, 10 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java index 3e7bf51299691..2220c20323778 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java @@ -36,7 +36,7 @@ public class TransactionBufferSnapshotServiceFactory { public TransactionBufferSnapshotServiceFactory(PulsarClient pulsarClient) { this.txnBufferSnapshotSegmentService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, - EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT, + EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS, TransactionBufferSnapshotSegment.class); this.txnBufferSnapshotIndexService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, EventType.TRANSACTION_BUFFER_SNAPSHOT_INDEXES, TransactionBufferSnapshotIndexes.class); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java index db51fb2763442..c86ad70f6fc06 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java @@ -59,8 +59,8 @@ public static TopicName getSystemTopicName(NamespaceName namespaceName, EventTyp SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME); case TRANSACTION_BUFFER_SNAPSHOT -> TopicName.get(TopicDomain.persistent.value(), namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); - case TRANSACTION_BUFFER_SNAPSHOT_SEGMENT -> TopicName.get(TopicDomain.persistent.value(), namespaceName, - SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT); + case TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS -> TopicName.get(TopicDomain.persistent.value(), namespaceName, + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS); case TRANSACTION_BUFFER_SNAPSHOT_INDEXES -> TopicName.get(TopicDomain.persistent.value(), namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES); }; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java index 4b9fd6b422034..1efa47ff81cad 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java @@ -74,7 +74,7 @@ protected CompletableFuture> newReaderAsyncInternal() { }); } - public static class TopicPolicyWriter implements Writer { + private static class TopicPolicyWriter implements Writer { private final Producer producer; private final SystemTopicClient systemTopicClient; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index fe168437399d7..6029f5ad0ac2f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -206,7 +206,7 @@ protected CompletableFuture> newReaderAsyncInternal() { .createAsync() .thenApply(reader -> { if (log.isDebugEnabled()) { - log.debug("[{}] A new {} reader is created", topicName, schemaType); + log.debug("[{}] A new {} reader is created", topicName, schemaType.getName()); } return new TransactionBufferSnapshotReader<>(reader, this); }); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index f258647a11298..a5292a2406c0d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -636,7 +636,7 @@ public static String buildKey( @Test public void testTransactionBufferSegmentSystemTopic() throws Exception { // init topic and topicName - String snapshotTopic = NAMESPACE1 + "/" + EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENT; + String snapshotTopic = NAMESPACE1 + "/" + EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS; TopicName snapshotSegmentTopicName = TopicName.getPartitionedTopicName(snapshotTopic); //send message to create manager ledger diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/events/EventType.java b/pulsar-common/src/main/java/org/apache/pulsar/common/events/EventType.java index f5506318b9bd6..7093665ade344 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/events/EventType.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/events/EventType.java @@ -36,7 +36,7 @@ public enum EventType { /** * Transaction buffer snapshot segment events. */ - TRANSACTION_BUFFER_SNAPSHOT_SEGMENT, + TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS, /** * Transaction buffer snapshot indexes events. diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java index 313660ef3c5df..e3b7b2cf05dc2 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java @@ -38,9 +38,9 @@ public class SystemTopicNames { public static final String TRANSACTION_BUFFER_SNAPSHOT = "__transaction_buffer_snapshot"; /** - * Local topic name for the transaction buffer snapshot segment. + * Local topic name for the transaction buffer snapshot segments. */ - public static final String TRANSACTION_BUFFER_SNAPSHOT_SEGMENT = "__transaction_buffer_snapshot_segments"; + public static final String TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS = "__transaction_buffer_snapshot_segments"; /** * Local topic name for the transaction buffer snapshot indexes. @@ -56,7 +56,7 @@ public class SystemTopicNames { */ public static final Set EVENTS_TOPIC_NAMES = Collections.unmodifiableSet(Sets.newHashSet(NAMESPACE_EVENTS_LOCAL_NAME, TRANSACTION_BUFFER_SNAPSHOT, - TRANSACTION_BUFFER_SNAPSHOT_INDEXES, TRANSACTION_BUFFER_SNAPSHOT_SEGMENT)); + TRANSACTION_BUFFER_SNAPSHOT_INDEXES, TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS)); public static final TopicName TRANSACTION_COORDINATOR_ASSIGN = TopicName.get(TopicDomain.persistent.value(), From 6ec7dc63e0e1cd22933b28d74ba194d43370db49 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Fri, 21 Oct 2022 15:04:42 +0800 Subject: [PATCH 26/27] fix --- .../TransactionBufferSnapshotBaseSystemTopicClient.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index 6029f5ad0ac2f..11908512d4f83 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -160,8 +160,7 @@ public CompletableFuture hasMoreEventsAsync() { @Override public void close() throws IOException { - this.reader.closeAsync().join(); - transactionBufferSnapshotBaseSystemTopicClient.removeReader(this); + this.closeAsync().join(); } @Override From 39da3713fcc88bb9683f9260e07783b157e2a564 Mon Sep 17 00:00:00 2001 From: liangyepianzhou Date: Mon, 24 Oct 2022 09:31:08 +0800 Subject: [PATCH 27/27] fix --- .../broker/transaction/TopicTransactionBufferRecoverTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index a5292a2406c0d..c1f6ff16e77ab 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -718,7 +718,7 @@ public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Ob assertEquals(snapshot.getSequenceId(), 2L); assertEquals(snapshot.getMaxReadPositionLedgerId(), 2L); assertEquals(snapshot.getMaxReadPositionEntryId(), 3L); - assertEquals(snapshot.getAborts().get(0), new TxnID(1, 1)); + assertEquals(snapshot.getAborts().get(0), new TxnIDData(1, 1)); } }