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 br...@apache.org on 2013/06/21 20:34:06 UTC

svn commit: r1495535 - 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/protocolPB/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/mai...

Author: brandonli
Date: Fri Jun 21 18:34:05 2013
New Revision: 1495535

URL: http://svn.apache.org/r1495535
Log:
HDFS-4772. Merging change r1495253 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/HdfsFileStatus.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
    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/INodeDirectory.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.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=1495535&r1=1495534&r2=1495535&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 Fri Jun 21 18:34:05 2013
@@ -8,6 +8,8 @@ Release 2.3.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    HDFS-4772. Add number of children in HdfsFileStatus. (brandonli)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.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/HdfsFileStatus.java?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java Fri Jun 21 18:34:05 2013
@@ -42,6 +42,9 @@ public class HdfsFileStatus {
   private String group;
   private long fileId;
   
+  // Used by dir, not including dot and dotdot. Always zero for a regular file.
+  private int childrenNum;
+  
   public static final byte[] EMPTY_NAME = new byte[0];
 
   /**
@@ -61,7 +64,7 @@ public class HdfsFileStatus {
   public HdfsFileStatus(long length, boolean isdir, int block_replication,
                     long blocksize, long modification_time, long access_time,
                     FsPermission permission, String owner, String group, 
-                    byte[] symlink, byte[] path, long fileId) {
+                    byte[] symlink, byte[] path, long fileId, int childrenNum) {
     this.length = length;
     this.isdir = isdir;
     this.block_replication = (short)block_replication;
@@ -78,6 +81,7 @@ public class HdfsFileStatus {
     this.symlink = symlink;
     this.path = path;
     this.fileId = fileId;
+    this.childrenNum = childrenNum;
   }
 
   /**
@@ -230,4 +234,8 @@ public class HdfsFileStatus {
   final public long getFileId() {
     return fileId;
   }
+  
+  final public int getChildrenNum() {
+    return childrenNum;
+  }
 }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.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/HdfsLocatedFileStatus.java?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java Fri Jun 21 18:34:05 2013
@@ -50,9 +50,11 @@ public class HdfsLocatedFileStatus exten
   public HdfsLocatedFileStatus(long length, boolean isdir,
       int block_replication, long blocksize, long modification_time,
       long access_time, FsPermission permission, String owner, String group,
-      byte[] symlink, byte[] path, long fileId, LocatedBlocks locations) {
+      byte[] symlink, byte[] path, long fileId, LocatedBlocks locations,
+      int childrenNum) {
     super(length, isdir, block_replication, blocksize, modification_time,
-        access_time, permission, owner, group, symlink, path, fileId);
+        access_time, permission, owner, group, symlink, path, fileId,
+        childrenNum);
     this.locations = locations;
   }
 	

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.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/SnapshottableDirectoryStatus.java?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java Fri Jun 21 18:34:05 2013
@@ -57,10 +57,11 @@ public class SnapshottableDirectoryStatu
   
   public SnapshottableDirectoryStatus(long modification_time, long access_time,
       FsPermission permission, String owner, String group, byte[] localName,
-      long inodeId,
+      long inodeId, int childrenNum,
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
-        access_time, permission, owner, group, null, localName, inodeId);
+        access_time, permission, owner, group, null, localName, inodeId,
+        childrenNum);
     this.snapshotNumber = snapshotNumber;
     this.snapshotQuota = snapshotQuota;
     this.parentFullPath = parentFullPath;

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java Fri Jun 21 18:34:05 2013
@@ -1055,7 +1055,8 @@ public class PBHelper {
             fs.getSymlink().toByteArray() : null,
         fs.getPath().toByteArray(),
         fs.hasFileId()? fs.getFileId(): INodeId.GRANDFATHER_INODE_ID,
-        fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null);
+        fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null,
+        fs.hasChildrenNum() ? fs.getChildrenNum() : 0);
   }
 
   public static SnapshottableDirectoryStatus convert(
@@ -1072,6 +1073,7 @@ public class PBHelper {
         status.getGroup(),
         status.getPath().toByteArray(),
         status.getFileId(),
+        status.getChildrenNum(),
         sdirStatusProto.getSnapshotNumber(),
         sdirStatusProto.getSnapshotQuota(),
         sdirStatusProto.getParentFullpath().toByteArray());
@@ -1099,6 +1101,7 @@ public class PBHelper {
       setOwner(fs.getOwner()).
       setGroup(fs.getGroup()).
       setFileId(fs.getFileId()).
+      setChildrenNum(fs.getChildrenNum()).
       setPath(ByteString.copyFrom(fs.getLocalNameInBytes()));
     if (fs.isSymlink())  {
       builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));

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=1495535&r1=1495534&r2=1495535&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 Fri Jun 21 18:34:05 2013
@@ -1582,6 +1582,13 @@ public class FSDirectory implements Clos
     }
   }
   
+  /**
+   * Currently we only support "ls /xxx/.snapshot" which will return all the
+   * snapshots of a directory. The FSCommand Ls will first call getFileInfo to
+   * make sure the file/directory exists (before the real getListing call).
+   * Since we do not have a real INode for ".snapshot", we return an empty
+   * non-null HdfsFileStatus here.
+   */
   private HdfsFileStatus getFileInfo4DotSnapshot(String src)
       throws UnresolvedLinkException {
     Preconditions.checkArgument(
@@ -1596,7 +1603,7 @@ public class FSDirectory implements Clos
         && node.isDirectory()
         && node.asDirectory() instanceof INodeDirectorySnapshottable) {
       return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
-          HdfsFileStatus.EMPTY_NAME, -1L);
+          HdfsFileStatus.EMPTY_NAME, -1L, 0);
     }
     return null;
   }
@@ -2522,6 +2529,9 @@ public class FSDirectory implements Clos
        replication = fileNode.getFileReplication(snapshot);
        blocksize = fileNode.getPreferredBlockSize();
      }
+     int childrenNum = node.isDirectory() ? 
+         node.asDirectory().getChildrenNum(snapshot) : 0;
+         
      return new HdfsFileStatus(
         size, 
         node.isDirectory(), 
@@ -2534,7 +2544,8 @@ public class FSDirectory implements Clos
         node.getGroupName(snapshot),
         node.isSymlink() ? node.asSymlink().getSymlink() : null,
         path,
-        node.getId());
+        node.getId(),
+        childrenNum);
   }
 
   /**
@@ -2564,12 +2575,15 @@ public class FSDirectory implements Clos
         loc = new LocatedBlocks();
       }
     }
+    int childrenNum = node.isDirectory() ? 
+        node.asDirectory().getChildrenNum(snapshot) : 0;
+        
     return new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
         blocksize, node.getModificationTime(snapshot),
         node.getAccessTime(snapshot), node.getFsPermission(snapshot),
         node.getUserName(snapshot), node.getGroupName(snapshot),
         node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
-        node.getId(), loc);
+        node.getId(), loc, childrenNum);
   }
 
     

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=1495535&r1=1495534&r2=1495535&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 Fri Jun 21 18:34:05 2013
@@ -655,4 +655,8 @@ public class INodeDirectory extends INod
       this(snapshot, snapshot.getRoot());
     }
   }
+
+  public final int getChildrenNum(final Snapshot snapshot) {
+    return getChildrenList(snapshot).size();
+  }
 }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.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/INodeWithAdditionalFields.java?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java Fri Jun 21 18:34:05 2013
@@ -130,6 +130,7 @@ public abstract class INodeWithAdditiona
   }
 
   /** Get inode id */
+  @Override
   public final long getId() {
     return this.id;
   }
@@ -226,6 +227,7 @@ public abstract class INodeWithAdditiona
 
 
   /** Update modification time if it is larger than the current value. */
+  @Override
   public final INode updateModificationTime(long mtime, Snapshot latest,
       final INodeMap inodeMap) throws QuotaExceededException {
     Preconditions.checkState(isDirectory());
@@ -256,6 +258,7 @@ public abstract class INodeWithAdditiona
   /**
    * Set last access time of inode.
    */
+  @Override
   public final void setAccessTime(long accessTime) {
     this.accessTime = accessTime;
   }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.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/snapshot/SnapshotManager.java?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java Fri Jun 21 18:34:05 2013
@@ -325,7 +325,8 @@ public class SnapshotManager implements 
         SnapshottableDirectoryStatus status = new SnapshottableDirectoryStatus(
             dir.getModificationTime(), dir.getAccessTime(),
             dir.getFsPermission(), dir.getUserName(), dir.getGroupName(),
-            dir.getLocalNameBytes(), dir.getId(), dir.getNumSnapshots(),
+            dir.getLocalNameBytes(), dir.getId(), dir.getChildrenNum(null),
+            dir.getNumSnapshots(),
             dir.getSnapshotQuota(), dir.getParent() == null ? 
                 DFSUtil.EMPTY_BYTES : 
                 DFSUtil.string2Bytes(dir.getParent().getFullPathName()));

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java Fri Jun 21 18:34:05 2013
@@ -221,6 +221,7 @@ public class JsonUtil {
     m.put("blockSize", status.getBlockSize());
     m.put("replication", status.getReplication());
     m.put("fileId", status.getFileId());
+    m.put("childrenNum", status.getChildrenNum());
     return includeType ? toJsonString(FileStatus.class, m): JSON.toString(m);
   }
 
@@ -247,9 +248,10 @@ public class JsonUtil {
     final short replication = (short) (long) (Long) m.get("replication");
     final long fileId = m.containsKey("fileId") ? (Long) m.get("fileId")
         : INodeId.GRANDFATHER_INODE_ID;
+    final int childrenNum = (int) (long) (Long) m.get("childrenNum");
     return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
         blockSize, mTime, aTime, permission, owner, group,
-        symlink, DFSUtil.string2Bytes(localName), fileId);
+        symlink, DFSUtil.string2Bytes(localName), fileId, childrenNum);
   }
 
   /** Convert an ExtendedBlock to a Json map. */

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto Fri Jun 21 18:34:05 2013
@@ -173,6 +173,7 @@ message HdfsFileStatusProto {
 
   // Optional field for fileId
   optional uint64 fileId = 13 [default = 0]; // default as an invalid id
+  optional uint32 childrenNum = 14 [default = 0];
 } 
 
 /**

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java Fri Jun 21 18:34:05 2013
@@ -252,12 +252,12 @@ public class TestDFSClientRetries {
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010)).when(mockNN).getFileInfo(anyString());
+                1010, 0)).when(mockNN).getFileInfo(anyString());
     
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010))
+                1010, 0))
         .when(mockNN)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java Fri Jun 21 18:34:05 2013
@@ -116,6 +116,16 @@ public class TestFileStatus {
     // Make sure getFileInfo returns null for files which do not exist
     HdfsFileStatus fileInfo = dfsClient.getFileInfo("/noSuchFile");
     assertEquals("Non-existant file should result in null", null, fileInfo);
+    
+    Path path1 = new Path("/name1");
+    Path path2 = new Path("/name1/name2");
+    assertTrue(fs.mkdirs(path1));
+    FSDataOutputStream out = fs.create(path2, false);
+    out.close();
+    fileInfo = dfsClient.getFileInfo(path1.toString());
+    assertEquals(1, fileInfo.getChildrenNum());
+    fileInfo = dfsClient.getFileInfo(path2.toString());
+    assertEquals(0, fileInfo.getChildrenNum());
 
     // Test getFileInfo throws the right exception given a non-absolute path.
     try {

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java Fri Jun 21 18:34:05 2013
@@ -293,12 +293,12 @@ public class TestLease {
     Mockito.doReturn(
         new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
             (short) 777), "owner", "group", new byte[0], new byte[0],
-            1010)).when(mcp).getFileInfo(anyString());
+            1010, 0)).when(mcp).getFileInfo(anyString());
     Mockito
         .doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010))
+                1010, 0))
         .when(mcp)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java?rev=1495535&r1=1495534&r2=1495535&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java Fri Jun 21 18:34:05 2013
@@ -46,7 +46,7 @@ public class TestJsonUtil {
     final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
         now, now + 10, new FsPermission((short) 0644), "user", "group",
         DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
-        INodeId.GRANDFATHER_INODE_ID);
+        INodeId.GRANDFATHER_INODE_ID, 0);
     final FileStatus fstatus = toFileStatus(status, parent);
     System.out.println("status  = " + status);
     System.out.println("fstatus = " + fstatus);