|
@@ -42,6 +42,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.AppendTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
+import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
+import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
@@ -50,6 +52,7 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
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;
|
|
@@ -1198,7 +1201,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
ALLOW_SNAPSHOT, DISALLOW_SNAPSHOT, DISALLOW_SNAPSHOT_EXCEPTION,
|
|
|
FILE_STATUS_ATTR, GET_SNAPSHOT_DIFF, GET_SNAPSHOTTABLE_DIRECTORY_LIST,
|
|
|
GET_SNAPSHOT_LIST, GET_SERVERDEFAULTS, CHECKACCESS, SETECPOLICY,
|
|
|
- SATISFYSTORAGEPOLICY
|
|
|
+ SATISFYSTORAGEPOLICY, GET_SNAPSHOT_DIFF_LISTING
|
|
|
}
|
|
|
|
|
|
private void operation(Operation op) throws Exception {
|
|
@@ -1335,6 +1338,9 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
case SATISFYSTORAGEPOLICY:
|
|
|
testStoragePolicySatisfier();
|
|
|
break;
|
|
|
+ case GET_SNAPSHOT_DIFF_LISTING:
|
|
|
+ testGetSnapshotDiffListing();
|
|
|
+ break;
|
|
|
}
|
|
|
|
|
|
}
|
|
@@ -1607,29 +1613,30 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
Path file2 = new Path(path, "file2");
|
|
|
testCreate(file2, false);
|
|
|
fs.createSnapshot(path, "snap2");
|
|
|
- // Get snapshot diff
|
|
|
- SnapshotDiffReport diffReport = null;
|
|
|
- if (fs instanceof HttpFSFileSystem) {
|
|
|
- HttpFSFileSystem httpFS = (HttpFSFileSystem) fs;
|
|
|
- diffReport = httpFS.getSnapshotDiffReport(path, "snap1", "snap2");
|
|
|
- } else if (fs instanceof WebHdfsFileSystem) {
|
|
|
- WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs;
|
|
|
- diffReport = webHdfsFileSystem.getSnapshotDiffReport(path,
|
|
|
- "snap1", "snap2");
|
|
|
- } else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() +
|
|
|
- " doesn't support getSnapshotDiff");
|
|
|
+
|
|
|
+ try {
|
|
|
+ // Get snapshot diff
|
|
|
+ SnapshotDiffReport diffReport = null;
|
|
|
+ if (fs instanceof HttpFSFileSystem) {
|
|
|
+ HttpFSFileSystem httpFS = (HttpFSFileSystem) fs;
|
|
|
+ diffReport = httpFS.getSnapshotDiffReport(path, "snap1", "snap2");
|
|
|
+ } else if (fs instanceof WebHdfsFileSystem) {
|
|
|
+ WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs;
|
|
|
+ diffReport = webHdfsFileSystem.getSnapshotDiffReport(path, "snap1", "snap2");
|
|
|
+ } else {
|
|
|
+ Assert.fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff");
|
|
|
+ }
|
|
|
+ // Verify result with DFS
|
|
|
+ DistributedFileSystem dfs =
|
|
|
+ (DistributedFileSystem) FileSystem.get(path.toUri(), this.getProxiedFSConf());
|
|
|
+ SnapshotDiffReport dfsDiffReport = dfs.getSnapshotDiffReport(path, "snap1", "snap2");
|
|
|
+ Assert.assertEquals(diffReport.toString(), dfsDiffReport.toString());
|
|
|
+ } finally {
|
|
|
+ // Cleanup
|
|
|
+ fs.deleteSnapshot(path, "snap2");
|
|
|
+ fs.deleteSnapshot(path, "snap1");
|
|
|
+ fs.delete(path, true);
|
|
|
}
|
|
|
- // Verify result with DFS
|
|
|
- DistributedFileSystem dfs = (DistributedFileSystem)
|
|
|
- FileSystem.get(path.toUri(), this.getProxiedFSConf());
|
|
|
- SnapshotDiffReport dfsDiffReport =
|
|
|
- dfs.getSnapshotDiffReport(path, "snap1", "snap2");
|
|
|
- Assert.assertEquals(diffReport.toString(), dfsDiffReport.toString());
|
|
|
- // Cleanup
|
|
|
- fs.deleteSnapshot(path, "snap2");
|
|
|
- fs.deleteSnapshot(path, "snap1");
|
|
|
- fs.delete(path, true);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1951,4 +1958,102 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
dfs.delete(path1, true);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private void testGetSnapshotDiffListing() throws Exception {
|
|
|
+ if (!this.isLocalFS()) {
|
|
|
+ // Create a directory with snapshot allowed
|
|
|
+ Path path = new Path("/tmp/tmp-snap-test");
|
|
|
+ createSnapshotTestsPreconditions(path);
|
|
|
+ // Get the FileSystem instance that's being tested
|
|
|
+ FileSystem fs = this.getHttpFSFileSystem();
|
|
|
+ // Check FileStatus
|
|
|
+ Assert.assertTrue(fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ // Create a file and take a snapshot
|
|
|
+ Path file1 = new Path(path, "file1");
|
|
|
+ testCreate(file1, false);
|
|
|
+ fs.createSnapshot(path, "snap1");
|
|
|
+ // Create another file and take a snapshot
|
|
|
+ Path file2 = new Path(path, "file2");
|
|
|
+ testCreate(file2, false);
|
|
|
+ fs.createSnapshot(path, "snap2");
|
|
|
+ // Get snapshot diff listing
|
|
|
+ try {
|
|
|
+ SnapshotDiffReportListing diffReportListing = null;
|
|
|
+ byte[] emptyBytes = new byte[] {};
|
|
|
+ if (fs instanceof HttpFSFileSystem) {
|
|
|
+ HttpFSFileSystem httpFS = (HttpFSFileSystem) fs;
|
|
|
+ diffReportListing =
|
|
|
+ httpFS.getSnapshotDiffReportListing(path, "snap1", "snap2", emptyBytes, -1);
|
|
|
+ } else if (fs instanceof WebHdfsFileSystem) {
|
|
|
+ WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs;
|
|
|
+ diffReportListing = webHdfsFileSystem
|
|
|
+ .getSnapshotDiffReportListing(path.toUri().getPath(), "snap1", "snap2", emptyBytes,
|
|
|
+ -1);
|
|
|
+ } else {
|
|
|
+ Assert.fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff");
|
|
|
+ }
|
|
|
+ // Verify result with DFS
|
|
|
+ DistributedFileSystem dfs =
|
|
|
+ (DistributedFileSystem) FileSystem.get(path.toUri(), this.getProxiedFSConf());
|
|
|
+ SnapshotDiffReportListing dfsDiffReportListing =
|
|
|
+ dfs.getSnapshotDiffReportListing(path, "snap1", "snap2",
|
|
|
+ DFSUtil.bytes2String(emptyBytes), -1);
|
|
|
+ assertHttpFsReportListingWithDfsClient(diffReportListing, dfsDiffReportListing);
|
|
|
+ } finally {
|
|
|
+ // Cleanup
|
|
|
+ fs.deleteSnapshot(path, "snap2");
|
|
|
+ fs.deleteSnapshot(path, "snap1");
|
|
|
+ fs.delete(path, true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void assertHttpFsReportListingWithDfsClient(SnapshotDiffReportListing diffReportListing,
|
|
|
+ SnapshotDiffReportListing dfsDiffReportListing) {
|
|
|
+ Assert.assertEquals(diffReportListing.getCreateList().size(),
|
|
|
+ dfsDiffReportListing.getCreateList().size());
|
|
|
+ Assert.assertEquals(diffReportListing.getDeleteList().size(),
|
|
|
+ dfsDiffReportListing.getDeleteList().size());
|
|
|
+ Assert.assertEquals(diffReportListing.getModifyList().size(),
|
|
|
+ dfsDiffReportListing.getModifyList().size());
|
|
|
+ Assert.assertEquals(diffReportListing.getIsFromEarlier(),
|
|
|
+ dfsDiffReportListing.getIsFromEarlier());
|
|
|
+ Assert.assertEquals(diffReportListing.getLastIndex(), dfsDiffReportListing.getLastIndex());
|
|
|
+ Assert.assertEquals(DFSUtil.bytes2String(diffReportListing.getLastPath()),
|
|
|
+ DFSUtil.bytes2String(dfsDiffReportListing.getLastPath()));
|
|
|
+ int i = 0;
|
|
|
+ for (SnapshotDiffReportListing.DiffReportListingEntry entry : diffReportListing
|
|
|
+ .getCreateList()) {
|
|
|
+ SnapshotDiffReportListing.DiffReportListingEntry dfsDiffEntry =
|
|
|
+ dfsDiffReportListing.getCreateList().get(i);
|
|
|
+ Assert.assertEquals(entry.getDirId(), dfsDiffEntry.getDirId());
|
|
|
+ Assert.assertEquals(entry.getFileId(), dfsDiffEntry.getFileId());
|
|
|
+ Assert.assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()),
|
|
|
+ DFSUtilClient.byteArray2bytes(dfsDiffEntry.getSourcePath()));
|
|
|
+ i++;
|
|
|
+ }
|
|
|
+ i = 0;
|
|
|
+ for (SnapshotDiffReportListing.DiffReportListingEntry entry : diffReportListing
|
|
|
+ .getDeleteList()) {
|
|
|
+ SnapshotDiffReportListing.DiffReportListingEntry dfsDiffEntry =
|
|
|
+ dfsDiffReportListing.getDeleteList().get(i);
|
|
|
+ Assert.assertEquals(entry.getDirId(), dfsDiffEntry.getDirId());
|
|
|
+ Assert.assertEquals(entry.getFileId(), dfsDiffEntry.getFileId());
|
|
|
+ Assert.assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()),
|
|
|
+ DFSUtilClient.byteArray2bytes(dfsDiffEntry.getSourcePath()));
|
|
|
+ i++;
|
|
|
+ }
|
|
|
+ i = 0;
|
|
|
+ for (SnapshotDiffReportListing.DiffReportListingEntry entry : diffReportListing
|
|
|
+ .getModifyList()) {
|
|
|
+ SnapshotDiffReportListing.DiffReportListingEntry dfsDiffEntry =
|
|
|
+ dfsDiffReportListing.getModifyList().get(i);
|
|
|
+ Assert.assertEquals(entry.getDirId(), dfsDiffEntry.getDirId());
|
|
|
+ Assert.assertEquals(entry.getFileId(), dfsDiffEntry.getFileId());
|
|
|
+ Assert.assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()),
|
|
|
+ DFSUtilClient.byteArray2bytes(dfsDiffEntry.getSourcePath()));
|
|
|
+ i++;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
}
|