HDFS-10313. Distcp need to enforce the order of snapshot names passed to -diff. (Lin Yiqun via Yongjun Zhang)

(cherry picked from commit 959a28dd12)
(cherry picked from commit 7955b6f66e)
This commit is contained in:
Yongjun Zhang 2016-04-26 16:08:03 -07:00
parent b98b057f45
commit 7ff4fb89ca
4 changed files with 59 additions and 1 deletions

View File

@ -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 {

View File

@ -180,7 +180,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);
} }
} }

View File

@ -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;
} }

View File

@ -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);
}
} }