[
https://issues.apache.org/jira/browse/BOOKKEEPER-846?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14497891#comment-14497891
]
Rakesh R commented on BOOKKEEPER-846:
-------------------------------------
bq. if the GC doesn't kick in, there is one fragment for this ledger. then the
ledger checker will respond 1, which it fails the test as what Flavio
encountered. right?
If the GC doesn't kick in, LedgerChecker will be able to read the ledger and
will return OK. I think this is not a straight forward case.
AFAIK, there are two places it is reading the entry from the ledger. For this
we have two different callbacks {{EntryExistsCallback}} and
{{FullLedgerCallback}}. Probably [[email protected]] can add more on the
reason, why two times ledger#read operation.
I was suspecting one case where it returns OK during {{EntryExistsCallback}}.
Then, it will go to read the ledger second time, now GC triggers and deletes
the ledger. The read callback would get {{NoSuchLedgerExistsException}}. But
{{FullLedgerCallback}} is handling in generic way {{rc != BKException.Code.OK}}
and treated as badFragments. IMHO, instead it could have conditions to filter
out {{NoSuchLedgerExistsException}} also, what do you say?
{code}
public void operationComplete(int rc, LedgerFragment result) {
if (rc != BKException.Code.OK) {
badFragments.add(result);
}
if (numFragments.decrementAndGet() == 0) {
cb.operationComplete(BKException.Code.OK, badFragments);
}
}
{code}
> 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)