|
@@ -21,7 +21,6 @@ import java.io.*;
|
|
|
import java.util.*;
|
|
|
|
|
|
import org.apache.hadoop.conf.*;
|
|
|
-import org.apache.hadoop.dfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.ipc.*;
|
|
|
import org.apache.hadoop.util.ToolBase;
|
|
|
|
|
@@ -138,18 +137,86 @@ public class FsShell extends ToolBase {
|
|
|
}
|
|
|
cat(srcf);
|
|
|
} else {
|
|
|
- Path [] srcs = fs.globPaths(new Path(srcf));
|
|
|
- if (srcs.length > 1 && !new File(dstf).isDirectory()) {
|
|
|
- throw new IOException("When copying multiple files, "
|
|
|
- + "destination should be a directory.");
|
|
|
+ copyToLocal(fs, new Path(srcf), new File(dstf), copyCrc);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The prefix for the tmp file used in copyToLocal.
|
|
|
+ * It must be at least three characters long, required by
|
|
|
+ * {@link java.io.File#createTempFile(String, String, File)}.
|
|
|
+ */
|
|
|
+ static final String COPYTOLOCAL_PREFIX = "_copyToLocal_";
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Copy a source file from a given file system to local destination.
|
|
|
+ * @param srcFS source file system
|
|
|
+ * @param src source path
|
|
|
+ * @param dst destination
|
|
|
+ * @param copyCrc copy CRC files?
|
|
|
+ * @exception IOException If some IO failed
|
|
|
+ */
|
|
|
+ private void copyToLocal(final FileSystem srcFS, final Path src,
|
|
|
+ final File dst, final boolean copyCrc)
|
|
|
+ throws IOException {
|
|
|
+ if (srcFS.isDirectory(src)) { //src is a directory
|
|
|
+ dst.mkdir();
|
|
|
+ if (!dst.isDirectory()) {
|
|
|
+ throw new IOException("cannot create directory for local destination \""
|
|
|
+ + dst + "\".");
|
|
|
+ }
|
|
|
+ for(Path p : srcFS.listPaths(src)) {
|
|
|
+ copyToLocal(srcFS, p,
|
|
|
+ srcFS.isDirectory(p)? new File(dst, p.getName()): dst, copyCrc);
|
|
|
}
|
|
|
- Path dst = new Path(dstf);
|
|
|
- for(int i=0; i<srcs.length; i++) {
|
|
|
- ((DistributedFileSystem)fs).copyToLocalFile(srcs[i], dst, copyCrc);
|
|
|
+ }
|
|
|
+ else {
|
|
|
+ Path [] srcs = srcFS.globPaths(src);
|
|
|
+ if (dst.isDirectory()) { //dst is a directory but src is not
|
|
|
+ for (Path p : srcs) {
|
|
|
+ copyToLocal(srcFS, p, new File(dst, p.getName()), copyCrc);
|
|
|
+ }
|
|
|
+ } else if (srcs.length == 1)
|
|
|
+ {
|
|
|
+ if (dst.exists()) {
|
|
|
+ throw new IOException("local destination \"" + dst
|
|
|
+ + "\" already exists.");
|
|
|
+ }
|
|
|
+ if (!srcFS.exists(src)) {
|
|
|
+ throw new IOException("src \"" + src + "\" does not exist.");
|
|
|
+ }
|
|
|
+
|
|
|
+ File tmp = FileUtil.createLocalTempFile(dst, COPYTOLOCAL_PREFIX, true);
|
|
|
+ if (FileUtil.copy(srcFS, src, tmp, false, srcFS.getConf())) {
|
|
|
+ if (!tmp.renameTo(dst)) {
|
|
|
+ //try to reanme tmp to another file since tmp will be deleted on exit
|
|
|
+ File another = FileUtil.createLocalTempFile(dst, COPYTOLOCAL_PREFIX,
|
|
|
+ false);
|
|
|
+ another.delete();
|
|
|
+ if (tmp.renameTo(another)) {
|
|
|
+ throw new IOException(
|
|
|
+ "Failed to rename tmp file to local destination \"" + dst
|
|
|
+ + "\". Remote source file \"" + src + "\" is saved to \""
|
|
|
+ + another + "\".");
|
|
|
+ } else {
|
|
|
+ throw new IOException("Failed to rename tmp file.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (copyCrc) {
|
|
|
+ ChecksumFileSystem csfs = (ChecksumFileSystem) srcFS;
|
|
|
+ File dstcs = FileSystem.getLocal(srcFS.getConf())
|
|
|
+ .pathToFile(csfs.getChecksumFile(new Path(dst.getCanonicalPath())));
|
|
|
+ copyToLocal(srcFS, csfs.getChecksumFile(src), dstcs, false);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ throw new IOException("When copying multiple files, "
|
|
|
+ + "destination should be a directory.");
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Get all the files in the directories that match the source file
|
|
|
* pattern and merge and sort them to only one file on local fs
|