Entry is lost when recovering a ledger with not enough bookies.
---------------------------------------------------------------
Key: BOOKKEEPER-150
URL: https://issues.apache.org/jira/browse/BOOKKEEPER-150
Project: Bookkeeper
Issue Type: Bug
Components: bookkeeper-client
Affects Versions: 4.0.0
Reporter: Sijie Guo
Assignee: Sijie Guo
Fix For: 4.1.0
suppose a ledger is created as ensemble size 3 and quorum size 3.
3 entries is added in this ledger, entry ids are 0, 1, 2.
this ledger is not closed. then a bookie server is down.
the ledger is opened. it would be recovered in following steps:
1) retrieve LAC from all bookie ensemble to get maxAddConfirmed. then
maxAddPushed would be 2 and maxAddConfirmed would be 1. then lastAddConfirmed
would be 1.
2) doRecovery read lastAddConfirmed + 1 (2). it would return right data since
there is still 2 replicas.
3) doRecovery add entry 2. but it would fail since there is not enough bookies
to form a new ensemble.
4) this ledger will be closed with lastAddConfirmed (1). entry 2 will be lost.
this issue happened in hub server. old ledger will be recovered and closed when
changing ownership. so published messages would be lost.
we should not close ledger when we encountered exception during recovery
adding, otherwise we would lose entries.
--
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