[
https://issues.apache.org/jira/browse/BOOKKEEPER-348?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13420183#comment-13420183
]
Wangda Tan commented on BOOKKEEPER-348:
---------------------------------------
Hi Flavio,
Thanks for the quick response, I found it's my fault. I mistakenly used the
openNoRecovery method, so it cannot get the entries not confirmed by client.
But I found another problem when try to close the ledger, it will throw an
BKException, following is the code can reproduce this problem.
{code:java}
@Test
public void testRecoverEntryInAnotherClient() throws Exception {
BookKeeper bk1 = new BookKeeper("localhost:2181");
final byte[] PASSWORD = "Password".getBytes();
final int ENTRY_COUNT = 1024;
// open the first ledger handle and add a number of entries to it
LedgerHandle lh1 = bk1.createLedger(DigestType.MAC, PASSWORD);
for (int i = 0; i < ENTRY_COUNT; i++) {
lh1.addEntry(String.valueOf(i).getBytes());
}
// open another ledger handler and try to read all entries from it
BookKeeper bk2 = new BookKeeper("localhost:2181");
LedgerHandle lh2 = bk2.openLedger(lh1.getId(), DigestType.MAC, PASSWORD);
long lastAddPushed = lh2.getLastAddPushed();
Enumeration<LedgerEntry> entries = lh2.readEntries(0, lastAddPushed);
int readCount = 0;
while (entries.hasMoreElements()) {
LedgerEntry entry = entries.nextElement();
Assert.assertEquals(Integer.valueOf(new
String(entry.getEntry())).intValue(),
readCount);
readCount++;
}
Assert.assertEquals(readCount, ENTRY_COUNT);
lh1.close();
lh2.close();
bk1.close();
bk2.close();
}
{code}
The step of the program is,
1) open a ledger from BK-client-1 (named ledger-1)
2) write some entries by ledger-1
3) open a ledger from BK-client-2 (named ledger-2)
4) read entries
5) close ledger-1
the exception will throw when close ledger-1
I don't know it's the expected result or something going wrong, do you have any
idea about this?
Thanks,
Wangda
> Last entry will be lost when open an un-closed ledger
> ------------------------------------------------------
>
> Key: BOOKKEEPER-348
> URL: https://issues.apache.org/jira/browse/BOOKKEEPER-348
> Project: Bookkeeper
> Issue Type: Bug
> Components: bookkeeper-client
> Affects Versions: 4.1.0
> Reporter: Wangda Tan
> Attachments: BookKeeper-348.patch
>
>
> This can be reproduced in following steps:
> 1) client-A created a ledger-x and write N entries to it
> 2) client-B open the ledger-x and try to read all entries from it. client-B
> can only get N-1 entries (except for the last entry)
> This problem caused by, when trying to open an unclosed ledger, it will enter
> the "recover" mode, it can get correct last entry-Id judged by the size of
> log file. But it will set the new opened ledger's lastAddConfirmed by the
> previous lastAddConfirmed, and the entry-id will be ignored.
> For an unclosed ledger, the lastAddConfirmed will always = (last-entry-id -
> 1).
> A patch attached to this jira.
--
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