Skip to content

Commit

Permalink
Fixed deadlock on txn semaphore permit exhaustion (apache#14131)
Browse files Browse the repository at this point in the history
### Motivation

Removing semaphore on the end of transactions operations. The semaphore is not very useful here as we are already closing the transactions (backpressure should eventually be applied at the starting of the transactions). 

The semaphore here is being acquired from a BK callback thread and it causes a deadlock in broker when the semaphore is full, because the response that will release the permits on the semaphore are coming from either the same thread or a thread in the same condition.

```
sun.misc.Unsafe.park(Unsafe.java)
java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
java.util.concurrent.Semaphore.acquire(Semaphore.java:312)
org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferHandlerImpl.canSendRequest(TransactionBufferHandlerImpl.java:216)
org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferHandlerImpl.endTxnOnTopic(TransactionBufferHandlerImpl.java:93)
org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl.commitTxnOnTopic(TransactionBufferClientImpl.java:50)
org.apache.pulsar.broker.TransactionMetadataStoreService.lambda$null$23(TransactionMetadataStoreService.java:484)
org.apache.pulsar.broker.TransactionMetadataStoreService$$Lambda$1253.accept()
java.util.ArrayList.forEach(ArrayList.java:1257)
org.apache.pulsar.broker.TransactionMetadataStoreService.lambda$endTxnInTransactionBuffer$25(TransactionMetadataStoreService.java:481)
org.apache.pulsar.broker.TransactionMetadataStoreService$$Lambda$1251.accept()
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
java.util.concurrent.CompletableFuture.uniWhenCompleteStage(CompletableFuture.java:778)
java.util.concurrent.CompletableFuture.whenComplete(CompletableFuture.java:2140)
org.apache.pulsar.broker.TransactionMetadataStoreService.endTxnInTransactionBuffer(TransactionMetadataStoreService.java:458)
org.apache.pulsar.broker.TransactionMetadataStoreService.lambda$null$11(TransactionMetadataStoreService.java:349)
org.apache.pulsar.broker.TransactionMetadataStoreService$$Lambda$1309.accept()
java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:656)
java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:632)
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1962)
org.apache.pulsar.transaction.coordinator.impl.MLTransactionLogImpl$3.addComplete(MLTransactionLogImpl.java:160)
org.apache.bookkeeper.mledger.impl.OpAddEntry.safeRun(OpAddEntry.java:228)
org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
java.lang.Thread.run(Thread.java:748)
```
  • Loading branch information
merlimat authored and nicklixinyang committed Apr 20, 2022
1 parent db8a80b commit 7a2d7bb
Show file tree
Hide file tree
Showing 2 changed files with 0 additions and 43 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -28,14 +28,12 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.transaction.TransactionBufferClientException;
import org.apache.pulsar.client.api.transaction.TransactionBufferClientException.ReachMaxPendingOpsException;
import org.apache.pulsar.client.api.transaction.TxnID;
import org.apache.pulsar.client.impl.ClientCnx;
import org.apache.pulsar.client.impl.PulsarClientImpl;
Expand All @@ -52,8 +50,6 @@ public class TransactionBufferHandlerImpl implements TransactionBufferHandler {
private final AtomicLong requestIdGenerator = new AtomicLong();
private final long operationTimeoutInMills;
private final HashedWheelTimer timer;
private final Semaphore semaphore;
private final boolean blockIfReachMaxPendingOps;
private final PulsarClient pulsarClient;

private final LoadingCache<String, CompletableFuture<ClientCnx>> cache = CacheBuilder.newBuilder()
Expand All @@ -77,8 +73,6 @@ public TransactionBufferHandlerImpl(PulsarClient pulsarClient,
this.pulsarClient = pulsarClient;
this.pendingRequests = new ConcurrentSkipListMap<>();
this.operationTimeoutInMills = 3000L;
this.semaphore = new Semaphore(10000);
this.blockIfReachMaxPendingOps = true;
this.timer = timer;
}

Expand All @@ -90,9 +84,6 @@ public CompletableFuture<TxnID> endTxnOnTopic(String topic, long txnIdMostBits,
topic, new TxnID(txnIdMostBits, txnIdLeastBits), action.getValue());
}
CompletableFuture<TxnID> cb = new CompletableFuture<>();
if (!canSendRequest(cb)) {
return cb;
}
long requestId = requestIdGenerator.getAndIncrement();
ByteBuf cmd = Commands.newEndTxnOnPartition(requestId, txnIdLeastBits, txnIdMostBits,
topic, action, lowWaterMark);
Expand All @@ -108,9 +99,6 @@ public CompletableFuture<TxnID> endTxnOnSubscription(String topic, String subscr
topic, new TxnID(txnIdMostBits, txnIdLeastBits), action.getValue());
}
CompletableFuture<TxnID> cb = new CompletableFuture<>();
if (!canSendRequest(cb)) {
return cb;
}
long requestId = requestIdGenerator.getAndIncrement();
ByteBuf cmd = Commands.newEndTxnOnSubscription(requestId, txnIdLeastBits, txnIdMostBits,
topic, subscription, action, lowWaterMark);
Expand Down Expand Up @@ -210,29 +198,9 @@ public void handleEndTxnOnSubscriptionResponse(long requestId,
onResponse(op);
}

private boolean canSendRequest(CompletableFuture<?> callback) {
try {
if (blockIfReachMaxPendingOps) {
semaphore.acquire();
} else {
if (!semaphore.tryAcquire()) {
callback.completeExceptionally(new ReachMaxPendingOpsException("Reach max pending ops."));
return false;
}
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
callback.completeExceptionally(TransactionBufferClientException.unwrap(e));
return false;
}
return true;
}


void onResponse(OpRequestSend op) {
ReferenceCountUtil.safeRelease(op.byteBuf);
op.recycle();
semaphore.release();
}

private static final class OpRequestSend {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Semaphore;

import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
Expand Down Expand Up @@ -248,16 +247,6 @@ public void testTransactionBufferLookUp() throws Exception {

@Test
public void testTransactionBufferHandlerSemaphore() throws Exception {

Field field = TransactionBufferClientImpl.class.getDeclaredField("tbHandler");
field.setAccessible(true);
TransactionBufferHandlerImpl transactionBufferHandler = (TransactionBufferHandlerImpl) field.get(tbClient);

field = TransactionBufferHandlerImpl.class.getDeclaredField("semaphore");
field.setAccessible(true);
field.set(transactionBufferHandler, new Semaphore(2));


String topic = "persistent://" + namespace + "/testTransactionBufferHandlerSemaphore";
String subName = "test";

Expand Down

0 comments on commit 7a2d7bb

Please sign in to comment.