|
@@ -20,6 +20,7 @@ package org.apache.hadoop.tools;
|
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
@@ -27,6 +28,9 @@ 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.hdfs.web.WebHdfsConstants;
|
|
|
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
|
|
|
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.SequenceFile;
|
|
|
import org.apache.hadoop.io.Text;
|
|
@@ -39,7 +43,6 @@ import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
-
|
|
|
import java.io.IOException;
|
|
|
import java.io.FileWriter;
|
|
|
import java.io.BufferedWriter;
|
|
@@ -48,12 +51,14 @@ import java.util.Arrays;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
+import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
|
public class TestDistCpSync {
|
|
|
private MiniDFSCluster cluster;
|
|
|
private final Configuration conf = new HdfsConfiguration();
|
|
|
private DistributedFileSystem dfs;
|
|
|
+ private WebHdfsFileSystem webfs;
|
|
|
private DistCpContext context;
|
|
|
private final Path source = new Path("/source");
|
|
|
private final Path target = new Path("/target");
|
|
@@ -65,6 +70,9 @@ public class TestDistCpSync {
|
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATA_NUM).build();
|
|
|
cluster.waitActive();
|
|
|
|
|
|
+ webfs = WebHdfsTestUtil.
|
|
|
+ getWebHdfsFileSystem(conf, WebHdfsConstants.WEBHDFS_SCHEME);
|
|
|
+
|
|
|
dfs = cluster.getFileSystem();
|
|
|
dfs.mkdirs(source);
|
|
|
dfs.mkdirs(target);
|
|
@@ -160,6 +168,10 @@ public class TestDistCpSync {
|
|
|
* f3 f4
|
|
|
*/
|
|
|
private void initData(Path dir) throws Exception {
|
|
|
+ initData(dfs, dir);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void initData(FileSystem fs, Path dir) throws Exception {
|
|
|
final Path foo = new Path(dir, "foo");
|
|
|
final Path bar = new Path(dir, "bar");
|
|
|
final Path d1 = new Path(foo, "d1");
|
|
@@ -169,10 +181,10 @@ public class TestDistCpSync {
|
|
|
final Path f3 = new Path(d1, "f3");
|
|
|
final Path f4 = new Path(d2, "f4");
|
|
|
|
|
|
- DFSTestUtil.createFile(dfs, f1, BLOCK_SIZE, DATA_NUM, 0);
|
|
|
- DFSTestUtil.createFile(dfs, f2, BLOCK_SIZE, DATA_NUM, 0);
|
|
|
- DFSTestUtil.createFile(dfs, f3, BLOCK_SIZE, DATA_NUM, 0);
|
|
|
- DFSTestUtil.createFile(dfs, f4, BLOCK_SIZE, DATA_NUM, 0);
|
|
|
+ DFSTestUtil.createFile(fs, f1, BLOCK_SIZE, DATA_NUM, 0);
|
|
|
+ DFSTestUtil.createFile(fs, f2, BLOCK_SIZE, DATA_NUM, 0);
|
|
|
+ DFSTestUtil.createFile(fs, f3, BLOCK_SIZE, DATA_NUM, 0);
|
|
|
+ DFSTestUtil.createFile(fs, f4, BLOCK_SIZE, DATA_NUM, 0);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -192,7 +204,7 @@ public class TestDistCpSync {
|
|
|
* foo/ f4
|
|
|
* f1(new)
|
|
|
*/
|
|
|
- private int changeData(Path dir) throws Exception {
|
|
|
+ private int changeData(FileSystem fs, Path dir) throws Exception {
|
|
|
final Path foo = new Path(dir, "foo");
|
|
|
final Path bar = new Path(dir, "bar");
|
|
|
final Path d1 = new Path(foo, "d1");
|
|
@@ -200,21 +212,21 @@ public class TestDistCpSync {
|
|
|
|
|
|
final Path bar_d1 = new Path(bar, "d1");
|
|
|
int numCreatedModified = 0;
|
|
|
- dfs.rename(d1, bar_d1);
|
|
|
+ fs.rename(d1, bar_d1);
|
|
|
numCreatedModified += 1; // modify ./foo
|
|
|
numCreatedModified += 1; // modify ./bar
|
|
|
final Path f3 = new Path(bar_d1, "f3");
|
|
|
- dfs.delete(f3, true);
|
|
|
+ fs.delete(f3, true);
|
|
|
final Path newfoo = new Path(bar_d1, "foo");
|
|
|
- dfs.rename(foo, newfoo);
|
|
|
+ fs.rename(foo, newfoo);
|
|
|
numCreatedModified += 1; // modify ./foo/d1
|
|
|
final Path f1 = new Path(newfoo, "f1");
|
|
|
- dfs.delete(f1, true);
|
|
|
- DFSTestUtil.createFile(dfs, f1, 2 * BLOCK_SIZE, DATA_NUM, 0);
|
|
|
+ fs.delete(f1, true);
|
|
|
+ DFSTestUtil.createFile(fs, f1, 2 * BLOCK_SIZE, DATA_NUM, 0);
|
|
|
numCreatedModified += 1; // create ./foo/f1
|
|
|
- DFSTestUtil.appendFile(dfs, f2, (int) BLOCK_SIZE);
|
|
|
+ DFSTestUtil.appendFile(fs, f2, (int) BLOCK_SIZE);
|
|
|
numCreatedModified += 1; // modify ./bar/f2
|
|
|
- dfs.rename(bar, new Path(dir, "foo"));
|
|
|
+ fs.rename(bar, new Path(dir, "foo"));
|
|
|
return numCreatedModified;
|
|
|
}
|
|
|
|
|
@@ -228,7 +240,7 @@ public class TestDistCpSync {
|
|
|
enableAndCreateFirstSnapshot();
|
|
|
|
|
|
// make changes under source
|
|
|
- int numCreatedModified = changeData(source);
|
|
|
+ int numCreatedModified = changeData(dfs, source);
|
|
|
dfs.createSnapshot(source, "s2");
|
|
|
|
|
|
// before sync, make some further changes on source. this should not affect
|
|
@@ -295,23 +307,51 @@ public class TestDistCpSync {
|
|
|
return values;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * By default, we are using DFS for both source and target.
|
|
|
+ * @param s source file status
|
|
|
+ * @param t target file status
|
|
|
+ * @param compareName whether will we compare the name of the files
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
private void verifyCopy(FileStatus s, FileStatus t, boolean compareName)
|
|
|
- throws Exception {
|
|
|
+ throws Exception {
|
|
|
+ verifyCopy(dfs, dfs, s, t, compareName);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Verify copy by using different file systems.
|
|
|
+ * @param sfs source file system
|
|
|
+ * @param tfs target file system
|
|
|
+ * @param s source file status
|
|
|
+ * @param t target file status
|
|
|
+ * @param compareName whether will we compare the name of the files
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ private void verifyCopyByFs(FileSystem sfs, FileSystem tfs,
|
|
|
+ FileStatus s, FileStatus t, boolean compareName)
|
|
|
+ throws Exception {
|
|
|
+ verifyCopy(sfs, tfs, s, t, compareName);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void verifyCopy(FileSystem sfs, FileSystem tfs,
|
|
|
+ FileStatus s, FileStatus t, boolean compareName)
|
|
|
+ throws Exception {
|
|
|
Assert.assertEquals(s.isDirectory(), t.isDirectory());
|
|
|
if (compareName) {
|
|
|
Assert.assertEquals(s.getPath().getName(), t.getPath().getName());
|
|
|
}
|
|
|
if (!s.isDirectory()) {
|
|
|
// verify the file content is the same
|
|
|
- byte[] sbytes = DFSTestUtil.readFileBuffer(dfs, s.getPath());
|
|
|
- byte[] tbytes = DFSTestUtil.readFileBuffer(dfs, t.getPath());
|
|
|
+ byte[] sbytes = DFSTestUtil.readFileBuffer(sfs, s.getPath());
|
|
|
+ byte[] tbytes = DFSTestUtil.readFileBuffer(tfs, t.getPath());
|
|
|
Assert.assertArrayEquals(sbytes, tbytes);
|
|
|
} else {
|
|
|
- FileStatus[] slist = dfs.listStatus(s.getPath());
|
|
|
- FileStatus[] tlist = dfs.listStatus(t.getPath());
|
|
|
+ FileStatus[] slist = sfs.listStatus(s.getPath());
|
|
|
+ FileStatus[] tlist = tfs.listStatus(t.getPath());
|
|
|
Assert.assertEquals(slist.length, tlist.length);
|
|
|
for (int i = 0; i < slist.length; i++) {
|
|
|
- verifyCopy(slist[i], tlist[i], true);
|
|
|
+ verifyCopy(sfs, tfs, slist[i], tlist[i], true);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -333,7 +373,7 @@ public class TestDistCpSync {
|
|
|
enableAndCreateFirstSnapshot();
|
|
|
|
|
|
// make changes under source
|
|
|
- changeData(source);
|
|
|
+ changeData(dfs, source);
|
|
|
|
|
|
// do the sync
|
|
|
sync();
|
|
@@ -907,4 +947,84 @@ public class TestDistCpSync {
|
|
|
deleteFilterFile(filterFile);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test DistCp ues diff option under (s)WebHDFSFileSyste.
|
|
|
+ * In this test, we are using DFS as source and WebHDFS as target
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testSyncSnapshotDiffWithWebHdfs1() throws Exception {
|
|
|
+ Path dfsSource = new Path(dfs.getUri().toString(), source);
|
|
|
+ Path webHdfsTarget = new Path(webfs.getUri().toString(), target);
|
|
|
+
|
|
|
+ snapshotDiffWithPaths(dfsSource, webHdfsTarget);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test DistCp ues diff option under (s)WebHDFSFileSyste.
|
|
|
+ * In this test, we are using WebHDFS as source and DFS as target
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testSyncSnapshotDiffWithWebHdfs2() throws Exception {
|
|
|
+ Path webHdfsSource = new Path(webfs.getUri().toString(), source);
|
|
|
+ Path dfsTarget = new Path(dfs.getUri().toString(), target);
|
|
|
+
|
|
|
+ snapshotDiffWithPaths(webHdfsSource, dfsTarget);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test DistCp ues diff option under (s)WebHDFSFileSyste.
|
|
|
+ * In this test, we are using WebHDFS for both source and target
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testSyncSnapshotDiffWithWebHdfs3() throws Exception {
|
|
|
+ Path webHdfsSource = new Path(webfs.getUri().toString(), source);
|
|
|
+ Path webHdfsTarget = new Path(webfs.getUri().toString(), target);
|
|
|
+
|
|
|
+ snapshotDiffWithPaths(webHdfsSource, webHdfsTarget);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void snapshotDiffWithPaths(Path sourceFSPath,
|
|
|
+ Path targetFSPath) throws Exception {
|
|
|
+
|
|
|
+ FileSystem sourceFS = sourceFSPath.getFileSystem(conf);
|
|
|
+ FileSystem targetFS = targetFSPath.getFileSystem(conf);
|
|
|
+
|
|
|
+ // Initialize both source and target file system
|
|
|
+ initData(sourceFS, sourceFSPath);
|
|
|
+ initData(targetFS, targetFSPath);
|
|
|
+
|
|
|
+ // create snapshots on both source and target side with the same name
|
|
|
+ List<Path> paths = Arrays.asList(sourceFSPath, targetFSPath);
|
|
|
+ for (Path path: paths) {
|
|
|
+ FileSystem fs = path.getFileSystem(conf);
|
|
|
+ if (fs instanceof DistributedFileSystem) {
|
|
|
+ ((DistributedFileSystem)fs).allowSnapshot(path);
|
|
|
+ } else if (fs instanceof WebHdfsFileSystem) {
|
|
|
+ ((WebHdfsFileSystem)fs).allowSnapshot(path);
|
|
|
+ } else {
|
|
|
+ throw new IOException("Unsupported fs: " + fs.getScheme());
|
|
|
+ }
|
|
|
+ fs.createSnapshot(path, "s1");
|
|
|
+ }
|
|
|
+
|
|
|
+ // do some modification on source side
|
|
|
+ changeData(sourceFS, sourceFSPath);
|
|
|
+
|
|
|
+ // create a new snapshot on source side
|
|
|
+ sourceFS.createSnapshot(sourceFSPath, "s2");
|
|
|
+
|
|
|
+ //try to copy the difference
|
|
|
+ final DistCpOptions options = new DistCpOptions.Builder(
|
|
|
+ Collections.singletonList(sourceFSPath), targetFSPath)
|
|
|
+ .withUseDiff("s1", "s2")
|
|
|
+ .withSyncFolder(true)
|
|
|
+ .build();
|
|
|
+ options.appendToConf(conf);
|
|
|
+
|
|
|
+ new DistCp(conf, options).execute();
|
|
|
+
|
|
|
+ verifyCopyByFs(sourceFS, targetFS, sourceFS.getFileStatus(sourceFSPath),
|
|
|
+ targetFS.getFileStatus(targetFSPath), false);
|
|
|
+ }
|
|
|
}
|