You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sr...@apache.org on 2010/03/03 03:48:50 UTC

svn commit: r918309 [1/2] - in /hadoop/common/trunk: ./ src/java/org/apache/hadoop/ src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/ipc/

Author: sradia
Date: Wed Mar  3 02:48:49 2010
New Revision: 918309

URL: http://svn.apache.org/viewvc?rev=918309&view=rev
Log:
   HADOOP-6537 Declare more detailed exceptions in FileContext and AbstractFileSystem
   (Suresh Srinivas via Sanjay Radia)

Added:
    hadoop/common/trunk/src/java/org/apache/hadoop/HadoopIllegalArgumentException.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/InvalidPathException.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/UnsupportedFileSystemException.java
    hadoop/common/trunk/src/java/org/apache/hadoop/ipc/RpcClientException.java
    hadoop/common/trunk/src/java/org/apache/hadoop/ipc/RpcException.java
    hadoop/common/trunk/src/java/org/apache/hadoop/ipc/RpcServerException.java
    hadoop/common/trunk/src/java/org/apache/hadoop/ipc/UnexpectedServerException.java
Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/AbstractFileSystem.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileContext.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=918309&r1=918308&r2=918309&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Wed Mar  3 02:48:49 2010
@@ -185,6 +185,9 @@
    HADOOP-6599  Split existing RpcMetrics into RpcMetrics & RpcDetailedMetrics.
    (Suresh Srinivas via Sanjay Radia)
 
+   HADOOP-6537 Declare more detailed exceptions in FileContext and AbstractFileSystem
+   (Suresh Srinivas via Sanjay Radia)
+
   OPTIMIZATIONS
 
     HADOOP-6467. Improve the performance on HarFileSystem.listStatus(..).

Added: hadoop/common/trunk/src/java/org/apache/hadoop/HadoopIllegalArgumentException.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/HadoopIllegalArgumentException.java?rev=918309&view=auto
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/HadoopIllegalArgumentException.java (added)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/HadoopIllegalArgumentException.java Wed Mar  3 02:48:49 2010
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop;
+
+/**
+ * Indicates that a method has been passed illegal or invalid argument. This
+ * exception is thrown instead of IllegalArgumentException to differentiate the
+ * exception thrown in Hadoop implementation from the one thrown in JDK.
+ */
+public class HadoopIllegalArgumentException extends IllegalArgumentException {
+  private static final long serialVersionUID = 1L;
+  
+  /**
+   * Constructs exception with the specified detail message. 
+   * @param message detailed message.
+   */
+  public HadoopIllegalArgumentException(final String message) {
+    super(message);
+  }
+}

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/AbstractFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/AbstractFileSystem.java?rev=918309&r1=918308&r2=918309&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/AbstractFileSystem.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/AbstractFileSystem.java Wed Mar  3 02:48:49 2010
@@ -31,6 +31,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -38,29 +39,31 @@
 import org.apache.hadoop.fs.Options.CreateOpts;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.Progressable;
 
 /**
- * This class provides an interface for implementors of a Hadoop filesystem
+ * This class provides an interface for implementors of a Hadoop file system
  * (analogous to the VFS of Unix). Applications do not access this class;
- * instead they access files across all filesystems using {@link FileContext}.
+ * instead they access files across all file systems using {@link FileContext}.
  * 
  * Pathnames passed to AbstractFileSystem can be fully qualified URI that
- * matches the "this" filesystem (ie same scheme and authority) 
+ * matches the "this" file system (ie same scheme and authority) 
  * or a Slash-relative name that is assumed to be relative
- * to the root of the "this" filesystem .
+ * to the root of the "this" file system .
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
 public abstract class AbstractFileSystem {
   static final Log LOG = LogFactory.getLog(AbstractFileSystem.class);
 
-  /** Recording statistics per a filesystem class. */
+  /** Recording statistics per a file system class. */
   private static final Map<Class<? extends AbstractFileSystem>, Statistics> 
   STATISTICS_TABLE =
       new IdentityHashMap<Class<? extends AbstractFileSystem>, Statistics>();
   
-  /** Cache of constructors for each filesystem class. */
+  /** Cache of constructors for each file system class. */
   private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE = 
     new ConcurrentHashMap<Class<?>, Constructor<?>>();
   
@@ -118,21 +121,25 @@
   }
   
   /**
-   * Create a file system instance for the specified uri using the conf.
-   * The conf is used to find the class name that implements the filesystem.
-   * The conf is also passed to the filesystem for its configuration.
-   * @param uri
-   * @param conf
-   * @return
-   * @throws IOException
+   * Create a file system instance for the specified uri using the conf. The
+   * conf is used to find the class name that implements the file system. The
+   * conf is also passed to the file system for its configuration.
+   *
+   * @param uri URI of the file system
+   * @param conf Configuration for the file system
+   * 
+   * @return Returns the file system for the given URI
+   *
+   * @throws UnsupportedFileSystemException file system for <code>uri</code> is
+   *           not found
    */
-  private static AbstractFileSystem createFileSystem(URI uri,
-    Configuration conf) throws IOException {
+  private static AbstractFileSystem createFileSystem(URI uri, Configuration conf)
+      throws UnsupportedFileSystemException {
     Class<?> clazz = conf.getClass("fs.AbstractFileSystem." + 
                                 uri.getScheme() + ".impl", null);
     if (clazz == null) {
-      throw new IOException("No AbstractFileSystem for scheme: "
-          + uri.getScheme());
+      throw new UnsupportedFileSystemException(
+          "No AbstractFileSystem for scheme: " + uri.getScheme());
     }
     return (AbstractFileSystem) newInstance(clazz, uri, conf);
   }
@@ -159,7 +166,7 @@
     }
   }
 
-  protected static synchronized void printStatistics() throws IOException {
+  protected static synchronized void printStatistics() {
     for (Map.Entry<Class<? extends AbstractFileSystem>, Statistics> pair: 
             STATISTICS_TABLE.entrySet()) {
       System.out.println("  FileSystem " + pair.getKey().getName() + 
@@ -167,20 +174,25 @@
     }
   }
 
-  
   /**
-   * The main factory method for creating a filesystem.
-   * Get a filesystem for the URI's scheme and authority.
-   * The scheme of the URI determines a configuration property name,
-   * <tt>fs.AbstractFileSystem.<i>scheme</i>.impl</tt> whose value names
-   * the AbstractFileSystem class. 
-   * The entire URI and conf is passed to the AbstractFileSystem factory
-   * method.
+   * The main factory method for creating a file system. Get a file system for
+   * the URI's scheme and authority. The scheme of the <code>uri</code>
+   * determines a configuration property name,
+   * <tt>fs.AbstractFileSystem.<i>scheme</i>.impl</tt> whose value names the
+   * AbstractFileSystem class.
+   * 
+   * The entire URI and conf is passed to the AbstractFileSystem factory method.
+   * 
    * @param uri for the file system to be created.
-   * @param conf which is passed to the filesystem impl.
+   * @param conf which is passed to the file system impl.
+   * 
+   * @return file system for the given URI.
+   * 
+   * @throws UnsupportedFileSystemException if the file system for
+   *           <code>uri</code> is not supported.
    */
   static AbstractFileSystem get(final URI uri, final Configuration conf)
-    throws IOException {
+      throws UnsupportedFileSystemException {
     return createFileSystem(uri, conf);
   }
 
@@ -191,10 +203,12 @@
    * @param supportedScheme the scheme supported by the implementor
    * @param authorityNeeded if true then theURI must have authority, if false
    *          then the URI must have null authority.
-   * @throws URISyntaxException
+   *
+   * @throws URISyntaxException <code>uri</code> has syntax error
    */
   protected AbstractFileSystem(final URI uri, final String supportedScheme,
-      final boolean authorityNeeded, final int defaultPort) throws URISyntaxException {
+      final boolean authorityNeeded, final int defaultPort)
+      throws URISyntaxException {
     myUri = getUri(uri, supportedScheme, authorityNeeded, defaultPort);
     statistics = getStatistics(supportedScheme, getClass()); 
   }
@@ -202,46 +216,48 @@
   protected void checkScheme(URI uri, String supportedScheme) {
     String scheme = uri.getScheme();
     if (scheme == null) {
-      throw new IllegalArgumentException("Uri without scheme: " + uri);
+      throw new HadoopIllegalArgumentException("Uri without scheme: " + uri);
     }
     if (!scheme.equals(supportedScheme)) {
-      throw new IllegalArgumentException("Uri scheme " + uri
+      throw new HadoopIllegalArgumentException("Uri scheme " + uri
           + " does not match the scheme " + supportedScheme);
     }
   }
 
   /**
    * Get the URI for the file system based on the given URI. The path, query
-   * part of the given URI is stripped out and default filesystem port is used
+   * part of the given URI is stripped out and default file system port is used
    * to form the URI.
    * 
    * @param uri FileSystem URI.
    * @param authorityNeeded if true authority cannot be null in the URI. If
    *          false authority must be null.
    * @param defaultPort default port to use if port is not specified in the URI.
+   * 
    * @return URI of the file system
-   * @throws URISyntaxException 
+   * 
+   * @throws URISyntaxException <code>uri</code> has syntax error
    */
   private URI getUri(URI uri, String supportedScheme,
       boolean authorityNeeded, int defaultPort) throws URISyntaxException {
     checkScheme(uri, supportedScheme);
-    // A filesystem implementation that requires authority must always
+    // A file system implementation that requires authority must always
     // specify default port
     if (defaultPort < 0 && authorityNeeded) {
-      throw new IllegalArgumentException(
+      throw new HadoopIllegalArgumentException(
           "FileSystem implementation error -  default port " + defaultPort
               + " is not valid");
     }
     String authority = uri.getAuthority();
     if (!authorityNeeded) {
       if (authority != null) {
-        throw new IllegalArgumentException("Scheme with non-null authority: "
+        throw new HadoopIllegalArgumentException("Scheme with non-null authority: "
             + uri);
       }
       return new URI(supportedScheme + ":///");
     }
     if (authority == null) {
-      throw new IllegalArgumentException("Uri without authority: " + uri);
+      throw new HadoopIllegalArgumentException("Uri without authority: " + uri);
     }
     int port = uri.getPort();
     port = port == -1 ? defaultPort : port;
@@ -249,15 +265,17 @@
   }
   
   /**
-   * The default port of this filesystem.
-   * @return default port of this filesystem's Uri scheme
-   * A uri with a port of -1 => default port;
+   * The default port of this file system.
+   * 
+   * @return default port of this file system's Uri scheme
+   *         A uri with a port of -1 => default port;
    */
   protected abstract int getUriDefaultPort();
 
   /**
    * Returns a URI whose scheme and authority identify this FileSystem.
-   * @return the uri of this filesystem.
+   * 
+   * @return the uri of this file system.
    */
   protected URI getUri() {
     return myUri;
@@ -269,6 +287,8 @@
    * If the path is fully qualified URI, then its scheme and authority
    * matches that of this file system. Otherwise the path must be 
    * slash-relative name.
+   * 
+   * @throws InvalidPathException if the path is invalid
    */
   protected void checkPath(Path path) {
     URI uri = path.toUri();
@@ -279,10 +299,10 @@
         if (path.isUriPathAbsolute()) {
           return;
         }
-        throw new IllegalArgumentException("relative paths not allowed:" + 
+        throw new InvalidPathException("relative paths not allowed:" + 
             path);
       } else {
-        throw new IllegalArgumentException(
+        throw new InvalidPathException(
             "Path without scheme with non-null autorhrity:" + path);
       }
     }
@@ -295,8 +315,8 @@
        (thisAuthority != null && 
             !thisAuthority.equalsIgnoreCase(thatAuthority)) ||
        (thisAuthority == null && thatAuthority != null)) {
-      throw new IllegalArgumentException("Wrong FS: " + path + 
-                                    ", expected: "+this.getUri());
+      throw new InvalidPathException("Wrong FS: " + path + ", expected: "
+          + this.getUri());
     }
     
     int thisPort = this.getUri().getPort();
@@ -305,42 +325,46 @@
       thatPort = this.getUriDefaultPort();
     }
     if (thisPort != thatPort) {
-      throw new IllegalArgumentException("Wrong FS: "+path+
-                                       ", expected: "+this.getUri());
+      throw new InvalidPathException("Wrong FS: " + path + ", expected: "
+          + this.getUri());
     }
   }
   
   /**
-   * Get the path-part of a pathname. Checks that URI matches this filesystem
+   * Get the path-part of a pathname. Checks that URI matches this file system
    * and that the path-part is a valid name.
-   * @param p
+   * 
+   * @param p path
+   * 
    * @return path-part of the Path p
    */
   protected String getUriPath(final Path p) {
     checkPath(p);
     String s = p.toUri().getPath();
     if (!isValidName(s)) {
-      throw new IllegalArgumentException("Path part " + s + " from URI" +
-          p + " is not a valid filename.");
+      throw new InvalidPathException("Path part " + s + " from URI" + p
+          + " is not a valid filename.");
     }
     return s;
   }
   
   /**
    * Some file systems like LocalFileSystem have an initial workingDir
-   * that we use as the starting workingDir. For other file systems
+   * that is used as the starting workingDir. For other file systems
    * like HDFS there is no built in notion of an initial workingDir.
    * 
-   * @return the initial workingDir if the filesystem if it has such a notion
-   * otherwise return a null.
+   * @return the initial workingDir if the file system has such a notion
+   *         otherwise return a null.
    */
   protected Path getInitialWorkingDirectory() {
     return null;
   }
   
   /** 
-   * Return the current user's home directory in this filesystem.
+   * Return the current user's home directory in this file system.
    * The default implementation returns "/user/$USER/".
+   * 
+   * @return current user's home directory.
    */
   protected Path getHomeDirectory() {
     return new Path("/user/"+System.getProperty("user.name")).makeQualified(
@@ -349,8 +373,10 @@
   
   /**
    * Return a set of server default configuration values.
+   * 
    * @return server default configuration values
-   * @throws IOException
+   * 
+   * @throws IOException an I/O error occurred
    */
   protected abstract FsServerDefaults getServerDefaults() throws IOException; 
 
@@ -361,8 +387,10 @@
    * (i.e. umask has been applied).
    */
   protected final FSDataOutputStream create(final Path f,
-    final EnumSet<CreateFlag> createFlag, Options.CreateOpts... opts)
-    throws IOException, UnresolvedLinkException {
+      final EnumSet<CreateFlag> createFlag, Options.CreateOpts... opts)
+      throws AccessControlException, FileAlreadyExistsException,
+      FileNotFoundException, ParentNotDirectoryException,
+      UnsupportedFileSystemException, UnresolvedLinkException, IOException {
     checkPath(f);
     int bufferSize = -1;
     short replication = -1;
@@ -375,46 +403,53 @@
     for (CreateOpts iOpt : opts) {
       if (CreateOpts.BlockSize.class.isInstance(iOpt)) {
         if (blockSize != -1) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
+          throw new HadoopIllegalArgumentException(
+              "BlockSize option is set multiple times");
         }
         blockSize = ((CreateOpts.BlockSize) iOpt).getValue();
       } else if (CreateOpts.BufferSize.class.isInstance(iOpt)) {
         if (bufferSize != -1) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
+          throw new HadoopIllegalArgumentException(
+              "BufferSize option is set multiple times");
         }
         bufferSize = ((CreateOpts.BufferSize) iOpt).getValue();
       } else if (CreateOpts.ReplicationFactor.class.isInstance(iOpt)) {
         if (replication != -1) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
+          throw new HadoopIllegalArgumentException(
+              "ReplicationFactor option is set multiple times");
         }
         replication = ((CreateOpts.ReplicationFactor) iOpt).getValue();
       } else if (CreateOpts.BytesPerChecksum.class.isInstance(iOpt)) {
         if (bytesPerChecksum != -1) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
+          throw new HadoopIllegalArgumentException(
+              "BytesPerChecksum option is set multiple times");
         }
         bytesPerChecksum = ((CreateOpts.BytesPerChecksum) iOpt).getValue();
       } else if (CreateOpts.Perms.class.isInstance(iOpt)) {
         if (permission != null) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
+          throw new HadoopIllegalArgumentException(
+              "Perms option is set multiple times");
         }
         permission = ((CreateOpts.Perms) iOpt).getValue();
       } else if (CreateOpts.Progress.class.isInstance(iOpt)) {
         if (progress != null) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
+          throw new HadoopIllegalArgumentException(
+              "Progress option is set multiple times");
         }
         progress = ((CreateOpts.Progress) iOpt).getValue();
       } else if (CreateOpts.CreateParent.class.isInstance(iOpt)) {
         if (createParent != null) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
+          throw new HadoopIllegalArgumentException(
+              "CreateParent option is set multiple times");
         }
         createParent = ((CreateOpts.CreateParent) iOpt).getValue();
       } else {
-        throw new IllegalArgumentException("Unkown CreateOpts of type " +
+        throw new HadoopIllegalArgumentException("Unkown CreateOpts of type " +
             iOpt.getClass().getName());
       }
     }
     if (permission == null) {
-      throw new IllegalArgumentException("no permission supplied");
+      throw new HadoopIllegalArgumentException("no permission supplied");
     }
 
 
@@ -441,7 +476,7 @@
     }
 
     if (blockSize % bytesPerChecksum != 0) {
-      throw new IllegalArgumentException(
+      throw new HadoopIllegalArgumentException(
              "blockSize should be a multiple of checksumsize");
     }
 
@@ -455,64 +490,70 @@
    * have been declared explicitly.
    */
   protected abstract FSDataOutputStream createInternal(Path f,
-      EnumSet<CreateFlag> flag, FsPermission absolutePermission, int bufferSize,
-      short replication, long blockSize, Progressable progress,
-      int bytesPerChecksum, boolean createParent) 
-      throws IOException, UnresolvedLinkException;
+      EnumSet<CreateFlag> flag, FsPermission absolutePermission,
+      int bufferSize, short replication, long blockSize, Progressable progress,
+      int bytesPerChecksum, boolean createParent)
+      throws AccessControlException, FileAlreadyExistsException,
+      FileNotFoundException, ParentNotDirectoryException,
+      UnsupportedFileSystemException, UnresolvedLinkException, IOException;
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#mkdir(Path, FsPermission, boolean)} except that the Path
-   * f must be fully qualified and the permission is absolute (ie umask has been
-   * applied).
+   * f must be fully qualified and the permission is absolute (i.e. 
+   * umask has been applied).
    */
-  protected abstract void mkdir(final Path dir,
-      final FsPermission permission, final boolean createParent)
-    throws IOException, UnresolvedLinkException;
+  protected abstract void mkdir(final Path dir, final FsPermission permission,
+      final boolean createParent) throws AccessControlException,
+      FileAlreadyExistsException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#delete(Path, boolean)} except that Path f must be for
-   * this filesystem.
+   * this file system.
    */
   protected abstract boolean delete(final Path f, final boolean recursive)
-    throws IOException, UnresolvedLinkException;
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#open(Path)} except that Path f must be for this
-   * filesystem.
+   * file system.
    */
-  protected FSDataInputStream open(final Path f) 
-    throws IOException, UnresolvedLinkException {
+  protected FSDataInputStream open(final Path f) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
     return open(f, getServerDefaults().getFileBufferSize());
   }
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#open(Path, int)} except that Path f must be for this
-   * filesystem.
-   * @throws UnresolvedLinkException 
+   * file system.
    */
   protected abstract FSDataInputStream open(final Path f, int bufferSize)
-    throws IOException, UnresolvedLinkException;
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#setReplication(Path, short)} except that Path f must be
-   * for this filesystem.
+   * for this file system.
    */
   protected abstract boolean setReplication(final Path f,
-    final short replication) throws IOException, UnresolvedLinkException;
+      final short replication) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException;
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#rename(Path, Path, Options.Rename...)} except that Path
-   * f must be for this filesystem.
+   * f must be for this file system.
    */
   protected final void rename(final Path src, final Path dst,
-    final Options.Rename... options) 
-    throws IOException, UnresolvedLinkException {
+      final Options.Rename... options) throws AccessControlException,
+      FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, UnresolvedLinkException, IOException {
     boolean overwrite = false;
     if (null != options) {
       for (Rename option : options) {
@@ -527,22 +568,26 @@
   /**
    * The specification of this method matches that of
    * {@link FileContext#rename(Path, Path, Options.Rename...)} except that Path
-   * f must be for this filesystem and NO OVERWRITE is performed.
+   * f must be for this file system and NO OVERWRITE is performed.
    * 
-   * Filesystems that do not have a built in overwrite need implement only this
+   * File systems that do not have a built in overwrite need implement only this
    * method and can take advantage of the default impl of the other
    * {@link #renameInternal(Path, Path, boolean)}
    */
   protected abstract void renameInternal(final Path src, final Path dst)
-    throws IOException, UnresolvedLinkException;
+      throws AccessControlException, FileAlreadyExistsException,
+      FileNotFoundException, ParentNotDirectoryException,
+      UnresolvedLinkException, IOException;
   
   /**
    * The specification of this method matches that of
    * {@link FileContext#rename(Path, Path, Options.Rename...)} except that Path
-   * f must be for this filesystem.
+   * f must be for this file system.
    */
   protected void renameInternal(final Path src, final Path dst,
-    boolean overwrite) throws IOException, UnresolvedLinkException {
+      boolean overwrite) throws AccessControlException,
+      FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, UnresolvedLinkException, IOException {
     // Default implementation deals with overwrite in a non-atomic way
     final FileStatus srcStatus = getFileLinkStatus(src);
     if (srcStatus == null) {
@@ -619,35 +664,38 @@
   /**
    * The specification of this method matches that of
    * {@link FileContext#setPermission(Path, FsPermission)} except that Path f
-   * must be for this filesystem.
+   * must be for this file system.
    */
   protected abstract void setPermission(final Path f,
-      final FsPermission permission) 
-      throws IOException, UnresolvedLinkException;
+      final FsPermission permission) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException;
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#setOwner(Path, String, String)} except that Path f must
-   * be for this filesystem.
+   * be for this file system.
    */
   protected abstract void setOwner(final Path f, final String username,
-      final String groupname) throws IOException, UnresolvedLinkException;
+      final String groupname) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException;
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#setTimes(Path, long, long)} except that Path f must be
-   * for this filesystem.
+   * for this file system.
    */
   protected abstract void setTimes(final Path f, final long mtime,
-    final long atime) throws IOException, UnresolvedLinkException;
+    final long atime) throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#getFileChecksum(Path)} except that Path f must be for
-   * this filesystem.
+   * this file system.
    */
   protected abstract FileChecksum getFileChecksum(final Path f)
-    throws IOException, UnresolvedLinkException;
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
   
   /**
    * The specification of this method matches that of
@@ -655,8 +703,9 @@
    * except that an UnresolvedLinkException may be thrown if a symlink is 
    * encountered in the path.
    */
-  protected abstract FileStatus getFileStatus(final Path f) 
-    throws IOException, UnresolvedLinkException;
+  protected abstract FileStatus getFileStatus(final Path f)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
 
   /**
    * The specification of this method matches that of
@@ -667,26 +716,27 @@
    * equivalent to {@link AbstractFileSystem#getFileStatus(Path)}.
    */
   protected FileStatus getFileLinkStatus(final Path f)
-    throws IOException, UnresolvedLinkException {
+      throws AccessControlException, FileNotFoundException,
+      UnsupportedFileSystemException, IOException {
     return getFileStatus(f);
   }
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#getFileBlockLocations(Path, long, long)} except that
-   * Path f must be for this filesystem.
+   * Path f must be for this file system.
    */
   protected abstract BlockLocation[] getFileBlockLocations(final Path f,
-    final long start, final long len) 
-    throws IOException, UnresolvedLinkException;
+      final long start, final long len) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException;
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#getFsStatus(Path)} except that Path f must be for this
-   * filesystem.
+   * file system.
    */
-  protected FsStatus getFsStatus(final Path f) 
-    throws IOException, UnresolvedLinkException {
+  protected FsStatus getFsStatus(final Path f) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
     // default impl gets FsStatus of root
     return getFsStatus();
   }
@@ -695,21 +745,23 @@
    * The specification of this method matches that of
    * {@link FileContext#getFsStatus(Path)}.
    */
-  protected abstract FsStatus getFsStatus() throws IOException;
+  protected abstract FsStatus getFsStatus() throws AccessControlException,
+      FileNotFoundException, IOException;
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#listStatus(Path)} except that Path f must be for this
-   * filesystem.
+   * file system.
    */
-  protected abstract FileStatus[] listStatus(final Path f) 
-    throws IOException, UnresolvedLinkException;
+  protected abstract FileStatus[] listStatus(final Path f)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
 
   /**
    * The specification of this method matches that of
    * {@link FileContext#setVerifyChecksum(boolean, Path)} except that Path f
-   * must be for this filesystem.
+   * must be for this file system.
    */
   protected abstract void setVerifyChecksum(final boolean verifyChecksum)
-    throws IOException;
+      throws AccessControlException, IOException;
 }