Browse Source

HDFS-15488. Add a command to list all snapshots for a snaphottable root with snapshot Ids. (#2166)

bshashikant 4 years ago
parent
commit
68287371cc
28 changed files with 833 additions and 6 deletions
  1. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
  2. 19 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  3. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java
  4. 14 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  5. 12 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  6. 226 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotStatus.java
  7. 22 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  8. 77 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  9. 10 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
  10. 21 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  11. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestReadOnly.java
  12. 7 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
  13. 7 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
  14. 35 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSnapshot.java
  15. 14 2
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
  16. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  17. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
  18. 24 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  19. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
  20. 33 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  21. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  22. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
  23. 38 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  24. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/AdminHelper.java
  25. 63 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/LsSnapshot.java
  26. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
  27. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsSnapshots.md
  28. 134 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestListSnapshot.java

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml

@@ -22,6 +22,7 @@
       <Class name="org.apache.hadoop.hdfs.util.StripedBlockUtil$ChunkByteArray"/>
       <Class name="org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing$DiffReportListingEntry"/>
       <Class name="org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.SnapshotStatus"/>
     </Or>
     <Bug pattern="EI_EXPOSE_REP,EI_EXPOSE_REP2" />
   </Match>

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -150,6 +150,7 @@ import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
@@ -2190,6 +2191,24 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  /**
+   * Get listing of all the snapshots for a snapshottable directory.
+   *
+   * @return Information about all the snapshots for a snapshottable directory
+   * @throws IOException If an I/O error occurred
+   * @see ClientProtocol#getSnapshotListing(String)
+   */
+  public SnapshotStatus[] getSnapshotListing(String snapshotRoot)
+      throws IOException {
+    checkOpen();
+    try (TraceScope ignored = tracer.newScope("getSnapshotListing")) {
+      return namenode.getSnapshotListing(snapshotRoot);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+
+
   /**
    * Allow snapshot on a directory.
    *

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java

@@ -111,6 +111,7 @@ public class DFSOpsCountStatistics extends StorageStatistics {
     SET_XATTR("op_set_xattr"),
     GET_SNAPSHOT_DIFF("op_get_snapshot_diff"),
     GET_SNAPSHOTTABLE_DIRECTORY_LIST("op_get_snapshottable_directory_list"),
+    GET_SNAPSHOT_LIST("op_get_snapshot_list"),
     TRUNCATE(CommonStatisticNames.OP_TRUNCATE),
     UNSET_EC_POLICY("op_unset_ec_policy"),
     UNSET_STORAGE_POLICY("op_unset_storage_policy");

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -109,6 +109,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
 import org.apache.hadoop.hdfs.client.impl.SnapshotDiffReportGenerator;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
@@ -2148,6 +2149,19 @@ public class DistributedFileSystem extends FileSystem
     return dfs.getSnapshottableDirListing();
   }
 
+  /**
+   * @return all the snapshots for a snapshottable directory
+   * @throws IOException
+   */
+  public SnapshotStatus[] getSnapshotListing(Path snapshotRoot)
+      throws IOException {
+    Path absF = fixRelativePart(snapshotRoot);
+    statistics.incrementReadOps(1);
+    storageStatistics
+        .incrementOpCounter(OpType.GET_SNAPSHOT_LIST);
+    return dfs.getSnapshotListing(getPathName(absF));
+  }
+
   @Override
   public void deleteSnapshot(final Path snapshotDir, final String snapshotName)
       throws IOException {

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -727,6 +727,18 @@ public interface ClientProtocol {
   SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException;
 
+  /**
+   * Get listing of all the snapshots for a snapshottable directory.
+   *
+   * @return Information about all the snapshots for a snapshottable directory
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  @ReadOnly(isCoordinated = true)
+  SnapshotStatus[] getSnapshotListing(String snapshotRoot)
+      throws IOException;
+
+
   ///////////////////////////////////////
   // System issues and management
   ///////////////////////////////////////

+ 226 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotStatus.java

@@ -0,0 +1,226 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.PrintStream;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.EnumSet;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+
+/**
+ * Metadata about a snapshottable directory.
+ */
+public class SnapshotStatus {
+  /**
+   * Basic information of the snapshot directory.
+   */
+  private final HdfsFileStatus dirStatus;
+
+  /**
+   * Snapshot ID for the snapshot.
+   */
+  private final int snapshotID;
+
+  /**
+   * Full path of the parent.
+   */
+  private byte[] parentFullPath;
+
+  public SnapshotStatus(long modificationTime, long accessTime,
+                        FsPermission permission,
+                        EnumSet<HdfsFileStatus.Flags> flags,
+                        String owner, String group, byte[] localName,
+                        long inodeId, int childrenNum, int snapshotID,
+                        byte[] parentFullPath) {
+    this.dirStatus = new HdfsFileStatus.Builder()
+        .isdir(true)
+        .mtime(modificationTime)
+        .atime(accessTime)
+        .perm(permission)
+        .flags(flags)
+        .owner(owner)
+        .group(group)
+        .path(localName)
+        .fileId(inodeId)
+        .children(childrenNum)
+        .build();
+    this.snapshotID = snapshotID;
+    this.parentFullPath = parentFullPath;
+  }
+
+  public SnapshotStatus(HdfsFileStatus dirStatus,
+                        int snapshotNumber, byte[] parentFullPath) {
+    this.dirStatus = dirStatus;
+    this.snapshotID = snapshotNumber;
+    this.parentFullPath = parentFullPath;
+  }
+
+  /**
+   * sets the prent path name.
+   * @param path parent path
+   */
+  public void setParentFullPath(byte[] path) {
+    parentFullPath = path;
+  }
+
+  /**
+   * @return snapshot id for the snapshot
+   */
+  public int getSnapshotID() {
+    return snapshotID;
+  }
+
+  /**
+   * @return The basic information of the directory
+   */
+  public HdfsFileStatus getDirStatus() {
+    return dirStatus;
+  }
+
+  /**
+   * @return Full path of the file
+   */
+  public byte[] getParentFullPath() {
+    return parentFullPath;
+  }
+
+  /**
+   * @return Full path of the snapshot
+   */
+  public Path getFullPath() {
+    String parentFullPathStr =
+        (parentFullPath == null || parentFullPath.length == 0) ?
+            "/" : DFSUtilClient.bytes2String(parentFullPath);
+    return new Path(getSnapshotPath(parentFullPathStr,
+        dirStatus.getLocalName()));
+  }
+
+  /**
+   * Print a list of {@link SnapshotStatus} out to a given stream.
+   *
+   * @param stats The list of {@link SnapshotStatus}
+   * @param out   The given stream for printing.
+   */
+  public static void print(SnapshotStatus[] stats,
+                           PrintStream out) {
+    if (stats == null || stats.length == 0) {
+      out.println();
+      return;
+    }
+    int maxRepl = 0, maxLen = 0, maxOwner = 0, maxGroup = 0;
+    int maxSnapshotID = 0;
+    for (SnapshotStatus status : stats) {
+      maxRepl = maxLength(maxRepl, status.dirStatus.getReplication());
+      maxLen = maxLength(maxLen, status.dirStatus.getLen());
+      maxOwner = maxLength(maxOwner, status.dirStatus.getOwner());
+      maxGroup = maxLength(maxGroup, status.dirStatus.getGroup());
+      maxSnapshotID = maxLength(maxSnapshotID, status.snapshotID);
+    }
+
+    String lineFormat = "%s%s " // permission string
+        + "%" + maxRepl + "s "
+        + (maxOwner > 0 ? "%-" + maxOwner + "s " : "%s")
+        + (maxGroup > 0 ? "%-" + maxGroup + "s " : "%s")
+        + "%" + maxLen + "s "
+        + "%s " // mod time
+        + "%" + maxSnapshotID + "s "
+        + "%s"; // path
+    SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm");
+
+    for (SnapshotStatus status : stats) {
+      String line = String.format(lineFormat, "d",
+          status.dirStatus.getPermission(),
+          status.dirStatus.getReplication(),
+          status.dirStatus.getOwner(),
+          status.dirStatus.getGroup(),
+          String.valueOf(status.dirStatus.getLen()),
+          dateFormat.format(new Date(status.dirStatus.getModificationTime())),
+          status.snapshotID,
+          getSnapshotPath(DFSUtilClient.bytes2String(status.parentFullPath),
+              status.dirStatus.getLocalName())
+      );
+      out.println(line);
+    }
+  }
+
+  private static int maxLength(int n, Object value) {
+    return Math.max(n, String.valueOf(value).length());
+  }
+
+  public static class Bean {
+    private final String path;
+    private final int snapshotID;
+    private final long modificationTime;
+    private final short permission;
+    private final String owner;
+    private final String group;
+
+    public Bean(String path, int snapshotID, long
+        modificationTime, short permission, String owner, String group) {
+      this.path = path;
+      this.snapshotID = snapshotID;
+      this.modificationTime = modificationTime;
+      this.permission = permission;
+      this.owner = owner;
+      this.group = group;
+    }
+
+    public String getPath() {
+      return path;
+    }
+
+    public int getSnapshotID() {
+      return snapshotID;
+    }
+
+    public long getModificationTime() {
+      return modificationTime;
+    }
+
+    public short getPermission() {
+      return permission;
+    }
+
+    public String getOwner() {
+      return owner;
+    }
+
+    public String getGroup() {
+      return group;
+    }
+  }
+
+  static String getSnapshotPath(String snapshottableDir,
+                                String snapshotRelativePath) {
+    String parentFullPathStr =
+        snapshottableDir == null || snapshottableDir.isEmpty() ?
+            "/" : snapshottableDir;
+    final StringBuilder b = new StringBuilder(parentFullPathStr);
+    if (b.charAt(b.length() - 1) != Path.SEPARATOR_CHAR) {
+      b.append(Path.SEPARATOR);
+    }
+    return b.append(HdfsConstants.DOT_SNAPSHOT_DIR)
+        .append(Path.SEPARATOR)
+        .append(snapshotRelativePath)
+        .toString();
+  }
+}

+ 22 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -89,6 +89,7 @@ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.ModifyAclEntriesRequestProto;
@@ -150,6 +151,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto;
@@ -1299,6 +1302,25 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
+  @Override
+  public SnapshotStatus[] getSnapshotListing(String path)
+      throws IOException {
+    GetSnapshotListingRequestProto req =
+        GetSnapshotListingRequestProto.newBuilder()
+            .setSnapshotRoot(path).build();
+    try {
+      GetSnapshotListingResponseProto result = rpcProxy
+          .getSnapshotListing(null, req);
+
+      if (result.hasSnapshotList()) {
+        return PBHelperClient.convert(result.getSnapshotList());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   @Override
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String fromSnapshot, String toSnapshot) throws IOException {

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

@@ -114,6 +114,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryScopeProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTypeProto;
@@ -184,6 +185,8 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryP
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotListingProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
@@ -1669,6 +1672,48 @@ public class PBHelperClient {
         sdirStatusProto.getParentFullpath().toByteArray());
   }
 
+  public static SnapshotStatus[] convert(
+      HdfsProtos.SnapshotListingProto sdlp) {
+    if (sdlp == null) {
+      return null;
+    }
+    List<HdfsProtos.SnapshotStatusProto> list = sdlp
+        .getSnapshotListingList();
+    if (list.isEmpty()) {
+      return new SnapshotStatus[0];
+    } else {
+      SnapshotStatus[] result =
+          new SnapshotStatus[list.size()];
+      for (int i = 0; i < list.size(); i++) {
+        result[i] = convert(list.get(i));
+      }
+      return result;
+    }
+  }
+
+  public static SnapshotStatus convert(
+      HdfsProtos.SnapshotStatusProto sdirStatusProto) {
+    if (sdirStatusProto == null) {
+      return null;
+    }
+    final HdfsFileStatusProto status = sdirStatusProto.getDirStatus();
+    EnumSet<HdfsFileStatus.Flags> flags = status.hasFlags()
+        ? convertFlags(status.getFlags())
+        : convertFlags(status.getPermission());
+    return new SnapshotStatus(
+        status.getModificationTime(),
+        status.getAccessTime(),
+        convert(status.getPermission()),
+        flags,
+        status.getOwner(),
+        status.getGroup(),
+        status.getPath().toByteArray(),
+        status.getFileId(),
+        status.getChildrenNum(),
+        sdirStatusProto.getSnapshotID(),
+        sdirStatusProto.getParentFullpath().toByteArray());
+  }
+
   // DataEncryptionKey
   public static DataEncryptionKey convert(DataEncryptionKeyProto bet) {
     String encryptionAlgorithm = bet.getEncryptionAlgorithm();
@@ -2367,6 +2412,23 @@ public class PBHelperClient {
     return builder.build();
   }
 
+  public static HdfsProtos.SnapshotStatusProto convert(SnapshotStatus status) {
+    if (status == null) {
+      return null;
+    }
+    byte[] parentFullPath = status.getParentFullPath();
+    ByteString parentFullPathBytes = getByteString(
+        parentFullPath == null ? DFSUtilClient.EMPTY_BYTES : parentFullPath);
+    HdfsFileStatusProto fs = convert(status.getDirStatus());
+    HdfsProtos.SnapshotStatusProto.Builder builder =
+        HdfsProtos.SnapshotStatusProto
+            .newBuilder()
+            .setSnapshotID(status.getSnapshotID())
+            .setParentFullpath(parentFullPathBytes)
+            .setDirStatus(fs);
+    return builder.build();
+  }
+
   public static HdfsFileStatusProto[] convert(HdfsFileStatus[] fs) {
     if (fs == null) return null;
     final int len = fs.length;
@@ -2649,6 +2711,21 @@ public class PBHelperClient {
         .addAllSnapshottableDirListing(protoList).build();
   }
 
+  public static HdfsProtos.SnapshotListingProto convert(
+      SnapshotStatus[] status) {
+    if (status == null) {
+      return null;
+    }
+    HdfsProtos.SnapshotStatusProto[] protos =
+        new HdfsProtos.SnapshotStatusProto[status.length];
+    for (int i = 0; i < status.length; i++) {
+      protos[i] = convert(status[i]);
+    }
+    List<SnapshotStatusProto> protoList = Arrays.asList(protos);
+    return SnapshotListingProto.newBuilder()
+        .addAllSnapshotListing(protoList).build();
+  }
+
   public static SnapshotDiffReportEntryProto convert(DiffReportEntry entry) {
     if (entry == null) {
       return null;

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

@@ -303,6 +303,14 @@ message GetSnapshottableDirListingResponseProto {
   optional SnapshottableDirectoryListingProto snapshottableDirList = 1;
 }
 
+message GetSnapshotListingRequestProto {
+  required string snapshotRoot = 1;
+}
+
+message GetSnapshotListingResponseProto {
+  optional SnapshotListingProto snapshotList = 1;
+}
+
 message GetSnapshotDiffReportRequestProto {
   required string snapshotRoot = 1;
   required string fromSnapshot = 2;
@@ -986,6 +994,8 @@ service ClientNamenodeProtocol {
       returns(DisallowSnapshotResponseProto);   
   rpc getSnapshottableDirListing(GetSnapshottableDirListingRequestProto)
       returns(GetSnapshottableDirListingResponseProto);
+  rpc getSnapshotListing(GetSnapshotListingRequestProto)
+      returns(GetSnapshotListingResponseProto);
   rpc deleteSnapshot(DeleteSnapshotRequestProto)
       returns(DeleteSnapshotResponseProto);
   rpc getSnapshotDiffReport(GetSnapshotDiffReportRequestProto)

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

@@ -563,6 +563,20 @@ message SnapshottableDirectoryStatusProto {
   required bytes parent_fullpath = 4;
 }
 
+/**
+ * Status of a snapshot directory: besides the normal information for
+ * a directory status, also include snapshot ID, and
+ * the full path of the parent directory.
+ */
+message SnapshotStatusProto {
+  required HdfsFileStatusProto dirStatus = 1;
+
+  // Fields specific for snapshot directory
+  required uint32 snapshotID = 2;
+  required bytes parent_fullpath = 3;
+}
+
+
 /**
  * Snapshottable directory listing
  */
@@ -570,6 +584,13 @@ message SnapshottableDirectoryListingProto {
   repeated SnapshottableDirectoryStatusProto snapshottableDirListing = 1;
 }
 
+/**
+ * Snapshot listing
+ */
+message SnapshotListingProto {
+  repeated SnapshotStatusProto snapshotListing = 1;
+}
+
 /**
  * Snapshot diff report entry
  */

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestReadOnly.java

@@ -41,6 +41,7 @@ public class TestReadOnly {
           "getListing",
           "getBatchedListing",
           "getSnapshottableDirListing",
+          "getSnapshotListing",
           "getPreferredBlockSize",
           "listCorruptFileBlocks",
           "getFileInfo",

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java

@@ -73,6 +73,7 @@ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
@@ -1314,6 +1315,12 @@ public class RouterClientProtocol implements ClientProtocol {
     return snapshotProto.getSnapshottableDirListing();
   }
 
+  @Override
+  public SnapshotStatus[] getSnapshotListing(String snapshotRoot)
+      throws IOException {
+    return snapshotProto.getSnapshotListing(snapshotRoot);
+  }
+
   @Override
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String earlierSnapshotName, String laterSnapshotName) throws IOException {

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java

@@ -112,6 +112,7 @@ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
@@ -1130,6 +1131,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol,
     return clientProto.getSnapshottableDirListing();
   }
 
+  @Override // ClientProtocol
+  public SnapshotStatus[] getSnapshotListing(String snapshotRoot)
+      throws IOException {
+    return clientProto.getSnapshotListing(snapshotRoot);
+  }
+
   @Override // ClientProtocol
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String earlierSnapshotName, String laterSnapshotName) throws IOException {

+ 35 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSnapshot.java

@@ -24,10 +24,12 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
@@ -157,6 +159,39 @@ public class RouterSnapshot {
     return RouterRpcServer.merge(ret, SnapshottableDirectoryStatus.class);
   }
 
+  public SnapshotStatus[] getSnapshotListing(String snapshotRoot)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(snapshotRoot, true, false);
+    RemoteMethod remoteMethod = new RemoteMethod("getSnapshotListing",
+        new Class<?>[]{String.class},
+        new RemoteParam());
+    SnapshotStatus[] response;
+    if (rpcServer.isInvokeConcurrent(snapshotRoot)) {
+      Map<RemoteLocation, SnapshotStatus[]> ret = rpcClient.invokeConcurrent(
+          locations, remoteMethod, true, false, SnapshotStatus[].class);
+      response = ret.values().iterator().next();
+      String src = ret.keySet().iterator().next().getSrc();
+      String dst = ret.keySet().iterator().next().getDest();
+      for (SnapshotStatus s : response) {
+        String mountPath = DFSUtil.bytes2String(s.getParentFullPath()).
+            replaceFirst(src, dst);
+        s.setParentFullPath(DFSUtil.string2Bytes(mountPath));
+      }
+    } else {
+      response = rpcClient.invokeSequential(
+          locations, remoteMethod, SnapshotStatus[].class, null);
+      RemoteLocation loc = locations.get(0);
+      for (SnapshotStatus s : response) {
+        String mountPath = DFSUtil.bytes2String(s.getParentFullPath()).
+            replaceFirst(loc.getDest(), loc.getSrc());
+        s.setParentFullPath(DFSUtil.string2Bytes(mountPath));
+      }
+    }
+    return response;
+  }
+
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String earlierSnapshotName, String laterSnapshotName)
           throws IOException {

+ 14 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java

@@ -84,8 +84,6 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -94,6 +92,9 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
@@ -110,6 +111,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
@@ -926,6 +928,16 @@ public class TestRouterRpc {
     SnapshottableDirectoryStatus snapshotDir0 = dirList[0];
     assertEquals(snapshotPath, snapshotDir0.getFullPath().toString());
 
+    // check for snapshot listing through the Router
+    SnapshotStatus[] snapshots = routerProtocol.
+        getSnapshotListing(snapshotPath);
+    assertEquals(2, snapshots.length);
+    assertEquals(SnapshotTestHelper.getSnapshotRoot(
+        new Path(snapshotPath), snapshot1),
+        snapshots[0].getFullPath());
+    assertEquals(SnapshotTestHelper.getSnapshotRoot(
+        new Path(snapshotPath), snapshot2),
+        snapshots[1].getFullPath());
     // Check for difference report in two snapshot
     SnapshotDiffReport diffReport = routerProtocol.getSnapshotDiffReport(
         snapshotPath, snapshot1, snapshot2);

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -54,6 +54,7 @@ function hadoop_usage
   hadoop_add_subcommand "jmxget" admin "get JMX exported values from NameNode or DataNode."
   hadoop_add_subcommand "journalnode" daemon "run the DFS journalnode"
   hadoop_add_subcommand "lsSnapshottableDir" client "list all snapshottable dirs owned by the current user"
+  hadoop_add_subcommand "lsSnapshot" client "list all snapshots for a snapshottable directory"
   hadoop_add_subcommand "mover" daemon "run a utility to move block replicas across storage types"
   hadoop_add_subcommand "namenode" daemon "run the DFS namenode"
   hadoop_add_subcommand "nfs3" daemon "run an NFS version 3 gateway"
@@ -166,6 +167,9 @@ function hdfscmd_case
     lsSnapshottableDir)
       HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.snapshot.LsSnapshottableDir
     ;;
+    lsSnapshot)
+      HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.snapshot.LsSnapshot
+    ;;
     mover)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME=org.apache.hadoop.hdfs.server.mover.Mover

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd

@@ -59,7 +59,7 @@ if "%1" == "--loglevel" (
     )
   )
 
-  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck fsImageValidation balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies classpath crypto dfsrouter dfsrouteradmin debug
+  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck fsImageValidation balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir lsSnapshot cacheadmin mover storagepolicies classpath crypto dfsrouter dfsrouteradmin debug
   for %%i in ( %hdfscommands% ) do (
     if %hdfs-command% == %%i set hdfscommand=true
   )
@@ -167,6 +167,10 @@ goto :eof
   set CLASS=org.apache.hadoop.hdfs.tools.snapshot.LsSnapshottableDir
   goto :eof
 
+:lsSnapshot
+  set CLASS=org.apache.hadoop.hdfs.tools.snapshot.LsSnapshot
+  goto :eof
+
 :cacheadmin
   set CLASS=org.apache.hadoop.hdfs.tools.CacheAdmin
   goto :eof
@@ -253,6 +257,8 @@ goto :eof
   @echo                        current directory contents with a snapshot
   @echo   lsSnapshottableDir   list all snapshottable dirs owned by the current user
   @echo 						Use -help to see options
+  @echo   lsSnapshot           list all snapshots for a snapshottable dir
+  @echo                         Use -help to see options
   @echo   cacheadmin           configure the HDFS cache
   @echo   crypto               configure HDFS encryption zones
   @echo   mover                run a utility to move block replicas across storage types

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

@@ -66,6 +66,7 @@ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
@@ -161,6 +162,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto;
@@ -325,6 +328,9 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   static final GetSnapshottableDirListingResponseProto 
       NULL_GET_SNAPSHOTTABLE_DIR_LISTING_RESPONSE = 
       GetSnapshottableDirListingResponseProto.newBuilder().build();
+  static final GetSnapshotListingResponseProto
+      NULL_GET_SNAPSHOT_LISTING_RESPONSE =
+      GetSnapshotListingResponseProto.newBuilder().build();
   static final SetStoragePolicyResponseProto VOID_SET_STORAGE_POLICY_RESPONSE =
       SetStoragePolicyResponseProto.newBuilder().build();
   static final UnsetStoragePolicyResponseProto
@@ -1349,6 +1355,24 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     }
   }
 
+  @Override
+  public GetSnapshotListingResponseProto getSnapshotListing(
+      RpcController controller, GetSnapshotListingRequestProto request)
+      throws ServiceException {
+    try {
+      SnapshotStatus[] result = server
+          .getSnapshotListing(request.getSnapshotRoot());
+      if (result != null) {
+        return GetSnapshotListingResponseProto.newBuilder().
+            setSnapshotList(PBHelperClient.convert(result)).build();
+      } else {
+        return NULL_GET_SNAPSHOT_LISTING_RESPONSE;
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
   @Override
   public GetSnapshotDiffReportResponseProto getSnapshotDiffReport(
       RpcController controller, GetSnapshotDiffReportRequestProto request)

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -154,6 +155,22 @@ class FSDirSnapshotOp {
     }
   }
 
+  static SnapshotStatus[] getSnapshotListing(
+      FSDirectory fsd, FSPermissionChecker pc, SnapshotManager snapshotManager,
+      String path)
+      throws IOException {
+    fsd.readLock();
+    try {
+      INodesInPath iip = fsd.getINodesInPath(path, DirOp.READ);
+      if (fsd.isPermissionEnabled()) {
+        fsd.checkPathAccess(pc, iip, FsAction.READ);
+      }
+      return snapshotManager.getSnapshotListing(iip);
+    } finally {
+      fsd.readUnlock();
+    }
+  }
+
   static SnapshotDiffReport getSnapshotDiffReport(FSDirectory fsd,
       FSPermissionChecker pc, SnapshotManager snapshotManager, String path,
       String fromSnapshot, String toSnapshot) throws IOException {

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

@@ -99,6 +99,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.commons.text.CaseUtils;
 import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 import static org.apache.hadoop.ha.HAServiceProtocol.HAServiceState.ACTIVE;
@@ -7001,7 +7002,38 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     logAuditEvent(true, operationName, null, null, null);
     return status;
   }
-  
+
+  /**
+   * Get the list of snapshots for a given snapshottable directory.
+   *
+   * @return The list of all the snapshots for a snapshottable directory
+   * @throws IOException
+   */
+  public SnapshotStatus[] getSnapshotListing(String snapshotRoot)
+      throws IOException {
+    final String operationName = "listSnapshotDirectory";
+    SnapshotStatus[] status;
+    checkOperation(OperationCategory.READ);
+    boolean success = false;
+    final FSPermissionChecker pc = getPermissionChecker();
+    FSPermissionChecker.setOperationType(operationName);
+    try {
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        status = FSDirSnapshotOp.getSnapshotListing(dir, pc, snapshotManager,
+            snapshotRoot);
+        success = true;
+      } finally {
+        readUnlock(operationName, getLockReportInfoSupplier(null));
+      }
+    } catch (AccessControlException ace) {
+      logAuditEvent(success, "listSnapshots", snapshotRoot);
+      throw ace;
+    }
+    logAuditEvent(success, "listSnapshots", snapshotRoot);
+    return status;
+  }
   /**
    * Get the difference between two snapshots (or between a snapshot and the
    * current status) of a snapshottable directory.

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -132,6 +132,7 @@ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
@@ -2004,6 +2005,16 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     return status;
   }
 
+  @Override // Client Protocol
+  public SnapshotStatus[] getSnapshotListing(String snapshotRoot)
+      throws IOException {
+    checkNNStartup();
+    SnapshotStatus[] status = namesystem
+        .getSnapshotListing(snapshotRoot);
+    metrics.incrListSnapshotsOps();
+    return status;
+  }
+
   @Override // ClientProtocol
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String earlierSnapshotName, String laterSnapshotName) throws IOException {

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

@@ -77,6 +77,8 @@ public class NameNodeMetrics {
   MutableCounterLong renameSnapshotOps;
   @Metric("Number of listSnapshottableDirectory operations")
   MutableCounterLong listSnapshottableDirOps;
+  @Metric("Number of listSnapshots operations")
+  MutableCounterLong listSnapshotOps;
   @Metric("Number of snapshotDiffReport operations")
   MutableCounterLong snapshotDiffReportOps;
   @Metric("Number of blockReceivedAndDeleted calls")
@@ -106,6 +108,7 @@ public class NameNodeMetrics {
       disallowSnapshotOps.value() +
       renameSnapshotOps.value() +
       listSnapshottableDirOps.value() +
+      listSnapshotOps.value() +
       createSymlinkOps.value() +
       snapshotDiffReportOps.value();
   }
@@ -319,6 +322,10 @@ public class NameNodeMetrics {
   public void incrListSnapshottableDirOps() {
     listSnapshottableDirOps.incr();
   }
+
+  public void incrListSnapshotsOps() {
+    listSnapshotOps.incr();
+  }
   
   public void incrSnapshotDiffReportOps() {
     snapshotDiffReportOps.incr();

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

@@ -50,10 +50,18 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshotInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.*;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
+import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.metrics2.util.MBeans;
 
 import com.google.common.base.Preconditions;
@@ -501,7 +509,36 @@ public class SnapshotManager implements SnapshotStatsMXBean {
     return statusList.toArray(
         new SnapshottableDirectoryStatus[statusList.size()]);
   }
-  
+
+  /**
+   * List all the snapshots under a snapshottable directory.
+   */
+  public SnapshotStatus[] getSnapshotListing(INodesInPath iip)
+      throws IOException {
+    INodeDirectory srcRoot = getSnapshottableRoot(iip);
+    ReadOnlyList<Snapshot> snapshotList = srcRoot.
+        getDirectorySnapshottableFeature().getSnapshotList();
+    SnapshotStatus[] statuses = new SnapshotStatus[snapshotList.size()];
+    for (int count = 0; count < snapshotList.size(); count++) {
+      Snapshot s = snapshotList.get(count);
+      Snapshot.Root dir = s.getRoot();
+      statuses[count] = new SnapshotStatus(dir.getModificationTime(),
+          dir.getAccessTime(), dir.getFsPermission(),
+          EnumSet.noneOf(HdfsFileStatus.Flags.class),
+          dir.getUserName(), dir.getGroupName(),
+          dir.getLocalNameBytes(), dir.getId(),
+          // the children number is same as the
+          // live fs as the children count is not cached per snashot.
+          // It is just used here to construct the HdfsFileStatus object.
+          // It is expensive to build the snapshot tree for the directory
+          // and determine the child count.
+          dir.getChildrenNum(Snapshot.CURRENT_STATE_ID),
+          s.getId(), DFSUtil.string2Bytes(dir.getParent().getFullPathName()));
+
+    }
+    return statuses;
+  }
+
   /**
    * Compute the difference between two snapshots of a directory, or between a
    * snapshot of the directory and its current tree.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/AdminHelper.java

@@ -43,7 +43,7 @@ public class AdminHelper {
   static final int MAX_LINE_WIDTH = 80;
   static final String HELP_COMMAND_NAME = "-help";
 
-  static DistributedFileSystem getDFS(Configuration conf)
+  public static DistributedFileSystem getDFS(Configuration conf)
       throws IOException {
     FileSystem fs = FileSystem.get(conf);
     return checkAndGetDFS(fs, conf);

+ 63 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/LsSnapshot.java

@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.tools.snapshot;
+
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
+import org.apache.hadoop.hdfs.tools.AdminHelper;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * A tool used to list all snapshottable directories that are owned by the
+ * current user. The tool returns all the snapshottable directories if the user
+ * is a super user.
+ */
+@InterfaceAudience.Private
+public class LsSnapshot extends Configured implements Tool {
+  @Override
+  public int run(String[] argv) throws Exception {
+    String description = "hdfs lsSnapshot <snapshotDir>: \n" +
+        "\tGet the list of snapshots for a snapshottable directory.\n";
+
+    if(argv.length != 1) {
+      System.err.println("Invalid no of arguments");
+      System.err.println("Usage: \n" + description);
+      return 1;
+    }
+    Path snapshotRoot = new Path(argv[0]);
+    try {
+      DistributedFileSystem dfs = AdminHelper.getDFS(getConf());
+      SnapshotStatus[] stats = dfs.getSnapshotListing(snapshotRoot);
+      SnapshotStatus.print(stats, System.out);
+    } catch (Exception e) {
+      String[] content = e.getLocalizedMessage().split("\n");
+      System.err.println("lsSnapshot: " + content[0]);
+      return 1;
+    }
+    return 0;
+  }
+  public static void main(String[] argv) throws Exception {
+    int rc = ToolRunner.run(new LsSnapshot(), argv);
+    System.exit(rc);
+  }
+}

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md

@@ -158,6 +158,16 @@ Usage: `hdfs lsSnapshottableDir [-help]`
 
 Get the list of snapshottable directories. When this is run as a super user, it returns all snapshottable directories. Otherwise it returns those directories that are owned by the current user.
 
+### `lsSnapshot`
+
+Usage: `hdfs lsSnapshot [-help]`
+
+| COMMAND\_OPTION | Description |
+|:---- |:---- |
+| `-help` | print help |
+
+Get the list of snapshots for a snapshottable directory.
+
 ### `jmxget`
 
 Usage: `hdfs jmxget [-localVM ConnectorURL | -port port | -server mbeanserver | -service service]`

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsSnapshots.md

@@ -236,6 +236,23 @@ See also the corresponding Java API
 `SnapshottableDirectoryStatus[] getSnapshottableDirectoryListing()`
 in `DistributedFileSystem`.
 
+#### Get Snapshot Listing
+
+Get all the snapshots for a snapshottable directory.
+
+* Command:
+
+        hdfs lsSnapshot <snapshotRoot>
+
+* Arguments:
+
+    | --- | --- |
+    | path | The path of the snapshottable directory. |
+
+See also the corresponding Java API
+`SnapshotStatus[] getSnapshotListing()`
+in `DistributedFileSystem`.
+
 
 #### Get Snapshots Difference Report
 

+ 134 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestListSnapshot.java

@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.snapshot;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Tests listSnapshot.
+ */
+public class TestListSnapshot {
+
+  static final short REPLICATION = 3;
+
+  private final Path dir1 = new Path("/TestSnapshot1");
+
+  Configuration conf;
+  MiniDFSCluster cluster;
+  FSNamesystem fsn;
+  DistributedFileSystem hdfs;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    hdfs.mkdirs(dir1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  /**
+   * Test listing all the snapshottable directories.
+   */
+  @Test(timeout = 60000)
+  public void testListSnapshot() throws Exception {
+    fsn.getSnapshotManager().setAllowNestedSnapshots(true);
+
+    // Initially there is no snapshottable directories in the system
+    SnapshotStatus[] snapshotStatuses = null;
+    SnapshottableDirectoryStatus[] dirs = hdfs.getSnapshottableDirListing();
+    assertNull(dirs);
+    LambdaTestUtils.intercept(SnapshotException.class,
+        "Directory is not a " + "snapshottable directory",
+        () -> hdfs.getSnapshotListing(dir1));
+    // Make root as snapshottable
+    final Path root = new Path("/");
+    hdfs.allowSnapshot(root);
+    dirs = hdfs.getSnapshottableDirListing();
+    assertEquals(1, dirs.length);
+    assertEquals("", dirs[0].getDirStatus().getLocalName());
+    assertEquals(root, dirs[0].getFullPath());
+    snapshotStatuses = hdfs.getSnapshotListing(root);
+    assertTrue(snapshotStatuses.length == 0);
+    // Make root non-snaphsottable
+    hdfs.disallowSnapshot(root);
+    dirs = hdfs.getSnapshottableDirListing();
+    assertNull(dirs);
+    snapshotStatuses = hdfs.getSnapshotListing(root);
+    assertTrue(snapshotStatuses.length == 0);
+
+    // Make dir1 as snapshottable
+    hdfs.allowSnapshot(dir1);
+    hdfs.createSnapshot(dir1, "s0");
+    snapshotStatuses = hdfs.getSnapshotListing(dir1);
+    assertEquals(1, snapshotStatuses.length);
+    assertEquals("s0", snapshotStatuses[0].getDirStatus().
+        getLocalName());
+    assertEquals(SnapshotTestHelper.getSnapshotRoot(dir1, "s0"),
+        snapshotStatuses[0].getFullPath());
+    // snapshot id is zero
+    assertEquals(0, snapshotStatuses[0].getSnapshotID());
+    // Create a snapshot for dir1
+    hdfs.createSnapshot(dir1, "s1");
+    hdfs.createSnapshot(dir1, "s2");
+    snapshotStatuses = hdfs.getSnapshotListing(dir1);
+    // There are now 3 snapshots for dir1
+    assertEquals(3, snapshotStatuses.length);
+    assertEquals("s0", snapshotStatuses[0].getDirStatus().
+        getLocalName());
+    assertEquals(SnapshotTestHelper.getSnapshotRoot(dir1, "s0"),
+        snapshotStatuses[0].getFullPath());
+    assertEquals("s1", snapshotStatuses[1].getDirStatus().
+        getLocalName());
+    assertEquals(SnapshotTestHelper.getSnapshotRoot(dir1, "s1"),
+        snapshotStatuses[1].getFullPath());
+    assertEquals("s2", snapshotStatuses[2].getDirStatus().
+        getLocalName());
+    assertEquals(SnapshotTestHelper.getSnapshotRoot(dir1, "s2"),
+        snapshotStatuses[2].getFullPath());
+    hdfs.deleteSnapshot(dir1, "s2");
+    snapshotStatuses = hdfs.getSnapshotListing(dir1);
+    // There are now 2 snapshots for dir1
+    assertEquals(2, snapshotStatuses.length);
+  }
+}