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 su...@apache.org on 2013/04/24 20:20:10 UTC

svn commit: r1471582 - in /hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/test/java/org/apache/hadoop/hdfs/server/namenode/ src/test/java/org/apache/hadoop/hdfs/server/namen...

Author: suresh
Date: Wed Apr 24 18:20:09 2013
New Revision: 1471582

URL: http://svn.apache.org/r1471582
Log:
HDFS-4334. Merge r1426429 from trunk

Added:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
      - copied unchanged from r1426429, hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Wed Apr 24 18:20:09 2013
@@ -74,6 +74,8 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4209. Clean up the addNode/addChild/addChildNoQuotaCheck methods in
     FSDirectory and INodeDirectory. (szetszwo)
 
+    HDFS-4334. Add a unique id to INode.  (Brandon Li via szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java Wed Apr 24 18:20:09 2013
@@ -76,8 +76,9 @@ import com.google.common.base.Preconditi
  *************************************************/
 public class FSDirectory implements Closeable {
   private static INodeDirectoryWithQuota createRoot(FSNamesystem namesystem) {
-    return new INodeDirectoryWithQuota(INodeDirectory.ROOT_NAME,
-        namesystem.createFsOwnerPermissions(new FsPermission((short)0755)));
+    return new INodeDirectoryWithQuota(namesystem.allocateNewInodeId(),
+        INodeDirectory.ROOT_NAME,
+        namesystem.createFsOwnerPermissions(new FsPermission((short) 0755)));
   }
 
   INodeDirectoryWithQuota rootDir;
@@ -252,7 +253,9 @@ public class FSDirectory implements Clos
     if (!mkdirs(parent.toString(), permissions, true, modTime)) {
       return null;
     }
+    long id = namesystem.allocateNewInodeId();
     INodeFileUnderConstruction newNode = new INodeFileUnderConstruction(
+                                 id,
                                  permissions,replication,
                                  preferredBlockSize, modTime, clientName, 
                                  clientMachine, clientNode);
@@ -274,7 +277,8 @@ public class FSDirectory implements Clos
     return newNode;
   }
 
-  INode unprotectedAddFile( String path, 
+  INode unprotectedAddFile( long id,
+                            String path, 
                             PermissionStatus permissions,
                             short replication,
                             long modificationTime,
@@ -286,13 +290,11 @@ public class FSDirectory implements Clos
     final INode newNode;
     assert hasWriteLock();
     if (underConstruction) {
-      newNode = new INodeFileUnderConstruction(
-          permissions, replication,
-          preferredBlockSize, modificationTime, clientName, 
-          clientMachine, null);
+      newNode = new INodeFileUnderConstruction(id, permissions, replication,
+          preferredBlockSize, modificationTime, clientName, clientMachine, null);
     } else {
-      newNode = new INodeFile(permissions, BlockInfo.EMPTY_ARRAY, replication,
-                              modificationTime, atime, preferredBlockSize);
+      newNode = new INodeFile(id, permissions, BlockInfo.EMPTY_ARRAY,
+          replication, modificationTime, atime, preferredBlockSize);
     }
 
     try {
@@ -1421,8 +1423,9 @@ public class FSDirectory implements Clos
       // create directories beginning from the first null index
       for(; i < inodes.length; i++) {
         pathbuilder.append(Path.SEPARATOR + names[i]);
-        unprotectedMkdir(inodesInPath, i, components[i],
-            (i < lastInodeIndex) ? parentPermissions : permissions, now);
+        unprotectedMkdir(namesystem.allocateNewInodeId(), inodesInPath, i,
+            components[i], (i < lastInodeIndex) ? parentPermissions
+                : permissions, now);
         if (inodes[i] == null) {
           return false;
         }
@@ -1444,7 +1447,7 @@ public class FSDirectory implements Clos
     return true;
   }
 
-  INode unprotectedMkdir(String src, PermissionStatus permissions,
+  INode unprotectedMkdir(long inodeId, String src, PermissionStatus permissions,
                           long timestamp) throws QuotaExceededException,
                           UnresolvedLinkException {
     assert hasWriteLock();
@@ -1453,7 +1456,8 @@ public class FSDirectory implements Clos
         components.length, false);
     INode[] inodes = inodesInPath.getINodes();
     final int pos = inodes.length - 1;
-    unprotectedMkdir(inodesInPath, pos, components[pos], permissions, timestamp);
+    unprotectedMkdir(inodeId, inodesInPath, pos, components[pos], permissions,
+        timestamp);
     return inodes[pos];
   }
 
@@ -1461,11 +1465,12 @@ public class FSDirectory implements Clos
    * The parent path to the directory is at [0, pos-1].
    * All ancestors exist. Newly created one stored at index pos.
    */
-  private void unprotectedMkdir(INodesInPath inodesInPath, int pos,
-      byte[] name, PermissionStatus permission,
-      long timestamp) throws QuotaExceededException {
+  private void unprotectedMkdir(long inodeId, INodesInPath inodesInPath,
+      int pos, byte[] name, PermissionStatus permission, long timestamp)
+      throws QuotaExceededException {
     assert hasWriteLock();
-    final INodeDirectory dir = new INodeDirectory(name, permission, timestamp);
+    final INodeDirectory dir = new INodeDirectory(inodeId, name, permission,
+        timestamp);
     if (addChild(inodesInPath, pos, dir, true)) {
       inodesInPath.setINode(pos, dir);
     }
@@ -2035,9 +2040,10 @@ public class FSDirectory implements Clos
     }
     final String userName = dirPerms.getUserName();
     INodeSymlink newNode  = null;
+    long id = namesystem.allocateNewInodeId();
     writeLock();
     try {
-      newNode = unprotectedAddSymlink(path, target, modTime, modTime,
+      newNode = unprotectedAddSymlink(id, path, target, modTime, modTime,
           new PermissionStatus(userName, null, FsPermission.getDefault()));
     } finally {
       writeUnlock();
@@ -2057,12 +2063,13 @@ public class FSDirectory implements Clos
   /**
    * Add the specified path into the namespace. Invoked from edit log processing.
    */
-  INodeSymlink unprotectedAddSymlink(String path, String target, long mtime, 
-                                  long atime, PermissionStatus perm) 
+  INodeSymlink unprotectedAddSymlink(long id, String path, String target,
+      long mtime, long atime, PermissionStatus perm)
       throws UnresolvedLinkException, QuotaExceededException {
     assert hasWriteLock();
-    final INodeSymlink symlink = new INodeSymlink(target, mtime, atime, perm);
-    return addINode(path, symlink)? symlink: null;
+    final INodeSymlink symlink = new INodeSymlink(id, target, mtime, atime,
+        perm);
+    return addINode(path, symlink) ? symlink : null;
   }
   
   /**

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java Wed Apr 24 18:20:09 2013
@@ -122,7 +122,8 @@ public class FSEditLogLoader {
     long lastTxId = in.getLastTxId();
     long numTxns = (lastTxId - expectedStartingTxId) + 1;
     long lastLogTime = now();
-
+    long lastInodeId = fsNamesys.getLastInodeId();
+    
     try {
       while (true) {
         try {
@@ -168,7 +169,10 @@ public class FSEditLogLoader {
             }
           }
           try {
-            applyEditLogOp(op, fsDir, in.getVersion());
+            long inodeId = applyEditLogOp(op, fsDir, in.getVersion());
+            if (lastInodeId < inodeId) {
+              lastInodeId = inodeId;
+            }
           } catch (Throwable e) {
             LOG.error("Encountered exception on operation " + op, e);
             MetaRecoveryContext.editLogLoaderPrompt("Failed to " +
@@ -203,6 +207,7 @@ public class FSEditLogLoader {
         }
       }
     } finally {
+      fsNamesys.resetLastInodeId(lastInodeId);
       if(closeOnExit) {
         in.close();
       }
@@ -221,9 +226,9 @@ public class FSEditLogLoader {
   }
   
   @SuppressWarnings("deprecation")
-  private void applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
+  private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
       int logVersion) throws IOException {
-
+    long inodeId = INodeId.GRANDFATHER_INODE_ID;
     if (LOG.isTraceEnabled()) {
       LOG.trace("replaying edit log: " + op);
     }
@@ -253,11 +258,11 @@ public class FSEditLogLoader {
         assert addCloseOp.blocks.length == 0;
 
         // add to the file tree
-        newFile = (INodeFile)fsDir.unprotectedAddFile(
-            addCloseOp.path, addCloseOp.permissions,
-            replication, addCloseOp.mtime,
-            addCloseOp.atime, addCloseOp.blockSize,
-            true, addCloseOp.clientName, addCloseOp.clientMachine);
+        inodeId = fsNamesys.allocateNewInodeId();
+        newFile = (INodeFile) fsDir.unprotectedAddFile(inodeId,
+            addCloseOp.path, addCloseOp.permissions, replication,
+            addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
+            addCloseOp.clientName, addCloseOp.clientMachine);
         fsNamesys.leaseManager.addLease(addCloseOp.clientName, addCloseOp.path);
 
       } else { // This is OP_ADD on an existing file
@@ -368,7 +373,8 @@ public class FSEditLogLoader {
     }
     case OP_MKDIR: {
       MkdirOp mkdirOp = (MkdirOp)op;
-      fsDir.unprotectedMkdir(mkdirOp.path, mkdirOp.permissions,
+      inodeId = fsNamesys.allocateNewInodeId();
+      fsDir.unprotectedMkdir(inodeId, mkdirOp.path, mkdirOp.permissions,
                              mkdirOp.timestamp);
       break;
     }
@@ -421,9 +427,10 @@ public class FSEditLogLoader {
     }
     case OP_SYMLINK: {
       SymlinkOp symlinkOp = (SymlinkOp)op;
-      fsDir.unprotectedAddSymlink(symlinkOp.path, symlinkOp.value,
-                               symlinkOp.mtime, symlinkOp.atime,
-                               symlinkOp.permissionStatus);
+      inodeId = fsNamesys.allocateNewInodeId();
+      fsDir.unprotectedAddSymlink(inodeId, symlinkOp.path,
+                                  symlinkOp.value, symlinkOp.mtime, 
+                                  symlinkOp.atime, symlinkOp.permissionStatus);
       break;
     }
     case OP_RENAME: {
@@ -483,6 +490,7 @@ public class FSEditLogLoader {
     default:
       throw new IOException("Invalid operation read " + op.opCode);
     }
+    return inodeId;
   }
   
   private static String formatEditLogReplayError(EditLogInputStream in,

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java Wed Apr 24 18:20:09 2013
@@ -216,7 +216,8 @@ class FSImageFormat {
         in = compression.unwrapInputStream(fin);
 
         LOG.info("Loading image file " + curFile + " using " + compression);
-
+        // reset INodeId. TODO: remove this after inodeId is persisted in fsimage
+        namesystem.resetLastInodeIdWithoutChecking(INodeId.LAST_RESERVED_ID); 
         // load all inodes
         LOG.info("Number of files = " + numFiles);
         if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
@@ -384,6 +385,8 @@ class FSImageFormat {
     long blockSize = 0;
     
     int imgVersion = getLayoutVersion();
+    long inodeId = namesystem.allocateNewInodeId();
+    
     short replication = in.readShort();
     replication = namesystem.getBlockManager().adjustReplication(replication);
     modificationTime = in.readLong();
@@ -421,7 +424,7 @@ class FSImageFormat {
     
     PermissionStatus permissions = PermissionStatus.read(in);
 
-    return INode.newINode(permissions, blocks, symlink, replication,
+    return INode.newINode(inodeId, permissions, blocks, symlink, replication,
         modificationTime, atime, nsQuota, dsQuota, blockSize);
   }
 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java Wed Apr 24 18:20:09 2013
@@ -107,7 +107,9 @@ public class FSImageSerialization {
     int numLocs = in.readInt();
     assert numLocs == 0 : "Unexpected block locations";
 
-    return new INodeFileUnderConstruction(name, 
+    //TODO: get inodeId from fsimage after inodeId is persisted
+    return new INodeFileUnderConstruction(INodeId.GRANDFATHER_INODE_ID,
+                                          name,
                                           blockReplication, 
                                           modificationTime,
                                           preferredBlockSize,

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Wed Apr 24 18:20:09 2013
@@ -378,6 +378,30 @@ public class FSNamesystem implements Nam
 
   private final boolean haEnabled;
     
+  private INodeId inodeId;
+  
+  /**
+   * Set the last allocated inode id when fsimage is loaded or editlog is
+   * applied. 
+   * @throws IOException
+   */
+  public void resetLastInodeId(long newValue) throws IOException {
+    inodeId.resetLastInodeId(newValue);
+  }
+
+  /** Should only be used for tests to reset to any value */
+  void resetLastInodeIdWithoutChecking(long newValue) {
+    inodeId.resetLastInodeIdWithoutChecking(newValue);
+  }
+  
+  public long getLastInodeId() {
+    return inodeId.getLastInodeId();
+  }
+
+  public long allocateNewInodeId() {
+    return inodeId.allocateNewInodeId();
+  }
+  
   /**
    * Clear all loaded data
    */
@@ -386,6 +410,7 @@ public class FSNamesystem implements Nam
     dtSecretManager.reset();
     generationStamp.setStamp(GenerationStamp.FIRST_VALID_STAMP);
     leaseManager.removeAllLeases();
+    inodeId.resetLastInodeIdWithoutChecking(INodeId.LAST_RESERVED_ID);
   }
 
   @VisibleForTesting
@@ -561,6 +586,8 @@ public class FSNamesystem implements Nam
       this.standbyShouldCheckpoint = conf.getBoolean(
           DFS_HA_STANDBY_CHECKPOINTS_KEY, DFS_HA_STANDBY_CHECKPOINTS_DEFAULT);
       
+      this.inodeId = new INodeId();
+      
       // For testing purposes, allow the DT secret manager to be started regardless
       // of whether security is enabled.
       alwaysUseDelegationTokensForTests = conf.getBoolean(
@@ -1895,6 +1922,7 @@ public class FSNamesystem implements Nam
       String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
       boolean writeToEditLog) throws IOException {
     INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
+                                    file.getId(),
                                     file.getLocalNameBytes(),
                                     file.getBlockReplication(),
                                     file.getModificationTime(),

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java Wed Apr 24 18:20:09 2013
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.ArrayList;
@@ -102,6 +103,11 @@ abstract class INode implements Comparab
   }
 
   /**
+   * The inode id
+   */
+  final private long id;
+
+  /**
    *  The inode name is in java UTF8 encoding; 
    *  The name in HdfsFileStatus should keep the same encoding as this.
    *  if this encoding is changed, implicitly getFileInfo and listStatus in
@@ -120,8 +126,9 @@ abstract class INode implements Comparab
   protected long modificationTime = 0L;
   protected long accessTime = 0L;
 
-  private INode(byte[] name, long permission, INodeDirectory parent,
+  private INode(long id, byte[] name, long permission, INodeDirectory parent,
       long modificationTime, long accessTime) {
+    this.id = id;
     this.name = name;
     this.permission = permission;
     this.parent = parent;
@@ -129,26 +136,31 @@ abstract class INode implements Comparab
     this.accessTime = accessTime;
   }
 
-  INode(byte[] name, PermissionStatus permissions, INodeDirectory parent,
-      long modificationTime, long accessTime) {
-    this(name, PermissionStatusFormat.toLong(permissions), parent,
+  INode(long id, byte[] name, PermissionStatus permissions,
+      INodeDirectory parent, long modificationTime, long accessTime) {
+    this(id, name, PermissionStatusFormat.toLong(permissions), parent,
         modificationTime, accessTime);
   }
-
-  INode(PermissionStatus permissions, long mtime, long atime) {
-    this(null, permissions, null, mtime, atime);
+  
+  INode(long id, PermissionStatus permissions, long mtime, long atime) {
+    this(id, null, PermissionStatusFormat.toLong(permissions), null, mtime, atime);
   }
-
-  protected INode(String name, PermissionStatus permissions) {
-    this(DFSUtil.string2Bytes(name), permissions, null, 0L, 0L);
+  
+  protected INode(long id, String name, PermissionStatus permissions) {
+    this(id, DFSUtil.string2Bytes(name), permissions, null, 0L, 0L);
   }
   
   /** @param other Other node to be copied */
   INode(INode other) {
-    this(other.getLocalNameBytes(), other.permission, other.getParent(), 
-        other.getModificationTime(), other.getAccessTime());
+    this(other.getId(), other.getLocalNameBytes(), other.permission, other
+        .getParent(), other.getModificationTime(), other.getAccessTime());
   }
 
+  /** Get inode id */
+  public long getId() {
+    return this.id;
+  }
+  
   /**
    * Check whether this is the root inode.
    */
@@ -463,6 +475,7 @@ abstract class INode implements Comparab
   /**
    * Create an INode; the inode's name is not set yet
    * 
+   * @param id preassigned inode id
    * @param permissions permissions
    * @param blocks blocks if a file
    * @param symlink symblic link if a symbolic link
@@ -474,7 +487,8 @@ abstract class INode implements Comparab
    * @param preferredBlockSize block size
    * @return an inode
    */
-  static INode newINode(PermissionStatus permissions,
+  static INode newINode(long id,
+                        PermissionStatus permissions,
                         BlockInfo[] blocks,
                         String symlink,
                         short replication,
@@ -484,17 +498,17 @@ abstract class INode implements Comparab
                         long dsQuota,
                         long preferredBlockSize) {
     if (symlink.length() != 0) { // check if symbolic link
-      return new INodeSymlink(symlink, modificationTime, atime, permissions);
+      return new INodeSymlink(id, symlink, modificationTime, atime, permissions);
     }  else if (blocks == null) { //not sym link and blocks null? directory!
       if (nsQuota >= 0 || dsQuota >= 0) {
         return new INodeDirectoryWithQuota(
-            permissions, modificationTime, nsQuota, dsQuota);
+             id, permissions, modificationTime, nsQuota, dsQuota);
       } 
       // regular directory
-      return new INodeDirectory(permissions, modificationTime);
+      return new INodeDirectory(id, permissions, modificationTime);
     }
     // file
-    return new INodeFile(permissions, blocks, replication,
+    return new INodeFile(id, permissions, blocks, replication,
         modificationTime, atime, preferredBlockSize);
   }
 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java Wed Apr 24 18:20:09 2013
@@ -53,17 +53,17 @@ class INodeDirectory extends INode {
 
   private List<INode> children = null;
 
-  INodeDirectory(String name, PermissionStatus permissions) {
-    super(name, permissions);
+  INodeDirectory(long id, String name, PermissionStatus permissions) {
+    super(id, name, permissions);
   }
 
-  public INodeDirectory(PermissionStatus permissions, long mTime) {
-    super(permissions, mTime, 0);
+  public INodeDirectory(long id, PermissionStatus permissions, long mTime) {
+    super(id, permissions, mTime, 0);
   }
-
+  
   /** constructor */
-  INodeDirectory(byte[] name, PermissionStatus permissions, long mtime) {
-    super(name, permissions, null, mtime, 0L);
+  INodeDirectory(long id, byte[] name, PermissionStatus permissions, long mtime) {
+    super(id, name, permissions, null, mtime, 0L);
   }
   
   /** copy constructor

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java Wed Apr 24 18:20:09 2013
@@ -54,16 +54,16 @@ class INodeDirectoryWithQuota extends IN
   }
   
   /** constructor with no quota verification */
-  INodeDirectoryWithQuota(PermissionStatus permissions, long modificationTime,
-      long nsQuota, long dsQuota) {
-    super(permissions, modificationTime);
+  INodeDirectoryWithQuota(long id, PermissionStatus permissions,
+      long modificationTime, long nsQuota, long dsQuota) {
+    super(id, permissions, modificationTime);
     this.nsQuota = nsQuota;
     this.dsQuota = dsQuota;
   }
   
   /** constructor with no quota verification */
-  INodeDirectoryWithQuota(String name, PermissionStatus permissions) {
-    super(name, permissions);
+  INodeDirectoryWithQuota(long id, String name, PermissionStatus permissions) {
+    super(id, name, permissions);
   }
   
   /** Get this directory's namespace quota

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java Wed Apr 24 18:20:09 2013
@@ -86,15 +86,15 @@ class INodeFile extends INode implements
 
   private BlockInfo[] blocks;
 
-  INodeFile(PermissionStatus permissions, BlockInfo[] blklist,
-                      short replication, long modificationTime,
-                      long atime, long preferredBlockSize) {
-    super(permissions, modificationTime, atime);
+  INodeFile(long id, PermissionStatus permissions, BlockInfo[] blklist,
+      short replication, long modificationTime, long atime,
+      long preferredBlockSize) {
+    super(id, permissions, modificationTime, atime);
     header = HeaderFormat.combineReplication(header, replication);
     header = HeaderFormat.combinePreferredBlockSize(header, preferredBlockSize);
     this.blocks = blklist;
   }
-
+  
   /** @return true unconditionally. */
   @Override
   public final boolean isFile() {

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java Wed Apr 24 18:20:09 2013
@@ -49,21 +49,23 @@ class INodeFileUnderConstruction extends
   private final String clientMachine;
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.
   
-  INodeFileUnderConstruction(PermissionStatus permissions,
+  INodeFileUnderConstruction(long id,
+                             PermissionStatus permissions,
                              short replication,
                              long preferredBlockSize,
                              long modTime,
                              String clientName,
                              String clientMachine,
                              DatanodeDescriptor clientNode) {
-    super(permissions.applyUMask(UMASK), BlockInfo.EMPTY_ARRAY, replication,
-        modTime, modTime, preferredBlockSize);
+    super(id, permissions.applyUMask(UMASK), BlockInfo.EMPTY_ARRAY,
+        replication, modTime, modTime, preferredBlockSize);
     this.clientName = clientName;
     this.clientMachine = clientMachine;
     this.clientNode = clientNode;
   }
 
-  INodeFileUnderConstruction(byte[] name,
+  INodeFileUnderConstruction(long id,
+                             byte[] name,
                              short blockReplication,
                              long modificationTime,
                              long preferredBlockSize,
@@ -72,8 +74,8 @@ class INodeFileUnderConstruction extends
                              String clientName,
                              String clientMachine,
                              DatanodeDescriptor clientNode) {
-    super(perm, blocks, blockReplication, modificationTime, modificationTime,
-          preferredBlockSize);
+    super(id, perm, blocks, blockReplication, modificationTime,
+        modificationTime, preferredBlockSize);
     setLocalName(name);
     this.clientName = clientName;
     this.clientMachine = clientMachine;
@@ -112,7 +114,8 @@ class INodeFileUnderConstruction extends
     assert allBlocksComplete() : "Can't finalize inode " + this
       + " since it contains non-complete blocks! Blocks are "
       + Arrays.asList(getBlocks());
-    INodeFile obj = new INodeFile(getPermissionStatus(),
+    INodeFile obj = new INodeFile(getId(),
+                                  getPermissionStatus(),
                                   getBlocks(),
                                   getBlockReplication(),
                                   getModificationTime(),

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java Wed Apr 24 18:20:09 2013
@@ -28,9 +28,9 @@ import org.apache.hadoop.hdfs.DFSUtil;
 public class INodeSymlink extends INode {
   private final byte[] symlink; // The target URI
 
-  INodeSymlink(String value, long mtime, long atime,
-               PermissionStatus permissions) {
-    super(permissions, mtime, atime);
+  INodeSymlink(long id, String value, long mtime, long atime,
+      PermissionStatus permissions) {
+    super(id, permissions, mtime, atime);
     this.symlink = DFSUtil.string2Bytes(value);
   }
 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java Wed Apr 24 18:20:09 2013
@@ -62,7 +62,8 @@ public class CreateEditsLog {
     
     PermissionStatus p = new PermissionStatus("joeDoe", "people",
                                       new FsPermission((short)0777));
-    INodeDirectory dirInode = new INodeDirectory(p, 0L);
+    INodeDirectory dirInode = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
+        p, 0L);
     editLog.logMkDir(BASE_PATH, dirInode);
     long blockSize = 10;
     BlockInfo[] blocks = new BlockInfo[blocksPerFile];
@@ -81,8 +82,9 @@ public class CreateEditsLog {
       }
 
       INodeFileUnderConstruction inode = new INodeFileUnderConstruction(
-                    null, replication, 0, blockSize, blocks, p, "", "", null);
-      // Append path to filename with information about blockIDs 
+          INodeId.GRANDFATHER_INODE_ID, null, replication, 0, blockSize,
+          blocks, p, "", "", null);
+     // Append path to filename with information about blockIDs 
       String path = "_" + iF + "_B" + blocks[0].getBlockId() + 
                     "_to_B" + blocks[blocksPerFile-1].getBlockId() + "_";
       String filePath = nameGenerator.getNextFileName("");
@@ -90,12 +92,12 @@ public class CreateEditsLog {
       // Log the new sub directory in edits
       if ((iF % nameGenerator.getFilesPerDirectory())  == 0) {
         String currentDir = nameGenerator.getCurrentDir();
-        dirInode = new INodeDirectory(p, 0L);
+        dirInode = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID, p, 0L);
         editLog.logMkDir(currentDir, dirInode);
       }
-      editLog.logOpenFile(filePath, 
-          new INodeFileUnderConstruction(
-              p, replication, 0, blockSize, "", "", null));
+      editLog.logOpenFile(filePath, new INodeFileUnderConstruction(
+          INodeId.GRANDFATHER_INODE_ID, p, replication, 0, blockSize, "", "",
+          null));
       editLog.logCloseFile(filePath, inode);
 
       if (currentBlockId - bidAtSync >= 2000) { // sync every 2K blocks

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java Wed Apr 24 18:20:09 2013
@@ -208,7 +208,7 @@ public abstract class FSImageTestUtil {
    * only a specified number of "mkdirs" operations.
    */
   public static void createAbortedLogWithMkdirs(File editsLogDir, int numDirs,
-      long firstTxId) throws IOException {
+      long firstTxId, long newInodeId) throws IOException {
     FSEditLog editLog = FSImageTestUtil.createStandaloneEditLog(editsLogDir);
     editLog.setNextTxId(firstTxId);
     editLog.openForWrite();
@@ -217,7 +217,7 @@ public abstract class FSImageTestUtil {
         FsPermission.createImmutable((short)0755));
     for (int i = 1; i <= numDirs; i++) {
       String dirName = "dir" + i;
-      INodeDirectory dir = new INodeDirectory(dirName, perms);
+      INodeDirectory dir = new INodeDirectory(newInodeId + i -1, dirName, perms);
       editLog.logMkDir("/" + dirName, dir);
     }
     editLog.logSync();

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java Wed Apr 24 18:20:09 2013
@@ -153,7 +153,8 @@ public class TestEditLog {
 
       for (int i = 0; i < numTransactions; i++) {
         INodeFileUnderConstruction inode = new INodeFileUnderConstruction(
-                            p, replication, blockSize, 0, "", "", null);
+            namesystem.allocateNewInodeId(), p, replication, blockSize, 0, "",
+            "", null);
         editLog.logOpenFile("/filename" + (startIndex + i), inode);
         editLog.logCloseFile("/filename" + (startIndex + i), inode);
         editLog.logSync();
@@ -318,6 +319,11 @@ public class TestEditLog {
       // we should now be writing to edits_inprogress_3
       fsimage.rollEditLog();
     
+      // Remember the current lastInodeId and will reset it back to test
+      // loading editlog segments.The transactions in the following allocate new
+      // inode id to write to editlogs but doesn't create ionde in namespace
+      long originalLastInodeId = namesystem.getLastInodeId();
+      
       // Create threads and make them run transactions concurrently.
       Thread threadId[] = new Thread[NUM_THREADS];
       for (int i = 0; i < NUM_THREADS; i++) {
@@ -350,6 +356,7 @@ public class TestEditLog {
       // If there were any corruptions, it is likely that the reading in
       // of these transactions will throw an exception.
       //
+      namesystem.resetLastInodeIdWithoutChecking(originalLastInodeId);
       for (Iterator<StorageDirectory> it = 
               fsimage.getStorage().dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
         FSEditLogLoader loader = new FSEditLogLoader(namesystem, 0);

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java Wed Apr 24 18:20:09 2013
@@ -73,7 +73,8 @@ public class TestFsLimits {
              fileAsURI(new File(MiniDFSCluster.getBaseDirectory(),
                                 "namenode")).toString());
 
-    rootInode = new INodeDirectoryWithQuota(INodeDirectory.ROOT_NAME, perms);
+    rootInode = new INodeDirectoryWithQuota(getMockNamesystem()
+        .allocateNewInodeId(), INodeDirectory.ROOT_NAME, perms);
     inodes = new INode[]{ rootInode, null };
     fs = null;
     fsIsReady = true;
@@ -152,7 +153,8 @@ public class TestFsLimits {
     // have to create after the caller has had a chance to set conf values
     if (fs == null) fs = new MockFSDirectory();
 
-    INode child = new INodeDirectory(name, perms);
+    INode child = new INodeDirectory(getMockNamesystem().allocateNewInodeId(),
+        name, perms);
     child.setLocalName(name);
     
     Class<?> generated = null;

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java Wed Apr 24 18:20:09 2013
@@ -25,10 +25,14 @@ import static org.junit.Assert.fail;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.junit.Test;
 
@@ -49,9 +53,9 @@ public class TestINodeFile {
   public void testReplication () {
     replication = 3;
     preferredBlockSize = 128*1024*1024;
-    INodeFile inf = new INodeFile(new PermissionStatus(userName, null, 
-                                  FsPermission.getDefault()), null, replication,
-                                  0L, 0L, preferredBlockSize);
+    INodeFile inf = new INodeFile(INodeId.GRANDFATHER_INODE_ID,
+        new PermissionStatus(userName, null, FsPermission.getDefault()), null,
+        replication, 0L, 0L, preferredBlockSize);
     assertEquals("True has to be returned in this case", replication,
                  inf.getBlockReplication());
   }
@@ -66,9 +70,9 @@ public class TestINodeFile {
               throws IllegalArgumentException {
     replication = -1;
     preferredBlockSize = 128*1024*1024;
-    new INodeFile(new PermissionStatus(userName, null,
-                                  FsPermission.getDefault()), null, replication,
-                                  0L, 0L, preferredBlockSize);
+    new INodeFile(INodeId.GRANDFATHER_INODE_ID, new PermissionStatus(userName,
+        null, FsPermission.getDefault()), null, replication, 0L, 0L,
+        preferredBlockSize);
   }
 
   /**
@@ -79,20 +83,20 @@ public class TestINodeFile {
   public void testPreferredBlockSize () {
     replication = 3;
     preferredBlockSize = 128*1024*1024;
-    INodeFile inf = new INodeFile(new PermissionStatus(userName, null,
-                                  FsPermission.getDefault()), null, replication,
-                                  0L, 0L, preferredBlockSize);
-    assertEquals("True has to be returned in this case", preferredBlockSize,
-           inf.getPreferredBlockSize());
-  }
+    INodeFile inf = new INodeFile(INodeId.GRANDFATHER_INODE_ID,
+        new PermissionStatus(userName, null, FsPermission.getDefault()), null,
+        replication, 0L, 0L, preferredBlockSize);
+   assertEquals("True has to be returned in this case", preferredBlockSize,
+        inf.getPreferredBlockSize());
+ }
 
   @Test
   public void testPreferredBlockSizeUpperBound () {
     replication = 3;
     preferredBlockSize = BLKSIZE_MAXVALUE;
-    INodeFile inf = new INodeFile(new PermissionStatus(userName, null, 
-                                  FsPermission.getDefault()), null, replication,
-                                  0L, 0L, preferredBlockSize);
+    INodeFile inf = new INodeFile(INodeId.GRANDFATHER_INODE_ID,
+        new PermissionStatus(userName, null, FsPermission.getDefault()), null,
+        replication, 0L, 0L, preferredBlockSize);
     assertEquals("True has to be returned in this case", BLKSIZE_MAXVALUE,
                  inf.getPreferredBlockSize());
   }
@@ -107,9 +111,9 @@ public class TestINodeFile {
               throws IllegalArgumentException {
     replication = 3;
     preferredBlockSize = -1;
-    new INodeFile(new PermissionStatus(userName, null, 
-                                  FsPermission.getDefault()), null, replication,
-                                  0L, 0L, preferredBlockSize);
+    new INodeFile(INodeId.GRANDFATHER_INODE_ID, new PermissionStatus(userName,
+        null, FsPermission.getDefault()), null, replication, 0L, 0L,
+        preferredBlockSize);
   } 
 
   /**
@@ -122,10 +126,10 @@ public class TestINodeFile {
               throws IllegalArgumentException {
     replication = 3;
     preferredBlockSize = BLKSIZE_MAXVALUE+1;
-    new INodeFile(new PermissionStatus(userName, null, 
-                                  FsPermission.getDefault()), null, replication,
-                                  0L, 0L, preferredBlockSize);
-  }
+    new INodeFile(INodeId.GRANDFATHER_INODE_ID, new PermissionStatus(userName,
+        null, FsPermission.getDefault()), null, replication, 0L, 0L,
+        preferredBlockSize);
+ }
 
   @Test
   public void testGetFullPathName() {
@@ -134,12 +138,14 @@ public class TestINodeFile {
 
     replication = 3;
     preferredBlockSize = 128*1024*1024;
-    INodeFile inf = new INodeFile(perms, null, replication,
-                                  0L, 0L, preferredBlockSize);
+    INodeFile inf = new INodeFile(INodeId.GRANDFATHER_INODE_ID, perms, null,
+        replication, 0L, 0L, preferredBlockSize);
     inf.setLocalName("f");
 
-    INodeDirectory root = new INodeDirectory(INodeDirectory.ROOT_NAME, perms);
-    INodeDirectory dir = new INodeDirectory("d", perms);
+    INodeDirectory root = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
+        INodeDirectory.ROOT_NAME, perms);
+    INodeDirectory dir = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID, "d",
+        perms);
 
     assertEquals("f", inf.getFullPathName());
     assertEquals("", inf.getLocalParentDir());
@@ -195,7 +201,7 @@ public class TestINodeFile {
     for (int i = 0; i < nCount; i++) {
       PermissionStatus perms = new PermissionStatus(userName, null,
           FsPermission.getDefault());
-      iNodes[i] = new INodeFile(perms, null, replication, 0L, 0L,
+      iNodes[i] = new INodeFile(i, perms, null, replication, 0L, 0L,
           preferredBlockSize);
       iNodes[i].setLocalName(fileNamePrefix +  Integer.toString(i));
       BlockInfo newblock = new BlockInfo(replication);
@@ -246,10 +252,10 @@ public class TestINodeFile {
     }
 
     {//cast from INodeFile
-      final INode from = new INodeFile(
-          perm, null, replication, 0L, 0L, preferredBlockSize);
-      
-      //cast to INodeFile, should success
+      final INode from = new INodeFile(INodeId.GRANDFATHER_INODE_ID, perm,
+          null, replication, 0L, 0L, preferredBlockSize);
+
+     //cast to INodeFile, should success
       final INodeFile f = INodeFile.valueOf(from, path);
       assertTrue(f == from);
 
@@ -271,8 +277,9 @@ public class TestINodeFile {
 
     {//cast from INodeFileUnderConstruction
       final INode from = new INodeFileUnderConstruction(
-          perm, replication, 0L, 0L, "client", "machine", null);
-      
+          INodeId.GRANDFATHER_INODE_ID, perm, replication, 0L, 0L, "client",
+          "machine", null);
+    
       //cast to INodeFile, should success
       final INodeFile f = INodeFile.valueOf(from, path);
       assertTrue(f == from);
@@ -291,7 +298,8 @@ public class TestINodeFile {
     }
 
     {//cast from INodeDirectory
-      final INode from = new INodeDirectory(perm, 0L);
+      final INode from = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID, perm,
+          0L);
 
       //cast to INodeFile, should fail
       try {
@@ -314,4 +322,47 @@ public class TestINodeFile {
       assertTrue(d == from);
     }
   }
+
+  /**
+   * Verify root always has inode id 1001 and new formated fsimage has last
+   * allocated inode id 1000. Validate correct lastInodeId is persisted.
+   * @throws IOException
+   */
+  @Test
+  public void TestInodeId() throws IOException {
+
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
+        DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
+        .build();
+    cluster.waitActive();
+    
+    FSNamesystem fsn = cluster.getNamesystem();
+    long lastId = fsn.getLastInodeId();
+
+    assertTrue(lastId == 1001);
+
+    // Create one directory and the last inode id should increase to 1002
+    FileSystem fs = cluster.getFileSystem();
+    Path path = new Path("/test1");
+    assertTrue(fs.mkdirs(path));
+    assertTrue(fsn.getLastInodeId() == 1002);
+
+    Path filePath = new Path("/test1/file");
+    fs.create(filePath);
+    assertTrue(fsn.getLastInodeId() == 1003);
+
+    // Rename doesn't increase inode id
+    Path renamedPath = new Path("/test2");
+    fs.rename(path, renamedPath);
+    assertTrue(fsn.getLastInodeId() == 1003);
+
+    cluster.restartNameNode();
+    cluster.waitActive();
+    // Make sure empty editlog can be handled
+    cluster.restartNameNode();
+    cluster.waitActive();
+    assertTrue(fsn.getLastInodeId() == 1003);
+  }
 }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java Wed Apr 24 18:20:09 2013
@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.MiniDFSClu
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.io.IOUtils;
@@ -139,7 +140,9 @@ public class TestEditLogsDuringFailover 
       // Create a fake in-progress edit-log in the shared directory
       URI sharedUri = cluster.getSharedEditsDir(0, 1);
       File sharedDir = new File(sharedUri.getPath(), "current");
-      FSImageTestUtil.createAbortedLogWithMkdirs(sharedDir, NUM_DIRS_IN_LOG, 1);
+      FSNamesystem fsn = cluster.getNamesystem(0);
+      FSImageTestUtil.createAbortedLogWithMkdirs(sharedDir, NUM_DIRS_IN_LOG, 1,
+          fsn.getLastInodeId() + 1);
       
       assertEditFiles(Collections.singletonList(sharedUri),
           NNStorage.getInProgressEditsFileName(1));

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java?rev=1471582&r1=1471581&r2=1471582&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java Wed Apr 24 18:20:09 2013
@@ -212,8 +212,9 @@ public class TestStandbyCheckpoints {
     File sharedDir = new File(sharedUri.getPath(), "current");
     File tmpDir = new File(MiniDFSCluster.getBaseDirectory(),
         "testCheckpointCancellation-tmp");
-    FSImageTestUtil.createAbortedLogWithMkdirs(tmpDir, NUM_DIRS_IN_LOG,
-        3);
+    FSNamesystem fsn = cluster.getNamesystem(0);
+    FSImageTestUtil.createAbortedLogWithMkdirs(tmpDir, NUM_DIRS_IN_LOG, 3,
+        fsn.getLastInodeId() + 1);
     String fname = NNStorage.getInProgressEditsFileName(3); 
     new File(tmpDir, fname).renameTo(new File(sharedDir, fname));