-
Notifications
You must be signed in to change notification settings - Fork 3.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Transaction] Transaction buffer snapshot implementation. (#9490)
## Motivation Transaction buffer snapshot to recover ongoing and abort transaction. ## implement snapshot metadata: ``` public class AbortTxnMetadata { long txnIdMostBits; long txnIdLeastBits; long ledgerId; long entryId; } public class TransactionBufferSnapshot { private String topicName; private long maxReadPositionLedgerId; private long maxReadPositionEntryId; private List<AbortTxnMetadata> aborts; } ``` - now only sore the max read position and aborts transaction, follow-up will store ongoing txns. - implement this by a namespace event topic. `__transaction_buffer_snapshot` - add `transactionBufferSnapshotMaxTransactionCount` and `transactionBufferSnapshotMinTimeInMills` to control taking snapshot - add `SystemTopicBaseTxnBufferSnapshotService` to create take snapshot `Writer` and `Reader` and cache the same namespace client.
- Loading branch information
1 parent
1d6aa57
commit 488d384
Showing
33 changed files
with
1,415 additions
and
161 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
86 changes: 86 additions & 0 deletions
86
...c/main/java/org/apache/pulsar/broker/service/SystemTopicBaseTxnBufferSnapshotService.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,86 @@ | ||
/** | ||
* 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.TransactionBufferSystemTopicClient; | ||
import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; | ||
import org.apache.pulsar.client.api.PulsarClient; | ||
import org.apache.pulsar.client.api.PulsarClientException.InvalidTopicNameException; | ||
import org.apache.pulsar.common.events.EventType; | ||
import org.apache.pulsar.common.naming.TopicName; | ||
import org.apache.pulsar.common.util.FutureUtil; | ||
|
||
public class SystemTopicBaseTxnBufferSnapshotService implements TransactionBufferSnapshotService { | ||
|
||
private final Map<TopicName, SystemTopicClient<TransactionBufferSnapshot>> clients; | ||
|
||
private final NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; | ||
|
||
public SystemTopicBaseTxnBufferSnapshotService(PulsarClient client) { | ||
this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); | ||
this.clients = new ConcurrentHashMap<>(); | ||
} | ||
|
||
@Override | ||
public CompletableFuture<Writer<TransactionBufferSnapshot>> createWriter(TopicName topicName) { | ||
return getTransactionBufferSystemTopicClient(topicName).thenCompose(SystemTopicClient::newWriterAsync); | ||
} | ||
|
||
private CompletableFuture<SystemTopicClient<TransactionBufferSnapshot>> getTransactionBufferSystemTopicClient( | ||
TopicName topicName) { | ||
TopicName systemTopicName = NamespaceEventsSystemTopicFactory | ||
.getSystemTopicName(topicName.getNamespaceObject(), EventType.TRANSACTION_BUFFER_SNAPSHOT); | ||
if (systemTopicName == null) { | ||
return FutureUtil.failedFuture( | ||
new InvalidTopicNameException("Can't create SystemTopicBaseTxnBufferSnapshotService, " | ||
+ "because the topicName is null!")); | ||
} | ||
return CompletableFuture.completedFuture(clients.computeIfAbsent(systemTopicName, | ||
(v) -> namespaceEventsSystemTopicFactory | ||
.createTransactionBufferSystemTopicClient(topicName.getNamespaceObject(), this))); | ||
} | ||
|
||
@Override | ||
public CompletableFuture<Reader<TransactionBufferSnapshot>> createReader(TopicName topicName) { | ||
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<TopicName, SystemTopicClient<TransactionBufferSnapshot>> entry : clients.entrySet()) { | ||
entry.getValue().close(); | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
62 changes: 62 additions & 0 deletions
62
...oker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotService.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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.TransactionBufferSystemTopicClient; | ||
import org.apache.pulsar.broker.transaction.buffer.matadata.TransactionBufferSnapshot; | ||
import org.apache.pulsar.common.naming.TopicName; | ||
|
||
public interface TransactionBufferSnapshotService { | ||
|
||
/** | ||
* Create a transaction buffer snapshot writer. | ||
* | ||
* @param topicName {@link TopicName} the topic name | ||
* | ||
* @return {@link CompletableFuture<Writer>} return the future of writer | ||
*/ | ||
CompletableFuture<Writer<TransactionBufferSnapshot>> createWriter(TopicName topicName); | ||
|
||
/** | ||
* Create a transaction buffer snapshot reader. | ||
* | ||
* @param topicName {@link TopicName} the topic name | ||
* | ||
* @return {@link CompletableFuture<Writer>} return the future of reader | ||
*/ | ||
CompletableFuture<Reader<TransactionBufferSnapshot>> createReader(TopicName topicName); | ||
|
||
/** | ||
* Remove a topic client from cache. | ||
* | ||
* @param topicName {@link TopicName} the topic name | ||
* @param transactionBufferSystemTopicClient {@link TransactionBufferSystemTopicClient} the topic client | ||
* | ||
*/ | ||
void removeClient(TopicName topicName, TransactionBufferSystemTopicClient transactionBufferSystemTopicClient); | ||
|
||
/** | ||
* Close transaction buffer snapshot service. | ||
*/ | ||
void close() throws Exception; | ||
|
||
} |
Oops, something went wrong.