You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/02/05 17:31:22 UTC

[GitHub] [pulsar] merlimat opened a new pull request #14131: Fixed deadlock on txn semaphore permit exhaustion

merlimat opened a new pull request #14131:
URL: https://github.com/apache/pulsar/pull/14131


   ### 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)
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on pull request #14131: Fixed deadlock on txn semaphore permit exhaustion

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on pull request #14131:
URL: https://github.com/apache/pulsar/pull/14131#issuecomment-1030802730


   > backpressure should eventually be applied at the start of the transactions
   
   I remember the semaphore is for limiting the transaction buffer calls while reply transaction logs. @congbobo184 Please help double confirm.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] github-actions[bot] removed a comment on pull request #14131: Fixed deadlock on txn semaphore permit exhaustion

Posted by GitBox <gi...@apache.org>.
github-actions[bot] removed a comment on pull request #14131:
URL: https://github.com/apache/pulsar/pull/14131#issuecomment-1030665433






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui merged pull request #14131: Fixed deadlock on txn semaphore permit exhaustion

Posted by GitBox <gi...@apache.org>.
codelipenghui merged pull request #14131:
URL: https://github.com/apache/pulsar/pull/14131


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] github-actions[bot] commented on pull request #14131: Fixed deadlock on txn semaphore permit exhaustion

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #14131:
URL: https://github.com/apache/pulsar/pull/14131#issuecomment-1030665429






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org