shustsud opened a new issue #3070:
URL: https://github.com/apache/bookkeeper/issues/3070


   **BUG REPORT**
   
   ***Describe the bug***
   
   checkAllLedgers gets stuck when read throttling is enabled.
   
   ***To Reproduce***
   
   Run TestLedgerChecker with low read throttle value, i.e.,
   
   1. Modify `LedgerChecker checker = new LedgerChecker(bkc);` to 
`LedgerChecker checker = new LedgerChecker(bkc, 1);`.
   
   
https://github.com/apache/bookkeeper/blob/d4d0060bc38995777da5cc2363c7af897a908f27/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java#L521
   
   2. Run TestLedgerChecker#testSingleEntryAfterEnsembleChange.
   
https://github.com/apache/bookkeeper/blob/d4d0060bc38995777da5cc2363c7af897a908f27/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java#L358
   
   3. The execution results are as follows.
   
   ```
   16:02:49.489 [Time-limited test] ERROR org.apache.bookkeeper.bookie.Journal 
- Problems reading from <path> (this is okay if it is the first time starting 
this bookie
   ...
   16:03:01.509 [Time-limited test] ERROR 
org.apache.bookkeeper.proto.PerChannelBookieClient - Cannot connect to 
<ip>:<port> as endpoint resolution failed (probably bookie is down) err 
org.apache.bookkeeper.proto.BookieAddressResolver$BookieIdNotResolvedException: 
Cannot resolve bookieId <ip>:<port>, bookie does not exist or it is not running
   
   test timed out after 120 seconds
   org.junit.runners.model.TestTimedOutException: test timed out after 120 
seconds
        at sun.misc.Unsafe.park(Native Method)
        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.CountDownLatch.await(CountDownLatch.java:231)
        at 
org.apache.bookkeeper.client.TestLedgerChecker$CheckerCallback.waitAndGetResult(TestLedgerChecker.java:68)
        at 
org.apache.bookkeeper.client.TestLedgerChecker.getUnderReplicatedFragments(TestLedgerChecker.java:525)
        at 
org.apache.bookkeeper.client.TestLedgerChecker.testSingleEntryAfterEnsembleChange(TestLedgerChecker.java:387)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
        at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
        at 
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
        at 
org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
        at 
org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.lang.Thread.run(Thread.java:748)
   ```
   
   ***Cause***
   
   The lock on the semaphore is acquired by main thread or also 
`BookKeeperClientWorker-OrderedExecutor` thread.
   
https://github.com/apache/bookkeeper/blob/525a4a03f99afe1dca120477273537a6db46873c/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java#L452
   
https://github.com/apache/bookkeeper/blob/525a4a03f99afe1dca120477273537a6db46873c/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java#L430-L431
   
   The lock on the semaphore is released by 
`BookKeeperClientWorker-OrderedExecutor` thread.
   
https://github.com/apache/bookkeeper/blob/525a4a03f99afe1dca120477273537a6db46873c/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java#L82
   
https://github.com/apache/bookkeeper/blob/525a4a03f99afe1dca120477273537a6db46873c/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java#L316
   
   So if all `BookKeeperClientWorker-OrderedExecutor` threads are waiting to 
acquire the lock, there will be no thread to release the lock. 
   


-- 
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: issues-unsubscr...@bookkeeper.apache.org

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


Reply via email to