[ 
https://issues.apache.org/jira/browse/BOOKKEEPER-846?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14497506#comment-14497506
 ] 

Rakesh R commented on BOOKKEEPER-846:
-------------------------------------

Hi [~fpj],

bq.I'm not sure your assessment and sijie's agree. Why are you referring to 
BKException, rakesh?

IIUC, Sijie's point was "getUnderReplicatedFragments checking the fragment of 
ledger would still get a fragment before bookie run garbage collection." To 
understand more about this part, I've modified the test by adding sleep in 
between {{deleteLedger}} and {{getUnderReplicatedFragments}}.

{code}
        bkc.deleteLedger(lh.getId());
        Thread.sleep(10000);
        Set<LedgerFragment> result = getUnderReplicatedFragments(lh);
{code}

Below is the output log which shows that GC is triggered and deleted the 
ledger. Now, calling {{getUnderReplicatedFragments}} function, it receives 
'NoSuchLedgerExistsException' and return the {{Set<LedgerFragment> result}} 
empty and passes the test case. After seeing the output, I ruled out the 
possibility of no GC cleanup and decided to dig(review) into the code to see 
any other possibilities in {{LedgerChecker}}. There I found one corner case of 
handling {{BKException}} inside {{FullLedgerCallback}} like I mentioned in my 
previous comment.

{code}
2015-04-16 07:51:46,112 - INFO  
[GarbageCollectorThread:GarbageCollectorThread@335] - Garbage collector thread 
forced to perform GC before expiry of wait time.
2015-04-16 07:51:46,112 - INFO  
[GarbageCollectorThread:GarbageCollectorThread$1@250] - delete ledger : 3

2015-04-16 07:51:55,162 - ERROR 
[BookieReadThread-15001-0:ReadEntryProcessorV3@121] - No ledger found while 
reading entry:0 from ledger: 3
2015-04-16 07:51:55,162 - ERROR 
[BookieReadThread-15004-0:ReadEntryProcessorV3@121] - No ledger found while 
reading entry:0 from ledger: 3
{code}

Is this still failing in your env. If yes, can you please re-run with debug 
logs enabled and share the logs. 

> TestLedgerChecker times out
> ---------------------------
>
>                 Key: BOOKKEEPER-846
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-846
>             Project: Bookkeeper
>          Issue Type: Test
>            Reporter: Flavio Junqueira
>            Assignee: Rakesh R
>            Priority: Blocker
>             Fix For: 4.4.0, 4.3.1
>
>         Attachments: BOOKKEEPER-846-001.patch, 
> org.apache.bookkeeper.client.TestLedgerChecker-output.txt
>
>
> {noformat}
> java.lang.Exception: test timed out after 3000 milliseconds
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:502)
>         at org.apache.bookkeeper.client.SyncCounter.block(SyncCounter.java:51)
>         at 
> org.apache.bookkeeper.client.LedgerHandle.addEntry(LedgerHandle.java:480)
>         at 
> org.apache.bookkeeper.client.LedgerHandle.addEntry(LedgerHandle.java:457)
>         at 
> org.apache.bookkeeper.client.TestLedgerChecker.testShouldGetTwoFrgamentsIfTwoBookiesFailedInSameEnsemble(TestLedgerChecker.java:185)
>         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:497)
>         at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:44)
>         at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:15)
>         at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:41)
>         at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:20)
>         at 
> org.junit.internal.runners.statements.FailOnTimeout$1.run(FailOnTimeout.java:28)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to