You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sz...@apache.org on 2011/05/10 23:29:34 UTC

svn commit: r1101653 - in /hadoop/common/trunk: ./ src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/fs/shell/ src/test/core/org/apache/hadoop/fs/ src/test/core/org/apache/hadoop/fs/shell/

Author: szetszwo
Date: Tue May 10 21:29:34 2011
New Revision: 1101653

URL: http://svn.apache.org/viewvc?rev=1101653&view=rev
Log:
HADOOP-7271. Standardize shell command error messages.  Contributed by Daryn Sharp

Added:
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathExceptions.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathExceptions.java
Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShell.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShellPermissions.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Command.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Count.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Display.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Ls.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Mkdir.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/SetReplication.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Tail.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestFsShellReturnCode.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Tue May 10 21:29:34 2011
@@ -130,6 +130,9 @@ Trunk (unreleased changes)
     HADOOP-7238. Refactor the cat and text commands to conform to new FsCommand
     class.  (Daryn Sharp via szetszwo)
 
+    HADOOP-7271. Standardize shell command error messages.  (Daryn Sharp
+    via szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShell.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShell.java?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShell.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShell.java Tue May 10 21:29:34 2011
@@ -28,6 +28,8 @@ import java.util.Arrays;
 import java.util.Date;
 import java.util.List;
 import java.util.TimeZone;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -38,6 +40,8 @@ import org.apache.hadoop.fs.shell.Comman
 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.PathData;
+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;
@@ -207,7 +211,7 @@ public class FsShell extends Configured 
       }
       FileStatus[] srcs = srcFS.globStatus(srcpath);
       if (null == srcs) {
-	throw new IOException(srcpath + ": No such file or directory");
+        throw new PathNotFoundException(srcstr);
       }
       boolean dstIsDir = dst.isDirectory(); 
       if (srcs.length > 1 && !dstIsDir) {
@@ -323,7 +327,7 @@ public class FsShell extends Configured 
     final Path srcPath = new Path(path);
     final FileSystem srcFs = srcPath.getFileSystem(getConf());
     if (! srcFs.exists(srcPath)) {
-      throw new FileNotFoundException("Cannot access "+srcPath.toString());
+      throw new PathNotFoundException(path);
     }
     final FsStatus stats = srcFs.getStatus(srcPath);
     final int PercentUsed = (int)(100.0f *  (float)stats.getUsed() / (float)stats.getCapacity());
@@ -379,8 +383,7 @@ public class FsShell extends Configured 
       }
       if ((statusToPrint == null) || ((statusToPrint.length == 0) &&
                                       (!srcFs.exists(srcPath)))){
-        throw new FileNotFoundException("Cannot access " + src
-                                        + ": No such file or directory.");
+        throw new PathNotFoundException(src);
       }
 
       if (!summary) {
@@ -463,15 +466,20 @@ public class FsShell extends Configured 
     if (!argv[i].startsWith("-") || argv[i].length() > 2)
       throw new IOException("Not a flag: " + argv[i]);
     char flag = argv[i].toCharArray()[1];
-    Path f = new Path(argv[++i]);
-    FileSystem srcFs = f.getFileSystem(getConf());
+    PathData item = new PathData(argv[++i], getConf());
+    
+    if ((flag != 'e') && !item.exists) { 
+      // TODO: it's backwards compat, but why is this throwing an exception?
+      // it's not like the shell test cmd
+      throw new PathNotFoundException(item.toString());
+    }
     switch(flag) {
       case 'e':
-        return srcFs.exists(f) ? 0 : 1;
+        return item.exists ? 0 : 1;
       case 'z':
-        return srcFs.getFileStatus(f).getLen() == 0 ? 0 : 1;
+        return (item.stat.getLen() == 0) ? 0 : 1;
       case 'd':
-        return srcFs.getFileStatus(f).isDirectory() ? 0 : 1;
+        return item.stat.isDirectory() ? 0 : 1;
       default:
         throw new IOException("Unknown flag: " + flag);
     }
@@ -492,7 +500,7 @@ public class FsShell extends Configured 
     FileSystem srcFs = srcPath.getFileSystem(getConf());
     FileStatus glob[] = srcFs.globStatus(srcPath);
     if (null == glob)
-      throw new IOException("cannot stat `" + src + "': No such file or directory");
+      throw new PathNotFoundException(src);
     for (FileStatus f : glob) {
       StringBuilder buf = new StringBuilder();
       for (int i = 0; i < fmt.length; ++i) {
@@ -565,8 +573,7 @@ public class FsShell extends Configured 
         try {
           srcFstatus = fs.getFileStatus(srcs[i]);
         } catch(FileNotFoundException e) {
-          throw new FileNotFoundException(srcs[i] + 
-          ": No such file or directory");
+          throw new PathNotFoundException(srcs[i].toString());
         }
         try {
           dstFstatus = fs.getFileStatus(dst);
@@ -637,10 +644,9 @@ public class FsShell extends Configured 
         LOG.debug("Error renaming " + argv[i], e);
         //
         // IO exception encountered locally.
-        //
+        // 
         exitCode = -1;
-        System.err.println(cmd.substring(1) + ": " +
-                           e.getLocalizedMessage());
+        displayError(cmd, e);
       }
     }
     return exitCode;
@@ -705,30 +711,10 @@ public class FsShell extends Configured 
         // issue the copy to the fs
         //
         copy(argv[i], dest, conf);
-      } catch (RemoteException e) {
-        LOG.debug("Error copying " + 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 copying " + argv[i], e);
-        //
-        // IO exception encountered locally.
-        //
         exitCode = -1;
-        System.err.println(cmd.substring(1) + ": " +
-                           e.getLocalizedMessage());
+        displayError(cmd, e);
       }
     }
     return exitCode;
@@ -767,8 +753,7 @@ public class FsShell extends Configured 
       fs = srcFs.getFileStatus(src);
     } catch (FileNotFoundException fnfe) {
       // Have to re-throw so that console output is as expected
-      throw new FileNotFoundException("cannot remove "
-          + src + ": No such file or directory.");
+      throw new PathNotFoundException(src.toString());
     }
     
     if (fs.isDirectory() && !recursive) {
@@ -1061,34 +1046,10 @@ public class FsShell extends Configured 
         } else if ("-touchz".equals(cmd)) {
           touchz(argv[i]);
         }
-      } catch (RemoteException e) {
-        LOG.debug("Error", e);
-        //
-        // This is a error returned by hadoop server. Print
-        // out the first line of the error message.
-        //
-        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", e);
-        //
-        // IO exception encountered locally.
-        //
         exitCode = -1;
-        String content = e.getLocalizedMessage();
-        if (content != null) {
-          content = content.split("\n")[0];
-        }
-        System.err.println(cmd.substring(1) + ": " +
-                          content);
+        displayError(cmd, e);
       }
     }
     return exitCode;
@@ -1306,38 +1267,36 @@ public class FsShell extends Configured 
       exitCode = -1;
       System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
       printUsage(cmd);
-    } catch (RemoteException e) {
-      LOG.debug("Error", e);
-      //
-      // This is a error returned by hadoop server. Print
-      // out the first line of the error mesage, ignore the stack trace.
-      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", e);
-      //
-      // IO exception encountered locally.
-      // 
-      exitCode = -1;
-      System.err.println(cmd.substring(1) + ": " + 
-                         e.getLocalizedMessage());  
     } catch (Exception re) {
       LOG.debug("Error", re);
       exitCode = -1;
-      System.err.println(cmd.substring(1) + ": " + re.getLocalizedMessage());  
+      displayError(cmd, re);
     } finally {
     }
     return exitCode;
   }
 
+  // TODO: this is a quick workaround to accelerate the integration of
+  // redesigned commands.  this will be removed this once all commands are
+  // converted.  this change will avoid having to change the hdfs tests
+  // every time a command is converted to use path-based exceptions
+  private static Pattern[] fnfPatterns = {
+    Pattern.compile("File (.*) does not exist\\."),
+    Pattern.compile("File does not exist: (.*)"),
+    Pattern.compile("`(.*)': specified destination directory doest not exist")
+  };
+  private void displayError(String cmd, Exception e) {
+    String message = e.getLocalizedMessage().split("\n")[0];
+    for (Pattern pattern : fnfPatterns) {
+      Matcher matcher = pattern.matcher(message);
+      if (matcher.matches()) {
+        message = new PathNotFoundException(matcher.group(1)).getMessage();
+        break;
+      }
+    }
+    System.err.println(cmd.substring(1) + ": " + message);  
+  }
+  
   public void close() throws IOException {
     if (fs != null) {
       fs.close();

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShellPermissions.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShellPermissions.java?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShellPermissions.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShellPermissions.java Tue May 10 21:29:34 2011
@@ -53,12 +53,6 @@ public class FsShellPermissions extends 
     factory.addClass(Chgrp.class, "-chgrp");
   }
 
-  @Override
-  protected String getFnfText(Path path) {
-    // TODO: printing the path twice is silly for backwards compatibility
-    return "could not get status for '"+path+"': File does not exist: "+path;   
-  }
-
   /**
    * The pattern is almost as flexible as mode allowed by chmod shell command.
    * The main restriction is that we recognize only rwxXt. To reduce errors we

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Command.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Command.java?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Command.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Command.java Tue May 10 21:29:34 2011
@@ -32,6 +32,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.PathExceptions.PathNotFoundException;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -199,7 +200,7 @@ abstract public class Command extends Co
     PathData[] items = PathData.expandAsGlob(arg, getConf());
     if (items.length == 0) {
       // it's a glob that failed to match
-      throw new FileNotFoundException(getFnfText(new Path(arg)));
+      throw new PathNotFoundException(arg);
     }
     return Arrays.asList(items);
   }
@@ -261,20 +262,7 @@ abstract public class Command extends Co
    *  @throws IOException if anything else goes wrong... 
    */
   protected void processNonexistentPath(PathData item) throws IOException {
-    // TODO: this should be more posix-like: ex. "No such file or directory"
-    throw new FileNotFoundException(getFnfText(item.path));
-  }
-
-  /**
-   *  TODO: A crutch until the text is standardized across commands...
-   *  Eventually an exception that takes the path as an argument will
-   *  replace custom text, until then, commands can supply custom text
-   *  for backwards compatibility
-   *  @param path the thing that doesn't exist
-   *  @returns String in printf format
-   */
-  protected String getFnfText(Path path) {
-    return path + ": No such file or directory";
+    throw new PathNotFoundException(item.toString());
   }
 
   /**

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Count.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Count.java?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Count.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Count.java Tue May 10 21:29:34 2011
@@ -26,7 +26,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FsShell;
-import org.apache.hadoop.fs.Path;
 
 /**
  * Count the number of directories, files, bytes, quota, and remaining quota.
@@ -87,10 +86,4 @@ public class Count extends FsCommand {
     ContentSummary summary = src.fs.getContentSummary(src.path);
     out.println(summary.toString(showQuotas) + src.path);
   }
-
-  // TODO: remove when the error is commonized...
-  @Override
-  protected String getFnfText(Path path) {
-    return "Can not find listing for " + path;
-  }
 }

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Display.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Display.java?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Display.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Display.java Tue May 10 21:29:34 2011
@@ -27,8 +27,8 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIsDirectoryException;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.IOUtils;
@@ -51,20 +51,6 @@ class Display extends FsCommand {
     factory.addClass(Text.class, "-text");
   }
 
-  @Override
-  protected String getFnfText(Path path) {
-    // TODO: this is a pretty inconsistent way to output the path...!!
-    //       but, it's backwards compatible
-    try {
-      FileSystem fs = path.getFileSystem(getConf());
-      path = fs.makeQualified(path);
-    } catch (IOException e) {
-      // shouldn't happen, so just use path as-is
-      displayWarning("can't fully qualify "+path);
-    }
-    return "File does not exist: " + path.toUri().getPath();
-  }
-
   /**
    * Displays file content to stdout
    */
@@ -87,6 +73,10 @@ class Display extends FsCommand {
 
     @Override
     protected void processPath(PathData item) throws IOException {
+      if (item.stat.isDirectory()) {
+        throw new PathIsDirectoryException(item.toString());
+      }
+      
       item.fs.setVerifyChecksum(verifyChecksum);
       printToStdout(getInputStream(item));
     }

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Ls.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Ls.java?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Ls.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Ls.java Tue May 10 21:29:34 2011
@@ -126,12 +126,6 @@ class Ls extends FsCommand {
     return Math.max(n, (value != null) ? String.valueOf(value).length() : 0);
   }
 
-  // TODO: remove when the error is commonized...
-  @Override
-  protected String getFnfText(Path path) {
-    return "Cannot access " + path.toUri() + ": No such file or directory.";
-  }
-
   @Override
   protected int exitCodeForError() { return -1; }
 

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Mkdir.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Mkdir.java?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Mkdir.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Mkdir.java Tue May 10 21:29:34 2011
@@ -23,6 +23,9 @@ import java.util.LinkedList;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.shell.PathExceptions.PathExistsException;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIOException;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIsNotDirectoryException;
 
 /**
  * Create the given dir
@@ -49,16 +52,16 @@ class Mkdir extends FsCommand {
   @Override
   protected void processPath(PathData item) throws IOException {
     if (item.stat.isDirectory()) {
-      throw new IOException("cannot create directory " + item + ": File exists");
+      throw new PathExistsException(item.toString());
     } else {
-      throw new IOException(item + " exists but is not a directory");
+      throw new PathIsNotDirectoryException(item.toString());
     }
   }
 
   @Override
   protected void processNonexistentPath(PathData item) throws IOException {
     if (!item.fs.mkdirs(item.path)) {
-      throw new IOException("failed to create " + item);
+      throw new PathIOException(item.toString());
     }
   }
 }

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java Tue May 10 21:29:34 2011
@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIsNotDirectoryException;
 
 /**
  * Encapsulates a Path (path), its FileStatus (stat), and its FileSystem (fs).
@@ -140,7 +141,7 @@ public class PathData {
    */
   public PathData[] getDirectoryContents() throws IOException {
     if (!stat.isDirectory()) {
-      throw new IOException(path + ": Not a directory");
+      throw new PathIsNotDirectoryException(string);
     }
 
     FileStatus[] stats = fs.listStatus(path);

Added: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathExceptions.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathExceptions.java?rev=1101653&view=auto
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathExceptions.java (added)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathExceptions.java Tue May 10 21:29:34 2011
@@ -0,0 +1,147 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Standardized posix/linux style exceptions for path related errors.
+ * Returns an IOException with the format "path: standard error string".
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+
+public class PathExceptions {
+
+  /** EIO */
+  public static class PathIOException extends IOException {
+    static final long serialVersionUID = 0L;
+    private static final String EIO = "Input/output error";
+    // 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 path;
+
+    /**
+     * Constructor a generic I/O error exception
+     *  @param path for the exception
+     */
+    public PathIOException(String path) {
+      this(path, EIO, null);
+    }
+
+    /**
+     * Appends the text of a Throwable to the default error message
+     * @param path for the exception
+     * @param cause a throwable to extract the error message
+     */
+    public PathIOException(String path, Throwable cause) {
+      this(path, EIO, cause);
+    }
+
+    /**
+     * Avoid using this method.  Use a subclass of PathIOException if
+     * possible.
+     * @param path for the exception
+     * @param error custom string to use an the error text
+     */
+    public PathIOException(String path, String error) {
+      this(path, error, null);
+    }
+
+    protected PathIOException(String path, String error, Throwable cause) {
+      super(error, cause);
+      this.path = path;
+    }
+
+    @Override
+    public String getMessage() {
+      String message = "`" + path + "': " + super.getMessage();
+      if (getCause() != null) {
+        message += ": " + getCause().getMessage();
+      }
+      return message;
+    }
+
+    /** @return Path that generated the exception */
+    public Path getPath()  { return new Path(path); }
+  }
+
+  /** ENOENT */
+  public static class PathNotFoundException extends PathIOException {
+    static final long serialVersionUID = 0L;
+    /** @param path for the exception */
+    public PathNotFoundException(String path) {
+      super(path, "No such file or directory");
+    }    
+  }
+
+  /** EEXISTS */
+  public static class PathExistsException extends PathIOException {
+    static final long serialVersionUID = 0L;
+    /** @param path for the exception */
+    public PathExistsException(String path) {
+      super(path, "File exists");
+    }
+    
+    protected PathExistsException(String path, String error) {
+      super(path, error);
+    }
+  }
+
+  /** EISDIR */
+  public static class PathIsDirectoryException extends PathExistsException {
+    static final long serialVersionUID = 0L;
+    /** @param path for the exception */
+    public PathIsDirectoryException(String path) {
+      super(path, "Is a directory");
+    }
+  }
+
+  /** ENOTDIR */
+  public static class PathIsNotDirectoryException extends PathExistsException {
+    static final long serialVersionUID = 0L;
+    /** @param path for the exception */
+    public PathIsNotDirectoryException(String path) {
+      super(path, "Is not a directory");
+    }
+  }
+
+  /** EACCES */
+  public static class PathAccessDeniedException extends PathIOException {
+    static final long serialVersionUID = 0L;
+    /** @param path for the exception */
+    public PathAccessDeniedException(String path) {
+      super(path, "Permission denied");
+    }
+  }
+
+  /** EPERM */
+  public static class PathPermissionException extends PathIOException {
+    static final long serialVersionUID = 0L;
+    /** @param path for the exception */
+    public PathPermissionException(String path) {
+      super(path, "Operation not permitted");
+    }
+  }
+}

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/SetReplication.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/SetReplication.java?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/SetReplication.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/SetReplication.java Tue May 10 21:29:34 2011
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIOException;
 
 /**
  * Modifies the replication factor
@@ -76,7 +76,9 @@ public class SetReplication extends FsCo
 
   @Override
   protected void processPath(PathData item) throws IOException {
-    if (item.stat.isSymlink()) throw new IOException("Symlinks unsupported");
+    if (item.stat.isSymlink()) {
+      throw new PathIOException(item.toString(), "Symlinks unsupported");
+    }
     
     if (item.stat.isFile()) {
       if (!item.fs.setReplication(item.path, newRep)) {
@@ -124,9 +126,4 @@ public class SetReplication extends FsCo
       out.println(" done");
     }
   }
-
-  @Override
-  protected String getFnfText(Path path) {
-    return "File does not exist: " + path;
-  }
 }
\ No newline at end of file

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Tail.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Tail.java?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Tail.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Tail.java Tue May 10 21:29:34 2011
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIsDirectoryException;
 import org.apache.hadoop.io.IOUtils;
 
 /**
@@ -67,7 +67,7 @@ class Tail extends FsCommand {
   @Override
   protected void processPath(PathData item) throws IOException {
     if (item.stat.isDirectory()) {
-      throw new IOException("Source must be a file.");
+      throw new PathIsDirectoryException(item.toString());
     }
 
     long offset = dumpFromOffset(item, startingOffset);
@@ -100,9 +100,4 @@ class Tail extends FsCommand {
     }
     return offset;
   }
-  
-  @Override
-  protected String getFnfText(Path path) {
-    return "File does not exist: " + path;
-  }
 }

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestFsShellReturnCode.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestFsShellReturnCode.java?rev=1101653&r1=1101652&r2=1101653&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestFsShellReturnCode.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestFsShellReturnCode.java Tue May 10 21:29:34 2011
@@ -288,7 +288,7 @@ public class TestFsShellReturnCode {
       results = bytes.toString();
       assertTrue("Return code should be -1", run == -1);
       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"));
+      assertTrue(" Not displaying the intended message ",results.contains("get: `"+args[1]+"': No such file or directory"));
     } finally {
       IOUtils.closeStream(out);
       System.setErr(oldErr);

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathExceptions.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathExceptions.java?rev=1101653&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathExceptions.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathExceptions.java Tue May 10 21:29:34 2011
@@ -0,0 +1,56 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIOException;
+import org.junit.Test;
+
+public class TestPathExceptions {
+
+  protected String path = "some/file";
+  protected String error = "KABOOM";
+
+  @Test
+  public void testWithDefaultString() throws Exception {
+    PathIOException pe = new PathIOException(path);
+    assertEquals(new Path(path), pe.getPath());
+    assertEquals("`" + path + "': Input/output error", pe.getMessage());
+  }
+
+  @Test
+  public void testWithThrowable() throws Exception {
+    IOException ioe = new IOException("KABOOM");    
+    PathIOException pe = new PathIOException(path, ioe);
+    assertEquals(new Path(path), pe.getPath());
+    assertEquals("`" + path + "': Input/output error: " + error, pe.getMessage());
+  }
+
+  @Test
+  public void testWithCustomString() throws Exception {
+    PathIOException pe = new PathIOException(path, error);
+    assertEquals(new Path(path), pe.getPath());
+    assertEquals("`" + path + "': " + error, pe.getMessage());
+  }
+  
+}