[
https://issues.apache.org/jira/browse/BOOKKEEPER-152?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13206370#comment-13206370
]
[email protected] commented on BOOKKEEPER-152:
----------------------------------------------------------
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3737/#review5029
-----------------------------------------------------------
ah, thanks Ivan for fixing the wrong assignment of ledger length.
the new patch is good to me. but I don't like the new callback name
'RecoveredDataCallback', which sounds like that the data is returned by some
recovery actions although it doesn't. This data is retrieved by reading last
confirmed without recovery. I prefer changing this callback to
'ReadLastConfirmedDataCallback' and the method to
'readLastConfirmedDataComplete'.
- Sijie
On 2012-02-10 15:23:20, Ivan Kelly wrote:
bq.
bq. -----------------------------------------------------------
bq. This is an automatically generated e-mail. To reply, visit:
bq. https://reviews.apache.org/r/3737/
bq. -----------------------------------------------------------
bq.
bq. (Updated 2012-02-10 15:23:20)
bq.
bq.
bq. Review request for bookkeeper.
bq.
bq.
bq. Summary
bq. -------
bq.
bq. Proposed fix ensures that at least one of each quorum replies to
ReadLastConfirmed.
bq.
bq. Refactors code a bit to make the read last confirmed common for recovery
and standalone read last confirmed.
bq.
bq. The bug here was actually that we were waiting for quorumSize responses,
from the bookies, when really all we need to get a response from one bookie in
each possible quorum. in the 2/2 case as above this means only 1 bookie need
response.
bq.
bq. There's a fix for the timeouts and an improvement in fencing which fixing
this uncovered.
bq.
bq.
bq. This addresses bug BOOKKEEPER-152.
bq. https://issues.apache.org/jira/browse/BOOKKEEPER-152
bq.
bq.
bq. Diffs
bq. -----
bq.
bq.
bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DigestManager.java
ae375ec
bq.
bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DistributionSchedule.java
f2ed6bd
bq.
bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
e3d1847
bq.
bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
4625bbb
bq.
bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java
43e999d
bq.
bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java
4a88747
bq.
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
a68fc8c
bq.
bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java
cbd2277
bq.
bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BaseTestCase.java
da52ca5
bq.
bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java
5873255
bq.
bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerRecoveryTest.java
77a2f69
bq.
bq. Diff: https://reviews.apache.org/r/3737/diff
bq.
bq.
bq. Testing
bq. -------
bq.
bq.
bq. Thanks,
bq.
bq. Ivan
bq.
bq.
> Can't recover a ledger whose current ensemble contain failed bookie.
> --------------------------------------------------------------------
>
> Key: BOOKKEEPER-152
> URL: https://issues.apache.org/jira/browse/BOOKKEEPER-152
> Project: Bookkeeper
> Issue Type: Bug
> Components: bookkeeper-client
> Affects Versions: 4.0.0
> Reporter: Sijie Guo
> Assignee: Ivan Kelly
> Fix For: 4.1.0
>
> Attachments: BK-152.draft.patch, BOOKKEEPER-152.diff,
> BOOKKEEPER-152.diff, BOOKKEEPER-152.diff, BOOKKEEPER-152.diff
>
>
> Suppose we have a unclosed ledger L, whose ensemble size is 2, quorum size is
> 2. the ledger's current ensemble is <bk1, bk2>.
> bk2 is crashed.
> we use recovery tool to recover entries in bk2. $
> bookkeeper-server/bin/bookkeeper org.apache.bookkeeper.tools.BookKeeperTools
> bk2
> recovery failed due to recovery tool can't open ledger L, since ledger L
> doesn't have enough quorum to readLastConfirmed entry.
> (asyncOpenLedgerNoRecovery)
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators:
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira