Browse Source

svn merge -c 1127591 from trunk for HADOOP-7320.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/yahoo-merge@1127731 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 14 years ago
parent
commit
467588fec0

+ 3 - 0
CHANGES.txt

@@ -80,6 +80,9 @@ Trunk (unreleased changes)
     HADOOP-7286. Refactor the du/dus/df commands to conform to new FsCommand
     class. (Daryn Sharp via todd)
 
+    HADOOP-7320. Refactor the copy and move commands to conform to new
+    FsCommand class. (Daryn Sharp via todd)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 10 - 551
src/java/org/apache/hadoop/fs/FsShell.java

@@ -17,15 +17,8 @@
  */
 package org.apache.hadoop.fs;
 
-import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -36,12 +29,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.shell.Command;
 import org.apache.hadoop.fs.shell.CommandFactory;
-import org.apache.hadoop.fs.shell.CommandFormat;
 import org.apache.hadoop.fs.shell.FsCommand;
 import org.apache.hadoop.fs.shell.PathExceptions.PathNotFoundException;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -56,14 +46,6 @@ public class FsShell extends Configured implements Tool {
   private Trash trash;
   protected CommandFactory commandFactory;
 
-  public static final SimpleDateFormat dateForm = 
-    new SimpleDateFormat("yyyy-MM-dd HH:mm");
-  static final int BORDER = 2;
-
-  static final String GET_SHORT_USAGE = "-get [-ignoreCrc] [-crc] <src> <localdst>";
-  static final String COPYTOLOCAL_SHORT_USAGE = GET_SHORT_USAGE.replace(
-      "-get", "-copyToLocal");
-
   /**
    */
   public FsShell() {
@@ -95,397 +77,6 @@ public class FsShell extends Configured implements Tool {
     getConf().setQuietMode(true);
   }
 
-  
-  /**
-   * Copies from stdin to the indicated file.
-   */
-  private void copyFromStdin(Path dst, FileSystem dstFs) throws IOException {
-    if (dstFs.isDirectory(dst)) {
-      throw new IOException("When source is stdin, destination must be a file.");
-    }
-    if (dstFs.exists(dst)) {
-      throw new IOException("Target " + dst.toString() + " already exists.");
-    }
-    FSDataOutputStream out = dstFs.create(dst); 
-    try {
-      IOUtils.copyBytes(System.in, out, getConf(), false);
-    } 
-    finally {
-      out.close();
-    }
-  }
-
-  /** 
-   * Print from src to stdout.
-   */
-  private void printToStdout(InputStream in) throws IOException {
-    try {
-      IOUtils.copyBytes(in, System.out, getConf(), false);
-    } finally {
-      in.close();
-    }
-  }
-
-  
-  /**
-   * Add local files to the indicated FileSystem name. src is kept.
-   */
-  void copyFromLocal(Path[] srcs, String dstf) throws IOException {
-    Path dstPath = new Path(dstf);
-    FileSystem dstFs = dstPath.getFileSystem(getConf());
-    if (srcs.length == 1 && srcs[0].toString().equals("-"))
-      copyFromStdin(dstPath, dstFs);
-    else
-      dstFs.copyFromLocalFile(false, false, srcs, dstPath);
-  }
-  
-  /**
-   * Add local files to the indicated FileSystem name. src is removed.
-   */
-  void moveFromLocal(Path[] srcs, String dstf) throws IOException {
-    Path dstPath = new Path(dstf);
-    FileSystem dstFs = dstPath.getFileSystem(getConf());
-    dstFs.moveFromLocalFile(srcs, dstPath);
-  }
-
-  /**
-   * Add a local file to the indicated FileSystem name. src is removed.
-   */
-  void moveFromLocal(Path src, String dstf) throws IOException {
-    moveFromLocal((new Path[]{src}), dstf);
-  }
-
-  /**
-   * Obtain the indicated files that match the file pattern <i>srcf</i>
-   * and copy them to the local name. srcf is kept.
-   * When copying multiple files, the destination must be a directory. 
-   * Otherwise, IOException is thrown.
-   * @param argv : arguments
-   * @param pos : Ignore everything before argv[pos]  
-   * @throws Exception 
-   * @see org.apache.hadoop.fs.FileSystem.globStatus 
-   */
-  void copyToLocal(String[]argv, int pos) throws Exception {
-    CommandFormat cf = new CommandFormat("copyToLocal", 2,2,"crc","ignoreCrc");
-    
-    String srcstr = null;
-    String dststr = null;
-    try {
-      List<String> parameters = cf.parse(argv, pos);
-      srcstr = parameters.get(0);
-      dststr = parameters.get(1);
-    }
-    catch(IllegalArgumentException iae) {
-      System.err.println("Usage: java FsShell " + GET_SHORT_USAGE);
-      throw iae;
-    }
-    boolean copyCrc = cf.getOpt("crc");
-    final boolean verifyChecksum = !cf.getOpt("ignoreCrc");
-
-    if (dststr.equals("-")) {
-      if (copyCrc) {
-        System.err.println("-crc option is not valid when destination is stdout.");
-      }
-      
-      List<String> catArgv = new ArrayList<String>();
-      catArgv.add("-cat");
-      if (cf.getOpt("ignoreCrc")) catArgv.add("-ignoreCrc");
-      catArgv.add(srcstr);      
-      run(catArgv.toArray(new String[0]));
-    } else {
-      File dst = new File(dststr);      
-      Path srcpath = new Path(srcstr);
-      FileSystem srcFS = getSrcFileSystem(srcpath, verifyChecksum);
-      if (copyCrc && !(srcFS instanceof ChecksumFileSystem)) {
-        System.err.println("-crc option is not valid when source file system " +
-            "does not have crc files. Automatically turn the option off.");
-        copyCrc = false;
-      }
-      FileStatus[] srcs = srcFS.globStatus(srcpath);
-      if (null == srcs) {
-        throw new PathNotFoundException(srcstr);
-      }
-      boolean dstIsDir = dst.isDirectory(); 
-      if (srcs.length > 1 && !dstIsDir) {
-        throw new IOException("When copying multiple files, "
-                              + "destination should be a directory.");
-      }
-      for (FileStatus status : srcs) {
-        Path p = status.getPath();
-        File f = dstIsDir? new File(dst, p.getName()): dst;
-        copyToLocal(srcFS, status, f, copyCrc);
-      }
-    }
-  }
-
-  /**
-   * Return the {@link FileSystem} specified by src and the conf.
-   * It the {@link FileSystem} supports checksum, set verifyChecksum.
-   */
-  private FileSystem getSrcFileSystem(Path src, boolean verifyChecksum
-      ) throws IOException { 
-    FileSystem srcFs = src.getFileSystem(getConf());
-    srcFs.setVerifyChecksum(verifyChecksum);
-    return srcFs;
-  }
-
-  /**
-   * 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 FileStatus srcStatus,
-                           final File dst, final boolean copyCrc)
-    throws IOException {
-    /* Keep the structure similar to ChecksumFileSystem.copyToLocal(). 
-     * Ideal these two should just invoke FileUtil.copy() and not repeat
-     * recursion here. Of course, copy() should support two more options :
-     * copyCrc and useTmpFile (may be useTmpFile need not be an option).
-     */
-    
-    Path src = srcStatus.getPath();
-    if (srcStatus.isFile()) {
-      if (dst.exists()) {
-        // match the error message in FileUtil.checkDest():
-        throw new IOException("Target " + dst + " already exists");
-      }
-      
-      // use absolute name so that tmp file is always created under dest dir
-      File tmp = FileUtil.createLocalTempFile(dst.getAbsoluteFile(),
-                                              COPYTOLOCAL_PREFIX, true);
-      if (!FileUtil.copy(srcFS, src, tmp, false, srcFS.getConf())) {
-        throw new IOException("Failed to copy " + src + " to " + dst); 
-      }
-      
-      if (!tmp.renameTo(dst)) {
-        throw new IOException("Failed to rename tmp file " + tmp + 
-                              " to local destination \"" + dst + "\".");
-      }
-
-      if (copyCrc) {
-        if (!(srcFS instanceof ChecksumFileSystem)) {
-          throw new IOException("Source file system does not have crc files");
-        }
-        
-        ChecksumFileSystem csfs = (ChecksumFileSystem) srcFS;
-        File dstcs = FileSystem.getLocal(srcFS.getConf())
-          .pathToFile(csfs.getChecksumFile(new Path(dst.getCanonicalPath())));
-        FileSystem fs = csfs.getRawFileSystem();
-        FileStatus status = csfs.getFileStatus(csfs.getChecksumFile(src));
-        copyToLocal(fs, status, dstcs, false);
-      } 
-    } else if (srcStatus.isSymlink()) {
-      throw new AssertionError("Symlinks unsupported");
-    } else {
-      // once FileUtil.copy() supports tmp file, we don't need to mkdirs().
-      if (!dst.mkdirs()) {
-        throw new IOException("Failed to create local destination \"" +
-                              dst + "\".");
-      }
-      for(FileStatus status : srcFS.listStatus(src)) {
-        copyToLocal(srcFS, status,
-                    new File(dst, status.getPath().getName()), copyCrc);
-      }
-    }
-  }
-
-  /**
-   * Obtain the indicated file and copy to the local name.
-   * srcf is removed.
-   */
-  void moveToLocal(String srcf, Path dst) throws IOException {
-    System.err.println("Option '-moveToLocal' is not implemented yet.");
-  }
-    
-  /**
-   * Move files that match the file pattern <i>srcf</i>
-   * to a destination file.
-   * When moving mutiple files, the destination must be a directory. 
-   * Otherwise, IOException is thrown.
-   * @param srcf a file pattern specifying source files
-   * @param dstf a destination local file/directory 
-   * @throws IOException  
-   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
-   */
-  void rename(String srcf, String dstf) throws IOException {
-    Path srcPath = new Path(srcf);
-    Path dstPath = new Path(dstf);
-    FileSystem fs = srcPath.getFileSystem(getConf());
-    URI srcURI = fs.getUri();
-    URI dstURI = dstPath.getFileSystem(getConf()).getUri();
-    if (srcURI.compareTo(dstURI) != 0) {
-      throw new IOException("src and destination filesystems do not match.");
-    }
-    Path[] srcs = FileUtil.stat2Paths(fs.globStatus(srcPath), srcPath);
-    Path dst = new Path(dstf);
-    if (srcs.length > 1 && !fs.isDirectory(dst)) {
-      throw new IOException("When moving multiple files, " 
-                            + "destination should be a directory.");
-    }
-    for(int i=0; i<srcs.length; i++) {
-      if (!fs.rename(srcs[i], dst)) {
-        FileStatus srcFstatus = null;
-        FileStatus dstFstatus = null;
-        try {
-          srcFstatus = fs.getFileStatus(srcs[i]);
-        } catch(FileNotFoundException e) {
-          throw new PathNotFoundException(srcs[i].toString());
-        }
-        try {
-          dstFstatus = fs.getFileStatus(dst);
-        } catch(IOException e) {
-          LOG.debug("Error getting file status of " + dst, e);
-        }
-        if((srcFstatus!= null) && (dstFstatus!= null)) {
-          if (srcFstatus.isDirectory()  && !dstFstatus.isDirectory()) {
-            throw new IOException("cannot overwrite non directory "
-                + dst + " with directory " + srcs[i]);
-          }
-        }
-        throw new IOException("Failed to rename " + srcs[i] + " to " + dst);
-      }
-    }
-  }
-
-  /**
-   * Move/rename file(s) to a destination file. Multiple source
-   * files can be specified. The destination is the last element of
-   * the argvp[] array.
-   * If multiple source files are specified, then the destination 
-   * must be a directory. Otherwise, IOException is thrown.
-   * @throws IOException on error
-   */
-  private int rename(String argv[], Configuration conf) throws IOException {
-    int i = 0;
-    int exitCode = 0;
-    String cmd = argv[i++];  
-    String dest = argv[argv.length-1];
-    //
-    // If the user has specified multiple source files, then
-    // the destination has to be a directory
-    //
-    if (argv.length > 3) {
-      Path dst = new Path(dest);
-      FileSystem dstFs = dst.getFileSystem(getConf());
-      if (!dstFs.isDirectory(dst)) {
-        throw new IOException("When moving multiple files, " 
-                              + "destination " + dest + " should be a directory.");
-      }
-    }
-    //
-    // for each source file, issue the rename
-    //
-    for (; i < argv.length - 1; i++) {
-      try {
-        //
-        // issue the rename to the fs
-        //
-        rename(argv[i], dest);
-      } catch (RemoteException e) {
-        LOG.debug("Error renaming " + argv[i], e);
-        //
-        // This is a error returned by hadoop server. Print
-        // out the first line of the error mesage.
-        //
-        exitCode = -1;
-        try {
-          String[] content;
-          content = e.getLocalizedMessage().split("\n");
-          System.err.println(cmd.substring(1) + ": " + content[0]);
-        } catch (Exception ex) {
-          System.err.println(cmd.substring(1) + ": " +
-                             ex.getLocalizedMessage());
-        }
-      } catch (IOException e) {
-        LOG.debug("Error renaming " + argv[i], e);
-        //
-        // IO exception encountered locally.
-        // 
-        exitCode = -1;
-        displayError(cmd, e);
-      }
-    }
-    return exitCode;
-  }
-
-  /**
-   * Copy files that match the file pattern <i>srcf</i>
-   * to a destination file.
-   * When copying mutiple files, the destination must be a directory. 
-   * Otherwise, IOException is thrown.
-   * @param srcf a file pattern specifying source files
-   * @param dstf a destination local file/directory 
-   * @throws IOException  
-   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
-   */
-  void copy(String srcf, String dstf, Configuration conf) throws IOException {
-    Path srcPath = new Path(srcf);
-    FileSystem srcFs = srcPath.getFileSystem(getConf());
-    Path dstPath = new Path(dstf);
-    FileSystem dstFs = dstPath.getFileSystem(getConf());
-    Path [] srcs = FileUtil.stat2Paths(srcFs.globStatus(srcPath), srcPath);
-    if (srcs.length > 1 && !dstFs.isDirectory(dstPath)) {
-      throw new IOException("When copying multiple files, " 
-                            + "destination should be a directory.");
-    }
-    for(int i=0; i<srcs.length; i++) {
-      FileUtil.copy(srcFs, srcs[i], dstFs, dstPath, false, conf);
-    }
-  }
-
-  /**
-   * Copy file(s) to a destination file. Multiple source
-   * files can be specified. The destination is the last element of
-   * the argvp[] array.
-   * If multiple source files are specified, then the destination 
-   * must be a directory. Otherwise, IOException is thrown.
-   * @throws IOException on error
-   */
-  private int copy(String argv[], Configuration conf) throws IOException {
-    int i = 0;
-    int exitCode = 0;
-    String cmd = argv[i++];  
-    String dest = argv[argv.length-1];
-    //
-    // If the user has specified multiple source files, then
-    // the destination has to be a directory
-    //
-    if (argv.length > 3) {
-      Path dst = new Path(dest);
-      FileSystem pFS = dst.getFileSystem(conf);
-      if (!pFS.isDirectory(dst)) {
-        throw new IOException("When copying multiple files, " 
-                              + "destination " + dest + " should be a directory.");
-      }
-    }
-    //
-    // for each source file, issue the copy
-    //
-    for (; i < argv.length - 1; i++) {
-      try {
-        //
-        // issue the copy to the fs
-        //
-        copy(argv[i], dest, conf);
-      } catch (IOException e) {
-        LOG.debug("Error copying " + argv[i], e);
-        exitCode = -1;
-        displayError(cmd, e);
-      }
-    }
-    return exitCode;
-  }
-
   /**
    * Returns the Trash object associated with this shell.
    */
@@ -515,11 +106,6 @@ public class FsShell extends Configured implements Tool {
       "The full syntax is: \n\n" +
       "hadoop fs [-fs <local | file system URI>] [-conf <configuration file>]\n\t" +
       "[-D <property=value>]\n\t" +
-      "[-mv <src> <dst>] [-cp <src> <dst>]\n\t" + 
-      "[-put <localsrc> ... <dst>] [-copyFromLocal <localsrc> ... <dst>]\n\t" +
-      "[-moveFromLocal <localsrc> ... <dst>] [" + 
-      GET_SHORT_USAGE + "\n\t" +
-      "[" + COPYTOLOCAL_SHORT_USAGE + "] [-moveToLocal <src> <localdst>]\n\t" +
       "[-report]";
 
     String conf ="-conf <configuration file>:  Specify an application configuration file.";
@@ -537,33 +123,6 @@ public class FsShell extends Configured implements Tool {
       "\t\tappear first on the command line.  Exactly one additional\n" +
       "\t\targument must be specified. \n";
 
-    String mv = "-mv <src> <dst>:   Move files that match the specified file pattern <src>\n" +
-      "\t\tto a destination <dst>.  When moving multiple files, the \n" +
-      "\t\tdestination must be a directory. \n";
-
-    String cp = "-cp <src> <dst>:   Copy files that match the file pattern <src> to a \n" +
-      "\t\tdestination.  When copying multiple files, the destination\n" +
-      "\t\tmust be a directory. \n";
-
-    String put = "-put <localsrc> ... <dst>: \tCopy files " + 
-    "from the local file system \n\t\tinto fs. \n";
-
-    String copyFromLocal = "-copyFromLocal <localsrc> ... <dst>:" +
-    " Identical to the -put command.\n";
-
-    String moveFromLocal = "-moveFromLocal <localsrc> ... <dst>:" +
-    " Same as -put, except that the source is\n\t\tdeleted after it's copied.\n"; 
-
-    String get = GET_SHORT_USAGE
-      + ":  Copy files that match the file pattern <src> \n" +
-      "\t\tto the local name.  <src> is kept.  When copying mutiple, \n" +
-      "\t\tfiles, the destination must be a directory. \n";
-
-    String copyToLocal = COPYTOLOCAL_SHORT_USAGE
-                         + ":  Identical to the -get command.\n";
-
-    String moveToLocal = "-moveToLocal <src> <localdst>:  Not implemented yet \n";
-        
     String help = "-help [cmd]: \tDisplays help for given command or all commands if none\n" +
       "\t\tis specified.\n";
 
@@ -576,24 +135,6 @@ public class FsShell extends Configured implements Tool {
       System.out.println(conf);
     } else if ("D".equals(cmd)) {
       System.out.println(D);
-    } else if ("mv".equals(cmd)) {
-      System.out.println(mv);
-    } else if ("cp".equals(cmd)) {
-      System.out.println(cp);
-    } else if ("put".equals(cmd)) {
-      System.out.println(put);
-    } else if ("copyFromLocal".equals(cmd)) {
-      System.out.println(copyFromLocal);
-    } else if ("moveFromLocal".equals(cmd)) {
-      System.out.println(moveFromLocal);
-    } else if ("get".equals(cmd)) {
-      System.out.println(get);
-    } else if ("copyToLocal".equals(cmd)) {
-      System.out.println(copyToLocal);
-    } else if ("moveToLocal".equals(cmd)) {
-      System.out.println(moveToLocal);
-    } else if ("get".equals(cmd)) {
-      System.out.println(get);
     } else if ("help".equals(cmd)) {
       System.out.println(help);
     } else {
@@ -607,14 +148,6 @@ public class FsShell extends Configured implements Tool {
       System.out.println("\t[-help [cmd]]\n");
       
       System.out.println(fs);
-      System.out.println(mv);
-      System.out.println(cp);
-      System.out.println(put);
-      System.out.println(copyFromLocal);
-      System.out.println(moveFromLocal);
-      System.out.println(get);
-      System.out.println(copyToLocal);
-      System.out.println(moveToLocal);
 
       for (String thisCmdName : commandFactory.getNames()) {
         instance = commandFactory.getInstance(thisCmdName);
@@ -661,30 +194,8 @@ public class FsShell extends Configured implements Tool {
     } else if ("-D".equals(cmd)) {
       System.err.println("Usage: java FsShell" + 
                          " [-D <[property=value>]");
-    } else if ("-mv".equals(cmd) || "-cp".equals(cmd)) {
-      System.err.println("Usage: java FsShell" + 
-                         " [" + cmd + " <src> <dst>]");
-    } else if ("-put".equals(cmd) || "-copyFromLocal".equals(cmd) ||
-               "-moveFromLocal".equals(cmd)) {
-      System.err.println("Usage: java FsShell" + 
-                         " [" + cmd + " <localsrc> ... <dst>]");
-    } else if ("-get".equals(cmd)) {
-      System.err.println("Usage: java FsShell [" + GET_SHORT_USAGE + "]"); 
-    } else if ("-copyToLocal".equals(cmd)) {
-      System.err.println("Usage: java FsShell [" + COPYTOLOCAL_SHORT_USAGE+ "]"); 
-    } else if ("-moveToLocal".equals(cmd)) {
-      System.err.println("Usage: java FsShell" + 
-                         " [" + cmd + " [-crc] <src> <localdst>]");
     } else {
       System.err.println("Usage: java FsShell");
-      System.err.println("           [-mv <src> <dst>]");
-      System.err.println("           [-cp <src> <dst>]");
-      System.err.println("           [-put <localsrc> ... <dst>]");
-      System.err.println("           [-copyFromLocal <localsrc> ... <dst>]");
-      System.err.println("           [-moveFromLocal <localsrc> ... <dst>]");
-      System.err.println("           [" + GET_SHORT_USAGE + "]");
-      System.err.println("           [" + COPYTOLOCAL_SHORT_USAGE + "]");
-      System.err.println("           [-moveToLocal [-crc] <src> <localdst>]");
       for (String name : commandFactory.getNames()) {
         instance = commandFactory.getInstance(name);
         if (!instance.isDeprecated()) {
@@ -715,27 +226,6 @@ public class FsShell extends Configured implements Tool {
     int exitCode = -1;
     int i = 0;
     String cmd = argv[i++];
-    //
-    // verify that we have enough command line parameters
-    //
-    if ("-put".equals(cmd) ||
-        "-copyFromLocal".equals(cmd) || "-moveFromLocal".equals(cmd)) {
-      if (argv.length < 3) {
-        printUsage(cmd);
-        return exitCode;
-      }
-    } else if ("-get".equals(cmd) || 
-               "-copyToLocal".equals(cmd) || "-moveToLocal".equals(cmd)) {
-      if (argv.length < 3) {
-        printUsage(cmd);
-        return exitCode;
-      }
-    } else if ("-mv".equals(cmd) || "-cp".equals(cmd)) {
-      if (argv.length < 3) {
-        printUsage(cmd);
-        return exitCode;
-      }
-    }
     // initialize FsShell
     try {
       init();
@@ -751,38 +241,10 @@ public class FsShell extends Configured implements Tool {
       return exitCode;
     }
 
-    exitCode = 0;
     try {
       Command instance = commandFactory.getInstance(cmd);
       if (instance != null) {
-        try {
-          exitCode = instance.run(Arrays.copyOfRange(argv, i, argv.length));
-        } catch (Exception e) {
-          exitCode = -1;
-          LOG.debug("Error", e);
-          instance.displayError(e);
-          if (e instanceof IllegalArgumentException) {
-            printUsage(cmd);
-          }
-        }
-      } else if ("-put".equals(cmd) || "-copyFromLocal".equals(cmd)) {
-        Path[] srcs = new Path[argv.length-2];
-        for (int j=0 ; i < argv.length-1 ;) 
-          srcs[j++] = new Path(argv[i++]);
-        copyFromLocal(srcs, argv[i++]);
-      } else if ("-moveFromLocal".equals(cmd)) {
-        Path[] srcs = new Path[argv.length-2];
-        for (int j=0 ; i < argv.length-1 ;) 
-          srcs[j++] = new Path(argv[i++]);
-        moveFromLocal(srcs, argv[i++]);
-      } else if ("-get".equals(cmd) || "-copyToLocal".equals(cmd)) {
-        copyToLocal(argv, i);
-      } else if ("-moveToLocal".equals(cmd)) {
-        moveToLocal(argv[i++], new Path(argv[i++]));
-      } else if ("-mv".equals(cmd)) {
-        exitCode = rename(argv, getConf());
-      } else if ("-cp".equals(cmd)) {
-        exitCode = copy(argv, getConf());
+        exitCode = instance.run(Arrays.copyOfRange(argv, i, argv.length));
       } else if ("-help".equals(cmd)) {
         if (i < argv.length) {
           printHelp(argv[i]);
@@ -790,20 +252,17 @@ public class FsShell extends Configured implements Tool {
           printHelp("");
         }
       } else {
-        exitCode = -1;
-        System.err.println(cmd.substring(1) + ": Unknown command");
+        System.err.println(cmd + ": Unknown command");
         printUsage("");
       }
-    } catch (IllegalArgumentException arge) {
-      LOG.debug("Error", arge);
-      exitCode = -1;
-      System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
-      printUsage(cmd);
-    } catch (Exception re) {
-      LOG.debug("Error", re);
-      exitCode = -1;
-      displayError(cmd, re);
-    } finally {
+    } catch (Exception e) {
+      exitCode = 1;
+      LOG.debug("Error", e);
+      displayError(cmd, e);
+      if (e instanceof IllegalArgumentException) {
+        exitCode = -1;
+        printUsage(cmd);
+      }
     }
     return exitCode;
   }

+ 151 - 0
src/java/org/apache/hadoop/fs/shell/CommandWithDestination.java

@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.shell;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.LinkedList;
+
+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.PathIsNotDirectoryException;
+import org.apache.hadoop.fs.shell.PathExceptions.PathNotFoundException;
+
+/**
+ * Provides: argument processing to ensure the destination is valid
+ * for the number of source arguments.  A processPaths that accepts both
+ * a source and resolved target.  Sources are resolved as children of
+ * a destination directory.
+ */
+abstract class CommandWithDestination extends FsCommand {  
+  protected PathData dst;
+  protected boolean overwrite = false;
+  
+  // TODO: commands should implement a -f to enable this
+  protected void setOverwrite(boolean flag) {
+    overwrite = flag;
+  }
+  
+  /**
+   *  The last arg is expected to be a local path, if only one argument is
+   *  given then the destination will be the current directory 
+   *  @param args is the list of arguments
+   */
+  protected void getLocalDestination(LinkedList<String> args)
+  throws IOException {
+    String pathString = (args.size() < 2) ? Path.CUR_DIR : args.removeLast();
+    dst = new PathData(new File(pathString), getConf());
+  }
+
+  /**
+   *  The last arg is expected to be a remote path, if only one argument is
+   *  given then the destination will be the remote user's directory 
+   *  @param args is the list of arguments
+   *  @throws PathIOException if path doesn't exist or matches too many times 
+   */
+  protected void getRemoteDestination(LinkedList<String> args)
+  throws IOException {
+    if (args.size() < 2) {
+      dst = new PathData(Path.CUR_DIR, getConf());
+    } else {
+      String pathString = args.removeLast();
+      // if the path is a glob, then it must match one and only one path
+      PathData[] items = PathData.expandAsGlob(pathString, getConf());
+      switch (items.length) {
+        case 0:
+          throw new PathNotFoundException(pathString);
+        case 1:
+          dst = items[0];
+          break;
+        default:
+          throw new PathIOException(pathString, "Too many matches");
+      }
+    }
+  }
+
+  @Override
+  protected void processArguments(LinkedList<PathData> args)
+  throws IOException {
+    // if more than one arg, the destination must be a directory
+    // if one arg, the dst must not exist or must be a directory
+    if (args.size() > 1) {
+      if (!dst.exists) {
+        throw new PathNotFoundException(dst.toString());
+      }
+      if (!dst.stat.isDirectory()) {
+        throw new PathIsNotDirectoryException(dst.toString());
+      }
+    } else {
+      if (dst.exists && !dst.stat.isDirectory() && !overwrite) {
+        throw new PathExistsException(dst.toString());
+      }
+    }
+    super.processArguments(args);
+  }
+
+  @Override
+  protected void processPaths(PathData parent, PathData ... items)
+  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);
+    } finally {
+      dst = savedDst;
+    }
+  }
+  
+  @Override
+  protected void processPath(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()) {
+      target = dst.getPathDataForChild(src);
+    } else {
+      target = dst;
+    }
+    if (target.exists && !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) {
+        e.setTargetPath(target.toString());
+      }
+      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;
+}

+ 0 - 72
src/java/org/apache/hadoop/fs/shell/Copy.java

@@ -1,72 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.shell;
-
-import java.io.IOException;
-import java.util.LinkedList;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.Path;
-
-/** Various commands for copy files */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-
-class Copy extends FsCommand {  
-  public static void registerCommands(CommandFactory factory) {
-    factory.addClass(Merge.class, "-getmerge");
-  }
-
-  /** merge multiple files together */
-  public static class Merge extends Copy {
-    public static final String NAME = "MergeToLocal";    
-    public static final String USAGE = "<src> <localdst> [addnl]";
-    public static final String DESCRIPTION =
-      "Get all the files in the directories that\n" +
-      "match the source file pattern and merge and sort them to only\n" +
-      "one file on local fs. <src> is kept.\n";
-
-    protected PathData dst = null;
-    protected String delimiter = null;
-
-    @Override
-    protected void processOptions(LinkedList<String> args) throws IOException {
-      CommandFormat cf = new CommandFormat(null, 2, 3);
-      cf.parse(args);
-
-      // TODO: this really should be a -nl option
-      if ((args.size() > 2) && Boolean.parseBoolean(args.removeLast())) {
-        delimiter = "\n";
-      } else {
-        delimiter = null;
-      }
-      
-      Path path = new Path(args.removeLast());
-      dst = new PathData(path.getFileSystem(getConf()), path);
-    }
-
-    @Override
-    protected void processPath(PathData src) throws IOException {
-      FileUtil.copyMerge(src.fs, src.path,
-          dst.fs, dst.path, false, getConf(), delimiter);
-    }
-  }
-}

+ 274 - 0
src/java/org/apache/hadoop/fs/shell/CopyCommands.java

@@ -0,0 +1,274 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.shell;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.LinkedList;
+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.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.LocalFileSystem;
+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.PathOperationException;
+import org.apache.hadoop.io.IOUtils;
+
+/** Various commands for copy files */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+class CopyCommands {  
+  public static void registerCommands(CommandFactory factory) {
+    factory.addClass(Merge.class, "-getmerge");
+    factory.addClass(Cp.class, "-cp");
+    factory.addClass(CopyFromLocal.class, "-copyFromLocal");
+    factory.addClass(CopyToLocal.class, "-copyToLocal");
+    factory.addClass(Get.class, "-get");
+    factory.addClass(Put.class, "-put");
+  }
+
+  /** merge multiple files together */
+  public static class Merge extends FsCommand {
+    public static final String NAME = "getmerge";    
+    public static final String USAGE = "<src> <localdst> [addnl]";
+    public static final String DESCRIPTION =
+      "Get all the files in the directories that\n" +
+      "match the source file pattern and merge and sort them to only\n" +
+      "one file on local fs. <src> is kept.";
+
+    protected PathData dst = null;
+    protected String delimiter = null;
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      CommandFormat cf = new CommandFormat(null, 2, 3);
+      cf.parse(args);
+
+      // TODO: this really should be a -nl option
+      if ((args.size() > 2) && Boolean.parseBoolean(args.removeLast())) {
+        delimiter = "\n";
+      } else {
+        delimiter = null;
+      }
+
+      dst = new PathData(new File(args.removeLast()), getConf());
+    }
+
+    @Override
+    protected void processPath(PathData src) throws IOException {
+      FileUtil.copyMerge(src.fs, src.path,
+          dst.fs, dst.path, false, getConf(), delimiter);
+    }
+  }
+
+  static class Cp extends CommandWithDestination {
+    public static final String NAME = "cp";
+    public static final String USAGE = "<src> ... <dst>";
+    public static final String DESCRIPTION =
+      "Copy files that match the file pattern <src> to a\n" +
+      "destination.  When copying multiple files, the destination\n" +
+      "must be a directory.";
+    
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      CommandFormat cf = new CommandFormat(null, 2, Integer.MAX_VALUE);
+      cf.parse(args);
+      getRemoteDestination(args);
+    }
+
+    @Override
+    protected void processPath(PathData src, PathData target)
+    throws IOException {
+      if (!FileUtil.copy(src.fs, src.path, target.fs, target.path, false, 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());
+      }
+    }
+  }
+  
+  /** 
+   * Copy local files to a remote filesystem
+   */
+  public static class Get extends CommandWithDestination {
+    public static final String NAME = "get";
+    public static final String USAGE =
+      "[-ignoreCrc] [-crc] <src> ... <localdst>";
+    public static final String DESCRIPTION =
+      "Copy files that match the file pattern <src>\n" +
+      "to the local name.  <src> is kept.  When copying multiple,\n" +
+      "files, the destination must be a directory.";
+
+    /**
+     * 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)}.
+     */
+    private static final String COPYTOLOCAL_PREFIX = "_copyToLocal_";
+    private boolean copyCrc;
+    private boolean verifyChecksum;
+    private LocalFileSystem localFs;
+
+    @Override
+    protected void processOptions(LinkedList<String> args)
+    throws IOException {
+      localFs = FileSystem.getLocal(getConf());
+      CommandFormat cf = new CommandFormat(
+          null, 1, Integer.MAX_VALUE, "crc", "ignoreCrc");
+      cf.parse(args);
+      copyCrc = cf.getOpt("crc");
+      verifyChecksum = !cf.getOpt("ignoreCrc");
+      
+      setRecursive(true);
+      getLocalDestination(args);
+    }
+
+    @Override
+    protected void processPath(PathData src, PathData target)
+    throws IOException {
+      src.fs.setVerifyChecksum(verifyChecksum);
+
+      if (copyCrc && !(src.fs instanceof ChecksumFileSystem)) {
+        displayWarning(src.fs + ": Does not support checksums");
+        copyCrc = false;
+      }      
+
+      File targetFile = localFs.pathToFile(target.path);
+      if (src.stat.isFile()) {
+        // copy the file and maybe its crc
+        copyFileToLocal(src, target.path);
+        if (copyCrc) {
+          copyCrcToLocal(src, target.path);
+        }
+      } else if (src.stat.isDirectory()) {
+        // create the remote directory structure locally
+        if (!targetFile.mkdirs()) {
+          throw new PathIOException(target.toString());
+        }
+      } else {
+        throw new PathOperationException(src.toString());
+      }
+    }
+
+    private void copyFileToLocal(PathData src, Path target)
+    throws IOException {
+      File targetFile = localFs.pathToFile(target);
+      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;
+      }
+    }
+
+    private void copyCrcToLocal(PathData src, Path target)
+    throws IOException {
+      ChecksumFileSystem srcFs = (ChecksumFileSystem)src.fs;
+      Path srcPath = srcFs.getChecksumFile(src.path);
+      src = new PathData(srcFs.getRawFileSystem(), srcPath);
+      copyFileToLocal(src, localFs.getChecksumFile(target));    
+    }
+  }
+
+  /**
+   *  Copy local files to a remote filesystem
+   */
+  public static class Put extends CommandWithDestination {
+    public static final String NAME = "put";
+    public static final String USAGE = "<localsrc> ... <dst>";
+    public static final String DESCRIPTION =
+      "Copy files from the local file system\n" +
+      "into fs.";
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      CommandFormat cf = new CommandFormat(null, 1, Integer.MAX_VALUE);
+      cf.parse(args);
+      getRemoteDestination(args);
+    }
+
+    // commands operating on local paths have no need for glob expansion
+    @Override
+    protected List<PathData> expandArgument(String arg) throws IOException {
+      List<PathData> items = new LinkedList<PathData>();
+      items.add(new PathData(new File(arg), getConf()));
+      return items;
+    }
+
+    @Override
+    protected void processArguments(LinkedList<PathData> args)
+    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();
+        return;
+      }
+      super.processArguments(args);
+    }
+
+    @Override
+    protected void processPath(PathData src, PathData target)
+    throws IOException {
+      target.fs.copyFromLocalFile(false, false, 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 {
+    public static final String NAME = "copyFromLocal";
+    public static final String USAGE = Put.USAGE;
+    public static final String DESCRIPTION = "Identical to the -put command.";
+  }
+ 
+  public static class CopyToLocal extends Get {
+    public static final String NAME = "copyToLocal";
+    public static final String USAGE = Get.USAGE;
+    public static final String DESCRIPTION = "Identical to the -get command.";
+  }
+}

+ 2 - 1
src/java/org/apache/hadoop/fs/shell/FsCommand.java

@@ -43,7 +43,7 @@ abstract public class FsCommand extends Command {
    * @param factory where to register the class
    */
   public static void registerCommands(CommandFactory factory) {
-    factory.registerCommands(Copy.class);
+    factory.registerCommands(CopyCommands.class);
     factory.registerCommands(Count.class);
     factory.registerCommands(Delete.class);
     factory.registerCommands(Display.class);
@@ -51,6 +51,7 @@ abstract public class FsCommand extends Command {
     factory.registerCommands(FsUsage.class);
     factory.registerCommands(Ls.class);
     factory.registerCommands(Mkdir.class);
+    factory.registerCommands(MoveCommands.class);
     factory.registerCommands(SetReplication.class);
     factory.registerCommands(Stat.class);
     factory.registerCommands(Tail.class);

+ 0 - 3
src/java/org/apache/hadoop/fs/shell/Ls.java

@@ -127,9 +127,6 @@ class Ls extends FsCommand {
     return Math.max(n, (value != null) ? String.valueOf(value).length() : 0);
   }
 
-  @Override
-  protected int exitCodeForError() { return -1; }
-
   /**
    * Get a recursive listing of all files in that match the file patterns.
    * Same as "-ls -R"

+ 98 - 0
src/java/org/apache/hadoop/fs/shell/MoveCommands.java

@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.shell;
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.shell.CopyCommands.CopyFromLocal;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIOException;
+
+/** Various commands for moving files */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+class MoveCommands {
+  public static void registerCommands(CommandFactory factory) {
+    factory.addClass(MoveFromLocal.class, "-moveFromLocal");
+    factory.addClass(MoveToLocal.class, "-moveToLocal");
+    factory.addClass(Rename.class, "-mv");
+  }
+
+  /**
+   *  Move local files to a remote filesystem
+   */
+  public static class MoveFromLocal extends CopyFromLocal {
+    public static final String NAME = "moveFromLocal";
+    public static final String USAGE = "<localsrc> ... <dst>";
+    public static final String DESCRIPTION = 
+      "Same as -put, except that the source is\n" +
+      "deleted after it's copied.";
+
+    @Override
+    protected void processPath(PathData src, PathData target) throws IOException {
+      target.fs.moveFromLocalFile(src.path, target.path);
+    }
+  }
+
+  /**
+   *  Move remote files to a local filesystem
+   */
+  public static class MoveToLocal extends FsCommand { 
+    public static final String NAME = "moveToLocal";
+    public static final String USAGE = "<src> <localdst>";
+    public static final String DESCRIPTION = "Not implemented yet";
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      throw new IOException("Option '-moveToLocal' is not implemented yet.");
+    }
+  }
+
+  /** move/rename paths on the same fileystem */
+  public static class Rename extends CommandWithDestination {
+    public static final String NAME = "mv";
+    public static final String USAGE = "<src> ... <dst>";
+    public static final String DESCRIPTION = 
+      "Move files that match the specified file pattern <src>\n" +
+      "to a destination <dst>.  When moving multiple files, the\n" +
+      "destination must be a directory.";
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      CommandFormat cf = new CommandFormat(null, 2, Integer.MAX_VALUE);
+      cf.parse(args);
+      getRemoteDestination(args);
+    }
+
+    @Override
+    protected void processPath(PathData src, PathData target) throws IOException {
+      if (!src.fs.getUri().equals(target.fs.getUri())) {
+        throw new PathIOException(src.toString(),
+            "Does not match target filesystem");
+      }
+      if (!target.fs.rename(src.path, target.path)) {
+        // we have no way to know the actual error...
+        throw new PathIOException(src.toString());
+      }
+    }
+  }
+}

+ 29 - 0
src/java/org/apache/hadoop/fs/shell/PathData.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.shell;
 
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 
@@ -58,6 +59,21 @@ public class PathData {
     setStat(getStat(fs, path));
   }
   
+  /**
+   * Creates an object to wrap the given parameters as fields.  The string
+   * used to create the path will be recorded since the Path object does not
+   * return exactly the same string used to initialize it
+   * @param localPath a local File
+   * @param conf the configuration file
+   * @throws IOException if anything goes wrong...
+   */
+  public PathData(File localPath, Configuration conf) throws IOException {
+    this.string = localPath.toString();
+    this.path = new Path(this.string);
+    this.fs = FileSystem.getLocal(conf);
+    setStat(getStat(fs, path));
+  }
+
   /**
    * Creates an object to wrap the given parameters as fields. 
    * @param fs the FileSystem
@@ -156,6 +172,19 @@ public class PathData {
     return items;
   }
 
+  /**
+   * Creates a new object for a child entry in this directory
+   * @param child the basename will be appended to this object's path
+   * @return PathData for the child
+   * @throws IOException if this object does not exist or is not a directory
+   */
+  public PathData getPathDataForChild(PathData child) throws IOException {
+    if (!stat.isDirectory()) {
+      throw new PathIsNotDirectoryException(string);
+    }
+    return new PathData(fs, new Path(path, child.path.getName()));
+  }
+
   /**
    * Expand the given path as a glob pattern.  Non-existent paths do not
    * throw an exception because creation commands like touch and mkdir need

+ 51 - 4
src/java/org/apache/hadoop/fs/shell/PathExceptions.java

@@ -40,7 +40,9 @@ public class PathExceptions {
     // NOTE: this really should be a Path, but a Path is buggy and won't
     // return the exact string used to construct the path, and it mangles
     // uris with no authority
+    private String operation;
     private String path;
+    private String targetPath;
 
     /**
      * Constructor a generic I/O error exception
@@ -74,17 +76,53 @@ public class PathExceptions {
       this.path = path;
     }
 
+    /** Format:
+     * cmd: {operation} `path' {to `target'}: error string
+     */
     @Override
     public String getMessage() {
-      String message = "`" + path + "': " + super.getMessage();
+      StringBuilder message = new StringBuilder();
+      if (operation != null) {
+        message.append(operation + " ");
+      }
+      message.append(formatPath(path));
+      if (targetPath != null) {
+        message.append(" to " + formatPath(targetPath));
+      }
+      message.append(": " + super.getMessage());
       if (getCause() != null) {
-        message += ": " + getCause().getMessage();
+        message.append(": " + getCause().getMessage());
       }
-      return message;
+      return message.toString();
     }
 
     /** @return Path that generated the exception */
     public Path getPath()  { return new Path(path); }
+
+    /** @return Path if the operation involved copying or moving, else null */
+    public Path getTargetPath() {
+      return (targetPath != null) ? new Path(targetPath) : null;
+    }    
+    
+    /**
+     * Optional operation that will preface the path
+     * @param operation a string
+     */
+    public void setOperation(String operation) {
+      this.operation = operation;
+    }
+    
+    /**
+     * Optional path if the exception involved two paths, ex. a copy operation
+     * @param targetPath the of the operation
+     */
+    public void setTargetPath(String targetPath) {
+      this.targetPath = targetPath;
+    }
+    
+    private String formatPath(String path) {
+      return "`" + path + "'";
+    }
   }
 
   /** ENOENT */
@@ -144,4 +182,13 @@ public class PathExceptions {
       super(path, "Operation not permitted");
     }
   }
-}
+  
+  /** ENOTSUP */
+  public static class PathOperationException extends PathExistsException {
+    static final long serialVersionUID = 0L;
+    /** @param path for the exception */
+    public PathOperationException(String path) {
+      super(path, "Operation not supported");
+    }
+  }
+}

+ 12 - 12
src/test/core/org/apache/hadoop/cli/testConf.xml

@@ -117,11 +117,11 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-get( )*\[-ignoreCrc\]( )*\[-crc\]( )*&lt;src&gt; &lt;localdst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt;( )*</expected-output>
+          <expected-output>^-get( )*\[-ignoreCrc\]( )*\[-crc\]( )*&lt;src&gt; \.\.\. &lt;localdst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt;( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*to the local name.( )*&lt;src&gt; is kept.( )*When copying mutiple,( )*</expected-output>
+          <expected-output>^( |\t)*to the local name.( )*&lt;src&gt; is kept.( )*When copying multiple,( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -237,7 +237,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-mv &lt;src&gt; &lt;dst&gt;:( |\t)*Move files that match the specified file pattern &lt;src&gt;( )*</expected-output>
+          <expected-output>^-mv &lt;src&gt; \.\.\. &lt;dst&gt;:( |\t)*Move files that match the specified file pattern &lt;src&gt;( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -260,7 +260,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-cp &lt;src&gt; &lt;dst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt; to a( )*</expected-output>
+          <expected-output>^-cp &lt;src&gt; \.\.\. &lt;dst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt; to a( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -325,7 +325,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-put &lt;localsrc&gt; ... &lt;dst&gt;:( |\t)*Copy files from the local file system( )*</expected-output>
+          <expected-output>^-put &lt;localsrc&gt; \.\.\. &lt;dst&gt;:\s+Copy files from the local file system</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -344,7 +344,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-copyFromLocal &lt;localsrc&gt; ... &lt;dst&gt;:( )*Identical to the -put command.( )*</expected-output>
+          <expected-output>^-copyFromLocal &lt;localsrc&gt; \.\.\. &lt;dst&gt;:\s+Identical to the -put command\.</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -359,11 +359,11 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-moveFromLocal &lt;localsrc&gt; ... &lt;dst&gt;: Same as -put, except that the source is( )*</expected-output>
+          <expected-output>^-moveFromLocal &lt;localsrc&gt; \.\.\. &lt;dst&gt;:\s+Same as -put, except that the source is</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*deleted after it's copied.( )*</expected-output>
+          <expected-output>^( |\t)*deleted after it's copied.</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -379,11 +379,11 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-get( )*\[-ignoreCrc\]( )*\[-crc\]( )*&lt;src&gt; &lt;localdst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt;( )*</expected-output>
+          <expected-output>^-get( )*\[-ignoreCrc\]( )*\[-crc\]( )*&lt;src&gt; \.\.\. &lt;localdst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt;( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*to the local name.( )*&lt;src&gt; is kept.( )*When copying mutiple,( )*</expected-output>
+          <expected-output>^( |\t)*to the local name.( )*&lt;src&gt; is kept.( )*When copying multiple,( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -445,7 +445,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-copyToLocal \[-ignoreCrc\] \[-crc\] &lt;src&gt; &lt;localdst&gt;:( )*Identical to the -get command.( )*</expected-output>
+          <expected-output>^-copyToLocal \[-ignoreCrc\] \[-crc\] &lt;src&gt; \.\.\. &lt;localdst&gt;:\s+Identical to the -get command.</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -460,7 +460,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-moveToLocal &lt;src&gt; &lt;localdst&gt;:( )*Not implemented yet( )*</expected-output>
+          <expected-output>^-moveToLocal &lt;src&gt; &lt;localdst&gt;:\s+Not implemented yet</expected-output>
         </comparator>
       </comparators>
     </test>

+ 9 - 3
src/test/core/org/apache/hadoop/fs/TestFsShellReturnCode.java

@@ -50,7 +50,7 @@ public class TestFsShellReturnCode {
   
   @BeforeClass
   public static void setup() throws IOException {
-    conf.setClass("fs.file.impl", LocalFileSystemExtn.class, RawLocalFileSystem.class);
+    conf.setClass("fs.file.impl", LocalFileSystemExtn.class, LocalFileSystem.class);
     fileSys = FileSystem.get(conf);
     fsShell = new FsShell(conf);
   }
@@ -285,7 +285,7 @@ public class TestFsShellReturnCode {
       assertTrue("file exists", !fileSys.exists(new Path(args[1])));
       int run = shell.run(args);
       results = bytes.toString();
-      assertTrue("Return code should be -1", run == -1);
+      assertEquals("Return code should be 1", 1, run);
       assertTrue(" Null is coming when source path is invalid. ",!results.contains("get: null"));
       assertTrue(" Not displaying the intended message ",results.contains("get: `"+args[1]+"': No such file or directory"));
     } finally {
@@ -325,7 +325,13 @@ public class TestFsShellReturnCode {
     
   }
   
-  static class LocalFileSystemExtn extends RawLocalFileSystem {
+  static class LocalFileSystemExtn extends LocalFileSystem {
+    public LocalFileSystemExtn() {
+      super(new RawLocalFileSystemExtn());
+    }
+  }
+
+  static class RawLocalFileSystemExtn extends RawLocalFileSystem {
     protected static HashMap<String,String> owners = new HashMap<String,String>();
     protected static HashMap<String,String> groups = new HashMap<String,String>();