Skip to content

Commit

Permalink
[Transaction][buffer] Add basic operation of transaction (apache#4738)
Browse files Browse the repository at this point in the history

*Modifications*

Add primary operation of transaction. Keep all actions persistently.

Describe the modifications you've done.

- add commit operation
- add abort operation
- add openreader operation
  • Loading branch information
zymap authored and sijie committed Aug 2, 2019
1 parent 3ae0e61 commit e7195eb
Show file tree
Hide file tree
Showing 15 changed files with 1,687 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1465,7 +1465,7 @@ void invalidateLedgerHandle(ReadHandle ledgerHandle, Throwable t) {
}
}

void asyncReadEntry(PositionImpl position, ReadEntryCallback callback, Object ctx) {
public void asyncReadEntry(PositionImpl position, ReadEntryCallback callback, Object ctx) {
LedgerHandle currentLedger = this.currentLedger;
if (log.isDebugEnabled()) {
log.debug("[{}] Reading entry ledger {}: {}", name, position.getLedgerId(), position.getEntryId());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ private ReadOnlyCursor createReadOnlyCursor(PositionImpl startPosition) {
}

@Override
void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) {
public void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) {
this.getLedgerHandle(position.getLedgerId()).thenAccept((ledger) -> {
asyncReadEntry(ledger, position, callback, ctx);
}).exceptionally((ex) -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@
public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCallback {

// Managed ledger associated with the topic
private final ManagedLedger ledger;
protected final ManagedLedger ledger;

// Subscriptions to this topic
private final ConcurrentOpenHashMap<String, PersistentSubscription> subscriptions;
Expand Down
44 changes: 44 additions & 0 deletions pulsar-transaction/buffer/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,50 @@
<artifactId>pulsar-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.pulsar</groupId>
<artifactId>pulsar-broker</artifactId>
<version>${project.version}</version>
<scope>compile</scope>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>managed-ledger-original</artifactId>
<version>${project.parent.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>managed-ledger-original</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>

</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.xolstice.maven.plugins</groupId>
<artifactId>protobuf-maven-plugin</artifactId>
<version>${protobuf-maven-plugin.version}</version>
<configuration>
<protocArtifact>com.google.protobuf:protoc:${protoc3.version}:exe:${os.detected.classifier}</protocArtifact>
<checkStaleness>true</checkStaleness>
</configuration>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>compile</goal>
<goal>test-compile</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@
* commits the buffer again.
*/
@Beta
public interface TransactionBuffer extends AutoCloseable {
public interface TransactionBuffer {

/**
* Return the metadata of a transaction in the buffer.
Expand Down Expand Up @@ -95,9 +95,7 @@ public interface TransactionBuffer extends AutoCloseable {
* @throws org.apache.pulsar.transaction.buffer.exceptions.TransactionNotFoundException if the transaction
* is not in the buffer.
*/
CompletableFuture<Void> commitTxn(TxnID txnID,
long committedAtLedgerId,
long committedAtEntryId);
CompletableFuture<Void> commitTxn(TxnID txnID, long committedAtLedgerId, long committedAtEntryId);

/**
* Abort the transaction and all the entries of this transaction will
Expand All @@ -123,9 +121,9 @@ CompletableFuture<Void> commitTxn(TxnID txnID,
CompletableFuture<Void> purgeTxns(List<Long> dataLedgers);

/**
* {@inheritDoc}
* Close the buffer asynchronously.
*
* @return
*/
@Override
void close();

CompletableFuture<Void> closeAsync();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/**
* 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.transaction.buffer;

import java.util.Set;
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.mledger.Position;
import org.apache.pulsar.transaction.impl.common.TxnID;

/**
* The transaction Cursor maintains the index of all transactions.
*/
public interface TransactionCursor {
/**
* Get the specified transaction meta.
*
* @param txnID
* @param createIfNotExist
* @return
*/
CompletableFuture<TransactionMeta> getTxnMeta(TxnID txnID, boolean createIfNotExist);

/**
* Commit transaction.
*
* @param committedLedgerId the ledger which txn committed at.
* @param committedEntryId the entry which txn committed at.
* @param txnID the id which txn committed.
* @param position the commit position at transaction log.
* @return
*/
CompletableFuture<Void> commitTxn(long committedLedgerId, long committedEntryId, TxnID txnID, Position position);

/**
* Abort transaction.
*
* @param txnID aborted transaction id.
* @return
*/
CompletableFuture<Void> abortTxn(TxnID txnID);

/**
* Get all the transaction id on the specified ledger.
*
* @param ledgerId the transaction committed ledger id
* @return
*/
CompletableFuture<Set<TxnID>> getAllTxnsCommittedAtLedger(long ledgerId);

/**
* Remove the transactions on the specified ledger.
*
* @param ledgerId the remove transaction id
* @return
*/
CompletableFuture<Void> removeTxnsCommittedAtLedger(long ledgerId);
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,9 @@
package org.apache.pulsar.transaction.buffer;

import com.google.common.annotations.Beta;
import java.util.SortedMap;
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.mledger.Position;
import org.apache.pulsar.transaction.impl.common.TxnID;
import org.apache.pulsar.transaction.impl.common.TxnStatus;

Expand Down Expand Up @@ -49,4 +52,59 @@ public interface TransactionMeta {
*/
int numEntries();

/**
* Return the committed ledger id at data ledger.
*
* @return the committed ledger id
*/
long committedAtLedgerId();

/**
* Return the committed entry id at data ledger.
*
* @return the committed entry id
*/
long committedAtEntryId();

/**
* Return the last sequence id.
*
* @return the last sequence id
*/
long lastSequenceId();

/**
* Read the entries from start sequence id.
*
* @param num the entries number need to read
* @param startSequenceId the start position of the entries
* @return
*/
CompletableFuture<SortedMap<Long, Position>> readEntries(int num, long startSequenceId);

/**
* Add transaction entry into the transaction.
*
* @param sequenceId the message sequence id
* @param position the position of transaction log
* @return
*/
CompletableFuture<Void> appendEntry(long sequenceId, Position position);

/**
* Mark the transaction is committed.
*
* @param committedAtLedgerId
* @param committedAtEntryId
* @return
*/
CompletableFuture<TransactionMeta> commitTxn(long committedAtLedgerId, long committedAtEntryId);

/**
* Mark the transaction is aborted.
*
* @return
*/
CompletableFuture<TransactionMeta> abortTxn();

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/**
* 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.transaction.buffer.exceptions;

/**
* Exception is thrown when no transactions found committed at a given ledger.
*/
public class NoTxnsCommittedAtLedgerException extends TransactionBufferException {

private static final long serialVersionUID = 0L;

public NoTxnsCommittedAtLedgerException(String message) {
super(message);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.bookkeeper.mledger.Position;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.transaction.buffer.TransactionBuffer;
import org.apache.pulsar.transaction.buffer.TransactionBufferReader;
import org.apache.pulsar.transaction.buffer.TransactionMeta;
Expand Down Expand Up @@ -79,6 +81,49 @@ public int numEntries() {
}
}

@Override
public long committedAtLedgerId() {
return committedAtLedgerId;
}

@Override
public long committedAtEntryId() {
return committedAtEntryId;
}

@Override
public long lastSequenceId() {
return entries.lastKey();
}

@Override
public CompletableFuture<SortedMap<Long, Position>> readEntries(int num, long startSequenceId) {
return FutureUtil.failedFuture(new UnsupportedOperationException());
}

@Override
public CompletableFuture<Void> appendEntry(long sequenceId, Position position) {
return FutureUtil.failedFuture(new UnsupportedOperationException());
}

@Override
public CompletableFuture<TransactionMeta> commitTxn(long committedAtLedgerId, long committedAtEntryId) {
try {
return CompletableFuture.completedFuture(commitAt(committedAtLedgerId, committedAtEntryId));
} catch (UnexpectedTxnStatusException e) {
return FutureUtil.failedFuture(e);
}
}

@Override
public CompletableFuture<TransactionMeta> abortTxn() {
try {
return CompletableFuture.completedFuture(abort());
} catch (UnexpectedTxnStatusException e) {
return FutureUtil.failedFuture(e);
}
}

synchronized TxnBuffer abort() throws UnexpectedTxnStatusException {
if (TxnStatus.OPEN != status) {
throw new UnexpectedTxnStatusException(txnid, TxnStatus.OPEN, status);
Expand Down Expand Up @@ -150,7 +195,6 @@ public TransactionBufferReader newReader(long sequenceId) throws TransactionNotS

final ConcurrentMap<TxnID, TxnBuffer> buffers;
final Map<Long, Set<TxnID>> txnIndex;

public InMemTransactionBuffer() {
this.buffers = new ConcurrentHashMap<>();
this.txnIndex = new HashMap<>();
Expand Down Expand Up @@ -285,7 +329,9 @@ public CompletableFuture<Void> purgeTxns(List<Long> dataLedgers) {
}

@Override
public void close() {
public CompletableFuture<Void> closeAsync() {
buffers.values().forEach(TxnBuffer::close);
return CompletableFuture.completedFuture(null);
}

}
Loading

0 comments on commit e7195eb

Please sign in to comment.