|
@@ -37,6 +37,7 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
+import org.apache.hadoop.fs.FileChecksum;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FsShell;
|
|
@@ -335,9 +336,9 @@ public class DistCp implements Tool {
|
|
|
* be meaningful in this context.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- private boolean needsUpdate(FileStatus src, FileSystem dstfs, Path dstpath
|
|
|
- ) throws IOException {
|
|
|
- return update && !sameFile(src, dstfs, dstpath);
|
|
|
+ private boolean needsUpdate(FileSystem srcfs, Path srcpath,
|
|
|
+ FileSystem dstfs, Path dstpath) throws IOException {
|
|
|
+ return update && !sameFile(srcfs, srcpath, dstfs, dstpath);
|
|
|
}
|
|
|
|
|
|
private FSDataOutputStream create(Path f, Reporter reporter,
|
|
@@ -386,8 +387,10 @@ public class DistCp implements Tool {
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
+ final Path srcpath = srcstat.getPath();
|
|
|
+ final FileSystem srcfs = srcpath.getFileSystem(job);
|
|
|
if (destFileSys.exists(absdst) && !overwrite
|
|
|
- && !needsUpdate(srcstat, destFileSys, absdst)) {
|
|
|
+ && !needsUpdate(srcfs, srcpath, destFileSys, absdst)) {
|
|
|
outc.collect(null, new Text("SKIP: " + srcstat.getPath()));
|
|
|
++skipcount;
|
|
|
reporter.incrCounter(Counter.SKIP, 1);
|
|
@@ -1048,7 +1051,7 @@ public class DistCp implements Tool {
|
|
|
else {
|
|
|
//skip file if the src and the dst files are the same.
|
|
|
final Path absdst = new Path(args.dst, dst);
|
|
|
- skipfile = update && sameFile(child, dstfs, absdst);
|
|
|
+ skipfile = update && sameFile(srcfs,child.getPath(),dstfs,absdst);
|
|
|
|
|
|
if (!skipfile) {
|
|
|
++fileCount;
|
|
@@ -1134,15 +1137,15 @@ public class DistCp implements Tool {
|
|
|
* Check whether the src and the dst are the same.
|
|
|
* Two files are considered as the same if they have the same size.
|
|
|
*/
|
|
|
- static private boolean sameFile(FileStatus src, FileSystem dstfs, Path dstpath
|
|
|
- ) throws IOException {
|
|
|
- FileStatus dst = null;
|
|
|
+ static private boolean sameFile(FileSystem srcfs, Path srcpath,
|
|
|
+ FileSystem dstfs, Path dstpath) throws IOException {
|
|
|
try {
|
|
|
- dst = dstfs.getFileStatus(dstpath);
|
|
|
- } catch (FileNotFoundException fnfe) {
|
|
|
+ final FileChecksum srccs = srcfs.getFileChecksum(srcpath);
|
|
|
+ final FileChecksum dstcs = dstfs.getFileChecksum(dstpath);
|
|
|
+ return srccs != null && srccs.equals(dstcs);
|
|
|
+ } catch(FileNotFoundException fnfe) {
|
|
|
return false;
|
|
|
}
|
|
|
- return src.getLen() == dst.getLen();
|
|
|
}
|
|
|
|
|
|
/** Delete the dst files/dirs which do not exist in src */
|