Skip to content
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 stable position and lowWaterMark implementation. #9195

Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Fix some comments
  • Loading branch information
congbo committed Jan 15, 2021
commit 50f494d98327b11abb4b9cfc70ce8b7967c86036
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.pulsar.broker.transaction.buffer.impl;

import io.netty.buffer.ByteBuf;

import java.util.List;
import java.util.concurrent.CompletableFuture;
import lombok.extern.slf4j.Slf4j;
Expand Down Expand Up @@ -63,11 +64,13 @@ public CompletableFuture<Position> appendBufferToTxn(TxnID txnId, long sequenceI
topic.getManagedLedger().asyncAddEntry(buffer, new AsyncCallbacks.AddEntryCallback() {
@Override
public void addComplete(Position position, Object ctx) {
if (!ongoingTxns.containsKey(txnId)) {
ongoingTxns.put(txnId, (PositionImpl) position);
PositionImpl firstPosition = ongoingTxns.get(ongoingTxns.firstKey());
//max read position is less than first ongoing transaction message position, so entryId -1
maxReadPosition = PositionImpl.get(firstPosition.getLedgerId(), firstPosition.getEntryId() - 1);
synchronized (TopicTransactionBuffer.this) {
if (!ongoingTxns.containsKey(txnId)) {
ongoingTxns.put(txnId, (PositionImpl) position);
PositionImpl firstPosition = ongoingTxns.get(ongoingTxns.firstKey());
//max read position is less than first ongoing transaction message position, so entryId -1
maxReadPosition = PositionImpl.get(firstPosition.getLedgerId(), firstPosition.getEntryId() - 1);
}
}
completableFuture.complete(position);
}
Expand Down Expand Up @@ -98,8 +101,10 @@ public CompletableFuture<Void> commitTxn(TxnID txnID, long lowWaterMark) {
topic.getManagedLedger().asyncAddEntry(commitMarker, new AsyncCallbacks.AddEntryCallback() {
@Override
public void addComplete(Position position, Object ctx) {
changeMaxReadPosition(txnID);
handleLowWaterMark(txnID, lowWaterMark);
synchronized (TopicTransactionBuffer.this) {
updateMaxReadPosition(txnID);
handleLowWaterMark(txnID, lowWaterMark);
}
completableFuture.complete(null);
}

Expand All @@ -123,8 +128,11 @@ public CompletableFuture<Void> abortTxn(TxnID txnID, long lowWaterMark) {
topic.getManagedLedger().asyncAddEntry(abortMarker, new AsyncCallbacks.AddEntryCallback() {
@Override
public void addComplete(Position position, Object ctx) {
aborts.put(txnID, (PositionImpl) position);
changeMaxReadPosition(txnID);
synchronized (TopicTransactionBuffer.this) {
aborts.put(txnID, (PositionImpl) position);
updateMaxReadPosition(txnID);
handleLowWaterMark(txnID, lowWaterMark);
}
completableFuture.complete(null);
}

Expand All @@ -146,8 +154,10 @@ private void handleLowWaterMark(TxnID txnID, long lowWaterMark) {
topic.getManagedLedger().asyncAddEntry(abortMarker, new AsyncCallbacks.AddEntryCallback() {
@Override
public void addComplete(Position position, Object ctx) {
aborts.put(firstTxn, (PositionImpl) position);
changeMaxReadPosition(firstTxn);
synchronized (TopicTransactionBuffer.this) {
aborts.put(firstTxn, (PositionImpl) position);
updateMaxReadPosition(firstTxn);
}
}

@Override
Expand All @@ -159,7 +169,7 @@ public void addFailed(ManagedLedgerException exception, Object ctx) {
}
}

void changeMaxReadPosition(TxnID txnID) {
void updateMaxReadPosition(TxnID txnID) {
ongoingTxns.remove(txnID);
if (!ongoingTxns.isEmpty()) {
PositionImpl position = ongoingTxns.get(ongoingTxns.firstKey());
Expand Down Expand Up @@ -189,8 +199,10 @@ public boolean isTxnAborted(TxnID txnID) {
public void syncMaxReadPositionForNormalPublish(PositionImpl position) {
// when ongoing transaction is empty, proved that lastAddConfirm is can read max position, because callback
// thread is the same tread, in this time the lastAddConfirm don't content transaction message.
if (ongoingTxns.isEmpty()) {
maxReadPosition = position;
synchronized (TopicTransactionBuffer.this) {
if (ongoingTxns.isEmpty()) {
maxReadPosition = position;
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,13 +36,15 @@
import org.apache.pulsar.client.api.SubscriptionInitialPosition;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.api.transaction.Transaction;
import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.client.impl.transaction.TransactionImpl;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.TenantInfo;
import org.awaitility.Awaitility;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
Expand Down Expand Up @@ -79,7 +81,8 @@ protected void setup() throws Exception {
.enableTransaction(true)
.build();

Thread.sleep(1000 * 3);
Awaitility.await().atMost(3, TimeUnit.SECONDS).until(() -> ((PulsarClientImpl) pulsarClient)
.getTcClient().getState() == TransactionCoordinatorClient.State.READY);
}

@AfterMethod(alwaysRun = true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,10 +35,13 @@
import org.apache.pulsar.client.api.SubscriptionInitialPosition;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.api.transaction.Transaction;
import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.TenantInfo;
import org.awaitility.Awaitility;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
Expand Down Expand Up @@ -75,7 +78,8 @@ protected void setup() throws Exception {
.enableTransaction(true)
.build();

Thread.sleep(1000 * 3);
Awaitility.await().atMost(3, TimeUnit.SECONDS).until(() -> ((PulsarClientImpl) pulsarClient)
.getTcClient().getState() == TransactionCoordinatorClient.State.READY);
}

@AfterMethod(alwaysRun = true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -252,21 +252,6 @@ public static ByteBuf newTxnAbortMarker(long sequenceId, long txnMostBits,
MarkerType.TXN_ABORT, sequenceId, txnMostBits, txnLeastBits);
}

public static TxnMarker parseTxnMarker(ByteBuf payload) throws IOException {
TxnMarker txnMarker = LOCAL_TXN_COMMIT_MARKER.get();
txnMarker.parseFrom(payload, payload.readableBytes());
return txnMarker;
}


private static final FastThreadLocal<TxnMarker> LOCAL_TXN_COMMIT_MARKER = //
new FastThreadLocal<TxnMarker>() {
@Override
protected TxnMarker initialValue() throws Exception {
return new TxnMarker();
}
};

private static ByteBuf newTxnMarker(MarkerType markerType, long sequenceId, long txnMostBits,
long txnLeastBits) {
MessageMetadata msgMetadata = LOCAL_MESSAGE_METADATA.get()
Expand All @@ -278,15 +263,11 @@ private static ByteBuf newTxnMarker(MarkerType markerType, long sequenceId, long
.setTxnidMostBits(txnMostBits)
.setTxnidLeastBits(txnLeastBits);

TxnMarker txnMarker = LOCAL_TXN_COMMIT_MARKER.get()
.clear();


ByteBuf payload = PooledByteBufAllocator.DEFAULT.buffer(txnMarker.getSerializedSize());
ByteBuf payload = PooledByteBufAllocator.DEFAULT.buffer(0);

try {
txnMarker.writeTo(payload);
return Commands.serializeMetadataAndPayload(ChecksumType.Crc32c, msgMetadata, payload);
return Commands.serializeMetadataAndPayload(ChecksumType.Crc32c,
msgMetadata, payload);
} finally {
payload.release();
}
Expand Down
5 changes: 0 additions & 5 deletions pulsar-common/src/main/proto/PulsarMarkers.proto
Original file line number Diff line number Diff line change
Expand Up @@ -81,8 +81,3 @@ message MarkersMessageIdData {
required uint64 ledger_id = 1;
required uint64 entry_id = 2;
}


/// --- Transaction marker ---
message TxnMarker {
}