You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sh...@apache.org on 2012/02/02 20:02:38 UTC

svn commit: r1239762 - in /hadoop/common/branches/branch-0.22/hdfs: ./ src/java/org/apache/hadoop/hdfs/server/namenode/ src/test/hdfs/org/apache/hadoop/hdfs/ src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/

Author: shv
Date: Thu Feb  2 19:02:38 2012
New Revision: 1239762

URL: http://svn.apache.org/viewvc?rev=1239762&view=rev
Log:
HDFS-2718. Optimize OP_ADD in edits loading. Contributed by Konstantin Shvachko.

Modified:
    hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt
    hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfoUnderConstruction.java
    hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
    hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
    hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
    hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
    hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java
    hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

Modified: hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt?rev=1239762&r1=1239761&r2=1239762&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt Thu Feb  2 19:02:38 2012
@@ -10,6 +10,8 @@ Release 0.22.1 - Unreleased
 
   OPTIMIZATIONS
 
+    HDFS-2718. Optimize OP_ADD in edits loading. (shv)
+
   BUG FIXES
 
     HDFS-1910. NameNode should not save fsimage twice. (shv)

Modified: hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfoUnderConstruction.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfoUnderConstruction.java?rev=1239762&r1=1239761&r2=1239762&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfoUnderConstruction.java (original)
+++ hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfoUnderConstruction.java Thu Feb  2 19:02:38 2012
@@ -154,9 +154,6 @@ class BlockInfoUnderConstruction extends
   BlockInfo convertToCompleteBlock() throws IOException {
     assert getBlockUCState() != BlockUCState.COMPLETE :
       "Trying to convert a COMPLETE block";
-    if(getBlockUCState() != BlockUCState.COMMITTED)
-      throw new IOException(
-          "Cannot complete block: block has not been COMMITTED by the client");
     return new BlockInfo(this);
   }
 

Modified: hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java?rev=1239762&r1=1239761&r2=1239762&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java (original)
+++ hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java Thu Feb  2 19:02:38 2012
@@ -322,15 +322,23 @@ public class BlockManager {
    */
   BlockInfo completeBlock(INodeFile fileINode, int blkIndex)
   throws IOException {
+    return completeBlock(fileINode, blkIndex, false);
+  }
+
+  BlockInfo completeBlock(INodeFile fileINode, int blkIndex, boolean force)
+  throws IOException {
     if(blkIndex < 0)
       return null;
     BlockInfo curBlock = fileINode.getBlocks()[blkIndex];
     if(curBlock.isComplete())
       return curBlock;
     BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)curBlock;
-    if(ucBlock.numNodes() < minReplication)
+    if(!force && ucBlock.numNodes() < minReplication)
       throw new IOException("Cannot complete block: " +
           "block does not satisfy minimal replication requirement.");
+    if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
+      throw new IOException(
+          "Cannot complete block: block has not been COMMITTED by the client");
     BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
     // replace penultimate block in file
     fileINode.setBlock(blkIndex, completeBlock);
@@ -1739,16 +1747,14 @@ public class BlockManager {
     return blocksMap.getINode(b);
   }
 
-  void removeFromCorruptReplicasMap(Block block) {
-    corruptReplicas.removeFromCorruptReplicasMap(block);
-  }
-
   int numCorruptReplicas(Block block) {
     return corruptReplicas.numCorruptReplicas(block);
   }
 
   void removeBlockFromMap(Block block) {
     blocksMap.removeBlock(block);
+    // If block is removed from blocksMap remove it from corruptReplicasMap
+    corruptReplicas.removeFromCorruptReplicasMap(block);
   }
 
   int getCapacity() {

Modified: hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java?rev=1239762&r1=1239761&r2=1239762&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java (original)
+++ hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java Thu Feb  2 19:02:38 2012
@@ -258,21 +258,27 @@ class FSDirectory implements Closeable {
                             short replication,
                             long modificationTime,
                             long atime,
-                            long preferredBlockSize) 
+                            long preferredBlockSize,
+                            String clientName,
+                            String clientMachine)
       throws UnresolvedLinkException {
     INode newNode;
-    long diskspace = UNKNOWN_DISK_SPACE;
     if (blocks == null)
       newNode = new INodeDirectory(permissions, modificationTime);
-    else {
+    else if(blocks.length == 0 || blocks[blocks.length-1].getBlockUCState()
+            == BlockUCState.UNDER_CONSTRUCTION) {
+      newNode = new INodeFileUnderConstruction(
+          permissions, blocks.length, replication,
+          preferredBlockSize, modificationTime, clientName, 
+          clientMachine, null);
+    } else {
       newNode = new INodeFile(permissions, blocks.length, replication,
                               modificationTime, atime, preferredBlockSize);
-      diskspace = ((INodeFile)newNode).diskspaceConsumed(blocks);
     }
     writeLock();
     try {
       try {
-        newNode = addNode(path, newNode, diskspace, false);
+        newNode = addNode(path, newNode, UNKNOWN_DISK_SPACE, false);
         if(newNode != null && blocks != null) {
           int nrBlocks = blocks.length;
           // Add file->block mapping
@@ -291,6 +297,74 @@ class FSDirectory implements Closeable {
 
   }
 
+  /**
+   * Update files in-memory data structures with new block information.
+   * @throws IOException 
+   */
+  void updateFile(INodeFile file,
+                  String path,
+                  PermissionStatus permissions,
+                  BlockInfo[] blocks, 
+                  short replication,
+                  long mtime,
+                  long atime,
+                  long preferredBlockSize) throws IOException {
+
+    // Update the salient file attributes.
+    file.setAccessTime(atime);
+    file.setModificationTimeForce(mtime);
+
+    // Update its block list
+    BlockInfo[] oldBlocks = file.getBlocks();
+
+    // Are we only updating the last block's gen stamp.
+    boolean isGenStampUpdate = oldBlocks.length == blocks.length;
+
+    // First, update blocks in common
+    BlockInfo oldBlock = null;
+    for (int i = 0; i < oldBlocks.length && i < blocks.length; i++) {
+      oldBlock = oldBlocks[i];
+      Block newBlock = blocks[i];
+
+      boolean isLastBlock = i == oldBlocks.length - 1;
+      if (oldBlock.getBlockId() != newBlock.getBlockId() ||
+          (oldBlock.getGenerationStamp() != newBlock.getGenerationStamp() && 
+              !(isGenStampUpdate && isLastBlock))) {
+        throw new IOException("Mismatched block IDs or generation stamps, " + 
+            "attempting to replace block " + oldBlock + " with " + newBlock +
+            " as block # " + i + "/" + blocks.length + " of " + path);
+      }
+
+      oldBlock.setNumBytes(newBlock.getNumBytes());
+      oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
+    }
+
+    if (blocks.length < oldBlocks.length) {
+      // We're removing a block from the file, e.g. abandonBlock(...)
+      if (!file.isUnderConstruction()) {
+        throw new IOException("Trying to remove a block from file " +
+            path + " which is not under construction.");
+      }
+      if (blocks.length != oldBlocks.length - 1) {
+        throw new IOException("Trying to remove more than one block from file "
+            + path);
+      }
+      unprotectedRemoveBlock(path,
+          (INodeFileUnderConstruction)file, oldBlocks[oldBlocks.length - 1]);
+    } else if (blocks.length > oldBlocks.length) {
+      // We're adding blocks
+      // First complete last old Block
+      getBlockManager().completeBlock(file, oldBlocks.length-1, true);
+      // Add the new blocks
+      for (int i = oldBlocks.length; i < blocks.length; i++) {
+        // addBlock();
+        BlockInfo newBI = blocks[i];
+        getBlockManager().addINode(newBI, file);
+        file.addBlock(newBI);
+      }
+    }
+  }
+
   INodeDirectory addToParent( byte[][] src,
                               INodeDirectory parentINode,
                               PermissionStatus permissions,
@@ -442,30 +516,33 @@ class FSDirectory implements Closeable {
 
     writeLock();
     try {
-      // modify file-> block and blocksMap
-      fileNode.removeLastBlock(block);
-      getBlockManager().removeBlockFromMap(block);
-      // If block is removed from blocksMap remove it from corruptReplicasMap
-      getBlockManager().removeFromCorruptReplicasMap(block);
-
+      unprotectedRemoveBlock(path, fileNode, block);
       // write modified block locations to log
       fsImage.getEditLog().logOpenFile(path, fileNode);
-      if(NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
-            +path+" with "+block
-            +" block is added to the file system");
-      }
-
-      // update space consumed
-      INode[] pathINodes = getExistingPathINodes(path);
-      updateCount(pathINodes, pathINodes.length-1, 0,
-          -fileNode.getPreferredBlockSize()*fileNode.getReplication(), true);
     } finally {
       writeUnlock();
     }
     return true;
   }
 
+  void unprotectedRemoveBlock(String path, INodeFileUnderConstruction fileNode, 
+      Block block) throws IOException {
+    // modify file-> block and blocksMap
+    fileNode.removeLastBlock(block);
+    getBlockManager().removeBlockFromMap(block);
+
+    if(NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("DIR* FSDirectory.removeBlock: "
+          +path+" with "+block
+          +" block is removed from the file system");
+    }
+
+    // update space consumed
+    INode[] pathINodes = getExistingPathINodes(path);
+    updateCount(pathINodes, pathINodes.length-1, 0,
+        -fileNode.getPreferredBlockSize()*fileNode.getReplication(), true);
+  }
+
   /**
    * @see #unprotectedRenameTo(String, String, long)
    * @deprecated Use {@link #renameTo(String, String, Rename...)} instead.

Modified: hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java?rev=1239762&r1=1239761&r2=1239762&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java (original)
+++ hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java Thu Feb  2 19:02:38 2012
@@ -160,9 +160,7 @@ public class FSEditLogLoader {
             blockSize = readLong(in);
           }
           // get blocks
-          boolean isFileUnderConstruction = (opcode == Ops.OP_ADD);
-          BlockInfo blocks[] = 
-            readBlocks(in, logVersion, isFileUnderConstruction, replication);
+          BlockInfo blocks[] = readBlocks(in, logVersion, replication);
 
           // Older versions of HDFS does not store the block size in inode.
           // If the file has more than one block, use the size of the
@@ -182,7 +180,7 @@ public class FSEditLogLoader {
             permissions = PermissionStatus.read(in);
           }
 
-          // clientname, clientMachine and block locations of last block.
+          // clientName, clientMachine and block locations of last block.
           if (opcode == Ops.OP_ADD && logVersion <= -12) {
             clientName = FSImageSerialization.readString(in);
             clientMachine = FSImageSerialization.readString(in);
@@ -203,31 +201,54 @@ public class FSEditLogLoader {
                                    " clientMachine " + clientMachine);
           }
 
-          fsDir.unprotectedDelete(path, mtime);
-
-          // add to the file tree
-          INodeFile node = (INodeFile)fsDir.unprotectedAddFile(
-                                                    path, permissions,
-                                                    blocks, replication, 
-                                                    mtime, atime, blockSize);
-          if (isFileUnderConstruction) {
+          // There are four cases here:
+          // 1. OP_ADD to create a new file
+          // 2. OP_ADD to update file blocks
+          // 3. OP_ADD to open file for append
+          // 4. OP_CLOSE to close the file
+
+          // See if the file already exists
+          INodeFile oldFile = fsDir.getFileINode(path);
+          if (oldFile == null) { // OP_ADD for a new file
+            assert opcode == Ops.OP_ADD : 
+              "Expected opcode OP_ADD, but got " + opcode;
+            fsDir.unprotectedAddFile(
+                path, permissions, blocks, replication,
+                mtime, atime, blockSize, clientName, clientMachine);
+          } else {
+            fsDir.updateFile(oldFile,
+                path, permissions, blocks, replication,
+                mtime, atime, blockSize);
+            if(opcode == Ops.OP_CLOSE) {  // OP_CLOSE
+              assert oldFile.isUnderConstruction() : 
+                "File is not under construction: " + path;
+              fsNamesys.blockManager.completeBlock(
+                  oldFile, blocks.length-1, true);
+              INodeFile newFile =
+                ((INodeFileUnderConstruction)oldFile).convertToInodeFile();
+              fsDir.replaceNode(path, oldFile, newFile);
+            } else if(! oldFile.isUnderConstruction()) {  // OP_ADD for append
+              INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
+                  oldFile.getLocalNameBytes(),
+                  oldFile.getReplication(), 
+                  oldFile.getModificationTime(),
+                  oldFile.getPreferredBlockSize(),
+                  oldFile.getBlocks(),
+                  oldFile.getPermissionStatus(),
+                  clientName, 
+                  clientMachine, 
+                  null);
+              fsDir.replaceNode(path, oldFile, cons);
+            }
+          }
+          // Update file lease
+          if(opcode == Ops.OP_ADD) {
             numOpAdd++;
-            //
-            // Replace current node with a INodeUnderConstruction.
-            // Recreate in-memory lease record.
-            //
-            INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
-                                      node.getLocalNameBytes(),
-                                      node.getReplication(), 
-                                      node.getModificationTime(),
-                                      node.getPreferredBlockSize(),
-                                      node.getBlocks(),
-                                      node.getPermissionStatus(),
-                                      clientName, 
-                                      clientMachine, 
-                                      null);
-            fsDir.replaceNode(path, node, cons);
-            fsNamesys.leaseManager.addLease(cons.getClientName(), path);
+            fsNamesys.leaseManager.addLease(clientName, path);
+          } else {  // Ops.OP_CLOSE
+            numOpClose++;
+            fsNamesys.leaseManager.removeLease(
+                ((INodeFileUnderConstruction)oldFile).getClientName(), path);
           }
           break;
         } 
@@ -553,7 +574,6 @@ public class FSEditLogLoader {
   static private BlockInfo[] readBlocks(
       DataInputStream in,
       int logVersion,
-      boolean isFileUnderConstruction,
       short replication) throws IOException {
     int numBlocks = in.readInt();
     BlockInfo[] blocks = new BlockInfo[numBlocks];
@@ -567,7 +587,7 @@ public class FSEditLogLoader {
         blk.set(oldblk.blkid, oldblk.len,
                 GenerationStamp.GRANDFATHER_GENERATION_STAMP);
       }
-      if(isFileUnderConstruction && i == numBlocks-1)
+      if(i == numBlocks-1)
         blocks[i] = new BlockInfoUnderConstruction(blk, replication);
       else
         blocks[i] = new BlockInfo(blk, replication);

Modified: hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java?rev=1239762&r1=1239761&r2=1239762&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java (original)
+++ hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java Thu Feb  2 19:02:38 2012
@@ -36,8 +36,20 @@ class INodeFileUnderConstruction extends
                              String clientName,
                              String clientMachine,
                              DatanodeDescriptor clientNode) {
-    super(permissions.applyUMask(UMASK), 0, replication, modTime, modTime,
-        preferredBlockSize);
+    this(permissions, 0, replication, preferredBlockSize, modTime,
+        clientName, clientMachine, clientNode);
+  }
+
+  INodeFileUnderConstruction(PermissionStatus permissions,
+                             int nrBlocks,
+                             short replication,
+                             long preferredBlockSize,
+                             long modTime,
+                             String clientName,
+                             String clientMachine,
+                             DatanodeDescriptor clientNode) {
+    super(permissions.applyUMask(UMASK), nrBlocks, replication,
+        modTime, modTime, preferredBlockSize);
     this.clientName = clientName;
     this.clientMachine = clientMachine;
     this.clientNode = clientNode;

Modified: hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java?rev=1239762&r1=1239761&r2=1239762&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java (original)
+++ hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java Thu Feb  2 19:02:38 2012
@@ -72,12 +72,20 @@ public class TestAbandonBlock {
 
     // Now abandon the last block
     DFSClient dfsclient = ((DistributedFileSystem)fs).getClient();
-    LocatedBlocks blocks = dfsclient.getNamenode().getBlockLocations(src, 0, 1);
+    LocatedBlocks blocks =
+      dfsclient.getNamenode().getBlockLocations(src, 0, Integer.MAX_VALUE);
+    int orginalNumBlocks = blocks.locatedBlockCount();
     LocatedBlock b = blocks.getLastLocatedBlock();
     dfsclient.getNamenode().abandonBlock(b.getBlock(), src, dfsclient.clientName);
 
     // And close the file
     fout.close();
+
+    // Close cluster and check the block has been abandoned after restart
+    cluster.restartNameNode();
+    blocks = dfsclient.getNamenode().getBlockLocations(src, 0, Integer.MAX_VALUE);
+    assert orginalNumBlocks == blocks.locatedBlockCount() + 1 :
+      "Blocks " + b + " has not been abandoned.";
   }
 
   @Test

Modified: hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java?rev=1239762&r1=1239761&r2=1239762&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java (original)
+++ hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java Thu Feb  2 19:02:38 2012
@@ -66,10 +66,12 @@ public class TestEditLog extends TestCas
     int numTransactions;
     short replication = 3;
     long blockSize = 64;
+    int startIndex;
 
-    Transactions(FSNamesystem ns, int num) {
+    Transactions(FSNamesystem ns, int numTx, int startIdx) {
       namesystem = ns;
-      numTransactions = num;
+      numTransactions = numTx;
+      startIndex = startIdx;
     }
 
     // add a bunch of transactions.
@@ -81,8 +83,8 @@ public class TestEditLog extends TestCas
       for (int i = 0; i < numTransactions; i++) {
         INodeFileUnderConstruction inode = new INodeFileUnderConstruction(
                             p, replication, blockSize, 0, "", "", null);
-        editLog.logOpenFile("/filename" + i, inode);
-        editLog.logCloseFile("/filename" + i, inode);
+        editLog.logOpenFile("/filename" + startIndex + i, inode);
+        editLog.logCloseFile("/filename" + startIndex + i, inode);
         editLog.logSync();
       }
     }
@@ -132,7 +134,8 @@ public class TestEditLog extends TestCas
       // Create threads and make them run transactions concurrently.
       Thread threadId[] = new Thread[NUM_THREADS];
       for (int i = 0; i < NUM_THREADS; i++) {
-        Transactions trans = new Transactions(namesystem, NUM_TRANSACTIONS);
+        Transactions trans =
+          new Transactions(namesystem, NUM_TRANSACTIONS, i*NUM_TRANSACTIONS);
         threadId[i] = new Thread(trans, "TransactionThread-" + i);
         threadId[i].start();
       }
@@ -145,10 +148,17 @@ public class TestEditLog extends TestCas
           i--;      // retry 
         }
       } 
-      
+
+      // Reopen some files as for append
+      Transactions trans = 
+        new Transactions(namesystem, NUM_TRANSACTIONS, NUM_TRANSACTIONS / 2);
+      trans.run();
+
       editLog.close();
       editLog.open();
-  
+
+      long expectedTxns = (NUM_THREADS+1) * 2 * NUM_TRANSACTIONS;
+
       // Verify that we can read in all the transactions that we have written.
       // If there were any corruptions, it is likely that the reading in
       // of these transactions will throw an exception.
@@ -164,9 +174,9 @@ public class TestEditLog extends TestCas
         System.out.println("Number of outstanding leases " + numLeases);
         assertEquals(0, numLeases);
         assertTrue("Verification for " + editFile + " failed. " +
-                   "Expected " + (NUM_THREADS * 2 * NUM_TRANSACTIONS) + " transactions. "+
-                   "Found " + numEdits + " transactions.",
-                   numEdits == NUM_THREADS * 2 * NUM_TRANSACTIONS);
+        		"Expected " + expectedTxns + " transactions. " +
+            "Found " + numEdits + " transactions.",
+            numEdits == expectedTxns);
   
       }
     } finally {