Browse Source

HDFS-16091. WebHDFS should support getSnapshotDiffReportListing. (#3374)

Masatake Iwasaki 3 years ago
parent
commit
634f8a2209

+ 78 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java

@@ -17,6 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
+import org.apache.commons.collections.list.TreeList;
+import org.apache.hadoop.ipc.RpcNoSuchMethodException;
 import org.apache.hadoop.net.DomainNameResolver;
 import org.apache.hadoop.net.DomainNameResolver;
 import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
 import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
 import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.util.Preconditions;
@@ -30,6 +32,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.impl.SnapshotDiffReportGenerator;
 import org.apache.hadoop.hdfs.net.BasicInetPeer;
 import org.apache.hadoop.hdfs.net.BasicInetPeer;
 import org.apache.hadoop.hdfs.net.NioInetPeer;
 import org.apache.hadoop.hdfs.net.NioInetPeer;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.Peer;
@@ -43,8 +46,11 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol;
 import org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
 import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.ReconfigurationProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.ReconfigurationProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -55,6 +61,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
@@ -360,6 +367,13 @@ public class DFSUtilClient {
     return path;
     return path;
   }
   }
 
 
+  /**
+   * Given a list of path components returns a string.
+   */
+  public static String byteArray2String(byte[][] pathComponents) {
+    return bytes2String(byteArray2bytes(pathComponents));
+  }
+
   /**
   /**
    * Decode a specific range of bytes of the given byte array to a string
    * Decode a specific range of bytes of the given byte array to a string
    * using UTF8.
    * using UTF8.
@@ -1129,4 +1143,68 @@ public class DFSUtilClient {
     return (ssRoot.equals("/") ? ssRoot : ssRoot + Path.SEPARATOR)
     return (ssRoot.equals("/") ? ssRoot : ssRoot + Path.SEPARATOR)
         + FileSystem.TRASH_PREFIX + Path.SEPARATOR + ugi.getShortUserName();
         + FileSystem.TRASH_PREFIX + Path.SEPARATOR + ugi.getShortUserName();
   }
   }
+
+  /**
+   * Returns true if the name of snapshot is vlaid.
+   * @param snapshotName name of the snapshot.
+   * @return true if the name of snapshot is vlaid.
+   */
+  public static boolean isValidSnapshotName(String snapshotName) {
+    // If any of the snapshots specified in the getSnapshotDiffReport call
+    // is null or empty, it points to the current tree.
+    return (snapshotName != null && !snapshotName.isEmpty());
+  }
+
+  public static SnapshotDiffReport getSnapshotDiffReport(
+      String snapshotDir, String fromSnapshot, String toSnapshot,
+      SnapshotDiffReportFunction withoutListing,
+      SnapshotDiffReportListingFunction withListing) throws IOException {
+    // In case the diff needs to be computed between a snapshot and the current
+    // tree, we should not do iterative diffReport computation as the iterative
+    // approach might fail if in between the rpc calls the current tree
+    // changes in absence of the global fsn lock.
+    if (!isValidSnapshotName(fromSnapshot) || !isValidSnapshotName(toSnapshot)) {
+      return withoutListing.apply(snapshotDir, fromSnapshot, toSnapshot);
+    }
+    byte[] startPath = EMPTY_BYTES;
+    int index = -1;
+    SnapshotDiffReportGenerator snapshotDiffReport;
+    List<DiffReportListingEntry> modifiedList = new TreeList();
+    List<DiffReportListingEntry> createdList = new ChunkedArrayList<>();
+    List<DiffReportListingEntry> deletedList = new ChunkedArrayList<>();
+    SnapshotDiffReportListing report;
+    do {
+      try {
+        report = withListing.apply(snapshotDir, fromSnapshot, toSnapshot, startPath, index);
+      } catch (RpcNoSuchMethodException|UnsupportedOperationException e) {
+        // In case the server doesn't support getSnapshotDiffReportListing,
+        // fallback to getSnapshotDiffReport.
+        LOG.warn("Falling back to getSnapshotDiffReport {}", e.getMessage());
+        return withoutListing.apply(snapshotDir, fromSnapshot, toSnapshot);
+      }
+      startPath = report.getLastPath();
+      index = report.getLastIndex();
+      modifiedList.addAll(report.getModifyList());
+      createdList.addAll(report.getCreateList());
+      deletedList.addAll(report.getDeleteList());
+    } while (!(Arrays.equals(startPath, EMPTY_BYTES)
+        && index == -1));
+    snapshotDiffReport =
+        new SnapshotDiffReportGenerator(snapshotDir, fromSnapshot, toSnapshot,
+            report.getIsFromEarlier(), modifiedList, createdList, deletedList);
+    return snapshotDiffReport.generateReport();
+  }
+
+  @FunctionalInterface
+  public interface SnapshotDiffReportFunction {
+    SnapshotDiffReport apply(String snapshotDir, String fromSnapshot, String toSnapshot)
+        throws IOException;
+  }
+
+  @FunctionalInterface
+  public interface SnapshotDiffReportListingFunction {
+    SnapshotDiffReportListing apply(String snapshotDir, String fromSnapshot, String toSnapshot,
+        byte[] startPath, int index)
+        throws IOException;
+  }
 }
 }

+ 4 - 51
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -18,12 +18,9 @@
 
 
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
-
-import org.apache.hadoop.ipc.RpcNoSuchMethodException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.util.Preconditions;
-import org.apache.commons.collections.list.TreeList;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -107,8 +104,6 @@ import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 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.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -116,7 +111,6 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.DelegationTokenIssuer;
 import org.apache.hadoop.security.token.DelegationTokenIssuer;
-import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
@@ -2298,8 +2292,8 @@ public class DistributedFileSystem extends FileSystem
       @Override
       @Override
       public RemoteIterator<SnapshotDiffReportListing> doCall(final Path p)
       public RemoteIterator<SnapshotDiffReportListing> doCall(final Path p)
           throws IOException {
           throws IOException {
-        if (!isValidSnapshotName(fromSnapshot) || !isValidSnapshotName(
-            toSnapshot)) {
+        if (!DFSUtilClient.isValidSnapshotName(fromSnapshot) ||
+            !DFSUtilClient.isValidSnapshotName(toSnapshot)) {
           throw new UnsupportedOperationException("Remote Iterator is"
           throw new UnsupportedOperationException("Remote Iterator is"
               + "supported for snapshotDiffReport between two snapshots");
               + "supported for snapshotDiffReport between two snapshots");
         }
         }
@@ -2364,52 +2358,11 @@ public class DistributedFileSystem extends FileSystem
     }
     }
   }
   }
 
 
-  private boolean isValidSnapshotName(String snapshotName) {
-    // If any of the snapshots specified in the getSnapshotDiffReport call
-    // is null or empty, it points to the current tree.
-    return (snapshotName != null && !snapshotName.isEmpty());
-  }
-
   private SnapshotDiffReport getSnapshotDiffReportInternal(
   private SnapshotDiffReport getSnapshotDiffReportInternal(
       final String snapshotDir, final String fromSnapshot,
       final String snapshotDir, final String fromSnapshot,
       final String toSnapshot) throws IOException {
       final String toSnapshot) throws IOException {
-    // In case the diff needs to be computed between a snapshot and the current
-    // tree, we should not do iterative diffReport computation as the iterative
-    // approach might fail if in between the rpc calls the current tree
-    // changes in absence of the global fsn lock.
-    if (!isValidSnapshotName(fromSnapshot) || !isValidSnapshotName(
-        toSnapshot)) {
-      return dfs.getSnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot);
-    }
-    byte[] startPath = DFSUtilClient.EMPTY_BYTES;
-    int index = -1;
-    SnapshotDiffReportGenerator snapshotDiffReport;
-    List<DiffReportListingEntry> modifiedList = new TreeList();
-    List<DiffReportListingEntry> createdList = new ChunkedArrayList<>();
-    List<DiffReportListingEntry> deletedList = new ChunkedArrayList<>();
-    SnapshotDiffReportListing report;
-    do {
-      try {
-        report = dfs.getSnapshotDiffReportListing(snapshotDir, fromSnapshot,
-            toSnapshot, startPath, index);
-      } catch (RpcNoSuchMethodException e) {
-        // In case the server doesn't support getSnapshotDiffReportListing,
-        // fallback to getSnapshotDiffReport.
-        DFSClient.LOG.warn(
-            "Falling back to getSnapshotDiffReport {}", e.getMessage());
-        return dfs.getSnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot);
-      }
-      startPath = report.getLastPath();
-      index = report.getLastIndex();
-      modifiedList.addAll(report.getModifyList());
-      createdList.addAll(report.getCreateList());
-      deletedList.addAll(report.getDeleteList());
-    } while (!(Arrays.equals(startPath, DFSUtilClient.EMPTY_BYTES)
-        && index == -1));
-    snapshotDiffReport =
-        new SnapshotDiffReportGenerator(snapshotDir, fromSnapshot, toSnapshot,
-            report.getIsFromEarlier(), modifiedList, createdList, deletedList);
-    return snapshotDiffReport.generateReport();
+    return  DFSUtilClient.getSnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot,
+        dfs::getSnapshotDiffReport, dfs::getSnapshotDiffReportListing);
   }
   }
 
 
   /**
   /**

+ 57 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java

@@ -39,6 +39,8 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
@@ -245,6 +247,14 @@ public class JsonUtilClient {
     return new ExtendedBlock(blockPoolId, blockId, numBytes, generationStamp);
     return new ExtendedBlock(blockPoolId, blockId, numBytes, generationStamp);
   }
   }
 
 
+  static boolean getBoolean(Map<?, ?> m, String key, final boolean defaultValue) {
+    Object value = m.get(key);
+    if (value == null) {
+      return defaultValue;
+    }
+    return ((Boolean) value).booleanValue();
+  }
+
   static int getInt(Map<?, ?> m, String key, final int defaultValue) {
   static int getInt(Map<?, ?> m, String key, final int defaultValue) {
     Object value = m.get(key);
     Object value = m.get(key);
     if (value == null) {
     if (value == null) {
@@ -834,6 +844,53 @@ public class JsonUtilClient {
     return new SnapshotDiffReport.DiffReportEntry(type, sourcePath, targetPath);
     return new SnapshotDiffReport.DiffReportEntry(type, sourcePath, targetPath);
   }
   }
 
 
+  public static SnapshotDiffReportListing toSnapshotDiffReportListing(
+      final Map<?, ?> json) {
+    if (json == null) {
+      return null;
+    }
+
+    Map<?, ?> m =
+        (Map<?, ?>) json.get(SnapshotDiffReportListing.class.getSimpleName());
+    byte[] lastPath = DFSUtilClient.string2Bytes(getString(m, "lastPath", ""));
+    int lastIndex = getInt(m, "lastIndex", -1);
+    boolean isFromEarlier = getBoolean(m, "isFromEarlier", false);
+    List<DiffReportListingEntry> modifyList =
+        toDiffListingList(getList(m, "modifyList"));
+    List<DiffReportListingEntry> createList =
+        toDiffListingList(getList(m, "createList"));
+    List<DiffReportListingEntry> deleteList =
+        toDiffListingList(getList(m, "deleteList"));
+
+    return new SnapshotDiffReportListing(
+        lastPath, modifyList, createList, deleteList, lastIndex, isFromEarlier);
+  }
+
+  public static List<DiffReportListingEntry> toDiffListingList(List<?> objs) {
+    if (objs == null) {
+      return null;
+    }
+    List<DiffReportListingEntry> diffList = new ChunkedArrayList<>();
+    for (int i = 0; i < objs.size(); i++) {
+      diffList.add(toDiffReportListingEntry((Map<?, ?>) objs.get(i)));
+    }
+    return diffList;
+  }
+
+  private static DiffReportListingEntry toDiffReportListingEntry(
+      Map<?, ?> json) {
+    if (json == null) {
+      return null;
+    }
+    long dirId = getLong(json, "dirId", 0);
+    long fileId = getLong(json, "fileId", 0);
+    byte[] sourcePath = toByteArray(getString(json, "sourcePath", null));
+    byte[] targetPath = toByteArray(getString(json, "targetPath", null));
+    boolean isReference = getBoolean(json, "isReference", false);
+    return new DiffReportListingEntry(
+        dirId, fileId, sourcePath, isReference, targetPath);
+  }
+
   private static byte[] toByteArray(String str) {
   private static byte[] toByteArray(String str) {
     if (str == null) {
     if (str == null) {
       return null;
       return null;

+ 36 - 12
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -46,7 +46,6 @@ import java.util.Collection;
 import java.util.EnumSet;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Optional;
 import java.util.Set;
 import java.util.Set;
@@ -63,7 +62,6 @@ import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer;
 import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.CommonPathCapabilities;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.DelegationTokenRenewer;
 import org.apache.hadoop.fs.DelegationTokenRenewer;
@@ -78,7 +76,6 @@ import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
 import org.apache.hadoop.fs.MultipartUploaderBuilder;
 import org.apache.hadoop.fs.MultipartUploaderBuilder;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.QuotaUsage;
-import org.apache.hadoop.fs.PathCapabilities;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.impl.FileSystemMultipartUploaderBuilder;
 import org.apache.hadoop.fs.impl.FileSystemMultipartUploaderBuilder;
@@ -105,6 +102,7 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 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.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FileEncryptionInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FileEncryptionInfoProto;
@@ -1445,19 +1443,45 @@ public class WebHdfsFileSystem extends FileSystem
         new SnapshotNameParam(snapshotNewName)).run();
         new SnapshotNameParam(snapshotNewName)).run();
   }
   }
 
 
+  private SnapshotDiffReport getSnapshotDiffReport(
+      final String snapshotDir, final String fromSnapshot, final String toSnapshot)
+      throws IOException {
+    return new FsPathResponseRunner<SnapshotDiffReport>(
+        GetOpParam.Op.GETSNAPSHOTDIFF,
+        new Path(snapshotDir),
+        new OldSnapshotNameParam(fromSnapshot),
+        new SnapshotNameParam(toSnapshot)) {
+          @Override
+          SnapshotDiffReport decodeResponse(Map<?, ?> json) {
+            return JsonUtilClient.toSnapshotDiffReport(json);
+          }
+        }.run();
+  }
+
+  private SnapshotDiffReportListing getSnapshotDiffReportListing(
+        String snapshotDir, final String fromSnapshot, final String toSnapshot,
+        byte[] startPath, int index) throws IOException {
+    return new FsPathResponseRunner<SnapshotDiffReportListing>(
+        GetOpParam.Op.GETSNAPSHOTDIFFLISTING,
+        new Path(snapshotDir),
+        new OldSnapshotNameParam(fromSnapshot),
+        new SnapshotNameParam(toSnapshot),
+        new SnapshotDiffStartPathParam(DFSUtilClient.bytes2String(startPath)),
+        new SnapshotDiffIndexParam(index)) {
+          @Override
+          SnapshotDiffReportListing decodeResponse(Map<?, ?> json) {
+            return JsonUtilClient.toSnapshotDiffReportListing(json);
+          }
+        }.run();
+  }
+
   public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
   public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
       final String fromSnapshot, final String toSnapshot) throws IOException {
       final String fromSnapshot, final String toSnapshot) throws IOException {
     statistics.incrementReadOps(1);
     statistics.incrementReadOps(1);
     storageStatistics.incrementOpCounter(OpType.GET_SNAPSHOT_DIFF);
     storageStatistics.incrementOpCounter(OpType.GET_SNAPSHOT_DIFF);
-    final HttpOpParam.Op op = GetOpParam.Op.GETSNAPSHOTDIFF;
-    return new FsPathResponseRunner<SnapshotDiffReport>(op, snapshotDir,
-        new OldSnapshotNameParam(fromSnapshot),
-        new SnapshotNameParam(toSnapshot)) {
-      @Override
-      SnapshotDiffReport decodeResponse(Map<?, ?> json) {
-        return JsonUtilClient.toSnapshotDiffReport(json);
-      }
-    }.run();
+    return DFSUtilClient.getSnapshotDiffReport(
+        snapshotDir.toUri().getPath(), fromSnapshot, toSnapshot,
+        this::getSnapshotDiffReport, this::getSnapshotDiffReportListing);
   }
   }
 
 
   public SnapshottableDirectoryStatus[] getSnapshottableDirectoryList()
   public SnapshottableDirectoryStatus[] getSnapshottableDirectoryList()

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java

@@ -62,6 +62,7 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     LISTSTATUS_BATCH(false, HttpURLConnection.HTTP_OK),
     LISTSTATUS_BATCH(false, HttpURLConnection.HTTP_OK),
     GETSERVERDEFAULTS(false, HttpURLConnection.HTTP_OK),
     GETSERVERDEFAULTS(false, HttpURLConnection.HTTP_OK),
     GETSNAPSHOTDIFF(false, HttpURLConnection.HTTP_OK),
     GETSNAPSHOTDIFF(false, HttpURLConnection.HTTP_OK),
+    GETSNAPSHOTDIFFLISTING(false, HttpURLConnection.HTTP_OK),
     GETSNAPSHOTTABLEDIRECTORYLIST(false, HttpURLConnection.HTTP_OK),
     GETSNAPSHOTTABLEDIRECTORYLIST(false, HttpURLConnection.HTTP_OK),
     GETSNAPSHOTLIST(false, HttpURLConnection.HTTP_OK);
     GETSNAPSHOTLIST(false, HttpURLConnection.HTTP_OK);
 
 

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/SnapshotDiffIndexParam.java

@@ -0,0 +1,49 @@
+/**
+ * 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.web.resources;
+
+/** resuming index of snapshotDiffReportListing operation. */
+public class SnapshotDiffIndexParam extends IntegerParam {
+  /** Parameter name. */
+  public static final String NAME = "snapshotdiffindex";
+  /** Default parameter value. */
+  public static final String DEFAULT = "-1";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public SnapshotDiffIndexParam(final Integer value) {
+    super(DOMAIN, value, -1, null);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public SnapshotDiffIndexParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 40 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/SnapshotDiffStartPathParam.java

@@ -0,0 +1,40 @@
+/**
+ * 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.web.resources;
+
+/**
+ * The snapshot startPath parameter used by snapshotDiffReportListing.
+ */
+public class SnapshotDiffStartPathParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "snapshotdiffstartpath";
+
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  public SnapshotDiffStartPathParam(final String str) {
+    super(DOMAIN, str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

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

@@ -72,6 +72,8 @@ import org.apache.hadoop.hdfs.web.resources.RenameOptionSetParam;
 import org.apache.hadoop.hdfs.web.resources.RenewerParam;
 import org.apache.hadoop.hdfs.web.resources.RenewerParam;
 import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
 import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
 import org.apache.hadoop.hdfs.web.resources.SnapshotNameParam;
 import org.apache.hadoop.hdfs.web.resources.SnapshotNameParam;
+import org.apache.hadoop.hdfs.web.resources.SnapshotDiffStartPathParam;
+import org.apache.hadoop.hdfs.web.resources.SnapshotDiffIndexParam;
 import org.apache.hadoop.hdfs.web.resources.StartAfterParam;
 import org.apache.hadoop.hdfs.web.resources.StartAfterParam;
 import org.apache.hadoop.hdfs.web.resources.StoragePolicyParam;
 import org.apache.hadoop.hdfs.web.resources.StoragePolicyParam;
 import org.apache.hadoop.hdfs.web.resources.StorageSpaceQuotaParam;
 import org.apache.hadoop.hdfs.web.resources.StorageSpaceQuotaParam;
@@ -335,6 +337,8 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
       final FsActionParam fsAction,
       final FsActionParam fsAction,
       final SnapshotNameParam snapshotName,
       final SnapshotNameParam snapshotName,
       final OldSnapshotNameParam oldSnapshotName,
       final OldSnapshotNameParam oldSnapshotName,
+      final SnapshotDiffStartPathParam snapshotDiffStartPath,
+      final SnapshotDiffIndexParam snapshotDiffIndex,
       final TokenKindParam tokenKind,
       final TokenKindParam tokenKind,
       final TokenServiceParam tokenService,
       final TokenServiceParam tokenService,
       final NoRedirectParam noredirectParam,
       final NoRedirectParam noredirectParam,
@@ -383,6 +387,7 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
         return super.get(ugi, delegation, username, doAsUser, fullpath, op,
         return super.get(ugi, delegation, username, doAsUser, fullpath, op,
             offset, length, renewer, bufferSize, xattrNames, xattrEncoding,
             offset, length, renewer, bufferSize, xattrNames, xattrEncoding,
             excludeDatanodes, fsAction, snapshotName, oldSnapshotName,
             excludeDatanodes, fsAction, snapshotName, oldSnapshotName,
+            snapshotDiffStartPath, snapshotDiffIndex,
             tokenKind, tokenService, noredirectParam, startAfter);
             tokenKind, tokenService, noredirectParam, startAfter);
       }
       }
       default:
       default:

+ 23 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -85,6 +85,7 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 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.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
@@ -1037,6 +1038,10 @@ public class NamenodeWebHdfsMethods {
           final SnapshotNameParam snapshotName,
           final SnapshotNameParam snapshotName,
       @QueryParam(OldSnapshotNameParam.NAME) @DefaultValue(OldSnapshotNameParam.DEFAULT)
       @QueryParam(OldSnapshotNameParam.NAME) @DefaultValue(OldSnapshotNameParam.DEFAULT)
           final OldSnapshotNameParam oldSnapshotName,
           final OldSnapshotNameParam oldSnapshotName,
+      @QueryParam(SnapshotDiffStartPathParam.NAME) @DefaultValue(SnapshotDiffStartPathParam.DEFAULT)
+          final SnapshotDiffStartPathParam snapshotDiffStartPath,
+      @QueryParam(SnapshotDiffIndexParam.NAME) @DefaultValue(SnapshotDiffIndexParam.DEFAULT)
+          final SnapshotDiffIndexParam snapshotDiffIndex,
       @QueryParam(TokenKindParam.NAME) @DefaultValue(TokenKindParam.DEFAULT)
       @QueryParam(TokenKindParam.NAME) @DefaultValue(TokenKindParam.DEFAULT)
           final TokenKindParam tokenKind,
           final TokenKindParam tokenKind,
       @QueryParam(TokenServiceParam.NAME) @DefaultValue(TokenServiceParam.DEFAULT)
       @QueryParam(TokenServiceParam.NAME) @DefaultValue(TokenServiceParam.DEFAULT)
@@ -1048,7 +1053,9 @@ public class NamenodeWebHdfsMethods {
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
     return get(ugi, delegation, username, doAsUser, ROOT, op, offset, length,
     return get(ugi, delegation, username, doAsUser, ROOT, op, offset, length,
         renewer, bufferSize, xattrNames, xattrEncoding, excludeDatanodes,
         renewer, bufferSize, xattrNames, xattrEncoding, excludeDatanodes,
-        fsAction, snapshotName, oldSnapshotName, tokenKind, tokenService,
+        fsAction, snapshotName, oldSnapshotName,
+        snapshotDiffStartPath, snapshotDiffIndex,
+        tokenKind, tokenService,
         noredirect, startAfter);
         noredirect, startAfter);
   }
   }
 
 
@@ -1088,6 +1095,10 @@ public class NamenodeWebHdfsMethods {
           final SnapshotNameParam snapshotName,
           final SnapshotNameParam snapshotName,
       @QueryParam(OldSnapshotNameParam.NAME) @DefaultValue(OldSnapshotNameParam.DEFAULT)
       @QueryParam(OldSnapshotNameParam.NAME) @DefaultValue(OldSnapshotNameParam.DEFAULT)
           final OldSnapshotNameParam oldSnapshotName,
           final OldSnapshotNameParam oldSnapshotName,
+      @QueryParam(SnapshotDiffStartPathParam.NAME) @DefaultValue(SnapshotDiffStartPathParam.DEFAULT)
+          final SnapshotDiffStartPathParam snapshotDiffStartPath,
+      @QueryParam(SnapshotDiffIndexParam.NAME) @DefaultValue(SnapshotDiffIndexParam.DEFAULT)
+          final SnapshotDiffIndexParam snapshotDiffIndex,
       @QueryParam(TokenKindParam.NAME) @DefaultValue(TokenKindParam.DEFAULT)
       @QueryParam(TokenKindParam.NAME) @DefaultValue(TokenKindParam.DEFAULT)
           final TokenKindParam tokenKind,
           final TokenKindParam tokenKind,
       @QueryParam(TokenServiceParam.NAME) @DefaultValue(TokenServiceParam.DEFAULT)
       @QueryParam(TokenServiceParam.NAME) @DefaultValue(TokenServiceParam.DEFAULT)
@@ -1108,6 +1119,7 @@ public class NamenodeWebHdfsMethods {
         return get(ugi, delegation, username, doAsUser, path.getAbsolutePath(),
         return get(ugi, delegation, username, doAsUser, path.getAbsolutePath(),
             op, offset, length, renewer, bufferSize, xattrNames, xattrEncoding,
             op, offset, length, renewer, bufferSize, xattrNames, xattrEncoding,
             excludeDatanodes, fsAction, snapshotName, oldSnapshotName,
             excludeDatanodes, fsAction, snapshotName, oldSnapshotName,
+            snapshotDiffStartPath, snapshotDiffIndex,
             tokenKind, tokenService, noredirect, startAfter);
             tokenKind, tokenService, noredirect, startAfter);
       }
       }
     });
     });
@@ -1137,6 +1149,8 @@ public class NamenodeWebHdfsMethods {
       final FsActionParam fsAction,
       final FsActionParam fsAction,
       final SnapshotNameParam snapshotName,
       final SnapshotNameParam snapshotName,
       final OldSnapshotNameParam oldSnapshotName,
       final OldSnapshotNameParam oldSnapshotName,
+      final SnapshotDiffStartPathParam snapshotDiffStartPath,
+      final SnapshotDiffIndexParam snapshotDiffIndex,
       final TokenKindParam tokenKind,
       final TokenKindParam tokenKind,
       final TokenServiceParam tokenService,
       final TokenServiceParam tokenService,
       final NoRedirectParam noredirectParam,
       final NoRedirectParam noredirectParam,
@@ -1335,6 +1349,14 @@ public class NamenodeWebHdfsMethods {
       final String js = JsonUtil.toJsonString(diffReport);
       final String js = JsonUtil.toJsonString(diffReport);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     }
+    case GETSNAPSHOTDIFFLISTING: {
+      SnapshotDiffReportListing diffReport = cp.getSnapshotDiffReportListing(
+          fullpath, oldSnapshotName.getValue(), snapshotName.getValue(),
+          DFSUtilClient.string2Bytes(snapshotDiffStartPath.getValue()),
+          snapshotDiffIndex.getValue());
+      final String js = JsonUtil.toJsonString(diffReport);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
     case GETSNAPSHOTTABLEDIRECTORYLIST: {
     case GETSNAPSHOTTABLEDIRECTORYLIST: {
       SnapshottableDirectoryStatus[] snapshottableDirectoryList =
       SnapshottableDirectoryStatus[] snapshottableDirectoryList =
           cp.getSnapshottableDirListing();
           cp.getSnapshottableDirListing();

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

@@ -577,6 +577,58 @@ public class JsonUtil {
     return m;
     return m;
   }
   }
 
 
+  public static String toJsonString(SnapshotDiffReportListing diffReport) {
+    return toJsonString(SnapshotDiffReportListing.class.getSimpleName(),
+        toJsonMap(diffReport));
+  }
+
+  private static Object toJsonMap(SnapshotDiffReportListing diffReport) {
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put("lastPath", DFSUtilClient.bytes2String(diffReport.getLastPath()));
+    m.put("lastIndex", diffReport.getLastIndex());
+    m.put("isFromEarlier", diffReport.getIsFromEarlier());
+
+    Object[] modifyList = new Object[diffReport.getModifyList().size()];
+    for (int i = 0; i < diffReport.getModifyList().size(); i++) {
+      modifyList[i] = toJsonMap(diffReport.getModifyList().get(i));
+    }
+    m.put("modifyList", modifyList);
+
+    Object[] createList = new Object[diffReport.getCreateList().size()];
+    for (int i = 0; i < diffReport.getCreateList().size(); i++) {
+      createList[i] = toJsonMap(diffReport.getCreateList().get(i));
+    }
+    m.put("createList", createList);
+
+    Object[] deleteList = new Object[diffReport.getDeleteList().size()];
+    for (int i = 0; i < diffReport.getDeleteList().size(); i++) {
+      deleteList[i] = toJsonMap(diffReport.getDeleteList().get(i));
+    }
+    m.put("deleteList", deleteList);
+
+    return m;
+  }
+
+  private static Object toJsonMap(
+      SnapshotDiffReportListing.DiffReportListingEntry diffReportEntry) {
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put("dirId", diffReportEntry.getDirId());
+    m.put("fileId", diffReportEntry.getFileId());
+
+    if (diffReportEntry.getSourcePath() != null) {
+      m.put("sourcePath",
+          DFSUtilClient.byteArray2String(diffReportEntry.getSourcePath()));
+    }
+
+    if (diffReportEntry.getTargetPath() != null) {
+      m.put("targetPath",
+          DFSUtilClient.byteArray2String(diffReportEntry.getTargetPath()));
+    }
+
+    m.put("isReference", diffReportEntry.isReference());
+    return m;
+  }
+
   public static String toJsonString(
   public static String toJsonString(
       SnapshottableDirectoryStatus[] snapshottableDirectoryList) {
       SnapshottableDirectoryStatus[] snapshottableDirectoryList) {
     if (snapshottableDirectoryList == null) {
     if (snapshottableDirectoryList == null) {

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

@@ -21,8 +21,6 @@ import static org.apache.hadoop.fs.permission.AclEntryScope.*;
 import static org.apache.hadoop.fs.permission.AclEntryType.*;
 import static org.apache.hadoop.fs.permission.AclEntryType.*;
 import static org.apache.hadoop.fs.permission.FsAction.*;
 import static org.apache.hadoop.fs.permission.FsAction.*;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.*;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.*;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.EnumSet;
 import java.util.EnumSet;
@@ -48,8 +46,11 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus.Flags;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus.Flags;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 
 
@@ -128,7 +129,7 @@ public class TestJsonUtil {
         .isdir(true)
         .isdir(true)
         .build();
         .build();
 
 
-    assertFalse(hdfsFileStatus.isSymlink());
+    Assert.assertFalse(hdfsFileStatus.isSymlink());
     LambdaTestUtils.intercept(IOException.class,
     LambdaTestUtils.intercept(IOException.class,
         "Path " + hdfsFileStatus.getPath() + " is not a symbolic link",
         "Path " + hdfsFileStatus.getPath() + " is not a symbolic link",
         () -> hdfsFileStatus.getSymlink());
         () -> hdfsFileStatus.getSymlink());
@@ -150,7 +151,7 @@ public class TestJsonUtil {
         .append("}")
         .append("}")
         .toString();
         .toString();
 
 
-    assertEquals(expectString, hdfsFileStatus.toString());
+    Assert.assertEquals(expectString, hdfsFileStatus.toString());
   }
   }
 
 
   @Test
   @Test
@@ -391,6 +392,83 @@ public class TestJsonUtil {
     Assert.assertArrayEquals(XAttrCodec.decodeValue("0x313131"), value);
     Assert.assertArrayEquals(XAttrCodec.decodeValue("0x313131"), value);
   }
   }
 
 
+  @Test
+  public void testSnapshotDiffReportListingEmptyReport() throws IOException {
+    SnapshotDiffReportListing report = new SnapshotDiffReportListing();
+    String jsonString = JsonUtil.toJsonString(report);
+    Map<?, ?> json = READER.readValue(jsonString);
+    SnapshotDiffReportListing parsed =
+        JsonUtilClient.toSnapshotDiffReportListing(json);
+
+    assertEquals(report, parsed);
+  }
+
+  @Test
+  public void testSnapshotDiffReportListing() throws IOException {
+    List<DiffReportListingEntry> mlist = new ChunkedArrayList<>();
+    List<DiffReportListingEntry> clist = new ChunkedArrayList<>();
+    List<DiffReportListingEntry> dlist = new ChunkedArrayList<>();
+    clist.add(new DiffReportListingEntry(
+        1L, 2L, DFSUtilClient.string2Bytes("dir1/file2"), false, null));
+    clist.add(new DiffReportListingEntry(
+        1L, 3L, DFSUtilClient.string2Bytes("dir1/file3"), false, null));
+    dlist.add(new DiffReportListingEntry(
+        1L, 4L, DFSUtilClient.string2Bytes("dir1/file4"), false, null));
+    dlist.add(new DiffReportListingEntry(
+        1L, 5L,
+        DFSUtilClient.string2Bytes("dir1/file5"),
+        true,
+        DFSUtilClient.string2Bytes("dir1/file6")));
+
+    SnapshotDiffReportListing report =
+        new SnapshotDiffReportListing(
+            DFSUtilClient.string2Bytes("dir1/file2"), mlist, clist, dlist, 3, true);
+    String jsonString = JsonUtil.toJsonString(report);
+    Map<?, ?> json = READER.readValue(jsonString);
+    SnapshotDiffReportListing parsed =
+        JsonUtilClient.toSnapshotDiffReportListing(json);
+
+    assertEquals(report, parsed);
+  }
+
+  private void assertEquals(
+      SnapshotDiffReportListing expected, SnapshotDiffReportListing actual) {
+    Assert.assertEquals(expected.getLastIndex(), actual.getLastIndex());
+    Assert.assertEquals(expected.getIsFromEarlier(), actual.getIsFromEarlier());
+    assertEquals(expected.getModifyList(), actual.getModifyList());
+    assertEquals(expected.getCreateList(), actual.getCreateList());
+    assertEquals(expected.getDeleteList(), actual.getDeleteList());
+    Assert.assertArrayEquals(expected.getLastPath(), actual.getLastPath());
+  }
+
+  private void assertEquals(
+      List<DiffReportListingEntry> expected, List<DiffReportListingEntry> actual) {
+    Assert.assertEquals(expected.size(), actual.size());
+
+    for (int i = 0; i < expected.size(); i++) {
+      DiffReportListingEntry a = expected.get(i);
+      DiffReportListingEntry b = actual.get(i);
+
+      Assert.assertEquals(a.getFileId(), b.getFileId());
+      Assert.assertEquals(a.getDirId(), b.getDirId());
+      Assert.assertEquals(a.isReference(), b.isReference());
+      if (a.getSourcePath() != null) {
+        Assert.assertArrayEquals(
+            DFSUtilClient.byteArray2bytes(a.getSourcePath()),
+            DFSUtilClient.byteArray2bytes(b.getSourcePath()));
+      } else {
+        Assert.assertArrayEquals(a.getSourcePath(), b.getSourcePath());
+      }
+      if (a.getTargetPath() != null) {
+        Assert.assertArrayEquals(
+            DFSUtilClient.byteArray2bytes(a.getTargetPath()),
+            DFSUtilClient.byteArray2bytes(b.getTargetPath()));
+      } else {
+        Assert.assertArrayEquals(a.getTargetPath(), b.getTargetPath());
+      }
+    }
+  }
+
   private void checkDecodeFailure(Map<String, Object> map) {
   private void checkDecodeFailure(Map<String, Object> map) {
     try {
     try {
       JsonUtilClient.toDatanodeInfo(map);
       JsonUtilClient.toDatanodeInfo(map);

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java

@@ -23,6 +23,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.TestDistributedFileSystem.checkOpStatistics;
 import static org.apache.hadoop.hdfs.TestDistributedFileSystem.checkOpStatistics;
 import static org.apache.hadoop.hdfs.TestDistributedFileSystem.checkStatistics;
 import static org.apache.hadoop.hdfs.TestDistributedFileSystem.checkStatistics;
@@ -723,6 +724,7 @@ public class TestWebHDFS {
   @Test
   @Test
   public void testWebHdfsSnapshotDiff() throws Exception {
   public void testWebHdfsSnapshotDiff() throws Exception {
     final Configuration conf = WebHdfsTestUtil.createConf();
     final Configuration conf = WebHdfsTestUtil.createConf();
+    conf.setInt(DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT, 2);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     cluster.waitActive();
     cluster.waitActive();
     final DistributedFileSystem dfs = cluster.getFileSystem();
     final DistributedFileSystem dfs = cluster.getFileSystem();