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:42:14 UTC

svn commit: r1471596 - in /hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/protocol/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/java/org/apache/hadoop/hdfs/tools/offlineImageV...

Author: suresh
Date: Wed Apr 24 18:42:14 2013
New Revision: 1471596

URL: http://svn.apache.org/r1471596
Log:
HDFS-4339. Merge change r1465835 from trunk.

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/protocol/LayoutVersion.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.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/FSEditLogOp.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/tools/offlineImageViewer/ImageLoaderCurrent.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.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/TestINodeFile.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

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=1471596&r1=1471595&r2=1471596&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:42:14 2013
@@ -11,6 +11,11 @@ Release 2.0.5-beta - UNRELEASED
 
     HDFS-4296. Reserve layout version for release 1.2.0. (suresh)
 
+    HDFS-4334. Add a unique id to INode.  (Brandon Li via szetszwo)
+
+    HDFS-4339. Persist inode id in fsimage and editlog. (Brandon Li via
+    suresh)
+
   IMPROVEMENTS
 
     HDFS-4222. NN is unresponsive and loses heartbeats from DNs when 
@@ -76,8 +81,6 @@ 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/protocol/LayoutVersion.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java?rev=1471596&r1=1471595&r2=1471596&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java Wed Apr 24 18:42:14 2013
@@ -96,7 +96,8 @@ public class LayoutVersion {
     OPTIMIZE_PERSIST_BLOCKS(-40,
         "Serialize block lists with delta-encoded variable length ints, " +
         "add OP_UPDATE_BLOCKS"),
-    RESERVED_REL1_2_0(-41, -32, "Reserved for release 1.2.0", true, CONCAT);
+    RESERVED_REL1_2_0(-41, -32, "Reserved for release 1.2.0", true, CONCAT),
+    ADD_INODE_ID(-42, -40, "Assign a unique inode id for each inode", false);
     
     final int lv;
     final int ancestorLV;

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.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/FSEditLog.java?rev=1471596&r1=1471595&r2=1471596&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java Wed Apr 24 18:42:14 2013
@@ -636,6 +636,7 @@ public class FSEditLog implements LogsPu
    */
   public void logOpenFile(String path, INodeFileUnderConstruction newNode) {
     AddOp op = AddOp.getInstance(cache.get())
+      .setInodeId(newNode.getId())
       .setPath(path)
       .setReplication(newNode.getBlockReplication())
       .setModificationTime(newNode.getModificationTime())
@@ -677,6 +678,7 @@ public class FSEditLog implements LogsPu
    */
   public void logMkDir(String path, INode newNode) {
     MkdirOp op = MkdirOp.getInstance(cache.get())
+      .setInodeId(newNode.getId())
       .setPath(path)
       .setTimestamp(newNode.getModificationTime())
       .setPermissionStatus(newNode.getPermissionStatus());
@@ -794,6 +796,7 @@ public class FSEditLog implements LogsPu
   void logSymlink(String path, String value, long mtime, 
                   long atime, INodeSymlink node) {
     SymlinkOp op = SymlinkOp.getInstance(cache.get())
+      .setId(node.getId())
       .setPath(path)
       .setValue(value)
       .setModificationTime(mtime)

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=1471596&r1=1471595&r2=1471596&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:42:14 2013
@@ -33,6 +33,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -169,7 +170,7 @@ public class FSEditLogLoader {
             }
           }
           try {
-            long inodeId = applyEditLogOp(op, fsDir, in.getVersion());
+            long inodeId = applyEditLogOp(op, fsDir, in.getVersion(), lastInodeId);
             if (lastInodeId < inodeId) {
               lastInodeId = inodeId;
             }
@@ -225,9 +226,30 @@ public class FSEditLogLoader {
     return numEdits;
   }
   
+  // allocate and update last allocated inode id
+  private long getAndUpdateLastInodeId(long inodeIdFromOp, int logVersion,
+      long lastInodeId) throws IOException {
+    long inodeId = inodeIdFromOp;
+
+    if (inodeId == INodeId.GRANDFATHER_INODE_ID) {
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+        throw new IOException("The layout version " + logVersion
+            + " supports inodeId but gave bogus inodeId");
+      }
+      inodeId = fsNamesys.allocateNewInodeId();
+    } else {
+      // need to reset lastInodeId. fsnamesys gets lastInodeId firstly from
+      // fsimage but editlog captures more recent inodeId allocations
+      if (inodeId > lastInodeId) {
+        fsNamesys.resetLastInodeId(inodeId);
+      }
+    }
+    return inodeId;
+  }
+
   @SuppressWarnings("deprecation")
   private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
-      int logVersion) throws IOException {
+      int logVersion, long lastInodeId) throws IOException {
     long inodeId = INodeId.GRANDFATHER_INODE_ID;
     if (LOG.isTraceEnabled()) {
       LOG.trace("replaying edit log: " + op);
@@ -258,7 +280,8 @@ public class FSEditLogLoader {
         assert addCloseOp.blocks.length == 0;
 
         // add to the file tree
-        inodeId = fsNamesys.allocateNewInodeId();
+        inodeId = getAndUpdateLastInodeId(addCloseOp.inodeId, logVersion,
+            lastInodeId);
         newFile = (INodeFile) fsDir.unprotectedAddFile(inodeId,
             addCloseOp.path, addCloseOp.permissions, replication,
             addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
@@ -373,7 +396,8 @@ public class FSEditLogLoader {
     }
     case OP_MKDIR: {
       MkdirOp mkdirOp = (MkdirOp)op;
-      inodeId = fsNamesys.allocateNewInodeId();
+      inodeId = getAndUpdateLastInodeId(mkdirOp.inodeId, logVersion,
+          lastInodeId);
       fsDir.unprotectedMkdir(inodeId, mkdirOp.path, mkdirOp.permissions,
                              mkdirOp.timestamp);
       break;
@@ -427,7 +451,8 @@ public class FSEditLogLoader {
     }
     case OP_SYMLINK: {
       SymlinkOp symlinkOp = (SymlinkOp)op;
-      inodeId = fsNamesys.allocateNewInodeId();
+      inodeId = getAndUpdateLastInodeId(symlinkOp.inodeId, logVersion,
+          lastInodeId);
       fsDir.unprotectedAddSymlink(inodeId, symlinkOp.path,
                                   symlinkOp.value, symlinkOp.mtime, 
                                   symlinkOp.atime, symlinkOp.permissionStatus);

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.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/FSEditLogOp.java?rev=1471596&r1=1471595&r2=1471596&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java Wed Apr 24 18:42:14 2013
@@ -158,6 +158,7 @@ public abstract class FSEditLogOp {
   @SuppressWarnings("unchecked")
   static abstract class AddCloseOp extends FSEditLogOp implements BlockListUpdatingOp {
     int length;
+    long inodeId;
     String path;
     short replication;
     long mtime;
@@ -172,6 +173,11 @@ public abstract class FSEditLogOp {
       super(opCode);
       assert(opCode == OP_ADD || opCode == OP_CLOSE);
     }
+    
+    <T extends AddCloseOp> T setInodeId(long inodeId) {
+      this.inodeId = inodeId;
+      return (T)this;
+    }
 
     <T extends AddCloseOp> T setPath(String path) {
       this.path = path;
@@ -235,6 +241,7 @@ public abstract class FSEditLogOp {
     @Override
     public 
     void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeLong(inodeId, out);
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeShort(replication, out);
       FSImageSerialization.writeLong(mtime, out);
@@ -255,6 +262,12 @@ public abstract class FSEditLogOp {
       if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
       }
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+        this.inodeId = in.readLong();
+      } else {
+        // The inodeId should be updated when this editLogOp is applied
+        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+      }
       if ((-17 < logVersion && length != 4) ||
           (logVersion <= -17 && length != 5 && !LayoutVersion.supports(
               Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
@@ -327,6 +340,8 @@ public abstract class FSEditLogOp {
       StringBuilder builder = new StringBuilder();
       builder.append("[length=");
       builder.append(length);
+      builder.append(", inodeId=");
+      builder.append(inodeId);
       builder.append(", path=");
       builder.append(path);
       builder.append(", replication=");
@@ -357,6 +372,8 @@ public abstract class FSEditLogOp {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
       XMLUtils.addSaxString(contentHandler, "LENGTH",
           Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "INODEID",
+          Long.valueOf(inodeId).toString());
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "REPLICATION",
           Short.valueOf(replication).toString());
@@ -376,6 +393,7 @@ public abstract class FSEditLogOp {
 
     @Override void fromXml(Stanza st) throws InvalidXmlException {
       this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.inodeId = Long.valueOf(st.getValue("INODEID"));
       this.path = st.getValue("PATH");
       this.replication = Short.valueOf(st.getValue("REPLICATION"));
       this.mtime = Long.valueOf(st.getValue("MTIME"));
@@ -907,6 +925,7 @@ public abstract class FSEditLogOp {
     
   static class MkdirOp extends FSEditLogOp {
     int length;
+    long inodeId;
     String path;
     long timestamp;
     PermissionStatus permissions;
@@ -919,6 +938,11 @@ public abstract class FSEditLogOp {
       return (MkdirOp)cache.get(OP_MKDIR);
     }
 
+    MkdirOp setInodeId(long inodeId) {
+      this.inodeId = inodeId;
+      return this;
+    }
+    
     MkdirOp setPath(String path) {
       this.path = path;
       return this;
@@ -937,6 +961,7 @@ public abstract class FSEditLogOp {
     @Override
     public 
     void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeLong(inodeId, out);
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeLong(timestamp, out); // mtime
       FSImageSerialization.writeLong(timestamp, out); // atime, unused at this
@@ -953,6 +978,12 @@ public abstract class FSEditLogOp {
           && !LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         throw new IOException("Incorrect data format. Mkdir operation.");
       }
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+        this.inodeId = FSImageSerialization.readLong(in);
+      } else {
+        // This id should be updated when this editLogOp is applied
+        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+      }
       this.path = FSImageSerialization.readString(in);
       if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.timestamp = FSImageSerialization.readLong(in);
@@ -979,6 +1010,8 @@ public abstract class FSEditLogOp {
       StringBuilder builder = new StringBuilder();
       builder.append("MkdirOp [length=");
       builder.append(length);
+      builder.append(", inodeId=");
+      builder.append(inodeId);
       builder.append(", path=");
       builder.append(path);
       builder.append(", timestamp=");
@@ -997,6 +1030,8 @@ public abstract class FSEditLogOp {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
       XMLUtils.addSaxString(contentHandler, "LENGTH",
           Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "INODEID",
+          Long.valueOf(inodeId).toString());
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
           Long.valueOf(timestamp).toString());
@@ -1005,6 +1040,7 @@ public abstract class FSEditLogOp {
     
     @Override void fromXml(Stanza st) throws InvalidXmlException {
       this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.inodeId = Long.valueOf(st.getValue("INODEID"));
       this.path = st.getValue("PATH");
       this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
       this.permissions =
@@ -1483,6 +1519,7 @@ public abstract class FSEditLogOp {
 
   static class SymlinkOp extends FSEditLogOp {
     int length;
+    long inodeId;
     String path;
     String value;
     long mtime;
@@ -1497,6 +1534,11 @@ public abstract class FSEditLogOp {
       return (SymlinkOp)cache.get(OP_SYMLINK);
     }
 
+    SymlinkOp setId(long inodeId) {
+      this.inodeId = inodeId;
+      return this;
+    }
+    
     SymlinkOp setPath(String path) {
       this.path = path;
       return this;
@@ -1525,6 +1567,7 @@ public abstract class FSEditLogOp {
     @Override
     public 
     void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeLong(inodeId, out);      
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeString(value, out);
       FSImageSerialization.writeLong(mtime, out);
@@ -1542,6 +1585,12 @@ public abstract class FSEditLogOp {
               + "symlink operation.");
         }
       }
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+        this.inodeId = FSImageSerialization.readLong(in);
+      } else {
+        // This id should be updated when the editLogOp is applied
+        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+      }
       this.path = FSImageSerialization.readString(in);
       this.value = FSImageSerialization.readString(in);
 
@@ -1560,6 +1609,8 @@ public abstract class FSEditLogOp {
       StringBuilder builder = new StringBuilder();
       builder.append("SymlinkOp [length=");
       builder.append(length);
+      builder.append(", inodeId=");
+      builder.append(inodeId);
       builder.append(", path=");
       builder.append(path);
       builder.append(", value=");
@@ -1582,6 +1633,8 @@ public abstract class FSEditLogOp {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
       XMLUtils.addSaxString(contentHandler, "LENGTH",
           Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "INODEID",
+          Long.valueOf(inodeId).toString());
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "VALUE", value);
       XMLUtils.addSaxString(contentHandler, "MTIME",
@@ -1593,6 +1646,7 @@ public abstract class FSEditLogOp {
 
     @Override void fromXml(Stanza st) throws InvalidXmlException {
       this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.inodeId = Long.valueOf(st.getValue("INODEID"));
       this.path = st.getValue("PATH");
       this.value = st.getValue("VALUE");
       this.mtime = Long.valueOf(st.getValue("MTIME"));

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=1471596&r1=1471595&r2=1471596&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:42:14 2013
@@ -206,6 +206,20 @@ class FSImageFormat {
           imgTxId = 0;
         }
 
+        // read the last allocated inode id in the fsimage
+        if (LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion)) {
+          long lastInodeId = in.readLong();
+          namesystem.resetLastInodeId(lastInodeId);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("load last allocated InodeId from fsimage:" + lastInodeId);
+          }
+        } else {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Old layout version doesn't have inode id."
+                + " Will assign new id for each inode.");
+          }
+        }
+        
         // read compression related info
         FSImageCompression compression;
         if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imgVersion)) {
@@ -216,8 +230,7 @@ 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,
@@ -264,8 +277,8 @@ class FSImageFormat {
    * @param in image input stream
    * @throws IOException
    */  
-   private void loadLocalNameINodes(long numFiles, DataInputStream in) 
-   throws IOException {
+   private void loadLocalNameINodes(long numFiles, DataInputStream in)
+        throws IOException {
      assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
          getLayoutVersion());
      assert numFiles > 0;
@@ -385,7 +398,8 @@ class FSImageFormat {
     long blockSize = 0;
     
     int imgVersion = getLayoutVersion();
-    long inodeId = namesystem.allocateNewInodeId();
+    long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? 
+           in.readLong() : namesystem.allocateNewInodeId();
     
     short replication = in.readShort();
     replication = namesystem.getBlockManager().adjustReplication(replication);
@@ -436,8 +450,8 @@ class FSImageFormat {
       LOG.info("Number of files under construction = " + size);
 
       for (int i = 0; i < size; i++) {
-        INodeFileUnderConstruction cons =
-          FSImageSerialization.readINodeUnderConstruction(in);
+        INodeFileUnderConstruction cons = FSImageSerialization
+            .readINodeUnderConstruction(in, namesystem, getLayoutVersion());
 
         // verify that file exists in namespace
         String path = cons.getLocalName();
@@ -566,7 +580,8 @@ class FSImageFormat {
         out.writeLong(fsDir.rootDir.numItemsInTree());
         out.writeLong(sourceNamesystem.getGenerationStamp());
         out.writeLong(context.getTxId());
-
+        out.writeLong(sourceNamesystem.getLastInodeId());
+        
         // write compression info and set up compressed stream
         out = compression.writeHeaderAndWrapStream(fos);
         LOG.info("Saving image file " + newFile +

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=1471596&r1=1471595&r2=1471596&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:42:14 2013
@@ -29,6 +29,8 @@ import org.apache.hadoop.fs.permission.P
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -79,8 +81,11 @@ public class FSImageSerialization {
   // from the input stream
   //
   static INodeFileUnderConstruction readINodeUnderConstruction(
-                            DataInputStream in) throws IOException {
+      DataInputStream in, FSNamesystem fsNamesys, int imgVersion)
+      throws IOException {
     byte[] name = readBytes(in);
+    long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? in
+        .readLong() : fsNamesys.allocateNewInodeId();
     short blockReplication = in.readShort();
     long modificationTime = in.readLong();
     long preferredBlockSize = in.readLong();
@@ -107,8 +112,7 @@ public class FSImageSerialization {
     int numLocs = in.readInt();
     assert numLocs == 0 : "Unexpected block locations";
 
-    //TODO: get inodeId from fsimage after inodeId is persisted
-    return new INodeFileUnderConstruction(INodeId.GRANDFATHER_INODE_ID,
+    return new INodeFileUnderConstruction(inodeId,
                                           name,
                                           blockReplication, 
                                           modificationTime,
@@ -128,6 +132,7 @@ public class FSImageSerialization {
                                            String path) 
                                            throws IOException {
     writeString(path, out);
+    out.writeLong(cons.getId());
     out.writeShort(cons.getBlockReplication());
     out.writeLong(cons.getModificationTime());
     out.writeLong(cons.getPreferredBlockSize());
@@ -151,6 +156,7 @@ public class FSImageSerialization {
     byte[] name = node.getLocalNameBytes();
     out.writeShort(name.length);
     out.write(name);
+    out.writeLong(node.getId());
     FsPermission filePerm = TL_DATA.get().FILE_PERM;
     if (node.isDirectory()) {
       out.writeShort(0);  // replication

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java?rev=1471596&r1=1471595&r2=1471596&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java Wed Apr 24 18:42:14 2013
@@ -123,7 +123,7 @@ class ImageLoaderCurrent implements Imag
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
-      -40};
+      -40, -41, -42};
   private int imageVersion = 0;
 
   /* (non-Javadoc)
@@ -163,6 +163,10 @@ class ImageLoaderCurrent implements Imag
         v.visit(ImageElement.TRANSACTION_ID, in.readLong());
       }
 
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
+        v.visit(ImageElement.LAST_INODE_ID, in.readLong());
+      }
+      
       if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imageVersion)) {
         boolean isCompressed = in.readBoolean();
         v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
@@ -440,6 +444,9 @@ class ImageLoaderCurrent implements Imag
     }
 
     v.visit(ImageElement.INODE_PATH, pathName);
+    if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
+      v.visit(ImageElement.INODE_ID, in.readLong());
+    }
     v.visit(ImageElement.REPLICATION, in.readShort());
     v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
     if(LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imageVersion))

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java?rev=1471596&r1=1471595&r2=1471596&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java Wed Apr 24 18:42:14 2013
@@ -80,7 +80,9 @@ abstract class ImageVisitor {
     DELEGATION_TOKEN_IDENTIFIER_MAX_DATE,
     DELEGATION_TOKEN_IDENTIFIER_EXPIRY_TIME,
     DELEGATION_TOKEN_IDENTIFIER_MASTER_KEY_ID,
-    TRANSACTION_ID
+    TRANSACTION_ID,
+    LAST_INODE_ID,
+    INODE_ID
   }
   
   /**

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=1471596&r1=1471595&r2=1471596&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:42:14 2013
@@ -217,7 +217,8 @@ public abstract class FSImageTestUtil {
         FsPermission.createImmutable((short)0755));
     for (int i = 1; i <= numDirs; i++) {
       String dirName = "dir" + i;
-      INodeDirectory dir = new INodeDirectory(newInodeId + i -1, 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/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=1471596&r1=1471595&r2=1471596&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:42:14 2013
@@ -26,12 +26,14 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 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.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.junit.Test;
@@ -334,6 +336,72 @@ public class TestINodeFile {
     Configuration conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
         DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT);
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+
+      FSNamesystem fsn = cluster.getNamesystem();
+      assertTrue(fsn.getLastInodeId() == 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);
+
+      int fileLen = 1024;
+      Path filePath = new Path("/test1/file");
+      DFSTestUtil.createFile(fs, filePath, fileLen, (short) 1, 0);
+      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();
+      fsn = cluster.getNamesystem();
+      assertTrue(fsn.getLastInodeId() == 1003);
+
+      DFSTestUtil.createFile(fs, new Path("/test2/file2"), fileLen, (short) 1,
+          0);
+      long id = fsn.getLastInodeId();
+      assertTrue(id == 1004);
+      fs.delete(new Path("/test2"), true);
+      // create a file under construction
+      FSDataOutputStream outStream = fs.create(new Path("/test3/file"));
+      assertTrue(outStream != null);
+      assertTrue(fsn.getLastInodeId() == 1006);
+
+      // Apply editlogs to fsimage, test fsimage with inodeUnderConstruction can
+      // be handled
+      fsn.enterSafeMode(false);
+      fsn.saveNamespace();
+      fsn.leaveSafeMode();
+
+      outStream.close();
+
+      // The lastInodeId in fsimage should remain 1006 after reboot
+      cluster.restartNameNode();
+      cluster.waitActive();
+      fsn = cluster.getNamesystem();
+      assertTrue(fsn.getLastInodeId() == 1006);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testWriteToRenamedFile() throws IOException {
+
+    Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
         .build();
     cluster.waitActive();

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored?rev=1471596&r1=1471595&r2=1471596&view=diff
==============================================================================
Binary files - no diff available.

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml?rev=1471596&r1=1471595&r2=1471596&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml Wed Apr 24 18:42:14 2013
@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <EDITS>
-  <EDITS_VERSION>-40</EDITS_VERSION>
+  <EDITS_VERSION>-42</EDITS_VERSION>
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
@@ -41,6 +41,7 @@
     <DATA>
       <TXID>5</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1002</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685834</MTIME>
@@ -60,6 +61,7 @@
     <DATA>
       <TXID>6</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685848</MTIME>
@@ -98,6 +100,7 @@
     <DATA>
       <TXID>9</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1003</INODEID>
       <PATH>/directory_mkdir</PATH>
       <TIMESTAMP>1330405685861</TIMESTAMP>
       <PERMISSION_STATUS>
@@ -119,6 +122,7 @@
     <DATA>
       <TXID>11</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1004</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685866</MTIME>
@@ -138,6 +142,7 @@
     <DATA>
       <TXID>12</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685868</MTIME>
@@ -218,6 +223,7 @@
     <DATA>
       <TXID>20</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1005</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685889</MTIME>
@@ -309,6 +315,7 @@
     <DATA>
       <TXID>27</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685978</MTIME>
@@ -350,6 +357,7 @@
     <DATA>
       <TXID>29</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1006</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685983</MTIME>
@@ -441,6 +449,7 @@
     <DATA>
       <TXID>36</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405686013</MTIME>
@@ -482,6 +491,7 @@
     <DATA>
       <TXID>38</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1007</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405686017</MTIME>
@@ -573,6 +583,7 @@
     <DATA>
       <TXID>45</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405686042</MTIME>
@@ -620,6 +631,7 @@
     <DATA>
       <TXID>47</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1008</INODEID>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
       <MTIME>1330405686051</MTIME>
@@ -693,6 +705,7 @@
     <DATA>
       <TXID>52</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1009</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405686084</MTIME>
@@ -759,6 +772,7 @@
     <DATA>
       <TXID>58</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405688726</MTIME>