Pārlūkot izejas kodu

svn merge -c 1195760 from trunk for HADOOP-7771.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1195762 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 gadi atpakaļ
vecāks
revīzija
7e7a610988

+ 4 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -433,6 +433,10 @@ Release 0.23.0 - Unreleased
 
     HADOOP-7743. Add Maven profile to create a full source tarball. (tucu)
 
+    HADOOP-7771. FsShell -copyToLocal, -get, etc. commands throw NPE if the
+    destination directory does not exist.  (John George and Daryn Sharp
+    via szetszwo)
+
   OPTIMIZATIONS
   
     HADOOP-7333. Performance improvement in PureJavaCrc32. (Eric Caspole

+ 12 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java

@@ -55,6 +55,7 @@ abstract public class Command extends Configured {
   protected int exitCode = 0;
   protected int numErrors = 0;
   protected boolean recursive = false;
+  private int depth = 0;
   protected ArrayList<Exception> exceptions = new ArrayList<Exception>();
 
   private static final Log LOG = LogFactory.getLog(Command.class);
@@ -86,6 +87,10 @@ abstract public class Command extends Configured {
     return recursive;
   }
 
+  protected int getDepth() {
+    return depth;
+  }
+  
   /** 
    * Execute the command on the input path
    * 
@@ -269,6 +274,7 @@ abstract public class Command extends Configured {
   protected void processPathArgument(PathData item) throws IOException {
     // null indicates that the call is not via recursion, ie. there is
     // no parent directory that was expanded
+    depth = 0;
     processPaths(null, item);
   }
   
@@ -326,7 +332,12 @@ abstract public class Command extends Configured {
    *  @throws IOException if anything goes wrong...
    */
   protected void recursePath(PathData item) throws IOException {
-    processPaths(item, item.getDirectoryContents());
+    try {
+      depth++;
+      processPaths(item, item.getDirectoryContents());
+    } finally {
+      depth--;
+    }
   }
 
   /**

+ 116 - 26
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java

@@ -20,13 +20,18 @@ package org.apache.hadoop.fs.shell;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.LinkedList;
 
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.shell.PathExceptions.PathExistsException;
 import org.apache.hadoop.fs.shell.PathExceptions.PathIOException;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIsDirectoryException;
 import org.apache.hadoop.fs.shell.PathExceptions.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.shell.PathExceptions.PathNotFoundException;
+import org.apache.hadoop.fs.shell.PathExceptions.PathOperationException;
+import org.apache.hadoop.io.IOUtils;
 
 /**
  * Provides: argument processing to ensure the destination is valid
@@ -106,51 +111,136 @@ abstract class CommandWithDestination extends FsCommand {
   }
 
   @Override
-  protected void processPaths(PathData parent, PathData ... items)
+  protected void processPathArgument(PathData src)
   throws IOException {
+    if (src.stat.isDirectory() && src.fs.equals(dst.fs)) {
+      PathData target = getTargetPath(src);
+      String srcPath = src.fs.makeQualified(src.path).toString();
+      String dstPath = dst.fs.makeQualified(target.path).toString();
+      if (dstPath.equals(srcPath)) {
+        PathIOException e = new PathIOException(src.toString(),
+            "are identical");
+        e.setTargetPath(dstPath.toString());
+        throw e;
+      }
+      if (dstPath.startsWith(srcPath+Path.SEPARATOR)) {
+        PathIOException e = new PathIOException(src.toString(),
+            "is a subdirectory of itself");
+        e.setTargetPath(target.toString());
+        throw e;
+      }
+    }
+    super.processPathArgument(src);
+  }
+
+  @Override
+  protected void processPath(PathData src) throws IOException {
+    processPath(src, getTargetPath(src));
+  }
+  
+  /**
+   * Called with a source and target destination pair
+   * @param src for the operation
+   * @param target for the operation
+   * @throws IOException if anything goes wrong
+   */
+  protected void processPath(PathData src, PathData dst) throws IOException {
+    if (src.stat.isSymlink()) {
+      // TODO: remove when FileContext is supported, this needs to either
+      // copy the symlink or deref the symlink
+      throw new PathOperationException(src.toString());        
+    } else if (src.stat.isFile()) {
+      copyFileToTarget(src, dst);
+    } else if (src.stat.isDirectory() && !isRecursive()) {
+      throw new PathIsDirectoryException(src.toString());
+    }
+  }
+
+  @Override
+  protected void recursePath(PathData src) throws IOException {
     PathData savedDst = dst;
     try {
       // modify dst as we descend to append the basename of the
       // current directory being processed
-      if (parent != null) dst = dst.getPathDataForChild(parent);
-      super.processPaths(parent, items);
+      dst = getTargetPath(src);
+      if (dst.exists) {
+        if (!dst.stat.isDirectory()) {
+          throw new PathIsNotDirectoryException(dst.toString());
+        }
+      } else {
+        if (!dst.fs.mkdirs(dst.path)) {
+          // too bad we have no clue what failed
+          PathIOException e = new PathIOException(dst.toString());
+          e.setOperation("mkdir");
+          throw e;
+        }    
+        dst.refreshStatus(); // need to update stat to know it exists now
+      }      
+      super.recursePath(src);
     } finally {
       dst = savedDst;
     }
   }
   
-  @Override
-  protected void processPath(PathData src) throws IOException {
+  protected PathData getTargetPath(PathData src) throws IOException {
     PathData target;
-    // if the destination is a directory, make target a child path,
-    // else use the destination as-is
-    if (dst.exists && dst.stat.isDirectory()) {
+    // on the first loop, the dst may be directory or a file, so only create
+    // a child path if dst is a dir; after recursion, it's always a dir
+    if ((getDepth() > 0) || (dst.exists && dst.stat.isDirectory())) {
       target = dst.getPathDataForChild(src);
     } else {
       target = dst;
     }
-    if (target.exists && !overwrite) {
+    return target;
+  }
+  
+  /**
+   * Copies the source file to the target.
+   * @param src item to copy
+   * @param target where to copy the item
+   * @throws IOException if copy fails
+   */ 
+  protected void copyFileToTarget(PathData src, PathData target) throws IOException {
+    copyStreamToTarget(src.fs.open(src.path), target);
+  }
+  
+  /**
+   * Copies the stream contents to a temporary file.  If the copy is
+   * successful, the temporary file will be renamed to the real path,
+   * else the temporary file will be deleted.
+   * @param in the input stream for the copy
+   * @param target where to store the contents of the stream
+   * @throws IOException if copy fails
+   */ 
+  protected void copyStreamToTarget(InputStream in, PathData target)
+  throws IOException {
+    if (target.exists && (target.stat.isDirectory() || !overwrite)) {
       throw new PathExistsException(target.toString());
     }
-
-    try { 
-      // invoke processPath with both a source and resolved target
-      processPath(src, target);
-    } catch (PathIOException e) {
-      // add the target unless it already has one
-      if (e.getTargetPath() == null) {
+    PathData tempFile = null;
+    try {
+      tempFile = target.createTempFile(target+"._COPYING_");
+      FSDataOutputStream out = target.fs.create(tempFile.path, true);
+      IOUtils.copyBytes(in, out, getConf(), true);
+      // the rename method with an option to delete the target is deprecated
+      if (target.exists && !target.fs.delete(target.path, false)) {
+        // too bad we don't know why it failed
+        PathIOException e = new PathIOException(target.toString());
+        e.setOperation("delete");
+        throw e;
+      }
+      if (!tempFile.fs.rename(tempFile.path, target.path)) {
+        // too bad we don't know why it failed
+        PathIOException e = new PathIOException(tempFile.toString());
+        e.setOperation("rename");
         e.setTargetPath(target.toString());
+        throw e;
+      }
+      tempFile = null;
+    } finally {
+      if (tempFile != null) {
+        tempFile.fs.delete(tempFile.path, false);
       }
-      throw e;
     }
   }
-
-  /**
-   * Called with a source and target destination pair
-   * @param src for the operation
-   * @param target for the operation
-   * @throws IOException if anything goes wrong
-   */
-  protected abstract void processPath(PathData src, PathData target)
-  throws IOException;
 }

+ 10 - 73
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java

@@ -26,13 +26,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ChecksumFileSystem;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.shell.PathExceptions.PathExistsException;
-import org.apache.hadoop.fs.shell.PathExceptions.PathIOException;
-import org.apache.hadoop.fs.shell.PathExceptions.PathOperationException;
-import org.apache.hadoop.io.IOUtils;
 
 /** Various commands for copy files */
 @InterfaceAudience.Private
@@ -95,18 +89,10 @@ class CopyCommands {
       CommandFormat cf = new CommandFormat(2, Integer.MAX_VALUE, "f");
       cf.parse(args);
       setOverwrite(cf.getOpt("f"));
+      // should have a -r option
+      setRecursive(true);
       getRemoteDestination(args);
     }
-
-    @Override
-    protected void processPath(PathData src, PathData target)
-    throws IOException {
-      if (!FileUtil.copy(src.fs, src.path, target.fs, target.path, false, overwrite, getConf())) {
-        // we have no idea what the error is...  FileUtils masks it and in
-        // some cases won't even report an error
-        throw new PathIOException(src.toString());
-      }
-    }
   }
   
   /** 
@@ -126,7 +112,6 @@ class CopyCommands {
      * It must be at least three characters long, required by
      * {@link java.io.File#createTempFile(String, String, File)}.
      */
-    private static final String COPYTOLOCAL_PREFIX = "_copyToLocal_";
     private boolean copyCrc;
     private boolean verifyChecksum;
 
@@ -144,7 +129,7 @@ class CopyCommands {
     }
 
     @Override
-    protected void processPath(PathData src, PathData target)
+    protected void copyFileToTarget(PathData src, PathData target)
     throws IOException {
       src.fs.setVerifyChecksum(verifyChecksum);
 
@@ -153,41 +138,10 @@ class CopyCommands {
         copyCrc = false;
       }      
 
-      if (src.stat.isFile()) {
-        // copy the file and maybe its crc
-        copyFileToLocal(src, target);
-        if (copyCrc) {
-          copyFileToLocal(src.getChecksumFile(), target.getChecksumFile());
-        }
-      } else if (src.stat.isDirectory()) {
-        // create the remote directory structure locally
-        if (!target.toFile().mkdirs()) {
-          throw new PathIOException(target.toString());
-        }
-      } else {
-        throw new PathOperationException(src.toString());
-      }
-    }
-
-    private void copyFileToLocal(PathData src, PathData target)
-    throws IOException {
-      File targetFile = target.toFile();
-      File tmpFile = FileUtil.createLocalTempFile(
-          targetFile, COPYTOLOCAL_PREFIX, true);
-      // too bad we can't tell exactly why it failed...
-      if (!FileUtil.copy(src.fs, src.path, tmpFile, false, getConf())) {
-        PathIOException e = new PathIOException(src.toString());
-        e.setOperation("copy");
-        e.setTargetPath(tmpFile.toString());
-        throw e;
-      }
-
-      // too bad we can't tell exactly why it failed...
-      if (!tmpFile.renameTo(targetFile)) {
-        PathIOException e = new PathIOException(tmpFile.toString());
-        e.setOperation("rename");
-        e.setTargetPath(targetFile.toString());
-        throw e;
+      super.copyFileToTarget(src, target);
+      if (copyCrc) {
+        // should we delete real file if crc copy fails?
+        super.copyFileToTarget(src.getChecksumFile(), target.getChecksumFile());
       }
     }
   }
@@ -208,6 +162,8 @@ class CopyCommands {
       cf.parse(args);
       setOverwrite(cf.getOpt("f"));
       getRemoteDestination(args);
+      // should have a -r option
+      setRecursive(true);
     }
 
     // commands operating on local paths have no need for glob expansion
@@ -223,30 +179,11 @@ class CopyCommands {
     throws IOException {
       // NOTE: this logic should be better, mimics previous implementation
       if (args.size() == 1 && args.get(0).toString().equals("-")) {
-        if (dst.exists && !overwrite) {
-          throw new PathExistsException(dst.toString());
-        }
-        copyFromStdin();
+        copyStreamToTarget(System.in, getTargetPath(args.get(0)));
         return;
       }
       super.processArguments(args);
     }
-
-    @Override
-    protected void processPath(PathData src, PathData target)
-    throws IOException {
-      target.fs.copyFromLocalFile(false, overwrite, src.path, target.path);
-    }
-
-    /** Copies from stdin to the destination file. */
-    protected void copyFromStdin() throws IOException {
-      FSDataOutputStream out = dst.fs.create(dst.path); 
-      try {
-        IOUtils.copyBytes(System.in, out, getConf(), false);
-      } finally {
-        out.close();
-      }
-    }
   }
 
   public static class CopyFromLocal extends Put {

+ 13 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java

@@ -182,6 +182,19 @@ public class PathData {
     return new PathData(srcFs.getRawFileSystem(), srcPath.toString());
   }
 
+  /**
+   * Returns a temporary file for this PathData with the given extension.
+   * The file will be deleted on exit.
+   * @param extension for the temporary file
+   * @return PathData
+   * @throws IOException shouldn't happen
+   */
+  public PathData createTempFile(String extension) throws IOException {
+    PathData tmpFile = new PathData(fs, uri+"._COPYING_");
+    fs.deleteOnExit(tmpFile.path);
+    return tmpFile;
+  }
+
   /**
    * Returns a list of PathData objects of the items contained in the given
    * directory.