瀏覽代碼

HDFS=1079. Throw exceptions as specified by the AbstractFileSystem in HDFS implemenation and protocols. (suresh)


git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@944096 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 15 年之前
父節點
當前提交
2143b1e3b6

+ 3 - 0
CHANGES.txt

@@ -10,6 +10,9 @@ Trunk (unreleased changes)
     HDFS-1061. Memory footprint optimization for INodeFile object. 
     (Bharath Mundlapudi via jghoman)
 
+    HDFS=1079. Throw exceptions as specified by the AbstractFileSystem
+    in HDFS implemenation and protocols. (suresh)
+
 Release 0.21.0 - Unreleased
 
   INCOMPATIBLE CHANGES

+ 171 - 155
src/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -53,7 +53,9 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.UnresolvedLinkException;
@@ -77,6 +79,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
@@ -323,6 +326,9 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     return defaultBlockSize;
   }
     
+  /**
+   * @see ClientProtocol#getPreferredBlockSize(String)
+   */
   public long getBlockSize(String f) throws IOException {
     try {
       return namenode.getPreferredBlockSize(f);
@@ -335,6 +341,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
 
   /**
    * Get server default values for a number of configuration params.
+   * @see ClientProtocol#getServerDefaults()
    */
   public FsServerDefaults getServerDefaults() throws IOException {
     long now = System.currentTimeMillis();
@@ -345,11 +352,17 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     return serverDefaults;
   }
 
+  /**
+   * @see ClientProtocol#getDelegationToken(Text)
+   */
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
     return namenode.getDelegationToken(renewer);
   }
 
+  /**
+   * @see ClientProtocol#renewDelegationToken(Token)
+   */
   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
     try {
@@ -360,6 +373,9 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     }
   }
 
+  /**
+   * @see ClientProtocol#cancelDelegationToken(Token)
+   */
   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
     try {
@@ -372,6 +388,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   
   /**
    * Report corrupt blocks that were discovered by the client.
+   * @see ClientProtocol#reportBadBlocks(LocatedBlock[])
    */
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
     namenode.reportBadBlocks(blocks);
@@ -381,9 +398,12 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     return defaultReplication;
   }
 
+  /**
+   * @see ClientProtocol#getBlockLocations(String, long, long)
+   */
   static LocatedBlocks callGetBlockLocations(ClientProtocol namenode,
       String src, long start, long length) 
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     try {
       return namenode.getBlockLocations(src, start, length);
     } catch(RemoteException re) {
@@ -469,116 +489,92 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   }
 
   /**
-   * Create a new dfs file and return an output stream for writing into it. 
-   * 
-   * @param src stream name
-   * @param overwrite do not check for file existence if true
-   * @return output stream
-   * @throws UnresolvedLinkException if a symlink is encountered in src.
-   * @throws IOException
+   * Get the namenode associated with this DFSClient object
+   * @return the namenode associated with this DFSClient object
+   */
+  public ClientProtocol getNamenode() {
+    return namenode;
+  }
+  
+  /**
+   * Call {@link #create(String, boolean, short, long, Progressable)} with
+   * default <code>replication</code> and <code>blockSize<code> and null <code>
+   * progress</code>.
    */
   public OutputStream create(String src, boolean overwrite) 
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     return create(src, overwrite, defaultReplication, defaultBlockSize, null);
   }
     
   /**
-   * Create a new dfs file and return an output stream for writing into it
-   * with write-progress reporting. 
-   * 
-   * @param src stream name
-   * @param overwrite do not check for file existence if true
-   * @return output stream
-   * @throws UnresolvedLinkException if a symlink is encountered in src.
-   * @throws IOException
+   * Call {@link #create(String, boolean, short, long, Progressable)} with
+   * default <code>replication</code> and <code>blockSize<code>.
    */
   public OutputStream create(String src, 
                              boolean overwrite,
-                             Progressable progress)
-      throws IOException, UnresolvedLinkException {
-    return create(src, overwrite, defaultReplication, defaultBlockSize, null);
+                             Progressable progress) throws IOException {
+    return create(src, overwrite, defaultReplication, defaultBlockSize, progress);
   }
     
   /**
-   * Create a new dfs file with the specified block replication 
-   * and return an output stream for writing into the file.  
-   * 
-   * @param src stream name
-   * @param overwrite do not check for file existence if true
-   * @param replication block replication
-   * @return output stream
-   * @throws UnresolvedLinkException if a symlink is encountered in src.
-   * @throws IOException
+   * Call {@link #create(String, boolean, short, long, Progressable)} with
+   * null <code>progress</code>.
    */
   public OutputStream create(String src, 
                              boolean overwrite, 
                              short replication,
-                             long blockSize)
-      throws IOException, UnresolvedLinkException {
+                             long blockSize) throws IOException {
     return create(src, overwrite, replication, blockSize, null);
   }
 
   /**
-   * Get the namenode associated with this DFSClient object
-   * @return the namenode associated with this DFSClient object
+   * Call {@link #create(String, boolean, short, long, Progressable, int)}
+   * with default bufferSize.
    */
-  public ClientProtocol getNamenode() {
-    return namenode;
+  public OutputStream create(String src, boolean overwrite, short replication,
+      long blockSize, Progressable progress) throws IOException {
+    return create(src, overwrite, replication, blockSize, progress,
+        conf.getInt("io.file.buffer.size", 4096));
   }
-  
-  
+
   /**
-   * Create a new dfs file with the specified block replication 
-   * with write-progress reporting and return an output stream for writing
-   * into the file.  
+   * Call {@link #create(String, FsPermission, EnumSet, short, long, 
+   * Progressable, int)} with default <code>permission</code>
+   * {@link FsPermission#getDefault()}.
+   * 
+   * @param src File name
+   * @param overwrite overwrite an existing file if true
+   * @param replication replication factor for the file
+   * @param blockSize maximum block size
+   * @param progress interface for reporting client progress
+   * @param buffersize underlying buffersize
    * 
-   * @param src stream name
-   * @param overwrite do not check for file existence if true
-   * @param replication block replication
    * @return output stream
-   * @throws UnresolvedLinkException if a symlink is encountered in src.
-   * @throws IOException
    */
-  public OutputStream create(String src, 
-                             boolean overwrite, 
+  public OutputStream create(String src,
+                             boolean overwrite,
                              short replication,
                              long blockSize,
-                             Progressable progress)
-      throws IOException, UnresolvedLinkException {
-    return create(src, overwrite, replication, blockSize, progress,
-        conf.getInt("io.file.buffer.size", 4096));
-  }
-  /**
-   * Call
-   * {@link #create(String,FsPermission,EnumSet,short,long,Progressable,int)}
-   * with default permission.
-   * @see FsPermission#getDefault()
-   */
-  public OutputStream create(String src,
-      boolean overwrite,
-      short replication,
-      long blockSize,
-      Progressable progress,
-      int buffersize)
-      throws IOException, UnresolvedLinkException {
+                             Progressable progress,
+                             int buffersize)
+      throws IOException {
     return create(src, FsPermission.getDefault(),
         overwrite ? EnumSet.of(CreateFlag.OVERWRITE) : EnumSet.of(CreateFlag.CREATE), 
         replication, blockSize, progress, buffersize);
   }
 
   /**
-   * Call
-   * {@link #create(String,FsPermission,EnumSet,boolean,short,long,Progressable,int)}
-   * with createParent set to true.
+   * Call {@link #create(String, FsPermission, EnumSet, boolean, short, 
+   * long, Progressable, int)} with <code>createParent</code> set to true.
    */
   public OutputStream create(String src, 
-      FsPermission permission,
-      EnumSet<CreateFlag> flag, 
-      short replication,
-      long blockSize,
-      Progressable progress,
-      int buffersize)
-      throws IOException, UnresolvedLinkException {
+                             FsPermission permission,
+                             EnumSet<CreateFlag> flag, 
+                             short replication,
+                             long blockSize,
+                             Progressable progress,
+                             int buffersize)
+      throws IOException {
     return create(src, permission, flag, true,
         replication, blockSize, progress, buffersize);
   }
@@ -588,16 +584,21 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * with write-progress reporting and return an output stream for writing
    * into the file.  
    * 
-   * @param src stream name
+   * @param src File name
    * @param permission The permission of the directory being created.
-   * If permission == null, use {@link FsPermission#getDefault()}.
-   * @param flag do not check for file existence if true
+   *          If null, use default permission {@link FsPermission#getDefault()}
+   * @param flag indicates create a new file or create/overwrite an
+   *          existing file or append to an existing file
    * @param createParent create missing parent directory if true
    * @param replication block replication
+   * @param blockSize maximum block size
+   * @param progress interface for reporting client progress
+   * @param buffersize underlying buffer size 
+   * 
    * @return output stream
-   * @throws IOException
-   * @throws UnresolvedLinkException if src contains a symlink. 
-   * @see ClientProtocol#create(String, FsPermission, String, EnumSetWritable, boolean, short, long)
+   * 
+   * @see ClientProtocol#create(String, FsPermission, String, EnumSetWritable,
+   * boolean, short, long) for detailed description of exceptions thrown
    */
   public OutputStream create(String src, 
                              FsPermission permission,
@@ -607,7 +608,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
                              long blockSize,
                              Progressable progress,
                              int buffersize)
-    throws IOException, UnresolvedLinkException {
+    throws IOException {
     checkOpen();
     if (permission == null) {
       permission = FsPermission.getDefault();
@@ -624,7 +625,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   
   /**
    * Same as {{@link #create(String, FsPermission, EnumSet, short, long,
-   *  Progressable, int)}   except that the permission
+   *  Progressable, int)} except that the permission
    *   is absolute (ie has already been masked with umask.
    */
   public OutputStream primitiveCreate(String src, 
@@ -651,16 +652,18 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @see ClientProtocol#createSymlink(String, String,FsPermission, boolean) 
    */
   public void createSymlink(String target, String link, boolean createParent)
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     try {
       FsPermission dirPerm = 
           FsPermission.getDefault().applyUMask(FsPermission.getUMask(conf)); 
       namenode.createSymlink(target, link, dirPerm, createParent);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileAlreadyExistsException.class, 
+                                     FileNotFoundException.class,
+                                     ParentNotDirectoryException.class,
                                      NSQuotaExceededException.class, 
                                      DSQuotaExceededException.class,
-                                     FileAlreadyExistsException.class, 
                                      UnresolvedPathException.class);
     }
   }
@@ -687,12 +690,11 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param buffersize buffer size
    * @param progress for reporting write-progress
    * @return an output stream for writing into the file
-   * @throws IOException
-   * @throws UnresolvedLinkException if the path contains a symlink.
-   * @see ClientProtocol#append(String, String)
+   * 
+   * @see ClientProtocol#append(String, String) 
    */
   OutputStream append(String src, int buffersize, Progressable progress)
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     checkOpen();
     HdfsFileStatus stat = null;
     LocatedBlock lastBlock = null;
@@ -700,10 +702,11 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       stat = getFileInfo(src);
       lastBlock = namenode.append(src, clientName);
     } catch(RemoteException re) {
-      throw re.unwrapRemoteException(FileNotFoundException.class,
-                                     AccessControlException.class,
-                                     NSQuotaExceededException.class,
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     SafeModeException.class,
                                      DSQuotaExceededException.class,
+                                     UnsupportedOperationException.class,
                                      UnresolvedPathException.class);
     }
     OutputStream result = new DFSOutputStream(this, src, buffersize, progress,
@@ -715,20 +718,19 @@ public class DFSClient implements FSConstants, java.io.Closeable {
 
   /**
    * Set replication for an existing file.
+   * @param src file name
+   * @param replication
    * 
    * @see ClientProtocol#setReplication(String, short)
-   * @param replication
-   * @throws IOException
-   * @return true is successful or false if file does not exist 
    */
-  public boolean setReplication(String src, 
-                                short replication)
-      throws IOException, UnresolvedLinkException {
+  public boolean setReplication(String src, short replication)
+      throws IOException {
     try {
       return namenode.setReplication(src, replication);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     NSQuotaExceededException.class,
+                                     FileNotFoundException.class,
+                                     SafeModeException.class,
                                      DSQuotaExceededException.class,
                                      UnresolvedPathException.class);
     }
@@ -736,12 +738,11 @@ public class DFSClient implements FSConstants, java.io.Closeable {
 
   /**
    * Rename file or directory.
-   * See {@link ClientProtocol#rename(String, String)}.
+   * @see ClientProtocol#rename(String, String)
    * @deprecated Use {@link #rename(String, String, Options.Rename...)} instead.
    */
   @Deprecated
-  public boolean rename(String src, String dst) 
-      throws IOException, UnresolvedLinkException {
+  public boolean rename(String src, String dst) throws IOException {
     checkOpen();
     try {
       return namenode.rename(src, dst);
@@ -757,31 +758,32 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * Move blocks from src to trg and delete src
    * See {@link ClientProtocol#concat(String, String [])}. 
    */
-  public void concat(String trg, String [] srcs) 
-      throws IOException, UnresolvedLinkException {
+  public void concat(String trg, String [] srcs) throws IOException {
     checkOpen();
     try {
       namenode.concat(trg, srcs);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class,
                                      UnresolvedPathException.class);
     }
   }
   /**
    * Rename file or directory.
-   * See {@link ClientProtocol#rename(String, String, Options.Rename...)}
+   * @see ClientProtocol#rename(String, String, Options.Rename...)
    */
-  public void rename(String src, String dst, Options.Rename... options) 
-      throws IOException, UnresolvedLinkException {
+  public void rename(String src, String dst, Options.Rename... options)
+      throws IOException {
     checkOpen();
     try {
       namenode.rename(src, dst, options);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     NSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
+                                     FileAlreadyExistsException.class,
+                                     FileNotFoundException.class,
+                                     ParentNotDirectoryException.class,
+                                     SafeModeException.class,
+                                     NSQuotaExceededException.class,
                                      UnresolvedPathException.class);
     }
   }
@@ -790,8 +792,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * See {@link ClientProtocol#delete(String)}. 
    */
   @Deprecated
-  public boolean delete(String src) 
-      throws IOException, UnresolvedLinkException {
+  public boolean delete(String src) throws IOException {
     checkOpen();
     return namenode.delete(src, true);
   }
@@ -800,14 +801,17 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * delete file or directory.
    * delete contents of the directory if non empty and recursive 
    * set to true
+   *
+   * @see ClientProtocol#delete(String, boolean)
    */
-  public boolean delete(String src, boolean recursive) 
-      throws IOException, UnresolvedLinkException {
+  public boolean delete(String src, boolean recursive) throws IOException {
     checkOpen();
     try {
       return namenode.delete(src, recursive);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     SafeModeException.class,
                                      UnresolvedPathException.class);
     }
   }
@@ -826,28 +830,35 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * if the application wants to fetch a listing starting from
    * the first entry in the directory
    *
-   * @param src the directory name
-   * @param startAfter the name to start listing after encoded in java UTF8
-   * @return a partial listing starting after startAfter
+   * @see ClientProtocol#getListing(String, byte[])
    */
   public DirectoryListing listPaths(String src,  byte[] startAfter) 
-    throws IOException, UnresolvedLinkException {
+    throws IOException {
     checkOpen();
     try {
       return namenode.getListing(src, startAfter);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
                                      UnresolvedPathException.class);
     }
   }
 
-  public HdfsFileStatus getFileInfo(String src) 
-      throws IOException, UnresolvedLinkException {
+  /**
+   * Get the file info for a specific file or directory.
+   * @param src The string representation of the path to the file
+   * @return object containing information regarding the file
+   *         or null if file not found
+   *         
+   * @see ClientProtocol#getFileInfo(String) for description of exceptions
+   */
+  public HdfsFileStatus getFileInfo(String src) throws IOException {
     checkOpen();
     try {
       return namenode.getFileInfo(src);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
                                      UnresolvedPathException.class);
     }
   }
@@ -856,12 +867,11 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * Get the file info for a specific file or directory. If src
    * refers to a symlink then the FileStatus of the link is returned.
    * @param src path to a file or directory.
-   * @throws IOException
-   * @throws UnresolvedLinkException if the path contains symlinks
-   * @return FileStatus describing src.
+   * 
+   * For description of exceptions thrown 
+   * @see ClientProtocol#getFileLinkInfo(String)
    */
-  public HdfsFileStatus getFileLinkInfo(String src) 
-      throws IOException, UnresolvedLinkException {
+  public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
     checkOpen();
     try {
       return namenode.getFileLinkInfo(src);
@@ -1013,17 +1023,18 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * Set permissions to a file or directory.
    * @param src path name.
    * @param permission
-   * @throws <code>FileNotFoundException</code> is file does not exist.
-   * @throws UnresolvedLinkException if the path contains a symlink.
+   * 
+   * @see ClientProtocol#setPermission(String, FsPermission)
    */
   public void setPermission(String src, FsPermission permission)
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     checkOpen();
     try {
       namenode.setPermission(src, permission);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
+                                     SafeModeException.class,
                                      UnresolvedPathException.class);
     }
   }
@@ -1033,21 +1044,25 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param src path name.
    * @param username user id.
    * @param groupname user group.
-   * @throws <code>FileNotFoundException</code> is file does not exist.
-   * @throws UnresolvedLinkException if the path contains a symlink.
+   * 
+   * @see ClientProtocol#setOwner(String, String, String)
    */
   public void setOwner(String src, String username, String groupname)
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     checkOpen();
     try {
       namenode.setOwner(src, username, groupname);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
-                                     UnresolvedPathException.class);
+                                     SafeModeException.class,
+                                     UnresolvedPathException.class);                                   
     }
   }
 
+  /**
+   * @see ClientProtocol#getStats()
+   */
   public FsStatus getDiskStatus() throws IOException {
     long rawNums[] = namenode.getStats();
     return new FsStatus(rawNums[0], rawNums[1], rawNums[2]);
@@ -1085,8 +1100,6 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     
   /**
    * Enter, leave or get safe mode.
-   * See {@link ClientProtocol#setSafeMode(FSConstants.SafeModeAction)} 
-   * for more details.
    * 
    * @see ClientProtocol#setSafeMode(FSConstants.SafeModeAction)
    */
@@ -1096,8 +1109,6 @@ public class DFSClient implements FSConstants, java.io.Closeable {
 
   /**
    * Save namespace image.
-   * See {@link ClientProtocol#saveNamespace()} 
-   * for more details.
    * 
    * @see ClientProtocol#saveNamespace()
    */
@@ -1111,9 +1122,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   
   /**
    * enable/disable restore failed storage.
-   * See {@link ClientProtocol#restoreFailedStorage(String arg)} 
-   * for more details.
    * 
+   * @see ClientProtocol#restoreFailedStorage(String arg)
    */
   boolean restoreFailedStorage(String arg) throws AccessControlException {
     return namenode.restoreFailedStorage(arg);
@@ -1132,8 +1142,6 @@ public class DFSClient implements FSConstants, java.io.Closeable {
 
   /**
    * Dumps DFS data structures into specified file.
-   * See {@link ClientProtocol#metaSave(String)} 
-   * for more details.
    * 
    * @see ClientProtocol#metaSave(String)
    */
@@ -1151,8 +1159,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   /**
    * @see ClientProtocol#distributedUpgradeProgress(FSConstants.UpgradeAction)
    */
-  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action
-                                                        ) throws IOException {
+  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
+      throws IOException {
     return namenode.distributedUpgradeProgress(action);
   }
 
@@ -1171,12 +1179,13 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param permission The permission of the directory being created.
    * If permission == null, use {@link FsPermission#getDefault()}.
    * @param createParent create missing parent directory if true
+   * 
    * @return True if the operation success.
-   * @throws UnresolvedLinkException if the path contains a symlink.
+   * 
    * @see ClientProtocol#mkdirs(String, FsPermission, boolean)
    */
-  public boolean mkdirs(String src, FsPermission permission, boolean createParent)
-      throws IOException, UnresolvedLinkException {
+  public boolean mkdirs(String src, FsPermission permission,
+      boolean createParent) throws IOException {
     checkOpen();
     if (permission == null) {
       permission = FsPermission.getDefault();
@@ -1187,10 +1196,12 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       return namenode.mkdirs(src, masked, createParent);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class,
-                                     FileNotFoundException.class,
+                                     InvalidPathException.class,
                                      FileAlreadyExistsException.class,
+                                     FileNotFoundException.class,
+                                     ParentNotDirectoryException.class,
+                                     SafeModeException.class,
+                                     NSQuotaExceededException.class,
                                      UnresolvedPathException.class);
     }
   }
@@ -1198,10 +1209,9 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   /**
    * Same {{@link #mkdirs(String, FsPermission, boolean)} except
    * that the permissions has already been masked against umask.
-   * @throws UnresolvedLinkException if the path contains a symlink.
    */
   public boolean primitiveMkdir(String src, FsPermission absPermission)
-    throws IOException, UnresolvedLinkException {
+    throws IOException {
     checkOpen();
     if (absPermission == null) {
       absPermission = 
@@ -1219,6 +1229,12 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     }
   }
 
+  /**
+   * Get {@link ContentSummary} rooted at the specified directory.
+   * @param path The string representation of the path
+   * 
+   * @see ClientProtocol#getContentSummary(String)
+   */
   ContentSummary getContentSummary(String src) throws IOException {
     try {
       return namenode.getContentSummary(src);
@@ -1231,10 +1247,10 @@ public class DFSClient implements FSConstants, java.io.Closeable {
 
   /**
    * Sets or resets quotas for a directory.
-   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long)
+   * @see ClientProtocol#setQuota(String, long, long)
    */
   void setQuota(String src, long namespaceQuota, long diskspaceQuota) 
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     // sanity check
     if ((namespaceQuota <= 0 && namespaceQuota != FSConstants.QUOTA_DONT_SET &&
          namespaceQuota != FSConstants.QUOTA_RESET) ||
@@ -1258,10 +1274,10 @@ public class DFSClient implements FSConstants, java.io.Closeable {
 
   /**
    * set the modification and access time of a file
-   * @throws FileNotFoundException if the path is not a file
+   * 
+   * @see ClientProtocol#setTimes(String, long, long)
    */
-  public void setTimes(String src, long mtime, long atime) 
-      throws IOException, UnresolvedLinkException {
+  public void setTimes(String src, long mtime, long atime) throws IOException {
     checkOpen();
     try {
       namenode.setTimes(src, mtime, atime);

+ 6 - 2
src/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -58,6 +59,7 @@ import org.apache.hadoop.hdfs.security.InvalidAccessTokenException;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
@@ -1042,7 +1044,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
   DFSOutputStream(DFSClient dfsClient, String src, FsPermission masked, EnumSet<CreateFlag> flag,
       boolean createParent, short replication, long blockSize, Progressable progress,
       int buffersize, int bytesPerChecksum) 
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     this(dfsClient, src, blockSize, progress, bytesPerChecksum, replication);
 
     computePacketChunkSize(dfsClient.writePacketSize, bytesPerChecksum);
@@ -1052,10 +1054,12 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
           src, masked, dfsClient.clientName, new EnumSetWritable<CreateFlag>(flag), createParent, replication, blockSize);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
+                                     DSQuotaExceededException.class,
                                      FileAlreadyExistsException.class,
                                      FileNotFoundException.class,
+                                     ParentNotDirectoryException.class,
                                      NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class,
+                                     SafeModeException.class,
                                      UnresolvedPathException.class);
     }
     streamer = new DataStreamer();

+ 263 - 114
src/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -27,11 +27,16 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
@@ -82,16 +87,20 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param src file name
    * @param offset range start offset
    * @param length range length
+   *
    * @return file length and array of blocks with their locations
-   * @throws IOException
-   * @throws UnresolvedLinkException if the path contains a symlink.
-   * @throws FileNotFoundException if the path does not exist.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> does not exist
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
    */
   @Nullable
   public LocatedBlocks getBlockLocations(String src,
                                          long offset,
                                          long length) 
-    throws IOException, UnresolvedLinkException, FileNotFoundException;
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
 
   /**
    * Get server default values for a number of configuration params.
@@ -113,7 +122,8 @@ public interface ClientProtocol extends VersionedProtocol {
    * or explicitly as a result of lease expiration.
    * <p>
    * Blocks have a maximum size.  Clients that intend to create
-   * multi-block files must also use {@link #addBlock(String, String, Block, DatanodeInfo[])}.
+   * multi-block files must also use 
+   * {@link #addBlock(String, String, Block, DatanodeInfo[])}
    *
    * @param src path of the file being created.
    * @param masked masked permission.
@@ -124,25 +134,31 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param replication block replication factor.
    * @param blockSize maximum block size.
    * 
-   * @throws AccessControlException if permission to create file is 
-   * denied by the system. As usually on the client side the exception will 
-   * be wrapped into {@link org.apache.hadoop.ipc.RemoteException}.
-   * @throws QuotaExceededException if the file creation violates 
-   *                                any quota restriction
-   * @throws IOException if other errors occur.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   * @throws AccessControlException If access is denied
    * @throws AlreadyBeingCreatedException if the path does not exist.
-   * @throws NSQuotaExceededException if the namespace quota is exceeded.
+   * @throws DSQuotaExceededException If file creation violates disk space 
+   *           quota restriction
+   * @throws FileAlreadyExistsException If file <code>src</code> already exists
+   * @throws FileNotFoundException If parent of <code>src</code> does not exist
+   *           and <code>createParent</code> is false
+   * @throws ParentNotDirectoryException If parent of <code>src</code> is not a
+   *           directory.
+   * @throws NSQuotaExceededException If file creation violates name space 
+   *           quota restriction
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   *
+   * RuntimeExceptions:
+   * @throws InvalidPathException Path <code>src</code> is invalid
    */
-  public void create(String src, 
-                     FsPermission masked,
-                     String clientName, 
-                     EnumSetWritable<CreateFlag> flag, 
-                     boolean createParent,
-                     short replication,
-                     long blockSize)
-    throws IOException, UnresolvedLinkException,
-           AlreadyBeingCreatedException, NSQuotaExceededException;
+  public void create(String src, FsPermission masked, String clientName,
+      EnumSetWritable<CreateFlag> flag, boolean createParent,
+      short replication, long blockSize) throws AccessControlException,
+      AlreadyBeingCreatedException, DSQuotaExceededException,
+      FileAlreadyExistsException, FileNotFoundException,
+      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
+      UnresolvedLinkException, IOException;
 
   /**
    * Append to the end of the file. 
@@ -155,11 +171,22 @@ public interface ClientProtocol extends VersionedProtocol {
    * Allows appending to an existing file if the server is
    * configured with the parameter dfs.support.append set to true, otherwise
    * throws an IOException.
-   * @throws IOException if other errors occur.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   * 
+   * @throws AccessControlException If permission to append to file is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws DSQuotaExceededException If append violates disk space quota 
+   *           restriction
+   * @throws SafeModeException append not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred.
+   *
+   * RuntimeExceptions:
+   * @throws UnsupportedOperationException if append is not supported
    */
-  public LocatedBlock append(String src, String clientName) 
-      throws IOException, UnresolvedLinkException;
+  public LocatedBlock append(String src, String clientName)
+      throws AccessControlException, DSQuotaExceededException,
+      FileNotFoundException, SafeModeException, UnresolvedLinkException,
+      IOException;
 
   /**
    * Set replication for an existing file.
@@ -171,20 +198,35 @@ public interface ClientProtocol extends VersionedProtocol {
    * 
    * @param src file name
    * @param replication new replication
-   * @throws IOException
+   * 
    * @return true if successful;
    *         false if file does not exist or is a directory
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   *
+   * @throws AccessControlException If access is denied
+   * @throws DSQuotaExceededException If replication violates disk space 
+   *           quota restriction
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException not allowed in safemode
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
    */
-  public boolean setReplication(String src, short replication) 
-      throws IOException, UnresolvedLinkException;
+  public boolean setReplication(String src, short replication)
+      throws AccessControlException, DSQuotaExceededException,
+      FileNotFoundException, SafeModeException, UnresolvedLinkException,
+      IOException;
 
   /**
    * Set permissions for an existing file/directory.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
    */
   public void setPermission(String src, FsPermission permission)
-    throws IOException, UnresolvedLinkException, SafeModeException;
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException;
 
   /**
    * Set Owner of a path (i.e. a file or a directory).
@@ -192,19 +234,31 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param src
    * @param username If it is null, the original username remains unchanged.
    * @param groupname If it is null, the original groupname remains unchanged.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
    */
   public void setOwner(String src, String username, String groupname)
-      throws IOException, UnresolvedLinkException;
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException;
 
   /**
-   * The client can give up on a block by calling abandonBlock().
-   * The client can then either obtain a new block, or complete or 
-   * abandon the file. Any partial writes to the block will be discarded.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   * The client can give up on a blcok by calling abandonBlock().
+   * The client can then
+   * either obtain a new block, or complete or abandon the file.
+   * Any partial writes to the block will be discarded.
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
    */
   public void abandonBlock(Block b, String src, String holder)
-      throws IOException, UnresolvedLinkException;
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
 
   /**
    * A client that wants to write an additional block to the 
@@ -221,16 +275,25 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param src the file being created
    * @param clientName the name of the client that adds the block
    * @param previous  previous block
-   * @param excludedNodes a list of nodes that should not be
+   * @param excludeNodes a list of nodes that should not be
    * allocated for the current block
+   *
    * @return LocatedBlock allocated block information.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   * @throws DSQuotaExceededException if the directory's quota is exceeded.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws NotReplicatedYetException previous blocks of the file are not
+   *           replicated yet. Blocks cannot be added until replication
+   *           completes.
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
    */
   public LocatedBlock addBlock(String src, String clientName,
-                               @Nullable Block previous,
-                               @Nullable DatanodeInfo[] excludedNodes) 
-    throws IOException, UnresolvedLinkException, DSQuotaExceededException;
+      @Nullable Block previous, @Nullable DatanodeInfo[] excludeNodes)
+      throws AccessControlException, FileNotFoundException,
+      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
+      IOException;
 
   /**
    * The client is done writing data to the given filename, and would 
@@ -247,11 +310,16 @@ public interface ClientProtocol extends VersionedProtocol {
    * blocks have been replicated the minimum number of times.  Thus,
    * DataNode failures may cause a client to call complete() several
    * times before succeeding.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink 
+   * @throws IOException If an I/O error occurred
    */
-  public boolean complete(String src, String clientName, Block last) 
-    throws IOException, UnresolvedLinkException;
-                          
+  public boolean complete(String src, String clientName, Block last)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException;
 
   /**
    * The client wants to report corrupted blocks (blocks with specified
@@ -269,15 +337,14 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param dst new name.
    * @return true if successful, or false if the old name does not exist
    * or if the new name already belongs to the namespace.
-   * @throws IOException if the new name is invalid.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   * @throws QuotaExceededException if the rename would violate 
-   *                                any quota restriction
+   * 
+   * @throws IOException an I/O error occurred
+   * 
    * @deprecated Use {@link #rename(String, String, Options.Rename...)} instead.
    */
   @Deprecated
   public boolean rename(String src, String dst) 
-      throws IOException, UnresolvedLinkException;
+      throws UnresolvedLinkException, IOException;
 
   /**
    * Moves blocks from srcs to trg and delete srcs
@@ -285,9 +352,8 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param trg existing file
    * @param srcs - list of existing files (same block size, same replication)
    * @throws IOException if some arguments are invalid
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   * @throws QuotaExceededException if the rename would violate 
-   *                                any quota restriction
+   * @throws UnresolvedLinkException if <code>trg</code> or <code>srcs</code>
+   *           contains a symlink
    */
   public void concat(String trg, String[] srcs) 
       throws IOException, UnresolvedLinkException;
@@ -309,11 +375,28 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param src existing file or directory name.
    * @param dst new name.
    * @param options Rename options
-   * @throws IOException if rename failed
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws DSQuotaExceededException If rename violates disk space 
+   *           quota restriction
+   * @throws FileAlreadyExistsException If <code>dst</code> already exists and
+   *           <code>options</options> has {@link Rename#OVERWRITE} option
+   *           false.
+   * @throws FileNotFoundException If <code>src</code> does not exist
+   * @throws NSQuotaExceededException If rename violates namespace 
+   *           quota restriction
+   * @throws ParentNotDirectoryException If parent of <code>dst</code> 
+   *           is not a directory
+   * @throws SafeModeException rename not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> or
+   *           <code>dst</code> contains a symlink
+   * @throws IOException If an I/O error occurred
    */
   public void rename(String src, String dst, Options.Rename... options)
-      throws IOException, UnresolvedLinkException;
+      throws AccessControlException, DSQuotaExceededException,
+      FileAlreadyExistsException, FileNotFoundException,
+      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
+      UnresolvedLinkException, IOException;
   
   /**
    * Delete the given file or directory from the file system.
@@ -323,7 +406,7 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param src existing name.
    * @return true only if the existing file or directory was actually removed 
    * from the file system. 
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
    * @deprecated use {@link #delete(String, boolean)} istead.
    */
   @Deprecated
@@ -339,11 +422,17 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param recursive if true deletes a non empty directory recursively,
    * else throws an exception.
    * @return true only if the existing file or directory was actually removed 
-   * from the file system. 
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   */
-  public boolean delete(String src, boolean recursive) 
-     throws IOException, UnresolvedLinkException;
+   * from the file system.
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   */
+  public boolean delete(String src, boolean recursive)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException;
   
   /**
    * Create a directory (or hierarchy of directories) with the given
@@ -352,27 +441,45 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param src The path of the directory being created
    * @param masked The masked permission of the directory being created
    * @param createParent create missing parent directory if true
+   *
    * @return True if the operation success.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   * @throws {@link AccessControlException} if permission to create file is 
-   * denied by the system. As usually on the client side the exception will 
-   * be wraped into {@link org.apache.hadoop.ipc.RemoteException}.
-   * @throws QuotaExceededException if the operation would violate 
-   *                                any quota restriction.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileAlreadyExistsException If <code>src</code> already exists
+   * @throws FileNotFoundException If parent of <code>src</code> does not exist
+   *           and <code>createParent</code> is false
+   * @throws NSQuotaExceededException If file creation violates quota restriction
+   * @throws ParentNotDirectoryException If parent of <code>src</code> 
+   *           is not a directory
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred.
+   *
+   * RunTimeExceptions:
+   * @throws InvalidPathException If <code>src</code> is invalid
    */
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
-    throws IOException, UnresolvedLinkException, NSQuotaExceededException;
+      throws AccessControlException, FileAlreadyExistsException,
+      FileNotFoundException, NSQuotaExceededException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      IOException;
 
   /**
    * Get a partial listing of the indicated directory
    *
    * @param src the directory name
    * @param startAfter the name to start listing after encoded in java UTF8
+   *
    * @return a partial listing starting after startAfter
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   *
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
    */
-  public DirectoryListing getListing(String src, byte[] startAfter) 
-      throws IOException, UnresolvedLinkException; 
+  public DirectoryListing getListing(String src, byte[] startAfter)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
 
   ///////////////////////////////////////
   // System issues and management
@@ -393,10 +500,12 @@ public interface ClientProtocol extends VersionedProtocol {
    * renewLease().  If a certain amount of time passes since
    * the last call to renewLease(), the NameNode assumes the
    * client has died.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   *
+   * @throws AccessControlException permission denied
+   * @throws IOException If an I/O error occurred
    */
-  public void renewLease(String clientName) 
-      throws IOException, UnresolvedLinkException;
+  public void renewLease(String clientName) throws AccessControlException,
+      IOException;
 
   public int GET_STATS_CAPACITY_IDX = 0;
   public int GET_STATS_USED_IDX = 1;
@@ -490,9 +599,11 @@ public interface ClientProtocol extends VersionedProtocol {
    *                <li>2 get safe mode state.</li></ul>
    * @return <ul><li>0 if the safe mode is OFF or</li> 
    *         <li>1 if the safe mode is ON.</li></ul>
+   ¥
    * @throws IOException
    */
-  public boolean setSafeMode(FSConstants.SafeModeAction action) throws IOException;
+  public boolean setSafeMode(FSConstants.SafeModeAction action) 
+      throws IOException;
 
   /**
    * Save namespace image.
@@ -503,7 +614,7 @@ public interface ClientProtocol extends VersionedProtocol {
    * @throws AccessControlException if the superuser privilege is violated.
    * @throws IOException if image creation failed.
    */
-  public void saveNamespace() throws IOException;
+  public void saveNamespace() throws AccessControlException, IOException;
 
   /**
    * Enable/Disable restore failed storage.
@@ -543,6 +654,7 @@ public interface ClientProtocol extends VersionedProtocol {
   /**
    * Dumps namenode data structures into specified file. If the file
    * already exists, then append.
+   *
    * @throws IOException
    */
   public void metaSave(String filename) throws IOException;
@@ -553,40 +665,50 @@ public interface ClientProtocol extends VersionedProtocol {
    * @throws AccessControlException
    * @throws IOException
    */
-  FileStatus[] getCorruptFiles() 
-    throws AccessControlException, IOException; 
+  FileStatus[] getCorruptFiles() throws AccessControlException, IOException; 
   
   /**
    * Get the file info for a specific file or directory.
    * @param src The string representation of the path to the file
-   * @throws UnresolvedLinkException if the path contains symlinks;
-   *         IOException if permission to access file is denied by the system
+   *
    * @return object containing information regarding the file
    *         or null if file not found
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if the path contains a symlink. 
+   * @throws IOException If an I/O error occurred        
    */
   @Nullable
-  public HdfsFileStatus getFileInfo(String src) 
-      throws IOException, UnresolvedLinkException;
+  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException;
 
   /**
    * Get the file info for a specific file or directory. If the path 
    * refers to a symlink then the FileStatus of the symlink is returned.
    * @param src The string representation of the path to the file
-   * @throws UnresolvedLinkException if the path contains symlinks;
-   *         IOException if permission to access file is denied by the system
+   *
    * @return object containing information regarding the file
    *         or null if file not found
+   *
+   * @throws AccessControlException permission denied
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred        
    */
-  public HdfsFileStatus getFileLinkInfo(String src) 
-      throws IOException, UnresolvedLinkException;
+  public HdfsFileStatus getFileLinkInfo(String src)
+      throws AccessControlException, UnresolvedLinkException, IOException;
   
   /**
    * Get {@link ContentSummary} rooted at the specified directory.
    * @param path The string representation of the path
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   *
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>path</code> is not found
+   * @throws UnresolvedLinkException if <code>path</code> contains a symlink. 
+   * @throws IOException If an I/O error occurred
    */
-  public ContentSummary getContentSummary(String path) 
-      throws IOException, UnresolvedLinkException;
+  public ContentSummary getContentSummary(String path)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
 
   /**
    * Set the quota for a directory.
@@ -601,25 +723,32 @@ public interface ClientProtocol extends VersionedProtocol {
    * the quota to that value, (2) {@link FSConstants#QUOTA_DONT_SET}  implies 
    * the quota will not be changed, and (3) {@link FSConstants#QUOTA_RESET} 
    * implies the quota will be reset. Any other value is a runtime error.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   * @throws FileNotFoundException if the path is a file or 
-   *                               does not exist 
+   * 
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>path</code> is not found
    * @throws QuotaExceededException if the directory size 
-   *                                is greater than the given quota
+   *           is greater than the given quota
+   * @throws UnresolvedLinkException if the <code>path</code> contains a symlink. 
+   * @throws IOException If an I/O error occurred
    */
   public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
-    throws IOException, UnresolvedLinkException,
-           FileNotFoundException, SafeModeException;
-  
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
+
   /**
    * Write all metadata for this file into persistent storage.
    * The file must be currently open for writing.
    * @param src The string representation of the path
    * @param client The string representation of the client
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   * 
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
+   * @throws IOException If an I/O error occurred
    */
   public void fsync(String src, String client) 
-      throws IOException, UnresolvedLinkException;
+      throws AccessControlException, FileNotFoundException, 
+      UnresolvedLinkException, IOException;
 
   /**
    * Sets the modification and access time of the file to the specified time.
@@ -630,32 +759,52 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param atime The number of milliseconds since Jan 1, 1970.
    *              Setting atime to -1 means that access time should not be set
    *              by this call.
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   *              
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
+   * @throws IOException If an I/O error occurred
    */
-  public void setTimes(String src, long mtime, long atime) 
-      throws IOException, UnresolvedLinkException;
+  public void setTimes(String src, long mtime, long atime)
+      throws AccessControlException, FileNotFoundException, 
+      UnresolvedLinkException, IOException;
 
   /**
-   * Create a symbolic link to a file or directory.
+   * Create symlink to a file or directory.
    * @param target The pathname of the destination that the
    *               link points to.
    * @param link The pathname of the link being created.
    * @param dirPerm permissions to use when creating parent directories
    * @param createParent - if true then missing parent dirs are created
    *                       if false then parent must exist
-   * @throws IOException
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   */
-  public void createSymlink(String target, String link, FsPermission dirPerm, 
-      boolean createParent) throws IOException, UnresolvedLinkException;
+   *
+   * @throws AccessControlException permission denied
+   * @throws FileAlreadyExistsException If file <code>link</code> already exists
+   * @throws FileNotFoundException If parent of <code>link</code> does not exist
+   *           and <code>createParent</code> is false
+   * @throws ParentNotDirectoryException If parent of <code>link</code> is not a
+   *           directory.
+   * @throws UnresolvedLinkException if <code>link</target> contains a symlink. 
+   * @throws IOException If an I/O error occurred
+   */
+  public void createSymlink(String target, String link, FsPermission dirPerm,
+      boolean createParent) throws AccessControlException,
+      FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      IOException;
 
   /**
    * Resolve the first symbolic link on the specified path.
    * @param path The pathname that needs to be resolved
+   * 
    * @return The pathname after resolving the first symbolic link if any.
-   * @throws IOException
+   * 
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException If <code>path</code> does not exist
+   * @throws IOException If an I/O error occurred
    */
-  public String getLinkTarget(String path) throws IOException; 
+  public String getLinkTarget(String path) throws AccessControlException,
+      FileNotFoundException, IOException; 
   
   /**
    * Get a new generation stamp together with an access token for 

+ 24 - 31
src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -153,6 +152,8 @@ class FSDirectory implements Closeable {
 
   /**
    * Add the given filename to the fs.
+   * @throws QuotaExceededException 
+   * @throws FileAlreadyExistsException 
    */
   INodeFileUnderConstruction addFile(String path, 
                 PermissionStatus permissions,
@@ -162,7 +163,8 @@ class FSDirectory implements Closeable {
                 String clientMachine,
                 DatanodeDescriptor clientNode,
                 long generationStamp) 
-                throws IOException, UnresolvedLinkException {
+    throws FileAlreadyExistsException, QuotaExceededException,
+      UnresolvedLinkException {
     waitForReady();
 
     // Always do an implicit mkdirs for parent directory tree.
@@ -293,7 +295,7 @@ class FSDirectory implements Closeable {
                      INode[] inodes,
                      Block block,
                      DatanodeDescriptor targets[]
-  ) throws QuotaExceededException, IOException  {
+  ) throws QuotaExceededException {
     waitForReady();
 
     synchronized (rootDir) {
@@ -403,7 +405,9 @@ class FSDirectory implements Closeable {
    * @see #unprotectedRenameTo(String, String, long, Options.Rename...)
    */
   void renameTo(String src, String dst, Options.Rename... options)
-      throws IOException, UnresolvedLinkException {
+      throws FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, QuotaExceededException,
+      UnresolvedLinkException, IOException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: " + src
           + " to " + dst);
@@ -534,20 +538,17 @@ class FSDirectory implements Closeable {
   /**
    * Rename src to dst.
    * See {@link DistributedFileSystem#rename(Path, Path, Options.Rename...)}
-   * for details related to rename semantics.
+   * for details related to rename semantics and exceptions.
    * 
    * @param src source path
    * @param dst destination path
    * @param timestamp modification time
    * @param options Rename options
-   * @throws IOException if the operation violates any quota limit
-   * @throws FileAlreadyExistsException if src equals dst or the src is a 
-   *         symlink that points to dst.
-   * @return true if rename overwrites {@code dst}
    */
   boolean unprotectedRenameTo(String src, String dst, long timestamp,
-      Options.Rename... options) throws IOException,
-      UnresolvedLinkException {
+      Options.Rename... options) throws FileAlreadyExistsException,
+      FileNotFoundException, ParentNotDirectoryException,
+      QuotaExceededException, UnresolvedLinkException, IOException {
     boolean overwrite = false;
     if (null != options) {
       for (Rename option : options) {
@@ -715,11 +716,8 @@ class FSDirectory implements Closeable {
    * @return array of file blocks
    * @throws QuotaExceededException
    */
-  Block[] setReplication(String src, 
-                         short replication,
-                         int[] oldReplication
-                         ) throws QuotaExceededException, 
-                         UnresolvedLinkException {
+  Block[] setReplication(String src, short replication, int[] oldReplication)
+      throws QuotaExceededException, UnresolvedLinkException {
     waitForReady();
     Block[] fileBlocks = unprotectedSetReplication(src, replication, oldReplication);
     if (fileBlocks != null)  // log replication change
@@ -765,10 +763,9 @@ class FSDirectory implements Closeable {
    * Get the blocksize of a file
    * @param filename the filename
    * @return the number of bytes 
-   * @throws IOException if it is a directory or does not exist.
    */
-  long getPreferredBlockSize(String filename) 
-      throws IOException, UnresolvedLinkException {
+  long getPreferredBlockSize(String filename) throws UnresolvedLinkException,
+      FileNotFoundException, IOException {
     synchronized (rootDir) {
       INode inode = rootDir.getNode(filename, false);
       if (inode == null) {
@@ -834,13 +831,10 @@ class FSDirectory implements Closeable {
   }
 
   /**
-   * 
-   * @param target
-   * @param srcs
-   * @throws IOException
+   * Concat all the blocks from srcs to trg and delete the srcs files
    */
   public void concatInternal(String target, String [] srcs) 
-      throws IOException, UnresolvedLinkException {
+      throws UnresolvedLinkException {
     synchronized(rootDir) {
       // actual move
       waitForReady();
@@ -854,15 +848,14 @@ class FSDirectory implements Closeable {
 
   
   /**
-   * Concat all the blocks from srcs to trg
-   * and delete the srcs files
+   * Concat all the blocks from srcs to trg and delete the srcs files
    * @param target target file to move the blocks to
    * @param srcs list of file to move the blocks from
    * Must be public because also called from EditLogs
    * NOTE: - it does not update quota (not needed for concat)
    */
   public void unprotectedConcat(String target, String [] srcs) 
-      throws IOException, UnresolvedLinkException {
+      throws UnresolvedLinkException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSNamesystem.concat to "+target);
     }
@@ -1806,10 +1799,10 @@ class FSDirectory implements Closeable {
   /**
    * Add the given symbolic link to the fs. Record it in the edits log.
    */
-  INodeSymlink addSymlink(String path, String target, 
-                          PermissionStatus dirPerms,
-                          boolean createParent) 
-      throws IOException, UnresolvedLinkException {
+  INodeSymlink addSymlink(String path, String target,
+      PermissionStatus dirPerms, boolean createParent)
+      throws UnresolvedLinkException, FileAlreadyExistsException,
+      QuotaExceededException, IOException {
     waitForReady();
 
     final long modTime = FSNamesystem.now();

+ 89 - 63
src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.commons.logging.*;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.*;
@@ -66,6 +67,8 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.Options;
@@ -659,7 +662,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @throws IOException
    */
   public synchronized void setPermission(String src, FsPermission permission)
-      throws IOException, UnresolvedLinkException {
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
     if (isInSafeMode())
       throw new SafeModeException("Cannot set permission for " + src, safeMode);
     checkOwner(src);
@@ -678,7 +682,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @throws IOException
    */
   public synchronized void setOwner(String src, String username, String group)
-      throws IOException, UnresolvedLinkException {
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
     if (isInSafeMode())
         throw new SafeModeException("Cannot set owner for " + src, safeMode);
     FSPermissionChecker pc = checkOwner(src);
@@ -706,7 +711,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @see ClientProtocol#getBlockLocations(String, long, long)
    */
   LocatedBlocks getBlockLocations(String clientMachine, String src,
-      long offset, long length) throws IOException, UnresolvedLinkException {
+      long offset, long length) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
     LocatedBlocks blocks = getBlockLocations(src, offset, length, true);
     if (blocks != null) {
       //sort the blocks
@@ -725,16 +731,19 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @throws FileNotFoundException
    */
   LocatedBlocks getBlockLocations(String src, long offset, long length,
-      boolean doAccessTime) throws IOException, UnresolvedLinkException {
+      boolean doAccessTime) throws FileNotFoundException,
+      UnresolvedLinkException, IOException {
     if (isPermissionEnabled) {
       checkPathAccess(src, FsAction.READ);
     }
 
     if (offset < 0) {
-      throw new IOException("Negative offset is not supported. File: " + src );
+      throw new HadoopIllegalArgumentException(
+          "Negative offset is not supported. File: " + src);
     }
     if (length < 0) {
-      throw new IOException("Negative length is not supported. File: " + src );
+      throw new HadoopIllegalArgumentException(
+          "Negative length is not supported. File: " + src);
     }
     final LocatedBlocks ret = getBlockLocationsInternal(src,
         offset, length, doAccessTime);  
@@ -750,7 +759,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                                                        long offset, 
                                                        long length,
                                                        boolean doAccessTime)
-      throws IOException, UnresolvedLinkException {
+      throws FileNotFoundException, UnresolvedLinkException, IOException {
     INodeFile inode = dir.getFileINode(src);
     if (inode == null)
       throw new FileNotFoundException("File does not exist: " + src);
@@ -1005,7 +1014,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       throw new SafeModeException("Cannot create symlink " + link, safeMode);
     }
     if (!DFSUtil.isValidName(link)) {
-      throw new IOException("Invalid file name: " + link);
+      throw new InvalidPathException("Invalid file name: " + link);
     }
     if (!dir.isValidToCreate(link)) {
       throw new IOException("failed to create link " + link 
@@ -1046,8 +1055,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     return status;
   }
 
-  private synchronized boolean setReplicationInternal(String src, short replication) 
-    throws IOException, UnresolvedLinkException {
+  private synchronized boolean setReplicationInternal(String src,
+      short replication) throws AccessControlException, QuotaExceededException,
+      SafeModeException, UnresolvedLinkException, IOException {
     if (isInSafeMode())
       throw new SafeModeException("Cannot set replication for " + src, safeMode);
     blockManager.verifyReplication(src, replication, null);
@@ -1092,8 +1102,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   /*
    * Verify that parent directory of src exists.
    */
-  private void verifyParentDir(String src) throws FileAlreadyExistsException,
-      FileNotFoundException, UnresolvedLinkException {
+  private void verifyParentDir(String src) throws FileNotFoundException,
+      ParentNotDirectoryException, UnresolvedLinkException {
     Path parent = new Path(src).getParent();
     if (parent != null) {
       INode[] pathINodes = dir.getExistingPathINodes(parent.toString());
@@ -1102,7 +1112,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         throw new FileNotFoundException("Parent directory doesn't exist: "
             + parent.toString());
       } else if (!parentNode.isDirectory() && !parentNode.isLink()) {
-        throw new FileAlreadyExistsException("Parent path is not a directory: "
+        throw new ParentNotDirectoryException("Parent path is not a directory: "
             + parent.toString());
       }
     }
@@ -1111,16 +1121,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   /**
    * Create a new file entry in the namespace.
    * 
-   * @see ClientProtocol#create(String, FsPermission, String, boolean, short, long)
-   * 
-   * @throws IOException if file name is invalid
-   *         {@link FSDirectory#isValidToCreate(String)}.
+   * For description of parameters and exceptions thrown see 
+   * {@link ClientProtocol#create()}
    */
-  void startFile(String src, PermissionStatus permissions,
-                 String holder, String clientMachine,
-                 EnumSet<CreateFlag> flag, boolean createParent, 
-                 short replication, long blockSize)
-      throws IOException, UnresolvedLinkException {
+  void startFile(String src, PermissionStatus permissions, String holder,
+      String clientMachine, EnumSet<CreateFlag> flag, boolean createParent,
+      short replication, long blockSize) throws AccessControlException,
+      SafeModeException, FileAlreadyExistsException, UnresolvedLinkException,
+      FileNotFoundException, ParentNotDirectoryException, IOException {
     startFileInternal(src, permissions, holder, clientMachine, flag,
         createParent, replication, blockSize);
     getEditLog().logSync();
@@ -1132,15 +1140,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     }
   }
 
+  /**
+   * For description of exceptions @see {@link ClientProtocol#create()}
+   */
   private synchronized void startFileInternal(String src,
-                                              PermissionStatus permissions,
-                                              String holder, 
-                                              String clientMachine, 
-                                              EnumSet<CreateFlag> flag,
-                                              boolean createParent,
-                                              short replication,
-                                              long blockSize)
-      throws IOException, UnresolvedLinkException {
+      PermissionStatus permissions, String holder, String clientMachine,
+      EnumSet<CreateFlag> flag, boolean createParent, short replication,
+      long blockSize) throws SafeModeException, FileAlreadyExistsException,
+      AccessControlException, UnresolvedLinkException, FileNotFoundException,
+      ParentNotDirectoryException, IOException {
     boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
     boolean append = flag.contains(CreateFlag.APPEND);
     boolean create = flag.contains(CreateFlag.CREATE);
@@ -1158,13 +1166,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     if (isInSafeMode())
       throw new SafeModeException("Cannot create file" + src, safeMode);
     if (!DFSUtil.isValidName(src)) {
-      throw new IOException("Invalid file name: " + src);
+      throw new InvalidPathException(src);
     }
 
     // Verify that the destination does not exist as a directory already.
     boolean pathExists = dir.exists(src);
     if (pathExists && dir.isDir(src)) {
-      throw new IOException("Cannot create file "+ src + "; already exists as a directory.");
+      throw new FileAlreadyExistsException("Cannot create file "+ src + "; already exists as a directory.");
     }
 
     if (isPermissionEnabled) {
@@ -1321,9 +1329,11 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * Append to an existing file in the namespace.
    */
   LocatedBlock appendFile(String src, String holder, String clientMachine)
-    throws IOException, UnresolvedLinkException {
+      throws AccessControlException, SafeModeException,
+      FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, IOException {
     if (supportAppends == false) {
-      throw new IOException("Append to hdfs not supported." +
+      throw new UnsupportedOperationException("Append to hdfs not supported." +
                             " Please refer to dfs.support.append configuration parameter.");
     }
     startFileInternal(src, null, holder, clientMachine, EnumSet.of(CreateFlag.APPEND), 
@@ -1405,7 +1415,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                                          Block previous,
                                          HashMap<Node, Node> excludedNodes
                                          ) 
-      throws IOException, UnresolvedLinkException {
+      throws LeaseExpiredException, NotReplicatedYetException,
+      QuotaExceededException, SafeModeException, UnresolvedLinkException,
+      IOException {
     long fileLength, blockSize;
     int replication;
     DatanodeDescriptor clientNode = null;
@@ -1481,7 +1493,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * The client would like to let go of the given block
    */
   public synchronized boolean abandonBlock(Block b, String src, String holder)
-    throws IOException, UnresolvedLinkException {
+      throws LeaseExpiredException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
     //
     // Remove the block from the pending creates list
     //
@@ -1497,14 +1510,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   
   // make sure that we still have the lease on this file.
   private INodeFileUnderConstruction checkLease(String src, String holder) 
-    throws IOException, UnresolvedLinkException {
+    throws LeaseExpiredException, UnresolvedLinkException {
     INodeFile file = dir.getFileINode(src);
     checkLease(src, holder, file);
     return (INodeFileUnderConstruction)file;
   }
 
-  private void checkLease(String src, String holder, INode file) 
-                                                     throws IOException {
+  private void checkLease(String src, String holder, INode file)
+      throws LeaseExpiredException {
 
     if (file == null || file.isDirectory()) {
       Lease lease = leaseManager.getLease(holder);
@@ -1536,14 +1549,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @throws IOException on error (eg lease mismatch, file not open, file deleted)
    */
   public boolean completeFile(String src, String holder, Block last) 
-    throws IOException, UnresolvedLinkException {
+    throws SafeModeException, UnresolvedLinkException, IOException {
     boolean success = completeFileInternal(src, holder, last);
     getEditLog().logSync();
     return success ;
   }
 
   private synchronized boolean completeFileInternal(String src, 
-    String holder, Block last) throws IOException, UnresolvedLinkException {
+      String holder, Block last) throws SafeModeException,
+      UnresolvedLinkException, IOException {
     NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src + " for " + holder);
     if (isInSafeMode())
       throw new SafeModeException("Cannot complete file " + src, safeMode);
@@ -1584,10 +1598,11 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @param src path to the file
    * @param inodes INode representing each of the components of src. 
    *        <code>inodes[inodes.length-1]</code> is the INode for the file.
+   *        
+   * @throws QuotaExceededException If addition of block exceeds space quota
    */
-  private Block allocateBlock(String src,
-                              INode[] inodes,
-                              DatanodeDescriptor targets[]) throws IOException {
+  private Block allocateBlock(String src, INode[] inodes,
+      DatanodeDescriptor targets[]) throws QuotaExceededException {
     Block b = new Block(FSNamesystem.randBlockId.nextLong(), 0, 0); 
     while(isValidBlock(b)) {
       b.setBlockId(FSNamesystem.randBlockId.nextLong());
@@ -1604,7 +1619,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * replicated.  If not, return false. If checkall is true, then check
    * all blocks, otherwise check only penultimate block.
    */
-  synchronized boolean checkFileProgress(INodeFile v, boolean checkall) throws IOException {
+  synchronized boolean checkFileProgress(INodeFile v, boolean checkall) {
     if (checkall) {
       //
       // check all blocks of the file.
@@ -1719,7 +1734,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   }
 
   private synchronized void renameToInternal(String src, String dst,
-      Options.Rename... options) throws IOException, UnresolvedLinkException {
+      Options.Rename... options) throws IOException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: with options - "
           + src + " to " + dst);
@@ -1728,7 +1743,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       throw new SafeModeException("Cannot rename " + src, safeMode);
     }
     if (!DFSUtil.isValidName(dst)) {
-      throw new IOException("Invalid name: " + dst);
+      throw new InvalidPathException("Invalid name: " + dst);
     }
     if (isPermissionEnabled) {
       checkParentAccess(src, FsAction.WRITE);
@@ -1741,11 +1756,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   }
   
   /**
-   * Remove the indicated filename from namespace. If the filename 
-   * is a directory (non empty) and recursive is set to false then throw exception.
+   * Remove the indicated file from namespace.
+   * 
+   * @see ClientProtocol#delete(String, boolean) for detailed descriptoin and 
+   * description of exceptions
    */
-    public boolean delete(String src, boolean recursive) 
-      throws IOException, UnresolvedLinkException {
+    public boolean delete(String src, boolean recursive)
+      throws AccessControlException, SafeModeException,
+      UnresolvedLinkException, IOException {
       if ((!recursive) && (!dir.isDirEmpty(src))) {
         throw new IOException(src + " is non empty");
       }
@@ -1769,9 +1787,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * the {@link FSNamesystem} lock.
    * <p>
    * For small directory or file the deletion is done in one shot.
+   * 
+   * @see ClientProtocol#delete(String, boolean) for description of exceptions
    */
-  private boolean deleteInternal(String src, boolean enforcePermission) 
-    throws IOException, UnresolvedLinkException {
+  private boolean deleteInternal(String src, boolean enforcePermission)
+      throws AccessControlException, SafeModeException,
+      UnresolvedLinkException, IOException{
     boolean deleteNow = false;
     ArrayList<Block> collectedBlocks = new ArrayList<Block>();
     synchronized(this) {
@@ -1833,15 +1854,17 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @param src The string representation of the path to the file
    * @param resolveLink whether to throw UnresolvedLinkException 
    *        if src refers to a symlinks
-   * @throws IOException if permission to access file is denied by the system 
+   *
+   * @throws AccessControlException if access is denied
    * @throws UnresolvedLinkException if a symlink is encountered.
+   *
    * @return object containing information regarding the file
    *         or null if file not found
    */
   HdfsFileStatus getFileInfo(String src, boolean resolveLink) 
-    throws IOException, UnresolvedLinkException {
+    throws AccessControlException, UnresolvedLinkException {
     if (!DFSUtil.isValidName(src)) {
-      throw new IOException("Invalid file name: " + src);
+      throw new InvalidPathException("Invalid file name: " + src);
     }
     if (isPermissionEnabled) {
       checkTraverse(src);
@@ -1883,7 +1906,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     if (isInSafeMode())
       throw new SafeModeException("Cannot create directory " + src, safeMode);
     if (!DFSUtil.isValidName(src)) {
-      throw new IOException("Invalid directory name: " + src);
+      throw new InvalidPathException(src);
     }
     if (isPermissionEnabled) {
       checkAncestorAccess(src, FsAction.WRITE);
@@ -1899,13 +1922,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     checkFsObjectLimit();
 
     if (!dir.mkdirs(src, permissions, false, now())) {
-      throw new IOException("Invalid directory name: " + src);
+      throw new IOException("Failed to create directory: " + src);
     }
     return true;
   }
 
-  ContentSummary getContentSummary(String src) 
-      throws IOException, UnresolvedLinkException {
+  ContentSummary getContentSummary(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException {
     if (isPermissionEnabled) {
       checkPermission(src, false, null, null, null, FsAction.READ_EXECUTE);
     }
@@ -2214,9 +2237,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @param src the directory name
    * @param startAfter the name to start after
    * @return a partial listing starting after startAfter
+   * 
+   * @throws AccessControlException if access is denied
+   * @throws UnresolvedLinkException if symbolic link is encountered
+   * @throws IOException if other I/O error occurred
    */
   public DirectoryListing getListing(String src, byte[] startAfter) 
-    throws IOException, UnresolvedLinkException {
+    throws AccessControlException, UnresolvedLinkException, IOException {
     if (isPermissionEnabled) {
       if (dir.isDir(src)) {
         checkPathAccess(src, FsAction.READ_EXECUTE);
@@ -4273,8 +4300,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   // rename was successful. If any part of the renamed subtree had
   // files that were being written to, update with new filename.
   //
-  void changeLease(String src, String dst, HdfsFileStatus dinfo) 
-                   throws IOException {
+  void changeLease(String src, String dst, HdfsFileStatus dinfo) {
     String overwrite;
     String replaceBy;
 

+ 3 - 3
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSMkdirs.java

@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs;
 import junit.framework.TestCase;
 import java.io.*;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 
@@ -97,9 +97,9 @@ public class TestDFSMkdirs extends TestCase {
         expectedException = e;
       }
       assertTrue("Create a directory when parent dir exists as file using"
-          + " mkdir() should throw FileAlreadyExistsException ",
+          + " mkdir() should throw ParentNotDirectoryException ",
           expectedException != null
-              && expectedException instanceof FileAlreadyExistsException);
+              && expectedException instanceof ParentNotDirectoryException);
       // Create a dir in a non-exist directory, should fail
       expectedException = null;
       try {

+ 5 - 4
src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -726,9 +727,9 @@ public class TestFileCreation extends junit.framework.TestCase {
         expectedException = e;
       }
       assertTrue("Create a file when parent directory exists as a file"
-          + " should throw FileAlreadyExistsException ",
+          + " should throw ParentNotDirectoryException ",
           expectedException != null
-              && expectedException instanceof FileAlreadyExistsException);
+              && expectedException instanceof ParentNotDirectoryException);
       fs.delete(path, true);
       // Create a file in a non-exist directory, should fail
       final Path path2 = new Path(nonExistDir + "/testCreateNonRecursive");
@@ -755,9 +756,9 @@ public class TestFileCreation extends junit.framework.TestCase {
         expectedException = e;
       }
       assertTrue("Overwrite a file when parent directory exists as a file"
-          + " should throw FileAlreadyExistsException ",
+          + " should throw ParentNotDirectoryException ",
           expectedException != null
-              && expectedException instanceof FileAlreadyExistsException);
+              && expectedException instanceof ParentNotDirectoryException);
       fs.delete(path, true);
       // Overwrite a file in a non-exist directory, should fail
       final Path path3 = new Path(nonExistDir + "/testOverwriteNonRecursive");