Kaynağa Gözat

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

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
arp 11 yıl önce
ebeveyn
işleme
0ee251ea26

+ 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;
 

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

@@ -1363,6 +1363,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;
   }
   
@@ -1379,6 +1382,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);
   }
 

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

@@ -2431,7 +2431,6 @@ public class FSDirectory implements Closeable {
        blocksize = fileNode.getPreferredBlockSize();
        isEncrypted = (feInfo != null) ||
            (isRawPath && isInAnEZ(INodesInPath.fromINode(node)));
-       isLazyPersist = fileNode.getLazyPersistFlag();
      } else {
        isEncrypted = isInAnEZ(INodesInPath.fromINode(node));
      }

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

@@ -2566,6 +2566,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();
 
@@ -2628,7 +2629,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       src = resolvePath(src, pathComponents);
       toRemoveBlocks = startFileInternal(pc, src, permissions, holder, 
           clientMachine, create, overwrite, createParent, replication, 
-          blockSize, suite, protocolVersion, edek, logRetryCache);
+          blockSize, isLazyPersist, suite, protocolVersion, edek, logRetryCache);
       stat = dir.getFileInfo(src, false,
           FSDirectory.isReservedRawName(srcArg), true);
     } catch (StandbyException se) {
@@ -2664,7 +2665,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       String src, PermissionStatus permissions, String holder, 
       String clientMachine, boolean create, boolean overwrite, 
       boolean createParent, short replication, long blockSize, 
-      CipherSuite suite, CryptoProtocolVersion version,
+      boolean isLazyPersist, CipherSuite suite, CryptoProtocolVersion version,
       EncryptedKeyVersion edek, boolean logRetryEntry)
       throws FileAlreadyExistsException, AccessControlException,
       UnresolvedLinkException, FileNotFoundException,

+ 0 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -108,9 +108,6 @@ public class INodeFile extends INodeWithAdditionalFields
       h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h);
       return h;
     }
-    static boolean getLazyPersistFlag(long header) {
-      return LAZY_PERSIST.BITS.retrieve(header) == 0 ? false : true;
-    }
   }
 
   private long header = 0L;
@@ -402,9 +399,6 @@ public class INodeFile extends INodeWithAdditionalFields
     recordModification(latestSnapshotId);
     setStoragePolicyID(storagePolicyId);
   }
-  public boolean getLazyPersistFlag() {
-    return HeaderFormat.getLazyPersistFlag(header);
-  }
 
   @Override
   public long getHeaderLong() {

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

@@ -73,9 +73,6 @@ public interface INodeFileAttributes extends INodeAttributes {
       return HeaderFormat.getStoragePolicyID(header);
     }
 
-    @Override
-    public boolean getLazyPersistFlag() { return HeaderFormat.getLazyPersistFlag(header); }
-
     @Override
     public long getHeaderLong() {
       return header;

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

@@ -65,6 +65,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/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -576,6 +576,7 @@ public class TestDirectoryScanner {
 
     @Override
     public void releaseReservedSpace(long bytesToRelease) {
+    }
 
     @Override
     public boolean isTransientStorage() {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java

@@ -147,7 +147,7 @@ public class TestFsDatasetImpl {
     for (int i = 0; i < NUM_BLOCKS; i++) {
       String bpid = BLOCK_POOL_IDS[NUM_BLOCKS % BLOCK_POOL_IDS.length];
       ExtendedBlock eb = new ExtendedBlock(bpid, i);
-      dataset.createRbw(StorageType.DEFAULT, eb);
+      dataset.createRbw(StorageType.DEFAULT, eb, false);
     }
     final String[] dataDirs =
         conf.get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY).split(",");