-
Notifications
You must be signed in to change notification settings - Fork 895
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
Issue 3070: Fix issue where checkAllLedgers could get stuck when read throttling is enabled #3655
Conversation
@horizonzy Would you please help take a look at this PR? thanks a lot. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM.
@hangc0276 @nicoloboschi @zymap |
@massakam Merged, thanks for your reminder. |
Thank you! |
…is enabled (#3655) ### Motivation Some time ago, I fixed a bug where check of all ledgers periodically run by the auditor got stuck (cf. #3214). That PR was merged so we cherry-picked it and released it, but `checkAllLedgers` still got stuck when the `inFlightReadEntryNumInLedgerChecker` value was very small. The cause is that multiple `BookKeeperClientWorker-OrderedExecutor` threads are blocked waiting for the semaphore to be released. This semaphore is released by the `BookKeeperClientWorker-OrderedExecutor` threads, so they should not run the `LedgerChecker#checkLedger` method. ``` "BookKeeperClientWorker-OrderedExecutor-1-0" #60 prio=5 os_prio=0 tid=0x00007f680401d800 nid=0x75e8 waiting on condition [0x00007f684a6e9000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000000a2121780> (a java.util.concurrent.Semaphore$NonfairSync) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304) at java.util.concurrent.Semaphore.acquire(Semaphore.java:467) at org.apache.bookkeeper.client.LedgerChecker.acquirePermit(LedgerChecker.java:163) at org.apache.bookkeeper.client.LedgerChecker.checkLedger(LedgerChecker.java:431) at org.apache.bookkeeper.replication.Auditor.lambda$null$4(Auditor.java:1216) at org.apache.bookkeeper.replication.Auditor$$Lambda$137/1553334572.openComplete(Unknown Source) at org.apache.bookkeeper.client.LedgerOpenOp.openComplete(LedgerOpenOp.java:232) at org.apache.bookkeeper.client.LedgerOpenOp$2.readLastConfirmedComplete(LedgerOpenOp.java:218) at org.apache.bookkeeper.client.LedgerHandle$14.getLacComplete(LedgerHandle.java:1718) at org.apache.bookkeeper.client.PendingReadLacOp.readLacComplete(PendingReadLacOp.java:154) at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion$1.readLacComplete(PerChannelBookieClient.java:1801) at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion.handleV3Response(PerChannelBookieClient.java:1840) at org.apache.bookkeeper.proto.PerChannelBookieClient$3.safeRun(PerChannelBookieClient.java:1473) at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(Thread.java:750) ``` ### Changes In the `Auditor` class, have the `LedgerChecker#checkLedger` method run in a different thread from the `BookKeeperClientWorker-OrderedExecutor` threads. Master Issue: #3070 (cherry picked from commit 4ca4b4e)
… throttling is enabled apache#3655
…is enabled (apache#3655) Some time ago, I fixed a bug where check of all ledgers periodically run by the auditor got stuck (cf. apache#3214). That PR was merged so we cherry-picked it and released it, but `checkAllLedgers` still got stuck when the `inFlightReadEntryNumInLedgerChecker` value was very small. The cause is that multiple `BookKeeperClientWorker-OrderedExecutor` threads are blocked waiting for the semaphore to be released. This semaphore is released by the `BookKeeperClientWorker-OrderedExecutor` threads, so they should not run the `LedgerChecker#checkLedger` method. ``` "BookKeeperClientWorker-OrderedExecutor-1-0" apache#60 prio=5 os_prio=0 tid=0x00007f680401d800 nid=0x75e8 waiting on condition [0x00007f684a6e9000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000000a2121780> (a java.util.concurrent.Semaphore$NonfairSync) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304) at java.util.concurrent.Semaphore.acquire(Semaphore.java:467) at org.apache.bookkeeper.client.LedgerChecker.acquirePermit(LedgerChecker.java:163) at org.apache.bookkeeper.client.LedgerChecker.checkLedger(LedgerChecker.java:431) at org.apache.bookkeeper.replication.Auditor.lambda$null$4(Auditor.java:1216) at org.apache.bookkeeper.replication.Auditor$$Lambda$137/1553334572.openComplete(Unknown Source) at org.apache.bookkeeper.client.LedgerOpenOp.openComplete(LedgerOpenOp.java:232) at org.apache.bookkeeper.client.LedgerOpenOp$2.readLastConfirmedComplete(LedgerOpenOp.java:218) at org.apache.bookkeeper.client.LedgerHandle$14.getLacComplete(LedgerHandle.java:1718) at org.apache.bookkeeper.client.PendingReadLacOp.readLacComplete(PendingReadLacOp.java:154) at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion$1.readLacComplete(PerChannelBookieClient.java:1801) at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion.handleV3Response(PerChannelBookieClient.java:1840) at org.apache.bookkeeper.proto.PerChannelBookieClient$3.safeRun(PerChannelBookieClient.java:1473) at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(Thread.java:750) ``` In the `Auditor` class, have the `LedgerChecker#checkLedger` method run in a different thread from the `BookKeeperClientWorker-OrderedExecutor` threads. Master Issue: apache#3070 (cherry picked from commit 4ca4b4e)
…is enabled (apache#3655) ### Motivation Some time ago, I fixed a bug where check of all ledgers periodically run by the auditor got stuck (cf. apache#3214). That PR was merged so we cherry-picked it and released it, but `checkAllLedgers` still got stuck when the `inFlightReadEntryNumInLedgerChecker` value was very small. The cause is that multiple `BookKeeperClientWorker-OrderedExecutor` threads are blocked waiting for the semaphore to be released. This semaphore is released by the `BookKeeperClientWorker-OrderedExecutor` threads, so they should not run the `LedgerChecker#checkLedger` method. ``` "BookKeeperClientWorker-OrderedExecutor-1-0" apache#60 prio=5 os_prio=0 tid=0x00007f680401d800 nid=0x75e8 waiting on condition [0x00007f684a6e9000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000000a2121780> (a java.util.concurrent.Semaphore$NonfairSync) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304) at java.util.concurrent.Semaphore.acquire(Semaphore.java:467) at org.apache.bookkeeper.client.LedgerChecker.acquirePermit(LedgerChecker.java:163) at org.apache.bookkeeper.client.LedgerChecker.checkLedger(LedgerChecker.java:431) at org.apache.bookkeeper.replication.Auditor.lambda$null$4(Auditor.java:1216) at org.apache.bookkeeper.replication.Auditor$$Lambda$137/1553334572.openComplete(Unknown Source) at org.apache.bookkeeper.client.LedgerOpenOp.openComplete(LedgerOpenOp.java:232) at org.apache.bookkeeper.client.LedgerOpenOp$2.readLastConfirmedComplete(LedgerOpenOp.java:218) at org.apache.bookkeeper.client.LedgerHandle$14.getLacComplete(LedgerHandle.java:1718) at org.apache.bookkeeper.client.PendingReadLacOp.readLacComplete(PendingReadLacOp.java:154) at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion$1.readLacComplete(PerChannelBookieClient.java:1801) at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion.handleV3Response(PerChannelBookieClient.java:1840) at org.apache.bookkeeper.proto.PerChannelBookieClient$3.safeRun(PerChannelBookieClient.java:1473) at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(Thread.java:750) ``` ### Changes In the `Auditor` class, have the `LedgerChecker#checkLedger` method run in a different thread from the `BookKeeperClientWorker-OrderedExecutor` threads. Master Issue: apache#3070
Motivation
Some time ago, I fixed a bug where check of all ledgers periodically run by the auditor got stuck (cf. #3214). That PR was merged so we cherry-picked it and released it, but
checkAllLedgers
still got stuck when theinFlightReadEntryNumInLedgerChecker
value was very small.The cause is that multiple
BookKeeperClientWorker-OrderedExecutor
threads are blocked waiting for the semaphore to be released. This semaphore is released by theBookKeeperClientWorker-OrderedExecutor
threads, so they should not run theLedgerChecker#checkLedger
method.Changes
In the
Auditor
class, have theLedgerChecker#checkLedger
method run in a different thread from theBookKeeperClientWorker-OrderedExecutor
threads.Master Issue: #3070