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 18:38:53 UTC

svn commit: r1471522 - 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/

Author: suresh
Date: Wed Apr 24 16:38:53 2013
New Revision: 1471522

URL: http://svn.apache.org/r1471522
Log:
HDFS-4206. Merge change r1410996 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/server/namenode/FSDirectory.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/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/INodeSymlink.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.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=1471522&r1=1471521&r2=1471522&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 16:38:53 2013
@@ -62,6 +62,9 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4152. Add a new class BlocksMapUpdateInfo for the parameter in
     INode.collectSubtreeBlocksAndClear(..). (Jing Zhao via szetszwo)
 
+    HDFS-4206. Change the fields in INode and its subclasses to private.
+    (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=1471522&r1=1471521&r2=1471522&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 16:38:53 2013
@@ -305,14 +305,14 @@ public class FSDirectory implements Clos
     return newNode;
   }
 
-  INodeDirectory addToParent(byte[] src, INodeDirectory parentINode,
+  INodeDirectory addToParent(INodeDirectory parentINode,
       INode newNode, boolean propagateModTime) {
     // NOTE: This does not update space counts for parents
     INodeDirectory newParent = null;
     writeLock();
     try {
       try {
-        newParent = rootDir.addToParent(src, newNode, parentINode,
+        newParent = rootDir.addToParent(newNode, parentINode,
                                         propagateModTime);
         cacheName(newNode);
       } catch (FileNotFoundException e) {
@@ -538,7 +538,7 @@ public class FSDirectory implements Clos
       return true;
     }
     if (srcInode.isSymlink() && 
-        dst.equals(((INodeSymlink)srcInode).getLinkValue())) {
+        dst.equals(((INodeSymlink)srcInode).getSymlinkString())) {
       throw new FileAlreadyExistsException(
           "Cannot rename symlink "+src+" to its target "+dst);
     }
@@ -662,7 +662,7 @@ public class FSDirectory implements Clos
           "The source "+src+" and destination "+dst+" are the same");
     }
     if (srcInode.isSymlink() && 
-        dst.equals(((INodeSymlink)srcInode).getLinkValue())) {
+        dst.equals(((INodeSymlink)srcInode).getSymlinkString())) {
       throw new FileAlreadyExistsException(
           "Cannot rename symlink "+src+" to its target "+dst);
     }
@@ -701,14 +701,15 @@ public class FSDirectory implements Clos
             + error);
         throw new FileAlreadyExistsException(error);
       }
-      List<INode> children = dstInode.isDirectory() ? 
-          ((INodeDirectory) dstInode).getChildren() : null;
-      if (children != null && children.size() != 0) {
-        error = "rename cannot overwrite non empty destination directory "
-            + dst;
-        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
-            + error);
-        throw new IOException(error);
+      if (dstInode.isDirectory()) {
+        final List<INode> children = ((INodeDirectory) dstInode
+            ).getChildrenList();
+        if (!children.isEmpty()) {
+          error = "rename destination directory is not empty: " + dst;
+          NameNode.stateChangeLog.warn(
+              "DIR* FSDirectory.unprotectedRenameTo: " + error);
+          throw new IOException(error);
+        }
       }
     }
     if (dstInodes[dstInodes.length - 2] == null) {
@@ -1172,7 +1173,7 @@ public class FSDirectory implements Clos
       HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
       for (int i=0; i<numOfListing; i++) {
         INode cur = contents.get(startChild+i);
-        listing[i] = createFileStatus(cur.name, cur, needLocation);
+        listing[i] = createFileStatus(cur.getLocalNameBytes(), cur, needLocation);
       }
       return new DirectoryListing(
           listing, totalNumChildren-startChild-numOfListing);
@@ -1360,7 +1361,7 @@ public class FSDirectory implements Clos
     for(int i = 0; i < numOfINodes; i++) {
       if (inodes[i].isQuotaSet()) { // a directory with quota
         INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i]; 
-        node.updateNumItemsInTree(nsDelta, dsDelta);
+        node.addSpaceConsumed(nsDelta, dsDelta);
       }
     }
   }
@@ -1393,7 +1394,7 @@ public class FSDirectory implements Clos
     for(int i=0; i < numOfINodes; i++) {
       if (inodes[i].isQuotaSet()) { // a directory with quota
         INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i]; 
-        node.unprotectedUpdateNumItemsInTree(nsDelta, dsDelta);
+        node.addSpaceConsumed(nsDelta, dsDelta);
       }
     }
   }

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=1471522&r1=1471521&r2=1471522&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 16:38:53 2013
@@ -307,7 +307,8 @@ class FSImageFormat {
        INode newNode = loadINode(in); // read rest of inode
 
        // add to parent
-       namesystem.dir.addToParent(localName, parent, newNode, false);
+       newNode.setLocalName(localName);
+       namesystem.dir.addToParent(parent, newNode, false);
      }
      return numChildren;
    }
@@ -341,8 +342,8 @@ class FSImageFormat {
       }
 
       // add new inode
-      parentINode = fsDir.addToParent(pathComponents[pathComponents.length-1], 
-          parentINode, newNode, false);
+      newNode.setLocalName(pathComponents[pathComponents.length-1]);
+      parentINode = fsDir.addToParent(parentINode, newNode, false);
     }
   }
 
@@ -580,8 +581,8 @@ class FSImageFormat {
     private void saveImage(ByteBuffer currentDirName,
                                   INodeDirectory current,
                                   DataOutputStream out) throws IOException {
-      List<INode> children = current.getChildren();
-      if (children == null || children.isEmpty())
+      final List<INode> children = current.getChildrenList();
+      if (children.isEmpty())
         return;
       // print prefix (parent directory name)
       int prefixLen = currentDirName.position();

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=1471522&r1=1471521&r2=1471522&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 16:38:53 2013
@@ -168,7 +168,7 @@ public class FSImageSerialization {
       out.writeLong(0);   // access time
       out.writeLong(0);   // preferred block size
       out.writeInt(-2);   // # of blocks
-      Text.writeString(out, ((INodeSymlink)node).getLinkValue());
+      Text.writeString(out, ((INodeSymlink)node).getSymlinkString());
       filePerm.fromShort(node.getFsPermissionShort());
       PermissionStatus.write(out, node.getUserName(),
                              node.getGroupName(),

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=1471522&r1=1471521&r2=1471522&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 16:38:53 2013
@@ -45,23 +45,12 @@ import com.google.common.primitives.Sign
 @InterfaceAudience.Private
 abstract class INode implements Comparable<byte[]> {
   static final List<INode> EMPTY_LIST = Collections.unmodifiableList(new ArrayList<INode>());
-  /**
-   *  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
-   *  clientProtocol are changed; The decoding at the client
-   *  side should change accordingly.
-   */
-  protected byte[] name;
-  protected INodeDirectory parent;
-  protected long modificationTime;
-  protected long accessTime;
 
-  /** Simple wrapper for two counters : 
-   *  nsCount (namespace consumed) and dsCount (diskspace consumed).
-   */
+  /** Wrapper of two counters for namespace consumed and diskspace consumed. */
   static class DirCounts {
+    /** namespace count */
     long nsCount = 0;
+    /** diskspace count */
     long dsCount = 0;
     
     /** returns namespace count */
@@ -74,10 +63,6 @@ abstract class INode implements Comparab
     }
   }
   
-  //Only updated by updatePermissionStatus(...).
-  //Other codes should not modify it.
-  private long permission;
-
   private static enum PermissionStatusFormat {
     MODE(0, 16),
     GROUP(MODE.OFFSET + MODE.LENGTH, 25),
@@ -100,31 +85,67 @@ abstract class INode implements Comparab
     long combine(long bits, long record) {
       return (record & ~MASK) | (bits << OFFSET);
     }
+
+    /** Set the {@link PermissionStatus} */
+    static long toLong(PermissionStatus ps) {
+      long permission = 0L;
+      final int user = SerialNumberManager.INSTANCE.getUserSerialNumber(
+          ps.getUserName());
+      permission = PermissionStatusFormat.USER.combine(user, permission);
+      final int group = SerialNumberManager.INSTANCE.getGroupSerialNumber(
+          ps.getGroupName());
+      permission = PermissionStatusFormat.GROUP.combine(group, permission);
+      final int mode = ps.getPermission().toShort();
+      permission = PermissionStatusFormat.MODE.combine(mode, permission);
+      return permission;
+    }
   }
 
-  INode(PermissionStatus permissions, long mTime, long atime) {
-    this.name = null;
-    this.parent = null;
-    this.modificationTime = mTime;
-    setAccessTime(atime);
-    setPermissionStatus(permissions);
+  /**
+   *  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
+   *  clientProtocol are changed; The decoding at the client
+   *  side should change accordingly.
+   */
+  private byte[] name = null;
+  /** 
+   * Permission encoded using PermissionStatusFormat.
+   * Codes other than {@link #updatePermissionStatus(PermissionStatusFormat, long)}.
+   * should not modify it.
+   */
+  private long permission = 0L;
+  protected INodeDirectory parent = null;
+  protected long modificationTime = 0L;
+  protected long accessTime = 0L;
+
+  private INode(byte[] name, long permission, INodeDirectory parent,
+      long modificationTime, long accessTime) {
+    this.name = name;
+    this.permission = permission;
+    this.parent = parent;
+    this.modificationTime = modificationTime;
+    this.accessTime = accessTime;
+  }
+
+  INode(byte[] name, PermissionStatus permissions, INodeDirectory parent,
+      long modificationTime, long accessTime) {
+    this(name, PermissionStatusFormat.toLong(permissions), parent,
+        modificationTime, accessTime);
+  }
+
+  INode(PermissionStatus permissions, long mtime, long atime) {
+    this(null, permissions, null, mtime, atime);
   }
 
   protected INode(String name, PermissionStatus permissions) {
-    this(permissions, 0L, 0L);
-    setLocalName(name);
+    this(DFSUtil.string2Bytes(name), permissions, null, 0L, 0L);
   }
   
-  /** copy constructor
-   * 
-   * @param other Other node to be copied
-   */
+  /** @param other Other node to be copied */
   INode(INode other) {
-    setLocalName(other.getLocalName());
-    this.parent = other.getParent();
-    setPermissionStatus(other.getPermissionStatus());
-    setModificationTime(other.getModificationTime());
-    setAccessTime(other.getAccessTime());
+    this(other.getLocalNameBytes(), other.permission, other.getParent(), 
+        other.getModificationTime(), other.getAccessTime());
   }
 
   /**

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=1471522&r1=1471521&r2=1471522&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 16:38:53 2013
@@ -50,22 +50,19 @@ class INodeDirectory extends INode {
   protected static final int DEFAULT_FILES_PER_DIRECTORY = 5;
   final static String ROOT_NAME = "";
 
-  private List<INode> children;
+  private List<INode> children = null;
 
   INodeDirectory(String name, PermissionStatus permissions) {
     super(name, permissions);
-    this.children = null;
   }
 
   public INodeDirectory(PermissionStatus permissions, long mTime) {
     super(permissions, mTime, 0);
-    this.children = null;
   }
 
   /** constructor */
-  INodeDirectory(byte[] localName, PermissionStatus permissions, long mTime) {
-    this(permissions, mTime);
-    this.name = localName;
+  INodeDirectory(byte[] name, PermissionStatus permissions, long mtime) {
+    super(name, permissions, null, mtime, 0L);
   }
   
   /** copy constructor
@@ -74,7 +71,7 @@ class INodeDirectory extends INode {
    */
   INodeDirectory(INodeDirectory other) {
     super(other);
-    this.children = other.getChildren();
+    this.children = other.children;
   }
   
   /** @return true unconditionally. */
@@ -83,25 +80,30 @@ class INodeDirectory extends INode {
     return true;
   }
 
-  INode removeChild(INode node) {
-    assert children != null;
-    int low = Collections.binarySearch(children, node.name);
-    if (low >= 0) {
-      return children.remove(low);
-    } else {
-      return null;
+  private void assertChildrenNonNull() {
+    if (children == null) {
+      throw new AssertionError("children is null: " + this);
     }
   }
 
+  private int searchChildren(INode inode) {
+    return Collections.binarySearch(children, inode.getLocalNameBytes());
+  }
+
+  INode removeChild(INode node) {
+    assertChildrenNonNull();
+    final int i = searchChildren(node);
+    return i >= 0? children.remove(i): null;
+  }
+
   /** Replace a child that has the same name as newChild by newChild.
    * 
    * @param newChild Child node to be added
    */
   void replaceChild(INode newChild) {
-    if ( children == null ) {
-      throw new IllegalArgumentException("The directory is empty");
-    }
-    int low = Collections.binarySearch(children, newChild.name);
+    assertChildrenNonNull();
+
+    final int low = searchChildren(newChild);
     if (low>=0) { // an old child exists so replace by the newChild
       children.set(low, newChild);
     } else {
@@ -210,7 +212,7 @@ class INodeDirectory extends INode {
         final String remainder =
           constructPath(components, count + 1, components.length);
         final String link = DFSUtil.bytes2String(components[count]);
-        final String target = ((INodeSymlink)curNode).getLinkValue();
+        final String target = ((INodeSymlink)curNode).getSymlinkString();
         if (NameNode.stateChangeLog.isDebugEnabled()) {
           NameNode.stateChangeLog.debug("UnresolvedPathException " +
             " path: " + path + " preceding: " + preceding +
@@ -284,7 +286,7 @@ class INodeDirectory extends INode {
     if (children == null) {
       children = new ArrayList<INode>(DEFAULT_FILES_PER_DIRECTORY);
     }
-    int low = Collections.binarySearch(children, node.name);
+    final int low = searchChildren(node);
     if(low >= 0)
       return null;
     node.parent = this;
@@ -324,13 +326,9 @@ class INodeDirectory extends INode {
    * @throws  FileNotFoundException if parent does not exist or 
    *          is not a directory.
    */
-  INodeDirectory addToParent( byte[] localname,
-                              INode newNode,
-                              INodeDirectory parent,
-                              boolean propagateModTime
-                              ) throws FileNotFoundException {
+  INodeDirectory addToParent(INode newNode, INodeDirectory parent,
+      boolean propagateModTime) throws FileNotFoundException {
     // insert into the parent children list
-    newNode.name = localname;
     if(parent.addChild(newNode, propagateModTime) == null)
       return null;
     return parent;
@@ -368,7 +366,7 @@ class INodeDirectory extends INode {
     if (pathComponents.length < 2) { // add root
       return null;
     }
-    newNode.name = pathComponents[pathComponents.length - 1];
+    newNode.setLocalName(pathComponents[pathComponents.length - 1]);
     // insert into the parent children list
     INodeDirectory parent = getParent(pathComponents);
     return parent.addChild(newNode, propagateModTime) == null? null: parent;
@@ -424,10 +422,6 @@ class INodeDirectory extends INode {
   public List<INode> getChildrenList() {
     return children==null ? EMPTY_LIST : children;
   }
-  /** @return the children list which is possibly null. */
-  public List<INode> getChildren() {
-    return children;
-  }
 
   @Override
   int collectSubtreeBlocksAndClear(BlocksMapUpdateInfo info) {

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=1471522&r1=1471521&r2=1471522&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 16:38:53 2013
@@ -27,9 +27,9 @@ import org.apache.hadoop.hdfs.protocol.Q
  */
 class INodeDirectoryWithQuota extends INodeDirectory {
   private long nsQuota; /// NameSpace quota
-  private long nsCount;
+  private long nsCount = 1L;
   private long dsQuota; /// disk space quota
-  private long diskspace;
+  private long diskspace = 0L;
   
   /** Convert an existing directory inode to one with the given quota
    * 
@@ -44,7 +44,8 @@ class INodeDirectoryWithQuota extends IN
     other.spaceConsumedInTree(counts);
     this.nsCount = counts.getNsCount();
     this.diskspace = counts.getDsCount();
-    setQuota(nsQuota, dsQuota);
+    this.nsQuota = nsQuota;
+    this.dsQuota = dsQuota;
   }
   
   /** constructor with no quota verification */
@@ -53,7 +54,6 @@ class INodeDirectoryWithQuota extends IN
     super(permissions, modificationTime);
     this.nsQuota = nsQuota;
     this.dsQuota = dsQuota;
-    this.nsCount = 1;
   }
   
   /** constructor with no quota verification */
@@ -62,7 +62,6 @@ class INodeDirectoryWithQuota extends IN
     super(name, permissions);
     this.nsQuota = nsQuota;
     this.dsQuota = dsQuota;
-    this.nsCount = 1;
   }
   
   /** Get this directory's namespace quota
@@ -116,19 +115,8 @@ class INodeDirectoryWithQuota extends IN
    * @param nsDelta the change of the tree size
    * @param dsDelta change to disk space occupied
    */
-  void updateNumItemsInTree(long nsDelta, long dsDelta) {
-    nsCount += nsDelta;
-    diskspace += dsDelta;
-  }
-  
-  /** Update the size of the tree
-   * 
-   * @param nsDelta the change of the tree size
-   * @param dsDelta change to disk space occupied
-   **/
-  void unprotectedUpdateNumItemsInTree(long nsDelta, long dsDelta) {
-    nsCount = nsCount + nsDelta;
-    diskspace = diskspace + dsDelta;
+  void addSpaceConsumed(long nsDelta, long dsDelta) {
+    setSpaceConsumed(nsCount + nsDelta, diskspace + dsDelta);
   }
   
   /** 

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=1471522&r1=1471521&r2=1471522&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 16:38:53 2013
@@ -45,14 +45,43 @@ class INodeFile extends INode implements
 
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
 
-  //Number of bits for Block size
-  static final short BLOCKBITS = 48;
 
-  //Header mask 64-bit representation
-  //Format: [16 bits for replication][48 bits for PreferredBlockSize]
-  static final long HEADERMASK = 0xffffL << BLOCKBITS;
+  /** Format: [16 bits for replication][48 bits for PreferredBlockSize] */
+  private static class HeaderFormat {
+    /** Number of bits for Block size */
+    static final int BLOCKBITS = 48;
+    /** Header mask 64-bit representation */
+    static final long HEADERMASK = 0xffffL << BLOCKBITS;
+    static final long MAX_BLOCK_SIZE = ~HEADERMASK; 
+    
+    static short getReplication(long header) {
+      return (short) ((header & HEADERMASK) >> BLOCKBITS);
+    }
+
+    static long combineReplication(long header, short replication) {
+      if (replication <= 0) {
+         throw new IllegalArgumentException(
+             "Unexpected value for the replication: " + replication);
+      }
+      return ((long)replication << BLOCKBITS) | (header & MAX_BLOCK_SIZE);
+    }
+    
+    static long getPreferredBlockSize(long header) {
+      return header & MAX_BLOCK_SIZE;
+    }
+
+    static long combinePreferredBlockSize(long header, long blockSize) {
+      if (blockSize < 0) {
+         throw new IllegalArgumentException("Block size < 0: " + blockSize);
+      } else if (blockSize > MAX_BLOCK_SIZE) {
+        throw new IllegalArgumentException("Block size = " + blockSize
+            + " > MAX_BLOCK_SIZE = " + MAX_BLOCK_SIZE);
+     }
+      return (header & HEADERMASK) | (blockSize & MAX_BLOCK_SIZE);
+    }
+  }
 
-  private long header;
+  private long header = 0L;
 
   private BlockInfo[] blocks;
 
@@ -60,8 +89,8 @@ class INodeFile extends INode implements
                       short replication, long modificationTime,
                       long atime, long preferredBlockSize) {
     super(permissions, modificationTime, atime);
-    this.setReplication(replication);
-    this.setPreferredBlockSize(preferredBlockSize);
+    header = HeaderFormat.combineReplication(header, replication);
+    header = HeaderFormat.combinePreferredBlockSize(header, preferredBlockSize);
     this.blocks = blklist;
   }
 
@@ -78,25 +107,17 @@ class INodeFile extends INode implements
   /** @return the replication factor of the file. */
   @Override
   public short getBlockReplication() {
-    return (short) ((header & HEADERMASK) >> BLOCKBITS);
+    return HeaderFormat.getReplication(header);
   }
 
   void setReplication(short replication) {
-    if(replication <= 0)
-       throw new IllegalArgumentException("Unexpected value for the replication");
-    header = ((long)replication << BLOCKBITS) | (header & ~HEADERMASK);
+    header = HeaderFormat.combineReplication(header, replication);
   }
 
   /** @return preferred block size (in bytes) of the file. */
   @Override
   public long getPreferredBlockSize() {
-    return header & ~HEADERMASK;
-  }
-
-  private void setPreferredBlockSize(long preferredBlkSize) {
-    if((preferredBlkSize < 0) || (preferredBlkSize > ~HEADERMASK ))
-       throw new IllegalArgumentException("Unexpected value for the block size");
-    header = (header & HEADERMASK) | (preferredBlkSize & ~HEADERMASK);
+    return HeaderFormat.getPreferredBlockSize(header);
   }
 
   /** @return the blocks of the file. */

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=1471522&r1=1471521&r2=1471522&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 16:38:53 2013
@@ -22,31 +22,24 @@ import org.apache.hadoop.fs.permission.P
 import org.apache.hadoop.hdfs.DFSUtil;
 
 /**
- * An INode representing a symbolic link.
+ * An {@link INode} representing a symbolic link.
  */
 @InterfaceAudience.Private
 public class INodeSymlink extends INode {
-  private byte[] symlink; // The target URI
+  private final byte[] symlink; // The target URI
 
-  INodeSymlink(String value, long modTime, long atime,
+  INodeSymlink(String value, long mtime, long atime,
                PermissionStatus permissions) {
-    super(permissions, modTime, atime);
-    assert value != null;
-    setLinkValue(value);
-    setModificationTimeForce(modTime);
-    setAccessTime(atime);
+    super(permissions, mtime, atime);
+    this.symlink = DFSUtil.string2Bytes(value);
   }
 
   @Override
   public boolean isSymlink() {
     return true;
   }
-  
-  void setLinkValue(String value) {
-    this.symlink = DFSUtil.string2Bytes(value);
-  }
 
-  public String getLinkValue() {
+  public String getSymlinkString() {
     return DFSUtil.bytes2String(symlink);
   }
 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.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/TestFSDirectory.java?rev=1471522&r1=1471521&r2=1471522&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java Wed Apr 24 16:38:53 2013
@@ -130,6 +130,16 @@ public class TestFSDirectory {
     Assert.assertTrue(diff.contains(file4.getName()));
   }
   
+  @Test
+  public void testReset() throws Exception {
+    fsdir.reset();
+    Assert.assertFalse(fsdir.isReady());
+    final INodeDirectory root = (INodeDirectory) fsdir.getINode("/");
+    Assert.assertTrue(root.getChildrenList().isEmpty());
+    fsdir.imageLoadComplete();
+    Assert.assertTrue(fsdir.isReady());
+  }
+  
   static void checkClassName(String line) {
     int i = line.lastIndexOf('(');
     int j = line.lastIndexOf('@');