-
Notifications
You must be signed in to change notification settings - Fork 3.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[Transaction] Transaction buffer snapshot implementation. #9490
Changes from 5 commits
874e7a6
0e96de3
e8c2399
ea517a5
688f27c
cbb1339
bb8bfca
ab97093
2860c65
a7504fb
3dc8464
bc53492
d434987
879a0ef
25290d7
cf58bf5
d4f123e
03da520
77db43d
fabe392
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -1193,6 +1193,13 @@ brokerServicePurgeInactiveFrequencyInSeconds=60 | |||||
transactionCoordinatorEnabled=false | ||||||
transactionMetadataStoreProviderClassName=org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider | ||||||
|
||||||
# Transaction buffer take snapshot interval number | ||||||
transactionBufferTakeSnapshotIntervalNumber=1000 | ||||||
|
||||||
# Transaction buffer take snapshot interval time | ||||||
# Unit : millisecond | ||||||
transactionBufferTakeSnapshotIntervalTime=5000 | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
|
||||||
### --- Packages management service configuration variables (begin) --- ### | ||||||
|
||||||
# Enable the packages management service or not | ||||||
|
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(); | ||
} | ||
} | ||
} |
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; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do we really need to throw a generic There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this method only close pulsarService use, so I think we don't need to think too much . There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Okay |
||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The number is the transactions count or entry count? And it's better to use
max
as the prefix?