You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ma...@apache.org on 2011/02/09 20:15:26 UTC

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/

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<Long, ArrayList<InetSocketAddress>> ensembles = new TreeMap<Long, ArrayList<InetSocketAddress>>();
     ArrayList<InetSocketAddress> 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<InetSocketAddress> 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<Long, ArrayList<InetSocketAddress>> 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<Void> 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