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

[fix][txn] Fix append txn message is lower than lowWaterMark decrease PendingWriteOps #16266

Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -2955,6 +2955,7 @@ public void publishTxnMessage(TxnID txnID, ByteBuf headersAndPayload, PublishCon
throwable = throwable.getCause();
if (throwable instanceof NotAllowedException) {
publishContext.completed((NotAllowedException) throwable, -1, -1);
decrementPendingWriteOpsAndCheck();
return null;
} else if (!(throwable instanceof ManagedLedgerException)) {
throwable = new ManagedLedgerException(throwable);
Expand Down
Expand Up @@ -29,6 +29,7 @@
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import lombok.Cleanup;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
Expand Down Expand Up @@ -60,6 +61,7 @@
import org.apache.pulsar.transaction.coordinator.TransactionMetadataStoreState;
import org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStore;
import org.awaitility.Awaitility;
import org.powermock.reflect.Whitebox;
import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
Expand Down Expand Up @@ -321,12 +323,18 @@ public void testTBLowWaterMarkEndToEnd() throws Exception {
Field field = TransactionImpl.class.getDeclaredField("state");
field.setAccessible(true);
field.set(txn1, TransactionImpl.State.OPEN);

AtomicLong pendingWriteOps = Whitebox.getInternalState(getPulsarServiceList().get(0)
.getBrokerService().getTopic(TopicName.get(TOPIC).toString(),
false).get().get(), "pendingWriteOps");
try {
producer.newMessage(txn1).send();
fail();
} catch (PulsarClientException.NotAllowedException ignore) {
// no-op
}

assertEquals(pendingWriteOps.get(), 0);
}

@Test
Expand Down