Browse Source

HDFS-6922. Add LazyPersist flag to INodeFile, save it in FsImage and edit logs. (Arpit Agarwal)

arp 11 years ago
parent
commit
042b33f20b
21 changed files with 107 additions and 33 deletions
  1. 2 1
      hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-6581.txt
  2. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
  3. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  4. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  5. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  6. 23 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  7. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  8. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  9. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  10. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  11. 17 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  12. 8 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
  13. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
  14. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
  15. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
  16. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
  17. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
  18. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  19. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  20. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
  21. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-6581.txt

@@ -4,5 +4,6 @@
 
     HDFS-6924. Add new RAM_DISK storage type. (Arpit Agarwal)
 
-
+    HDFS-6922. Add LazyPersist flag to INodeFile, save it in FsImage and
+    edit logs. (Arpit Agarwal)
 

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java

@@ -54,6 +54,12 @@ public interface BlockCollection {
    */
   public long getPreferredBlockSize();
 
+  /**
+   * Return true if the file was created with {@Link CreateFlag#LAZY_PERSIST}.
+   * @return
+   */
+  public boolean getLazyPersistFlag();
+
   /**
    * Get block replication for the collection 
    * @return block replication value

+ 11 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -278,6 +278,7 @@ public class FSDirectory implements Closeable {
    */
   INodeFile addFile(String path, PermissionStatus permissions,
                     short replication, long preferredBlockSize,
+                    boolean isLazyPersist,
                     String clientName, String clientMachine)
     throws FileAlreadyExistsException, QuotaExceededException,
       UnresolvedLinkException, SnapshotAccessControlException, AclException {
@@ -285,7 +286,7 @@ public class FSDirectory implements Closeable {
     long modTime = now();
     INodeFile newNode = new INodeFile(namesystem.allocateNewInodeId(), null,
         permissions, modTime, modTime, BlockInfo.EMPTY_ARRAY, replication,
-        preferredBlockSize);
+        preferredBlockSize, isLazyPersist);
     newNode.toUnderConstruction(clientName, clientMachine);
 
     boolean added = false;
@@ -315,6 +316,7 @@ public class FSDirectory implements Closeable {
                             long modificationTime,
                             long atime,
                             long preferredBlockSize,
+                            boolean isLazyPersist,
                             boolean underConstruction,
                             String clientName,
                             String clientMachine) {
@@ -323,12 +325,12 @@ public class FSDirectory implements Closeable {
     if (underConstruction) {
       newNode = new INodeFile(id, null, permissions, modificationTime,
           modificationTime, BlockInfo.EMPTY_ARRAY, replication,
-          preferredBlockSize);
+          preferredBlockSize, isLazyPersist);
       newNode.toUnderConstruction(clientName, clientMachine);
 
     } else {
       newNode = new INodeFile(id, null, permissions, modificationTime, atime,
-          BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize);
+          BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize, isLazyPersist);
     }
 
     try {
@@ -2283,11 +2285,13 @@ public class FSDirectory implements Closeable {
      long size = 0;     // length is zero for directories
      short replication = 0;
      long blocksize = 0;
+     boolean isLazyPersist = false;
      if (node.isFile()) {
        final INodeFile fileNode = node.asFile();
        size = fileNode.computeFileSize(snapshot);
        replication = fileNode.getFileReplication(snapshot);
        blocksize = fileNode.getPreferredBlockSize();
+       isLazyPersist = fileNode.getLazyPersistFlag();
      }
      int childrenNum = node.isDirectory() ? 
          node.asDirectory().getChildrenNum(snapshot) : 0;
@@ -2300,7 +2304,7 @@ public class FSDirectory implements Closeable {
         node.isDirectory(), 
         replication, 
         blocksize,
-        false,
+        isLazyPersist,
         node.getModificationTime(snapshot),
         node.getAccessTime(snapshot),
         getPermissionForFileStatus(node, snapshot),
@@ -2322,6 +2326,7 @@ public class FSDirectory implements Closeable {
     long size = 0; // length is zero for directories
     short replication = 0;
     long blocksize = 0;
+    boolean isLazyPersist = false;
     LocatedBlocks loc = null;
     final FileEncryptionInfo feInfo = isRawPath ? null :
         getFileEncryptionInfo(node, snapshot);
@@ -2329,7 +2334,7 @@ public class FSDirectory implements Closeable {
       final INodeFile fileNode = node.asFile();
       size = fileNode.computeFileSize(snapshot);
       replication = fileNode.getFileReplication(snapshot);
-      blocksize = fileNode.getPreferredBlockSize();
+      isLazyPersist = fileNode.getLazyPersistFlag();
 
       final boolean inSnapshot = snapshot != Snapshot.CURRENT_STATE_ID; 
       final boolean isUc = !inSnapshot && fileNode.isUnderConstruction();
@@ -2348,7 +2353,7 @@ public class FSDirectory implements Closeable {
 
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
-          blocksize, false, node.getModificationTime(snapshot),
+          blocksize, isLazyPersist, node.getModificationTime(snapshot),
           node.getAccessTime(snapshot),
           getPermissionForFileStatus(node, snapshot),
           node.getUserName(snapshot), node.getGroupName(snapshot),

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -697,6 +697,7 @@ public class FSEditLog implements LogsPurgeable {
       .setModificationTime(newNode.getModificationTime())
       .setAccessTime(newNode.getAccessTime())
       .setBlockSize(newNode.getPreferredBlockSize())
+      .setLazyPersistFlag(newNode.getLazyPersistFlag())
       .setBlocks(newNode.getBlocks())
       .setPermissionStatus(permissions)
       .setClientName(newNode.getFileUnderConstructionFeature().getClientName())
@@ -727,6 +728,7 @@ public class FSEditLog implements LogsPurgeable {
       .setModificationTime(newNode.getModificationTime())
       .setAccessTime(newNode.getAccessTime())
       .setBlockSize(newNode.getPreferredBlockSize())
+      .setLazyPersistFlag(newNode.getLazyPersistFlag())
       .setBlocks(newNode.getBlocks())
       .setPermissionStatus(newNode.getPermissionStatus());
     

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -357,7 +357,8 @@ public class FSEditLogLoader {
             path, addCloseOp.permissions, addCloseOp.aclEntries,
             addCloseOp.xAttrs,
             replication, addCloseOp.mtime, addCloseOp.atime,
-            addCloseOp.blockSize, true, addCloseOp.clientName,
+            addCloseOp.blockSize, addCloseOp.isLazyPersist,
+            true, addCloseOp.clientName,
             addCloseOp.clientMachine);
         fsNamesys.leaseManager.addLease(addCloseOp.clientName, path);
 

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -401,6 +401,7 @@ public abstract class FSEditLogOp {
     long mtime;
     long atime;
     long blockSize;
+    boolean isLazyPersist;
     Block[] blocks;
     PermissionStatus permissions;
     List<AclEntry> aclEntries;
@@ -448,6 +449,11 @@ public abstract class FSEditLogOp {
       return (T)this;
     }
 
+    <T extends AddCloseOp> T setLazyPersistFlag(boolean isLazyPersist) {
+      this.isLazyPersist = isLazyPersist;
+      return (T)this;
+    }
+
     <T extends AddCloseOp> T setBlocks(Block[] blocks) {
       if (blocks.length > MAX_BLOCKS) {
         throw new RuntimeException("Can't have more than " + MAX_BLOCKS +
@@ -495,6 +501,7 @@ public abstract class FSEditLogOp {
       FSImageSerialization.writeLong(mtime, out);
       FSImageSerialization.writeLong(atime, out);
       FSImageSerialization.writeLong(blockSize, out);
+      FSImageSerialization.writeInt((isLazyPersist ? 1 : 0), out);
       new ArrayWritable(Block.class, blocks).write(out);
       permissions.write(out);
 
@@ -562,6 +569,13 @@ public abstract class FSEditLogOp {
         this.blockSize = readLong(in);
       }
 
+      if (NameNodeLayoutVersion.supports(
+          NameNodeLayoutVersion.Feature.LAZY_PERSIST_FILES, logVersion)) {
+        this.isLazyPersist = (FSImageSerialization.readInt(in) != 0);
+      } else {
+        this.isLazyPersist = false;
+      }
+
       this.blocks = readBlocks(in, logVersion);
       this.permissions = PermissionStatus.read(in);
 
@@ -615,6 +629,8 @@ public abstract class FSEditLogOp {
       builder.append(atime);
       builder.append(", blockSize=");
       builder.append(blockSize);
+      builder.append(", lazyPersist");
+      builder.append(isLazyPersist);
       builder.append(", blocks=");
       builder.append(Arrays.toString(blocks));
       builder.append(", permissions=");
@@ -651,6 +667,8 @@ public abstract class FSEditLogOp {
           Long.toString(atime));
       XMLUtils.addSaxString(contentHandler, "BLOCKSIZE",
           Long.toString(blockSize));
+      XMLUtils.addSaxString(contentHandler, "LAZY_PERSIST",
+          Boolean.toString(isLazyPersist));
       XMLUtils.addSaxString(contentHandler, "CLIENT_NAME", clientName);
       XMLUtils.addSaxString(contentHandler, "CLIENT_MACHINE", clientMachine);
       for (Block b : blocks) {
@@ -674,6 +692,11 @@ public abstract class FSEditLogOp {
       this.mtime = Long.parseLong(st.getValue("MTIME"));
       this.atime = Long.parseLong(st.getValue("ATIME"));
       this.blockSize = Long.parseLong(st.getValue("BLOCKSIZE"));
+
+      String lazyPersistString = st.getValueOrNull("LAZY_PERSIST");
+      this.isLazyPersist =
+          lazyPersistString != null && Boolean.parseBoolean(lazyPersistString);
+
       this.clientName = st.getValue("CLIENT_NAME");
       this.clientMachine = st.getValue("CLIENT_MACHINE");
       if (st.hasChildren("BLOCK")) {

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -783,8 +783,11 @@ public class FSImageFormat {
       if (counter != null) {
         counter.increment();
       }
+
+      // Images in the old format will not have the lazyPersist flag so it is
+      // safe to pass false always.
       final INodeFile file = new INodeFile(inodeId, localName, permissions,
-          modificationTime, atime, blocks, replication, blockSize);
+          modificationTime, atime, blocks, replication, blockSize, false);
       if (underConstruction) {
         file.toUnderConstruction(clientName, clientMachine);
       }
@@ -884,8 +887,10 @@ public class FSImageFormat {
           in.readShort());
       final long preferredBlockSize = in.readLong();
 
+      // LazyPersist flag will not be present in old image formats and hence
+      // can be safely set to false always.
       return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
-          accessTime, replication, preferredBlockSize, null);
+          accessTime, replication, preferredBlockSize, false, null);
     }
 
     public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -283,7 +283,8 @@ public final class FSImageFormatPBINode {
 
       final INodeFile file = new INodeFile(n.getId(),
           n.getName().toByteArray(), permissions, f.getModificationTime(),
-          f.getAccessTime(), blocks, replication, f.getPreferredBlockSize());
+          f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
+          f.hasIsLazyPersist() ? f.getIsLazyPersist() : false);
 
       if (f.hasAcl()) {
         file.addAclFeature(new AclFeature(loadAclEntries(f.getAcl(),
@@ -391,7 +392,8 @@ public final class FSImageFormatPBINode {
           .setModificationTime(file.getModificationTime())
           .setPermission(buildPermissionStatus(file, state.getStringMap()))
           .setPreferredBlockSize(file.getPreferredBlockSize())
-          .setReplication(file.getFileReplication());
+          .setReplication(file.getFileReplication())
+          .setIsLazyPersist(file.getLazyPersistFlag());
 
       AclFeature f = file.getAclFeature();
       if (f != null) {

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -146,14 +146,16 @@ public class FSImageSerialization {
     int numLocs = in.readInt();
     assert numLocs == 0 : "Unexpected block locations";
 
+    // Images in the pre-protobuf format will not have the lazyPersist flag,
+    // so it is safe to pass false always.
     INodeFile file = new INodeFile(inodeId, name, perm, modificationTime,
-        modificationTime, blocks, blockReplication, preferredBlockSize);
+        modificationTime, blocks, blockReplication, preferredBlockSize, false);
     file.toUnderConstruction(clientName, clientMachine);
     return file;
   }
 
   // Helper function that writes an INodeUnderConstruction
-  // into the input stream
+  // into the output stream
   //
   static void writeINodeUnderConstruction(DataOutputStream out, INodeFile cons,
       String path) throws IOException {

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

@@ -2616,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);
+                              isLazyPersist, holder, clientMachine);
       }
 
       if (newNode == null) {

+ 17 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -74,7 +74,9 @@ public class INodeFile extends INodeWithAdditionalFields
   /** Format: [16 bits for replication][48 bits for PreferredBlockSize] */
   static enum HeaderFormat {
     PREFERRED_BLOCK_SIZE(null, 48, 1),
-    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 16, 1);
+    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 1),
+    LAZY_PERSIST(REPLICATION.BITS, 4, 0);
+
 
     private final LongBitFormat BITS;
 
@@ -90,12 +92,18 @@ public class INodeFile extends INodeWithAdditionalFields
       return PREFERRED_BLOCK_SIZE.BITS.retrieve(header);
     }
 
-    static long toLong(long preferredBlockSize, short replication) {
+    static boolean getLazyPersistFlag(long header) {
+      return LAZY_PERSIST.BITS.retrieve(header) == 0 ? false : true;
+    }
+
+    static long toLong(long preferredBlockSize, short replication, boolean isLazyPersist) {
       long h = 0;
       h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h);
       h = REPLICATION.BITS.combine(replication, h);
+      h = LAZY_PERSIST.BITS.combine(isLazyPersist ? 1 : 0, h);
       return h;
     }
+
   }
 
   private long header = 0L;
@@ -104,9 +112,9 @@ public class INodeFile extends INodeWithAdditionalFields
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
       long atime, BlockInfo[] blklist, short replication,
-      long preferredBlockSize) {
+      long preferredBlockSize, boolean isLazyPersist) {
     super(id, name, permissions, mtime, atime);
-    header = HeaderFormat.toLong(preferredBlockSize, replication);
+    header = HeaderFormat.toLong(preferredBlockSize, replication, isLazyPersist);
     this.blocks = blklist;
   }
   
@@ -160,7 +168,6 @@ public class INodeFile extends INodeWithAdditionalFields
     return getFileUnderConstructionFeature() != null;
   }
 
-  /** Convert this file to an {@link INodeFileUnderConstruction}. */
   INodeFile toUnderConstruction(String clientName, String clientMachine) {
     Preconditions.checkState(!isUnderConstruction(),
         "file is already under construction");
@@ -355,6 +362,11 @@ public class INodeFile extends INodeWithAdditionalFields
     return HeaderFormat.getPreferredBlockSize(header);
   }
 
+  @Override
+  public boolean getLazyPersistFlag() {
+    return HeaderFormat.getLazyPersistFlag(header);
+  }
+
   @Override
   public long getHeaderLong() {
     return header;

+ 8 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java

@@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile.HeaderFormat;
 import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
-
 /**
  * The attributes of a file.
  */
@@ -32,6 +31,8 @@ public interface INodeFileAttributes extends INodeAttributes {
 
   /** @return preferred block size in bytes */
   public long getPreferredBlockSize();
+
+  public boolean getLazyPersistFlag();
   
   /** @return the header as a long. */
   public long getHeaderLong();
@@ -45,10 +46,11 @@ public interface INodeFileAttributes extends INodeAttributes {
 
     public SnapshotCopy(byte[] name, PermissionStatus permissions,
         AclFeature aclFeature, long modificationTime, long accessTime,
-        short replication, long preferredBlockSize, XAttrFeature xAttrsFeature) {
+        short replication, long preferredBlockSize,
+        boolean isTransient, XAttrFeature xAttrsFeature) {
       super(name, permissions, aclFeature, modificationTime, accessTime, 
           xAttrsFeature);
-      header = HeaderFormat.toLong(preferredBlockSize, replication);
+      header = HeaderFormat.toLong(preferredBlockSize, replication, isTransient);
     }
 
     public SnapshotCopy(INodeFile file) {
@@ -66,6 +68,9 @@ public interface INodeFileAttributes extends INodeAttributes {
       return HeaderFormat.getPreferredBlockSize(header);
     }
 
+    @Override
+    public boolean getLazyPersistFlag() { return HeaderFormat.getLazyPersistFlag(header); }
+
     @Override
     public long getHeaderLong() {
       return header;

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java

@@ -65,7 +65,9 @@ public class NameNodeLayoutVersion {
   public static enum Feature implements LayoutFeature {
     ROLLING_UPGRADE(-55, -53, "Support rolling upgrade", false),
     EDITLOG_LENGTH(-56, "Add length field to every edit log op"),
-    XATTRS(-57, "Extended attributes");
+    XATTRS(-57, "Extended attributes"),
+    LAZY_PERSIST_FILES(-58, "Support for optional lazy persistence of "
+        + " files with reduced durability guarantees");
     
     private final FeatureInfo info;
 

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java

@@ -220,7 +220,9 @@ public class FSImageFormatPBSnapshot {
           copy = new INodeFileAttributes.SnapshotCopy(pbf.getName()
               .toByteArray(), permission, acl, fileInPb.getModificationTime(),
               fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
-              fileInPb.getPreferredBlockSize(), xAttrs);
+              fileInPb.getPreferredBlockSize(),
+              fileInPb.hasIsLazyPersist() ? fileInPb.getIsLazyPersist() : false,
+              xAttrs);
         }
 
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java

@@ -391,6 +391,7 @@ class FSImageLoader {
             f.getPermission(), stringTable);
         map.put("accessTime", f.getAccessTime());
         map.put("blockSize", f.getPreferredBlockSize());
+        map.put("lazyPersist", f.getIsLazyPersist());
         map.put("group", p.getGroupName());
         map.put("length", getFileSize(f));
         map.put("modificationTime", f.getModificationTime());

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java

@@ -247,6 +247,10 @@ public final class PBImageXmlWriter {
         .o("perferredBlockSize", f.getPreferredBlockSize())
         .o("permission", dumpPermission(f.getPermission()));
 
+    if (f.hasIsLazyPersist()) {
+      o("lazyPersist", f.getIsLazyPersist());
+    }
+
     if (f.getBlocksCount() > 0) {
       out.print("<blocks>");
       for (BlockProto b : f.getBlocksList()) {

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

@@ -134,6 +134,7 @@ message INodeSection {
     optional FileUnderConstructionFeature fileUC = 7;
     optional AclFeatureProto acl = 8;
     optional XAttrFeatureProto xAttrs = 9;
+    optional bool isLazyPersist = 10 [default = false];
   }
 
   message INodeDirectory {

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

@@ -82,7 +82,7 @@ public class CreateEditsLog {
       }
 
       final INodeFile inode = new INodeFile(inodeId.nextValue(), null,
-          p, 0L, 0L, blocks, replication, blockSize);
+          p, 0L, 0L, blocks, replication, blockSize, false);
       inode.toUnderConstruction("", "");
 
      // Append path to filename with information about blockIDs 
@@ -97,7 +97,7 @@ public class CreateEditsLog {
         editLog.logMkDir(currentDir, dirInode);
       }
       INodeFile fileUc = new INodeFile(inodeId.nextValue(), null,
-          p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
+          p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize, false);
       fileUc.toUnderConstruction("", "");
       editLog.logOpenFile(filePath, fileUc, false);
       editLog.logCloseFile(filePath, inode);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

@@ -194,7 +194,7 @@ public class TestEditLog {
 
       for (int i = 0; i < numTransactions; i++) {
         INodeFile inode = new INodeFile(namesystem.allocateNewInodeId(), null,
-            p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
+            p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize, false);
         inode.toUnderConstruction("", "");
 
         editLog.logOpenFile("/filename" + (startIndex + i), inode, false);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java

@@ -423,7 +423,7 @@ public class TestFSPermissionChecker {
       FsPermission.createImmutable(perm));
     INodeFile inodeFile = new INodeFile(INodeId.GRANDFATHER_INODE_ID,
       name.getBytes("UTF-8"), permStatus, 0L, 0L, null, REPLICATION,
-      PREFERRED_BLOCK_SIZE);
+      PREFERRED_BLOCK_SIZE, false);
     parent.addChild(inodeFile);
     return inodeFile;
   }

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -82,7 +82,7 @@ public class TestINodeFile {
 
   INodeFile createINodeFile(short replication, long preferredBlockSize) {
     return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-        null, replication, preferredBlockSize);
+        null, replication, preferredBlockSize, false);
   }
   /**
    * Test for the Replication value. Sets a value and checks if it was set
@@ -259,7 +259,7 @@ public class TestINodeFile {
     INodeFile[] iNodes = new INodeFile[nCount];
     for (int i = 0; i < nCount; i++) {
       iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication,
-          preferredBlockSize);
+          preferredBlockSize, false);
       iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i));
       BlockInfo newblock = new BlockInfo(replication);
       iNodes[i].addBlock(newblock);
@@ -316,7 +316,7 @@ public class TestINodeFile {
 
     {//cast from INodeFileUnderConstruction
       final INode from = new INodeFile(
-          INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication, 1024L);
+          INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication, 1024L, false);
       from.asFile().toUnderConstruction("client", "machine");
     
       //cast to INodeFile, should success
@@ -1079,7 +1079,7 @@ public class TestINodeFile {
   public void testFileUnderConstruction() {
     replication = 3;
     final INodeFile file = new INodeFile(INodeId.GRANDFATHER_INODE_ID, null,
-        perm, 0L, 0L, null, replication, 1024L);
+        perm, 0L, 0L, null, replication, 1024L, false);
     assertFalse(file.isUnderConstruction());
 
     final String clientName = "client";