HDFS-10313. Distcp need to enforce the order of snapshot names passed to -diff. (Lin Yiqun via Yongjun Zhang)
This commit is contained in:
parent
6be22ddbf1
commit
959a28dd12
|
@ -268,6 +268,10 @@ public abstract class CopyListing extends Configured {
|
||||||
public InvalidInputException(String message) {
|
public InvalidInputException(String message) {
|
||||||
super(message);
|
super(message);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public InvalidInputException(String message, Throwable cause) {
|
||||||
|
super(message, cause);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class AclsNotSupportedException extends RuntimeException {
|
public static class AclsNotSupportedException extends RuntimeException {
|
||||||
|
|
|
@ -179,7 +179,8 @@ public class DistCp extends Configured implements Tool {
|
||||||
if (distCpSync.sync()) {
|
if (distCpSync.sync()) {
|
||||||
createInputFileListingWithDiff(job, distCpSync);
|
createInputFileListingWithDiff(job, distCpSync);
|
||||||
} else {
|
} else {
|
||||||
inputOptions.disableUsingDiff();
|
throw new Exception("DistCp sync failed, input options: "
|
||||||
|
+ inputOptions);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -17,14 +17,18 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.tools;
|
package org.apache.hadoop.tools;
|
||||||
|
|
||||||
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
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.DFSUtilClient;
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
||||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
||||||
|
import org.apache.hadoop.tools.CopyListing.InvalidInputException;
|
||||||
|
|
||||||
|
import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -91,6 +95,28 @@ class DistCpSync {
|
||||||
inputOptions.getToSnapshot())));
|
inputOptions.getToSnapshot())));
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final String from = getSnapshotName(inputOptions.getFromSnapshot());
|
||||||
|
final String to = getSnapshotName(inputOptions.getToSnapshot());
|
||||||
|
|
||||||
|
try {
|
||||||
|
final FileStatus fromSnapshotStat =
|
||||||
|
sfs.getFileStatus(getSourceSnapshotPath(sourceDir, from));
|
||||||
|
|
||||||
|
final FileStatus toSnapshotStat =
|
||||||
|
sfs.getFileStatus(getSourceSnapshotPath(sourceDir, to));
|
||||||
|
|
||||||
|
// If toSnapshot isn't current dir then do a time check
|
||||||
|
if (!to.equals("")
|
||||||
|
&& fromSnapshotStat.getModificationTime() > toSnapshotStat
|
||||||
|
.getModificationTime()) {
|
||||||
|
throw new HadoopIllegalArgumentException("Snapshot " + to
|
||||||
|
+ " should be newer than " + from);
|
||||||
|
}
|
||||||
|
} catch (FileNotFoundException nfe) {
|
||||||
|
throw new InvalidInputException("Input snapshot is not found", nfe);
|
||||||
|
}
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.tools;
|
package org.apache.hadoop.tools;
|
||||||
|
|
||||||
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
@ -31,6 +32,7 @@ import org.apache.hadoop.io.SequenceFile;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
import org.apache.hadoop.security.Credentials;
|
import org.apache.hadoop.security.Credentials;
|
||||||
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.hadoop.tools.mapred.CopyMapper;
|
import org.apache.hadoop.tools.mapred.CopyMapper;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -712,4 +714,29 @@ public class TestDistCpSync {
|
||||||
verifyCopy(dfs.getFileStatus(sourcePath),
|
verifyCopy(dfs.getFileStatus(sourcePath),
|
||||||
dfs.getFileStatus(target), false);
|
dfs.getFileStatus(target), false);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSyncSnapshotTimeStampChecking() throws Exception {
|
||||||
|
initData(source);
|
||||||
|
initData(target);
|
||||||
|
dfs.allowSnapshot(source);
|
||||||
|
dfs.allowSnapshot(target);
|
||||||
|
dfs.createSnapshot(source, "s2");
|
||||||
|
dfs.createSnapshot(target, "s1");
|
||||||
|
// Sleep one second to make snapshot s1 created later than s2
|
||||||
|
Thread.sleep(1000);
|
||||||
|
dfs.createSnapshot(source, "s1");
|
||||||
|
|
||||||
|
boolean threwException = false;
|
||||||
|
try {
|
||||||
|
DistCpSync distCpSync = new DistCpSync(options, conf);
|
||||||
|
// do the sync
|
||||||
|
distCpSync.sync();
|
||||||
|
} catch (HadoopIllegalArgumentException e) {
|
||||||
|
threwException = true;
|
||||||
|
GenericTestUtils.assertExceptionContains(
|
||||||
|
"Snapshot s2 should be newer than s1", e);
|
||||||
|
}
|
||||||
|
Assert.assertTrue(threwException);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue