Return-Path: Delivered-To: apmail-zookeeper-commits-archive@www.apache.org Received: (qmail 69008 invoked from network); 9 Feb 2011 19:15:58 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 9 Feb 2011 19:15:58 -0000 Received: (qmail 61993 invoked by uid 500); 9 Feb 2011 19:15:56 -0000 Delivered-To: apmail-zookeeper-commits-archive@zookeeper.apache.org Received: (qmail 60849 invoked by uid 500); 9 Feb 2011 19:15:54 -0000 Mailing-List: contact commits-help@zookeeper.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ Delivered-To: mailing list commits@zookeeper.apache.org Received: (qmail 60841 invoked by uid 99); 9 Feb 2011 19:15:53 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 09 Feb 2011 19:15:53 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 09 Feb 2011 19:15:49 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 03ED623888FE; Wed, 9 Feb 2011 19:15:27 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1069039 - in /zookeeper/trunk: ./ src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/ src/contrib/bookkeeper/src/java/org/apache/bookkeeper/tools/ src/contrib/bookkeeper/test/org/apache/bookkeeper/test/ Date: Wed, 09 Feb 2011 19:15:26 -0000 To: commits@zookeeper.apache.org From: mahadev@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20110209191527.03ED623888FE@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: mahadev Date: Wed Feb 9 19:15:26 2011 New Revision: 1069039 URL: http://svn.apache.org/viewvc?rev=1069039&view=rev Log: ZOOKEEPER-465. Ledger size in bytes. (flavio via mahadev) Modified: zookeeper/trunk/CHANGES.txt zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/DigestManager.java zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerEntry.java zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerHandle.java zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerMetadata.java zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/PendingReadOp.java zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/tools/BookKeeperTools.java zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/AsyncLedgerOpsTest.java zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java Modified: zookeeper/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1069039&r1=1069038&r2=1069039&view=diff ============================================================================== --- zookeeper/trunk/CHANGES.txt (original) +++ zookeeper/trunk/CHANGES.txt Wed Feb 9 19:15:26 2011 @@ -234,6 +234,8 @@ IMPROVEMENTS: ZOOKEEPER-977. passing null for path_buffer in zoo_create (breed via mahadev) + ZOOKEEPER-465. Ledger size in bytes. (flavio via mahadev) + NEW FEATURES: ZOOKEEPER-729. Java client API to recursively delete a subtree. (Kay Kay via henry) Modified: zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/DigestManager.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/DigestManager.java?rev=1069039&r1=1069038&r2=1069039&view=diff ============================================================================== --- zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/DigestManager.java (original) +++ zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/DigestManager.java Wed Feb 9 19:15:26 2011 @@ -37,7 +37,9 @@ import org.jboss.netty.buffer.ChannelBuf public abstract class DigestManager { static final Logger logger = Logger.getLogger(DigestManager.class); - + + static final int METADATA_LENGTH = 32; + long ledgerId; abstract int getMacCodeLength(); @@ -67,21 +69,32 @@ public abstract class DigestManager { } } - public ChannelBuffer computeDigestAndPackageForSending(long entryId, long lastAddConfirmed, byte[] data) { + /** + * Computes the digest for an entry and put bytes together for sending. + * + * @param entryId + * @param lastAddConfirmed + * @param length + * @param data + * @return + */ + + public ChannelBuffer computeDigestAndPackageForSending(long entryId, long lastAddConfirmed, long length, byte[] data) { - byte[] bufferArray = new byte[24+macCodeLength]; + byte[] bufferArray = new byte[METADATA_LENGTH + macCodeLength]; ByteBuffer buffer = ByteBuffer.wrap(bufferArray); buffer.putLong(ledgerId); buffer.putLong(entryId); buffer.putLong(lastAddConfirmed); + buffer.putLong(length); buffer.flip(); - update(buffer.array(), 0, 24); + update(buffer.array(), 0, METADATA_LENGTH); update(data); byte[] digest = getValueAndReset(); buffer.limit(buffer.capacity()); - buffer.position(24); + buffer.position(METADATA_LENGTH); buffer.put(digest); buffer.flip(); @@ -102,14 +115,14 @@ public abstract class DigestManager { ByteBuffer dataReceivedBuffer = dataReceived.toByteBuffer(); byte[] digest; - update(dataReceivedBuffer.array(), dataReceivedBuffer.position(), 24); + update(dataReceivedBuffer.array(), dataReceivedBuffer.position(), METADATA_LENGTH); - int offset = 24 + macCodeLength; + int offset = METADATA_LENGTH + macCodeLength; update(dataReceivedBuffer.array(), dataReceivedBuffer.position() + offset, dataReceived.readableBytes() - offset); digest = getValueAndReset(); for (int i = 0; i < digest.length; i++) { - if (digest[i] != dataReceived.getByte(24 + i)) { + if (digest[i] != dataReceived.getByte(METADATA_LENGTH + i)) { logger.error("Mac mismatch for ledger-id: " + ledgerId + ", entry-id: " + entryId); throw new BKDigestMatchException(); } @@ -131,11 +144,19 @@ public abstract class DigestManager { } } - + + /** + * Verify that the digest matches and returns the data in the entry. + * + * @param entryId + * @param dataReceived + * @return + * @throws BKDigestMatchException + */ ChannelBufferInputStream verifyDigestAndReturnData(long entryId, ChannelBuffer dataReceived) throws BKDigestMatchException { verifyDigest(entryId, dataReceived); - dataReceived.readerIndex(24 + macCodeLength); + dataReceived.readerIndex(METADATA_LENGTH + macCodeLength); return new ChannelBufferInputStream(dataReceived); } @@ -156,6 +177,7 @@ public abstract class DigestManager { long entryId = dataReceived.readLong(); long lastAddConfirmed = dataReceived.readLong(); + long length = dataReceived.readLong(); return new RecoveryData(lastAddConfirmed, entryId); } Modified: zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerEntry.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerEntry.java?rev=1069039&r1=1069038&r2=1069039&view=diff ============================================================================== --- zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerEntry.java (original) +++ zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerEntry.java Wed Feb 9 19:15:26 2011 @@ -38,6 +38,7 @@ public class LedgerEntry { long ledgerId; long entryId; + long length; ChannelBufferInputStream entryDataStream; int nextReplicaIndexToReadFrom = 0; @@ -54,6 +55,10 @@ public class LedgerEntry { public long getEntryId() { return entryId; } + + public long getLength() { + return length; + } public byte[] getEntry() { try { Modified: zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerHandle.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerHandle.java?rev=1069039&r1=1069038&r2=1069039&view=diff ============================================================================== --- zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerHandle.java (original) +++ zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerHandle.java Wed Feb 9 19:15:26 2011 @@ -60,6 +60,7 @@ public class LedgerHandle implements Rea final long ledgerId; long lastAddPushed; long lastAddConfirmed; + long length; final DigestManager macManager; final DistributionSchedule distributionSchedule; @@ -75,8 +76,10 @@ public class LedgerHandle implements Rea this.metadata = metadata; if (metadata.isClosed()) { lastAddConfirmed = lastAddPushed = metadata.close; + length = metadata.length; } else { lastAddConfirmed = lastAddPushed = -1; + length = 0; } this.ledgerId = ledgerId; @@ -158,6 +161,26 @@ public class LedgerHandle implements Rea } /** + * Add to the length of the ledger in bytes. + * + * @param delta + * @return + */ + long addToLength(long delta){ + this.length += delta; + return this.length; + } + + /** + * Returns the length of the ledger in bytes. + * + * @return + */ + public long getLength(){ + return this.length; + } + + /** * Get the Distribution Schedule * * @return DistributionSchedule for the LedgerHandle @@ -212,6 +235,7 @@ public class LedgerHandle implements Rea @Override public void safeRun() { + metadata.length = length; // Close operation is idempotent, so no need to check if we are // already closed metadata.close(lastAddConfirmed); @@ -220,7 +244,7 @@ public class LedgerHandle implements Rea if (LOG.isDebugEnabled()) { LOG.debug("Closing ledger: " + ledgerId + " at entryId: " - + metadata.close); + + metadata.close + " with this many bytes: " + metadata.length); } writeLedgerConfig(new StatCallback() { @@ -343,10 +367,11 @@ public class LedgerHandle implements Rea } long entryId = ++lastAddPushed; + long currentLength = addToLength(data.length); PendingAddOp op = new PendingAddOp(LedgerHandle.this, cb, ctx, entryId); pendingAddOps.add(op); ChannelBuffer toSend = macManager.computeDigestAndPackageForSending( - entryId, lastAddConfirmed, data); + entryId, lastAddConfirmed, currentLength, data); op.initiate(toSend); } }); Modified: zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerMetadata.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerMetadata.java?rev=1069039&r1=1069038&r2=1069039&view=diff ============================================================================== --- zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerMetadata.java (original) +++ zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerMetadata.java Wed Feb 9 19:15:26 2011 @@ -45,6 +45,7 @@ public class LedgerMetadata { public static final int NOTCLOSED = -101; int ensembleSize; int quorumSize; + long length; long close; private SortedMap> ensembles = new TreeMap>(); ArrayList currentEnsemble; @@ -52,6 +53,12 @@ public class LedgerMetadata { public LedgerMetadata(int ensembleSize, int quorumSize) { this.ensembleSize = ensembleSize; this.quorumSize = quorumSize; + + /* + * It is set in PendingReadOp.readEntryComplete, and + * we read it in LedgerRecoveryOp.readComplete. + */ + this.length = 0; this.close = NOTCLOSED; }; @@ -77,7 +84,7 @@ public class LedgerMetadata { void close(long entryId) { close = entryId; } - + void addEnsemble(long startEntryId, ArrayList ensemble) { assert ensembles.isEmpty() || startEntryId >= ensembles.lastKey(); @@ -117,7 +124,7 @@ public class LedgerMetadata { */ public byte[] serialize() { StringBuilder s = new StringBuilder(); - s.append(quorumSize).append(lSplitter).append(ensembleSize); + s.append(quorumSize).append(lSplitter).append(ensembleSize).append(lSplitter).append(length); for (Map.Entry> entry : ensembles.entrySet()) { s.append(lSplitter).append(entry.getKey()); @@ -126,7 +133,7 @@ public class LedgerMetadata { StringUtils.addrToString(s, addr); } } - + if (close != NOTCLOSED) { s.append(lSplitter).append(close).append(tSplitter).append(closed); } @@ -166,8 +173,9 @@ public class LedgerMetadata { try { lc.quorumSize = new Integer(lines[0]); lc.ensembleSize = new Integer(lines[1]); - - for (int i = 2; i < lines.length; i++) { + lc.length = new Long(lines[2]); + + for (int i = 3; i < lines.length; i++) { String parts[] = lines[i].split(tSplitter); if (parts[1].equals(closed)) { Modified: zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java?rev=1069039&r1=1069038&r2=1069039&view=diff ============================================================================== --- zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java (original) +++ zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java Wed Feb 9 19:15:26 2011 @@ -44,6 +44,7 @@ class LedgerRecoveryOp implements ReadEn boolean proceedingWithRecovery = false; long maxAddPushed = -1; long maxAddConfirmed = -1; + long maxLength = 0; GenericCallback cb; @@ -81,8 +82,7 @@ class LedgerRecoveryOp implements ReadEn heardValidResponse = true; } catch (BKDigestMatchException e) { // Too bad, this bookie didnt give us a valid answer, we - // still - // might be able to recover though so continue + // still might be able to recover though so continue LOG.error("Mac mismatch while reading last entry from bookie: " + lh.metadata.currentEnsemble.get(bookieIndex)); } @@ -99,6 +99,7 @@ class LedgerRecoveryOp implements ReadEn if (heardValidResponse && lh.distributionSchedule.canProceedWithRecovery(bookieIndex)) { proceedingWithRecovery = true; lh.lastAddPushed = lh.lastAddConfirmed = maxAddConfirmed; + lh.length = maxLength; doRecoveryRead(); return; } @@ -126,7 +127,17 @@ class LedgerRecoveryOp implements ReadEn // get back to prev value lh.lastAddConfirmed--; if (rc == BKException.Code.OK) { - lh.asyncAddEntry(seq.nextElement().getEntry(), this, null); + LedgerEntry entry = seq.nextElement(); + byte[] data = entry.getEntry(); + + /* + * We will add this entry again to make sure it is written to enough + * replicas. We subtract the length of the data itself, since it will + * be added again when processing the call to add it. + */ + lh.length = entry.getLength() - (long) data.length; + lh.asyncAddEntry(data, this, null); + return; } @@ -138,6 +149,7 @@ class LedgerRecoveryOp implements ReadEn // it cb.operationComplete(BKException.Code.OK, null); + LOG.debug("After closing length is: " + lh.getLength()); return; } Modified: zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/PendingReadOp.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/PendingReadOp.java?rev=1069039&r1=1069038&r2=1069039&view=diff ============================================================================== --- zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/PendingReadOp.java (original) +++ zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/PendingReadOp.java Wed Feb 9 19:15:26 2011 @@ -22,6 +22,7 @@ package org.apache.bookkeeper.client; */ import java.net.InetSocketAddress; +import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Enumeration; @@ -34,6 +35,8 @@ import org.apache.log4j.Logger; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBufferInputStream; +import java.io.IOException; + /** * Sequence of entries of a ledger that represents a pending read operation. * When all the data read has come back, the application callback is called. @@ -126,8 +129,14 @@ class PendingReadOp implements Enumerati logErrorAndReattemptRead(entry, "Mac mismatch", BKException.Code.DigestMatchException); return; } - + entry.entryDataStream = is; + + /* + * The length is a long and it is the last field of the metadata of an entry. + * Consequently, we have to subtract 8 from METADATA_LENGTH to get the length. + */ + entry.length = buffer.getLong(DigestManager.METADATA_LENGTH - 8); numPendingReads--; if (numPendingReads == 0) { Modified: zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/tools/BookKeeperTools.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/tools/BookKeeperTools.java?rev=1069039&r1=1069038&r2=1069039&view=diff ============================================================================== --- zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/tools/BookKeeperTools.java (original) +++ zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/tools/BookKeeperTools.java Wed Feb 9 19:15:26 2011 @@ -686,8 +686,9 @@ public class BookKeeperTools { * Now that we've read the ledger entry, write it to the new * bookie we've selected. */ + LedgerEntry entry = seq.nextElement(); ChannelBuffer toSend = lh.getDigestManager().computeDigestAndPackageForSending(entryId, - lh.getLastAddConfirmed(), seq.nextElement().getEntry()); + lh.getLastAddConfirmed(), entry.getLength(), entry.getEntry()); bkc.getBookieClient().addEntry(newBookie, lh.getId(), lh.getLedgerKey(), entryId, toSend, new WriteCallback() { @Override Modified: zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/AsyncLedgerOpsTest.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/AsyncLedgerOpsTest.java?rev=1069039&r1=1069038&r2=1069039&view=diff ============================================================================== --- zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/AsyncLedgerOpsTest.java (original) +++ zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/AsyncLedgerOpsTest.java Wed Feb 9 19:15:26 2011 @@ -139,7 +139,7 @@ public class AsyncLedgerOpsTest extends } } - LOG.debug("*** WRITE COMPLETE ***"); + LOG.info("*** WRITE COMPLETE ***"); // close ledger synchronized (ctx) { lh.asyncClose(this, ctx); Modified: zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java?rev=1069039&r1=1069038&r2=1069039&view=diff ============================================================================== --- zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java (original) +++ zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java Wed Feb 9 19:15:26 2011 @@ -609,6 +609,60 @@ public class BookieReadWriteTest extends } } + @Test + public void testReadWriteAsyncLength() throws IOException { + try { + // Create a BookKeeper client and a ledger + bkc = new BookKeeper("127.0.0.1"); + lh = bkc.createLedger(digestType, ledgerPassword); + // bkc.initMessageDigest("SHA1"); + ledgerId = lh.getId(); + LOG.info("Ledger ID: " + lh.getId()); + for (int i = 0; i < numEntriesToWrite; i++) { + ByteBuffer entry = ByteBuffer.allocate(4); + entry.putInt(rng.nextInt(maxInt)); + entry.position(0); + + entries.add(entry.array()); + entriesSize.add(entry.array().length); + lh.asyncAddEntry(entry.array(), this, sync); + } + + // wait for all entries to be acknowledged + synchronized (sync) { + while (sync.counter < numEntriesToWrite) { + LOG.debug("Entries counter = " + sync.counter); + sync.wait(); + } + } + long length = numEntriesToWrite * 4; + assertTrue("Ledger length before closing: " + lh.getLength(), lh.getLength() == length); + + LOG.debug("*** WRITE COMPLETE ***"); + // close ledger + lh.close(); + + // *** WRITING PART COMPLETE // READ PART BEGINS *** + + // open ledger + lh = bkc.openLedger(ledgerId, digestType, ledgerPassword); + assertTrue("Ledger length after opening: " + lh.getLength(), lh.getLength() == length); + + + lh.close(); + } catch (KeeperException e) { + LOG.error("Test failed", e); + fail("Test failed due to ZooKeeper exception"); + } catch (BKException e) { + LOG.error("Test failed", e); + fail("Test failed due to BookKeeper exception"); + } catch (InterruptedException e) { + LOG.error("Test failed", e); + fail("Test failed due to interruption"); + } + } + + public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { if(rc != BKException.Code.OK) fail("Return code is not OK: " + rc); Modified: zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java?rev=1069039&r1=1069038&r2=1069039&view=diff ============================================================================== --- zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java (original) +++ zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java Wed Feb 9 19:15:26 2011 @@ -29,7 +29,6 @@ import org.apache.log4j.Logger; /** * This unit test tests ledger recovery. * - * */ public class LedgerRecoveryTest extends BaseTestCase { @@ -54,6 +53,8 @@ public class LedgerRecoveryTest extends beforelh.addEntry(tmp.getBytes()); } + long length = (long) (numEntries * tmp.length()); + /* * Try to open ledger. */ @@ -63,7 +64,9 @@ public class LedgerRecoveryTest extends * Check if has recovered properly. */ assertTrue("Has not recovered correctly: " + afterlh.getLastAddConfirmed(), - afterlh.getLastAddConfirmed() == numEntries - 1); + afterlh.getLastAddConfirmed() == numEntries - 1); + assertTrue("Has not set the length correctly: " + afterlh.getLength() + ", " + length, + afterlh.getLength() == length); } @Test