瀏覽代碼

HDFS-6921. Add LazyPersist flag to FileStatus. (Arpit Agarwal)

arp 10 年之前
父節點
當前提交
a7bcc95358
共有 19 個文件被更改,包括 103 次插入35 次删除
  1. 15 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
  2. 21 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  3. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
  4. 5 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  5. 1 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
  6. 4 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-6581.txt
  7. 14 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
  8. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
  10. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  11. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  12. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  13. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  14. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  15. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  16. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  17. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
  18. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  19. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java

+ 15 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java

@@ -79,7 +79,21 @@ public enum CreateFlag {
   /**
    * Force closed blocks to disk. Similar to POSIX O_SYNC. See javadoc for description.
    */
-  SYNC_BLOCK((short) 0x08);
+  SYNC_BLOCK((short) 0x08),
+
+  /**
+   * Create the block on transient storage (RAM) if available. If
+   * transient storage is unavailable then the block will be created
+   * on disk.
+   *
+   * HDFS will make a best effort to lazily write these files to persistent
+   * storage, however file contents may be lost at any time due to process/
+   * node restarts, hence there is no guarantee of data durability.
+   *
+   * This flag must only be used for intermediate data whose loss can be
+   * tolerated by the application.
+   */
+  LAZY_PERSIST((short) 0x10);
 
   private final short mode;
 

+ 21 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java

@@ -38,6 +38,7 @@ public class FileStatus implements Writable, Comparable {
   private boolean isdir;
   private short block_replication;
   private long blocksize;
+  private boolean isLazyPersist;
   private long modification_time;
   private long access_time;
   private FsPermission permission;
@@ -73,6 +74,18 @@ public class FileStatus implements Writable, Comparable {
                     FsPermission permission, String owner, String group, 
                     Path symlink,
                     Path path) {
+    this(length, isdir, block_replication, blocksize, false,
+        modification_time, access_time, permission, owner, group,
+        symlink, path);
+  }
+
+  public FileStatus(long length, boolean isdir,
+                    int block_replication,
+                    long blocksize, boolean isLazyPersist,
+                    long modification_time, long access_time,
+                    FsPermission permission, String owner, String group,
+                    Path symlink,
+                    Path path) {
     this.length = length;
     this.isdir = isdir;
     this.block_replication = (short)block_replication;
@@ -92,6 +105,7 @@ public class FileStatus implements Writable, Comparable {
     this.group = (group == null) ? "" : group;
     this.symlink = symlink;
     this.path = path;
+    this.isLazyPersist = isLazyPersist;
     // The variables isdir and symlink indicate the type:
     // 1. isdir implies directory, in which case symlink must be null.
     // 2. !isdir implies a file or symlink, symlink != null implies a
@@ -167,6 +181,13 @@ public class FileStatus implements Writable, Comparable {
     return blocksize;
   }
 
+  /**
+   * Get whether the file is lazyPersist.
+   */
+  public boolean isLazyPersist() {
+    return isLazyPersist;
+  }
+
   /**
    * Get the replication factor of a file.
    * @return the replication factor of a file.

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -762,6 +762,7 @@ public class RawLocalFileSystem extends FileSystem {
           false,
           fs.getReplication(),
           fs.getBlockSize(),
+          fs.isLazyPersist(),
           fs.getModificationTime(),
           fs.getAccessTime(),
           fs.getPermission(),
@@ -777,7 +778,7 @@ public class RawLocalFileSystem extends FileSystem {
        * when available.
        */
       if (!target.isEmpty()) {
-        return new FileStatus(0, false, 0, 0, 0, 0, FsPermission.getDefault(),
+        return new FileStatus(0, false, 0, 0, false, 0, 0, FsPermission.getDefault(),
             "", "", new Path(target), f);
       }
       // f refers to a file or directory that does not exist

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java

@@ -160,6 +160,7 @@ public class HttpFSFileSystem extends FileSystem
   public static final String XATTR_NAME_JSON = "name";
   public static final String XATTR_VALUE_JSON = "value";
   public static final String XATTRNAMES_JSON = "XAttrNames";
+  public static final String LAZY_PERSIST_JSON = "LazyPersist";
 
   public static final String FILE_CHECKSUM_JSON = "FileChecksum";
   public static final String CHECKSUM_ALGORITHM_JSON = "algorithm";
@@ -954,19 +955,20 @@ public class HttpFSFileSystem extends FileSystem
     long mTime = (Long) json.get(MODIFICATION_TIME_JSON);
     long blockSize = (Long) json.get(BLOCK_SIZE_JSON);
     short replication = ((Long) json.get(REPLICATION_JSON)).shortValue();
+    boolean isLazyPersist = ((Boolean) json.get(LAZY_PERSIST_JSON)).booleanValue();
     FileStatus fileStatus = null;
 
     switch (type) {
       case FILE:
       case DIRECTORY:
         fileStatus = new FileStatus(len, (type == FILE_TYPE.DIRECTORY),
-                                    replication, blockSize, mTime, aTime,
-                                    permission, owner, group, path);
+                                    replication, blockSize, false, mTime, aTime,
+                                    permission, owner, group, null, path);
         break;
       case SYMLINK:
         Path symLink = null;
         fileStatus = new FileStatus(len, false,
-                                    replication, blockSize, mTime, aTime,
+                                    replication, blockSize, isLazyPersist, mTime, aTime,
                                     permission, owner, group, symLink,
                                     path);
     }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java

@@ -125,6 +125,7 @@ public class FSOperations {
               fileStatus.getModificationTime());
       json.put(HttpFSFileSystem.BLOCK_SIZE_JSON, fileStatus.getBlockSize());
       json.put(HttpFSFileSystem.REPLICATION_JSON, fileStatus.getReplication());
+      json.put(HttpFSFileSystem.LAZY_PERSIST_JSON, fileStatus.isLazyPersist());
       if ( (aclStatus != null) && !(aclStatus.getEntries().isEmpty()) ) {
         json.put(HttpFSFileSystem.ACL_BIT_JSON,true);
       }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-6581.txt

@@ -0,0 +1,4 @@
+  BREAKDOWN OF HDFS-6581 SUBTASKS AND RELATED JIRAS
+
+    HDFS-6921. Add LazyPersist flag to FileStatus. (Arpit Agarwal)
+

+ 14 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java

@@ -39,6 +39,7 @@ public class HdfsFileStatus {
   private final boolean isdir;
   private final short block_replication;
   private final long blocksize;
+  private final boolean isLazyPersist;
   private final long modification_time;
   private final long access_time;
   private final FsPermission permission;
@@ -69,13 +70,15 @@ public class HdfsFileStatus {
    * @param feInfo the file's encryption info
    */
   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, int childrenNum, FileEncryptionInfo feInfo) {
+      long blocksize, boolean isLazyPersist, long modification_time,
+      long access_time, FsPermission permission, String owner,
+      String group, byte[] symlink, byte[] path, long fileId,
+      int childrenNum, FileEncryptionInfo feInfo) {
     this.length = length;
     this.isdir = isdir;
     this.block_replication = (short)block_replication;
     this.blocksize = blocksize;
+    this.isLazyPersist = isLazyPersist;
     this.modification_time = modification_time;
     this.access_time = access_time;
     this.permission = (permission == null) ? 
@@ -124,6 +127,13 @@ public class HdfsFileStatus {
     return blocksize;
   }
 
+  /**
+   * @return true if the file is lazyPersist.
+   */
+  final public boolean isLazyPersist() {
+    return isLazyPersist;
+  }
+
   /**
    * Get the replication factor of a file.
    * @return the replication factor of a file.
@@ -253,7 +263,7 @@ public class HdfsFileStatus {
 
   public final FileStatus makeQualified(URI defaultUri, Path path) {
     return new FileStatus(getLen(), isDir(), getReplication(),
-        getBlockSize(), getModificationTime(),
+        getBlockSize(), isLazyPersist(), getModificationTime(),
         getAccessTime(),
         getPermission(), getOwner(), getGroup(),
         isSymlink() ? new Path(getSymlink()) : null,

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java

@@ -55,13 +55,13 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
    * @param feInfo file encryption info
    */
   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,
-    int childrenNum, FileEncryptionInfo feInfo) {
-    super(length, isdir, block_replication, blocksize, modification_time,
-      access_time, permission, owner, group, symlink, path, fileId,
-      childrenNum, feInfo);
+      int block_replication, long blocksize, boolean isLazyPersist,
+      long modification_time, long access_time, FsPermission permission,
+      String owner, String group, byte[] symlink, byte[] path, long fileId,
+      LocatedBlocks locations, int childrenNum, FileEncryptionInfo feInfo) {
+    super(length, isdir, block_replication, blocksize, isLazyPersist,
+        modification_time, access_time, permission, owner, group, symlink,
+        path, fileId, childrenNum, feInfo);
     this.locations = locations;
   }
 	

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java

@@ -59,7 +59,7 @@ public class SnapshottableDirectoryStatus {
       FsPermission permission, String owner, String group, byte[] localName,
       long inodeId, int childrenNum,
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
-    this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
+    this.dirStatus = new HdfsFileStatus(0, true, 0, 0, false, modification_time,
         access_time, permission, owner, group, null, localName, inodeId,
         childrenNum, null);
     this.snapshotNumber = snapshotNumber;

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -1277,6 +1277,9 @@ public class PBHelper {
     if (flag.contains(CreateFlag.OVERWRITE)) {
       value |= CreateFlagProto.OVERWRITE.getNumber();
     }
+    if (flag.contains(CreateFlag.LAZY_PERSIST)) {
+      value |= CreateFlagProto.LAZY_PERSIST.getNumber();
+    }
     return value;
   }
   
@@ -1293,6 +1296,10 @@ public class PBHelper {
         == CreateFlagProto.OVERWRITE_VALUE) {
       result.add(CreateFlag.OVERWRITE);
     }
+    if ((flag & CreateFlagProto.LAZY_PERSIST_VALUE)
+        == CreateFlagProto.LAZY_PERSIST_VALUE) {
+      result.add(CreateFlag.LAZY_PERSIST);
+    }
     return new EnumSetWritable<CreateFlag>(result);
   }
 
@@ -1318,6 +1325,7 @@ public class PBHelper {
     return new HdfsLocatedFileStatus(
         fs.getLength(), fs.getFileType().equals(FileType.IS_DIR), 
         fs.getBlockReplication(), fs.getBlocksize(),
+        fs.hasIsLazyPersist() ? fs.getIsLazyPersist() : false,
         fs.getModificationTime(), fs.getAccessTime(),
         PBHelper.convert(fs.getPermission()), fs.getOwner(), fs.getGroup(), 
         fs.getFileType().equals(FileType.IS_SYMLINK) ? 
@@ -1366,6 +1374,7 @@ public class PBHelper {
       setFileType(fType).
       setBlockReplication(fs.getReplication()).
       setBlocksize(fs.getBlockSize()).
+      setIsLazyPersist(fs.isLazyPersist()).
       setModificationTime(fs.getModificationTime()).
       setAccessTime(fs.getAccessTime()).
       setPermission(PBHelper.convert(fs.getPermission())).

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -1438,7 +1438,7 @@ public class FSDirectory implements Closeable {
   private HdfsFileStatus getFileInfo4DotSnapshot(String src)
       throws UnresolvedLinkException {
     if (getINode4DotSnapshot(src) != null) {
-      return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
+      return new HdfsFileStatus(0, true, 0, 0, false, 0, 0, null, null, null, null,
           HdfsFileStatus.EMPTY_NAME, -1L, 0, null);
     }
     return null;
@@ -2300,6 +2300,7 @@ public class FSDirectory implements Closeable {
         node.isDirectory(), 
         replication, 
         blocksize,
+        false,
         node.getModificationTime(snapshot),
         node.getAccessTime(snapshot),
         getPermissionForFileStatus(node, snapshot),
@@ -2347,7 +2348,7 @@ public class FSDirectory implements Closeable {
 
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
-          blocksize, node.getModificationTime(snapshot),
+          blocksize, false, node.getModificationTime(snapshot),
           node.getAccessTime(snapshot),
           getPermissionForFileStatus(node, snapshot),
           node.getUserName(snapshot), node.getGroupName(snapshot),

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -358,7 +358,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       Path symlink = stat.isSymlink() ? new Path(stat.getSymlink()) : null;
       Path path = dst != null ? new Path(dst) : new Path(src);
       status = new FileStatus(stat.getLen(), stat.isDir(),
-          stat.getReplication(), stat.getBlockSize(), stat.getModificationTime(),
+          stat.getReplication(), stat.getBlockSize(), stat.isLazyPersist(),
+          stat.getModificationTime(),
           stat.getAccessTime(), stat.getPermission(), stat.getOwner(),
           stat.getGroup(), symlink, path);
     }
@@ -2435,6 +2436,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     boolean create = flag.contains(CreateFlag.CREATE);
     boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
+    boolean isLazyPersist = flag.contains(CreateFlag.LAZY_PERSIST);
 
     waitForLoadingFSImage();
 
@@ -2497,8 +2499,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           checkNameNodeSafeMode("Cannot create file" + src);
           src = resolvePath(src, pathComponents);
           startFileInternal(pc, src, permissions, holder, clientMachine, create,
-              overwrite, createParent, replication, blockSize, suite, edek,
-              logRetryCache);
+              overwrite, createParent, replication, blockSize, isLazyPersist,
+              suite, edek, logRetryCache);
           stat = dir.getFileInfo(src, false,
               FSDirectory.isReservedRawName(srcArg));
         } catch (StandbyException se) {
@@ -2538,8 +2540,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private void startFileInternal(FSPermissionChecker pc, String src,
       PermissionStatus permissions, String holder, String clientMachine,
       boolean create, boolean overwrite, boolean createParent,
-      short replication, long blockSize, CipherSuite suite,
-      EncryptedKeyVersion edek, boolean logRetryEntry)
+      short replication, long blockSize, boolean isLazyPersist,
+      CipherSuite suite, EncryptedKeyVersion edek, boolean logRetryEntry)
       throws FileAlreadyExistsException, AccessControlException,
       UnresolvedLinkException, FileNotFoundException,
       ParentNotDirectoryException, RetryStartFileException, IOException {
@@ -2614,7 +2616,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (parent != null && mkdirsRecursively(parent.toString(),
               permissions, true, now())) {
         newNode = dir.addFile(src, permissions, replication, blockSize,
-                holder, clientMachine);
+                              holder, clientMachine);
       }
 
       if (newNode == null) {

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -244,6 +244,8 @@ public class JsonUtil {
     final long aTime = (Long) m.get("accessTime");
     final long mTime = (Long) m.get("modificationTime");
     final long blockSize = (Long) m.get("blockSize");
+    final boolean isLazyPersist = m.containsKey("lazyPersist")
+        ? (Boolean) m.get("lazyPersist") : false;
     final short replication = (short) (long) (Long) m.get("replication");
     final long fileId = m.containsKey("fileId") ? (Long) m.get("fileId")
         : INodeId.GRANDFATHER_INODE_ID;
@@ -251,9 +253,8 @@ public class JsonUtil {
     final int childrenNum = (childrenNumLong == null) ? -1
             : childrenNumLong.intValue();
     return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
-        blockSize, mTime, aTime, permission, owner, group,
-        symlink, DFSUtil.string2Bytes(localName), fileId, childrenNum,
-        null);
+        blockSize, isLazyPersist, mTime, aTime, permission, owner, group,
+        symlink, DFSUtil.string2Bytes(localName), fileId, childrenNum, null);
   }
 
   /** Convert an ExtendedBlock to a Json map. */

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -64,6 +64,7 @@ enum CreateFlagProto {
   CREATE = 0x01;    // Create a file
   OVERWRITE = 0x02; // Truncate/overwrite a file. Same as POSIX O_TRUNC
   APPEND = 0x04;    // Append to a file
+  LAZY_PERSIST = 0x10; // File with reduced durability guarantees.
 }
 
 message CreateRequestProto {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -263,6 +263,7 @@ message HdfsFileStatusProto {
 
   // Optional field for file encryption
   optional FileEncryptionInfoProto fileEncryptionInfo = 15;
+  optional bool isLazyPersist = 16 [default = false];
 } 
 
 /**

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -253,12 +253,12 @@ public class TestDFSClientRetries {
                          anyLong(), any(String[].class))).thenAnswer(answer);
     
     Mockito.doReturn(
-            new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
+            new HdfsFileStatus(0, false, 1, 1024, false, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
                 1010, 0, null)).when(mockNN).getFileInfo(anyString());
     
     Mockito.doReturn(
-            new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
+            new HdfsFileStatus(0, false, 1, 1024, false, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
                 1010, 0, null))
         .when(mockNN)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java

@@ -340,12 +340,12 @@ public class TestLease {
     }
 
     Mockito.doReturn(
-        new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
+        new HdfsFileStatus(0, false, 1, 1024, false, 0, 0, new FsPermission(
             (short) 777), "owner", "group", new byte[0], new byte[0],
             1010, 0, null)).when(mcp).getFileInfo(anyString());
     Mockito
         .doReturn(
-            new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
+            new HdfsFileStatus(0, false, 1, 1024, false, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
                 1010, 0, null))
         .when(mcp)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -1017,8 +1017,8 @@ public class TestFsck {
     int numChildren = 1;
 
     HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication,
-        blockSize, modTime, accessTime, perms, owner, group, symlink, path,
-        fileId, numChildren, null);
+        blockSize, false, modTime, accessTime, perms, owner, group, symlink,
+        path, fileId, numChildren, null);
     Result res = new Result(conf);
 
     try {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java

@@ -62,7 +62,7 @@ public class TestJsonUtil {
     final long now = Time.now();
     final String parent = "/dir";
     final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
-        now, now + 10, new FsPermission((short) 0644), "user", "group",
+        false, now, now + 10, new FsPermission((short) 0644), "user", "group",
         DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
         INodeId.GRANDFATHER_INODE_ID, 0, null);
     final FileStatus fstatus = toFileStatus(status, parent);