|
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.SequenceFile;
|
|
@@ -97,6 +98,8 @@ public class TestDistCpSync {
|
|
|
dfs.createSnapshot(source, "s2");
|
|
|
dfs.createSnapshot(target, "s1");
|
|
|
Assert.assertTrue(DistCpSync.sync(options, conf));
|
|
|
+ // reset source paths in options
|
|
|
+ options.setSourcePaths(Arrays.asList(source));
|
|
|
|
|
|
// changes have been made in target
|
|
|
final Path subTarget = new Path(target, "sub");
|
|
@@ -183,9 +186,21 @@ public class TestDistCpSync {
|
|
|
changeData(source);
|
|
|
dfs.createSnapshot(source, "s2");
|
|
|
|
|
|
+ // before sync, make some further changes on source. this should not affect
|
|
|
+ // the later distcp since we're copying (s2-s1) to target
|
|
|
+ final Path toDelete = new Path(source, "foo/d1/foo/f1");
|
|
|
+ dfs.delete(toDelete, true);
|
|
|
+ final Path newdir = new Path(source, "foo/d1/foo/newdir");
|
|
|
+ dfs.mkdirs(newdir);
|
|
|
+
|
|
|
// do the sync
|
|
|
Assert.assertTrue(DistCpSync.sync(options, conf));
|
|
|
|
|
|
+ // make sure the source path has been updated to the snapshot path
|
|
|
+ final Path spath = new Path(source,
|
|
|
+ HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s2");
|
|
|
+ Assert.assertEquals(spath, options.getSourcePaths().get(0));
|
|
|
+
|
|
|
// build copy listing
|
|
|
final Path listingPath = new Path("/tmp/META/fileList.seq");
|
|
|
CopyListing listing = new GlobbedCopyListing(conf, new Credentials());
|
|
@@ -209,7 +224,7 @@ public class TestDistCpSync {
|
|
|
.getCounter(CopyMapper.Counter.BYTESCOPIED).getValue());
|
|
|
|
|
|
// verify the source and target now has the same structure
|
|
|
- verifyCopy(dfs.getFileStatus(source), dfs.getFileStatus(target), false);
|
|
|
+ verifyCopy(dfs.getFileStatus(spath), dfs.getFileStatus(target), false);
|
|
|
}
|
|
|
|
|
|
private Map<Text, CopyListingFileStatus> getListing(Path listingPath)
|
|
@@ -248,6 +263,29 @@ public class TestDistCpSync {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Similar test with testSync, but the "to" snapshot is specified as "."
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testSyncWithCurrent() throws Exception {
|
|
|
+ options.setUseDiff(true, "s1", ".");
|
|
|
+ initData(source);
|
|
|
+ initData(target);
|
|
|
+ dfs.allowSnapshot(source);
|
|
|
+ dfs.allowSnapshot(target);
|
|
|
+ dfs.createSnapshot(source, "s1");
|
|
|
+ dfs.createSnapshot(target, "s1");
|
|
|
+
|
|
|
+ // make changes under source
|
|
|
+ changeData(source);
|
|
|
+
|
|
|
+ // do the sync
|
|
|
+ Assert.assertTrue(DistCpSync.sync(options, conf));
|
|
|
+ // make sure the source path is still unchanged
|
|
|
+ Assert.assertEquals(source, options.getSourcePaths().get(0));
|
|
|
+ }
|
|
|
+
|
|
|
private void initData2(Path dir) throws Exception {
|
|
|
final Path test = new Path(dir, "test");
|
|
|
final Path foo = new Path(dir, "foo");
|