Browse Source

HDFS-12455. WebHDFS - Adding "snapshot enabled" status to ListStatus query result. Contributed by Ajay Kumar.

Xiaoyu Yao 7 years ago
parent
commit
107c177782

+ 26 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java

@@ -60,7 +60,8 @@ public class FileStatus implements Writable, Comparable<Object>,
     HAS_ACL,
     HAS_CRYPT,
     HAS_EC,
-  };
+    SNAPSHOT_ENABLED
+  }
   private static final Set<AttrFlags> NONE = Collections.<AttrFlags>emptySet();
   private static Set<AttrFlags> flags(boolean acl, boolean crypt, boolean ec) {
     if (!(acl || crypt || ec)) {
@@ -273,6 +274,15 @@ public class FileStatus implements Writable, Comparable<Object>,
     return attr.contains(AttrFlags.HAS_EC);
   }
 
+  /**
+   * Check if directory is Snapshot enabled or not.
+   *
+   * @return true if directory is snapshot enabled
+   */
+  public boolean isSnapshotEnabled() {
+    return attr.contains(AttrFlags.SNAPSHOT_ENABLED);
+  }
+
   /**
    * Get the owner of the file.
    * @return owner of the file. The string could be empty if there is no
@@ -330,6 +340,19 @@ public class FileStatus implements Writable, Comparable<Object>,
     this.group = (group == null) ? "" :  group;
   }
 
+  /**
+   * Sets Snapshot enabled flag.
+   *
+   * @param isSnapShotEnabled When true, SNAPSHOT_ENABLED flag is set
+   */
+  public void setSnapShotEnabledFlag(boolean isSnapShotEnabled) {
+    if (isSnapShotEnabled) {
+      attr.add(AttrFlags.SNAPSHOT_ENABLED);
+    } else {
+      attr.remove(AttrFlags.SNAPSHOT_ENABLED);
+    }
+  }
+
   /**
    * @return The contents of the symbolic link.
    */
@@ -486,4 +509,6 @@ public class FileStatus implements Writable, Comparable<Object>,
     }
   }
 
+
+
 }

+ 10 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java

@@ -94,11 +94,15 @@ public final class PBHelper {
     owner = proto.getOwner();
     group = proto.getGroup();
     int flags = proto.getFlags();
-    return new FileStatus(length, isdir, blockReplication, blocksize,
-        mtime, atime, permission, owner, group, symlink, path,
-        (flags & FileStatusProto.Flags.HAS_ACL_VALUE)   != 0,
+    FileStatus fileStatus = new FileStatus(length, isdir, blockReplication,
+        blocksize, mtime, atime, permission, owner, group, symlink, path,
+        (flags & FileStatusProto.Flags.HAS_ACL_VALUE) != 0,
         (flags & FileStatusProto.Flags.HAS_CRYPT_VALUE) != 0,
-        (flags & FileStatusProto.Flags.HAS_EC_VALUE)    != 0);
+        (flags & FileStatusProto.Flags.HAS_EC_VALUE) != 0);
+
+    fileStatus.setSnapShotEnabledFlag((flags & FileStatusProto.Flags
+        .SNAPSHOT_ENABLED_VALUE) != 0);
+    return fileStatus;
   }
 
   public static FileStatusProto convert(FileStatus stat) throws IOException {
@@ -124,6 +128,8 @@ public final class PBHelper {
     flags |= stat.hasAcl()         ? FileStatusProto.Flags.HAS_ACL_VALUE   : 0;
     flags |= stat.isEncrypted()    ? FileStatusProto.Flags.HAS_CRYPT_VALUE : 0;
     flags |= stat.isErasureCoded() ? FileStatusProto.Flags.HAS_EC_VALUE    : 0;
+    flags |= stat.isSnapshotEnabled() ? FileStatusProto.Flags
+        .SNAPSHOT_ENABLED_VALUE : 0;
     bld.setFlags(flags);
     return bld.build();
   }

+ 4 - 3
hadoop-common-project/hadoop-common/src/main/proto/FSProtos.proto

@@ -44,9 +44,10 @@ message FileStatusProto {
     FT_SYMLINK = 3;
   }
   enum Flags {
-    HAS_ACL    = 0x01; // has ACLs
-    HAS_CRYPT  = 0x02; // encrypted
-    HAS_EC     = 0x04; // erasure coded
+    HAS_ACL           = 0x01; // has ACLs
+    HAS_CRYPT         = 0x02; // encrypted
+    HAS_EC            = 0x04; // erasure coded
+    SNAPSHOT_ENABLED  = 0x08; // snapshot enabled
   }
   required FileType fileType            = 1;
   required string path                  = 2;

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java

@@ -56,7 +56,8 @@ public class HdfsFileStatus extends FileStatus {
   public enum Flags {
     HAS_ACL,
     HAS_CRYPT,
-    HAS_EC;
+    HAS_EC,
+    SNAPSHOT_ENABLED
   }
   private final EnumSet<Flags> flags;
 
@@ -244,6 +245,15 @@ public class HdfsFileStatus extends FileStatus {
     return storagePolicy;
   }
 
+  /**
+   * Check if directory is Snapshot enabled or not.
+   *
+   * @return true if directory is snapshot enabled
+   */
+  public boolean isSnapshotEnabled() {
+    return flags.contains(Flags.SNAPSHOT_ENABLED);
+  }
+
   @Override
   public boolean equals(Object o) {
     // satisfy findbugs

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -1602,6 +1602,9 @@ public class PBHelperClient {
         case HAS_EC:
           f.add(HdfsFileStatus.Flags.HAS_EC);
           break;
+        case SNAPSHOT_ENABLED:
+          f.add(HdfsFileStatus.Flags.SNAPSHOT_ENABLED);
+          break;
         default:
           // ignore unknown
           break;
@@ -2155,6 +2158,8 @@ public class PBHelperClient {
     int flags = fs.hasAcl()   ? HdfsFileStatusProto.Flags.HAS_ACL_VALUE   : 0;
     flags |= fs.isEncrypted() ? HdfsFileStatusProto.Flags.HAS_CRYPT_VALUE : 0;
     flags |= fs.isErasureCoded() ? HdfsFileStatusProto.Flags.HAS_EC_VALUE : 0;
+    flags |= fs.isSnapshotEnabled() ? HdfsFileStatusProto.Flags
+        .SNAPSHOT_ENABLED_VALUE : 0;
     builder.setFlags(flags);
     return builder.build();
   }

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

@@ -410,6 +410,7 @@ message HdfsFileStatusProto {
     HAS_ACL   = 0x01; // has ACLs
     HAS_CRYPT = 0x02; // encrypted
     HAS_EC    = 0x04; // erasure coded
+    SNAPSHOT_ENABLED    = 0x08; // SNAPSHOT ENABLED
   }
   required FileType fileType = 1;
   required bytes path = 2;          // local name of inode encoded java UTF8

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

@@ -198,6 +198,7 @@ public class HttpFSFileSystem extends FileSystem
 
   public static final String ENC_BIT_JSON = "encBit";
   public static final String EC_BIT_JSON = "ecBit";
+  public static final String SNAPSHOT_BIT_JSON = "seBit";
 
   public static final String DIRECTORY_LISTING_JSON = "DirectoryListing";
   public static final String PARTIAL_LISTING_JSON = "partialListing";
@@ -1066,19 +1067,27 @@ public class HttpFSFileSystem extends FileSystem
     final Boolean aclBit = (Boolean) json.get(ACL_BIT_JSON);
     final Boolean encBit = (Boolean) json.get(ENC_BIT_JSON);
     final Boolean erasureBit = (Boolean) json.get(EC_BIT_JSON);
+    final Boolean snapshotEnabledBit = (Boolean) json.get(SNAPSHOT_BIT_JSON);
     final boolean aBit = (aclBit != null) ? aclBit : false;
     final boolean eBit = (encBit != null) ? encBit : false;
     final boolean ecBit = (erasureBit != null) ? erasureBit : false;
-    if (aBit || eBit || ecBit) {
+    final boolean seBit =
+        (snapshotEnabledBit != null) ? snapshotEnabledBit : false;
+    if (aBit || eBit || ecBit || seBit) {
       // include this for compatibility with 2.x
       FsPermissionExtension deprecatedPerm =
           new FsPermissionExtension(permission, aBit, eBit, ecBit);
-      return new FileStatus(len, FILE_TYPE.DIRECTORY == type,
+      FileStatus fileStatus = new FileStatus(len, FILE_TYPE.DIRECTORY == type,
           replication, blockSize, mTime, aTime, deprecatedPerm, owner, group,
           null, path, aBit, eBit, ecBit);
+      if (seBit) {
+        fileStatus.setSnapShotEnabledFlag(seBit);
+      }
+      return fileStatus;
+    } else {
+      return new FileStatus(len, FILE_TYPE.DIRECTORY == type,
+          replication, blockSize, mTime, aTime, permission, owner, group, path);
     }
-    return new FileStatus(len, FILE_TYPE.DIRECTORY == type,
-        replication, blockSize, mTime, aTime, permission, owner, group, path);
   }
 
   /**

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -948,6 +948,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
+
   public GetFileLinkInfoResponseProto getFileLinkInfo(RpcController controller,
       GetFileLinkInfoRequestProto req) throws ServiceException {
     try {

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java

@@ -413,6 +413,8 @@ class FSDirStatAndListingOp {
         .unprotectedGetErasureCodingPolicy(fsd.getFSNamesystem(), iip);
     final boolean isErasureCoded = (ecPolicy != null);
 
+    boolean isSnapShottable = false;
+
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
       size = fileNode.computeFileSize(snapshot);
@@ -433,6 +435,8 @@ class FSDirStatAndListingOp {
           loc = new LocatedBlocks();
         }
       }
+    } else if (node.isDirectory()) {
+      isSnapShottable = node.asDirectory().isSnapshottable();
     }
 
     int childrenNum = node.isDirectory() ?
@@ -451,6 +455,9 @@ class FSDirStatAndListingOp {
     if (isErasureCoded) {
       flags.add(HdfsFileStatus.Flags.HAS_EC);
     }
+    if(isSnapShottable){
+      flags.add(HdfsFileStatus.Flags.SNAPSHOT_ENABLED);
+    }
     return createFileStatus(
         size,
         node.isDirectory(),

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

@@ -135,6 +135,10 @@ public class JsonUtil {
     if (status.isErasureCoded()) {
       m.put("ecBit", true);
     }
+    if (status.isSnapshotEnabled()) {
+      m.put("snapshotEnabled", status.isSnapshotEnabled());
+    }
+
     m.put("accessTime", status.getAccessTime());
     m.put("modificationTime", status.getModificationTime());
     m.put("blockSize", status.getBlockSize());

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md

@@ -440,6 +440,7 @@ See also: [`newlength`](#New_Length), [FileSystem](../../api/org/apache/hadoop/f
             "pathSuffix"      : "",
             "permission"      : "777",
             "replication"     : 0,
+            "snapshotEnabled" : true
             "type"            : "DIRECTORY"    //enum {FILE, DIRECTORY, SYMLINK}
           }
         }
@@ -485,6 +486,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileSt
                 "pathSuffix"      : "bar",
                 "permission"      : "711",
                 "replication"     : 0,
+                "snapshotEnabled" : true
                 "type"            : "DIRECTORY"
               },
               ...

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -1269,6 +1269,25 @@ public class TestDistributedFileSystem {
     }
   }
 
+  @Test
+  public void testListStatusOfSnapshotDirs() throws IOException {
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
+        .build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      dfs.create(new Path("/parent/test1/dfsclose/file-0"));
+      Path snapShotDir = new Path("/parent/test1/");
+      dfs.allowSnapshot(snapShotDir);
+
+      FileStatus status = dfs.getFileStatus(new Path("/parent/test1"));
+      assertTrue(status.isSnapshotEnabled());
+      status = dfs.getFileStatus(new Path("/parent/"));
+      assertFalse(status.isSnapshotEnabled());
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   @Test(timeout=10000)
   public void testDFSClientPeerReadTimeout() throws IOException {
     final int timeout = 1000;