Explorar el Código

Fix for HADOOP-126. 'bin/hadoop dfs -cp' now correctly handles .crc files. This also consolidates a lot of file copying code. Contributed by Konstantin Shvachko.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@393025 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting hace 19 años
padre
commit
dbccfa12b1

+ 6 - 1
CHANGES.txt

@@ -1,6 +1,12 @@
 Hadoop Change Log
 
 
+Trunk (unreleased)
+
+ 1. Fix HADOOP-126. 'bin/hadoop dfs -cp' now correctly copies .crc
+    files.  (Konstantin Shvachko via cutting)
+
+
 Release 0.1.1 - 2006-04-08
 
  1. Added CHANGES.txt, logging all significant changes to Hadoop.  (cutting)
@@ -18,7 +24,6 @@ Release 0.1.1 - 2006-04-08
 
  5. Fix HADOOP-125. Fix handling of absolute paths on Windows (cutting)
 
-
 Release 0.1.0 - 2006-04-01
 
  1. The first release of Hadoop.

+ 1 - 5
src/java/org/apache/hadoop/dfs/DFSShell.java

@@ -137,11 +137,7 @@ public class DFSShell {
      * Copy an DFS file
      */
     public void copy(String srcf, String dstf, Configuration conf) throws IOException {
-        if (FileUtil.copyContents(fs, new File(srcf), new File(dstf), true, conf)) {
-            System.out.println("Copied " + srcf + " to " + dstf);
-        } else {
-            System.out.println("Copy failed");
-        }
+      DistributedFileSystem.doCopy(fs, new File(srcf), fs, new File(dstf), true, conf);
     }
 
     /**

+ 23 - 49
src/java/org/apache/hadoop/dfs/DistributedFileSystem.java

@@ -157,30 +157,39 @@ public class DistributedFileSystem extends FileSystem {
     }
 
     private void doFromLocalFile(File src, File dst, boolean deleteSource) throws IOException {
-        if (exists(dst)) {
-            if (! isDirectory(dst)) {
+        FileSystem localFs = getNamed("local", getConf());
+        doCopy( localFs, src, this, dst, deleteSource, getConf() );
+    }
+
+    public static void doCopy(FileSystem srcFS, 
+                        File src, 
+                        FileSystem dstFS, 
+                        File dst, 
+                        boolean deleteSource,
+                        Configuration conf
+                       ) throws IOException {
+        if (dstFS.exists(dst)) {
+            if (! dstFS.isDirectory(dst)) {
                 throw new IOException("Target " + dst + " already exists");
             } else {
                 dst = new File(dst, src.getName());
-                if (exists(dst)) {
+                if (dstFS.exists(dst)) {
                     throw new IOException("Target " + dst + " already exists");
                 }
             }
         }
 
-        FileSystem localFs = getNamed("local", getConf());
-
-        if (localFs.isDirectory(src)) {
-            mkdirs(dst);
-            File contents[] = localFs.listFiles(src);
+        if (srcFS.isDirectory(src)) {
+            dstFS.mkdirs(dst);
+            File contents[] = srcFS.listFiles(src);
             for (int i = 0; i < contents.length; i++) {
-                doFromLocalFile(contents[i], new File(dst, contents[i].getName()), deleteSource);
+                doCopy( srcFS, contents[i], dstFS, new File(dst, contents[i].getName()), deleteSource, conf);
             }
         } else {
-            byte buf[] = new byte[getConf().getInt("io.file.buffer.size", 4096)];
-            InputStream in = localFs.open(src);
+            byte buf[] = new byte[conf.getInt("io.file.buffer.size", 4096)];
+            InputStream in = srcFS.open(src);
             try {
-                OutputStream out = create(dst);
+                OutputStream out = dstFS.create(dst);
                 try {
                     int bytesRead = in.read(buf);
                     while (bytesRead >= 0) {
@@ -195,48 +204,13 @@ public class DistributedFileSystem extends FileSystem {
             } 
         }
         if (deleteSource)
-            localFs.delete(src);
+          srcFS.delete(src);
     }
 
     public void copyToLocalFile(File src, File dst) throws IOException {
-        if (dst.exists()) {
-            if (! dst.isDirectory()) {
-                throw new IOException("Target " + dst + " already exists");
-            } else {
-                dst = new File(dst, src.getName());
-                if (dst.exists()) {
-                    throw new IOException("Target " + dst + " already exists");
-                }
-            }
-        }
         dst = dst.getCanonicalFile();
-
         FileSystem localFs = getNamed("local", getConf());
-
-        if (isDirectory(src)) {
-            localFs.mkdirs(dst);
-            File contents[] = listFiles(src);
-            for (int i = 0; i < contents.length; i++) {
-                copyToLocalFile(contents[i], new File(dst, contents[i].getName()));
-            }
-        } else {
-            byte buf[] = new byte[getConf().getInt("io.file.buffer.size", 4096)];
-            InputStream in = open(src);
-            try {
-                OutputStream out = localFs.create(dst);
-                try {
-                    int bytesRead = in.read(buf);
-                    while (bytesRead >= 0) {
-                        out.write(buf, 0, bytesRead);
-                        bytesRead = in.read(buf);
-                    }
-                } finally {
-                    out.close();
-                }
-            } finally {
-                in.close();
-            } 
-        }
+        doCopy( this, src, localFs, dst, false, getConf() );
     }
 
     public File startLocalOutput(File fsOutputFile, File tmpLocalFile) throws IOException {