001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.fs;
019
020import java.io.FileNotFoundException;
021import java.io.IOException;
022import java.io.InputStream;
023import java.io.OutputStream;
024import java.net.URI;
025import java.security.PrivilegedExceptionAction;
026import java.util.ArrayList;
027import java.util.Collection;
028import java.util.EnumSet;
029import java.util.HashSet;
030import java.util.IdentityHashMap;
031import java.util.List;
032import java.util.Map;
033import java.util.Set;
034import java.util.Stack;
035import java.util.TreeSet;
036import java.util.Map.Entry;
037
038import org.apache.commons.logging.Log;
039import org.apache.commons.logging.LogFactory;
040import org.apache.hadoop.HadoopIllegalArgumentException;
041import org.apache.hadoop.classification.InterfaceAudience;
042import org.apache.hadoop.classification.InterfaceStability;
043import org.apache.hadoop.conf.Configuration;
044import org.apache.hadoop.fs.FileSystem.Statistics;
045import org.apache.hadoop.fs.Options.CreateOpts;
046import org.apache.hadoop.fs.permission.AclEntry;
047import org.apache.hadoop.fs.permission.AclStatus;
048import org.apache.hadoop.fs.permission.FsAction;
049import org.apache.hadoop.fs.permission.FsPermission;
050import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
051import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_DEFAULT;
052
053import org.apache.hadoop.io.IOUtils;
054import org.apache.hadoop.ipc.RpcClientException;
055import org.apache.hadoop.ipc.RpcServerException;
056import org.apache.hadoop.ipc.UnexpectedServerException;
057import org.apache.hadoop.fs.InvalidPathException;
058import org.apache.hadoop.security.AccessControlException;
059import org.apache.hadoop.security.UserGroupInformation;
060import org.apache.hadoop.security.token.Token;
061import org.apache.hadoop.util.ShutdownHookManager;
062
063import com.google.common.base.Preconditions;
064
065/**
066 * The FileContext class provides an interface to the application writer for
067 * using the Hadoop file system.
068 * It provides a set of methods for the usual operation: create, open, 
069 * list, etc 
070 * 
071 * <p>
072 * <b> *** Path Names *** </b>
073 * <p>
074 * 
075 * The Hadoop file system supports a URI name space and URI names.
076 * It offers a forest of file systems that can be referenced using fully
077 * qualified URIs.
078 * Two common Hadoop file systems implementations are
079 * <ul>
080 * <li> the local file system: file:///path
081 * <li> the hdfs file system hdfs://nnAddress:nnPort/path
082 * </ul>
083 * 
084 * While URI names are very flexible, it requires knowing the name or address
085 * of the server. For convenience one often wants to access the default system
086 * in one's environment without knowing its name/address. This has an
087 * additional benefit that it allows one to change one's default fs
088 *  (e.g. admin moves application from cluster1 to cluster2).
089 * <p>
090 * 
091 * To facilitate this, Hadoop supports a notion of a default file system.
092 * The user can set his default file system, although this is
093 * typically set up for you in your environment via your default config.
094 * A default file system implies a default scheme and authority; slash-relative
095 * names (such as /for/bar) are resolved relative to that default FS.
096 * Similarly a user can also have working-directory-relative names (i.e. names
097 * not starting with a slash). While the working directory is generally in the
098 * same default FS, the wd can be in a different FS.
099 * <p>
100 *  Hence Hadoop path names can be one of:
101 *  <ul>
102 *  <li> fully qualified URI: scheme://authority/path
103 *  <li> slash relative names: /path relative to the default file system
104 *  <li> wd-relative names: path  relative to the working dir
105 *  </ul>   
106 *  Relative paths with scheme (scheme:foo/bar) are illegal.
107 *  
108 *  <p>
109 *  <b>****The Role of the FileContext and configuration defaults****</b>
110 *  <p>
111 *  The FileContext provides file namespace context for resolving file names;
112 *  it also contains the umask for permissions, In that sense it is like the
113 *  per-process file-related state in Unix system.
114 *  These two properties
115 *  <ul> 
116 *  <li> default file system i.e your slash)
117 *  <li> umask
118 *  </ul>
119 *  in general, are obtained from the default configuration file
120 *  in your environment,  (@see {@link Configuration}).
121 *  
122 *  No other configuration parameters are obtained from the default config as 
123 *  far as the file context layer is concerned. All file system instances
124 *  (i.e. deployments of file systems) have default properties; we call these
125 *  server side (SS) defaults. Operation like create allow one to select many 
126 *  properties: either pass them in as explicit parameters or use
127 *  the SS properties.
128 *  <p>
129 *  The file system related SS defaults are
130 *  <ul>
131 *  <li> the home directory (default is "/user/userName")
132 *  <li> the initial wd (only for local fs)
133 *  <li> replication factor
134 *  <li> block size
135 *  <li> buffer size
136 *  <li> encryptDataTransfer 
137 *  <li> checksum option. (checksumType and  bytesPerChecksum)
138 *  </ul>
139 *
140 * <p>
141 * <b> *** Usage Model for the FileContext class *** </b>
142 * <p>
143 * Example 1: use the default config read from the $HADOOP_CONFIG/core.xml.
144 *   Unspecified values come from core-defaults.xml in the release jar.
145 *  <ul>  
146 *  <li> myFContext = FileContext.getFileContext(); // uses the default config
147 *                                                // which has your default FS 
148 *  <li>  myFContext.create(path, ...);
149 *  <li>  myFContext.setWorkingDir(path)
150 *  <li>  myFContext.open (path, ...);  
151 *  </ul>  
152 * Example 2: Get a FileContext with a specific URI as the default FS
153 *  <ul>  
154 *  <li> myFContext = FileContext.getFileContext(URI)
155 *  <li> myFContext.create(path, ...);
156 *   ...
157 * </ul> 
158 * Example 3: FileContext with local file system as the default
159 *  <ul> 
160 *  <li> myFContext = FileContext.getLocalFSFileContext()
161 *  <li> myFContext.create(path, ...);
162 *  <li> ...
163 *  </ul> 
164 * Example 4: Use a specific config, ignoring $HADOOP_CONFIG
165 *  Generally you should not need use a config unless you are doing
166 *   <ul> 
167 *   <li> configX = someConfigSomeOnePassedToYou.
168 *   <li> myFContext = getFileContext(configX); // configX is not changed,
169 *                                              // is passed down 
170 *   <li> myFContext.create(path, ...);
171 *   <li>...
172 *  </ul>                                          
173 *    
174 */
175
176@InterfaceAudience.Public
177@InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
178public class FileContext {
179  
180  public static final Log LOG = LogFactory.getLog(FileContext.class);
181  /**
182   * Default permission for directory and symlink
183   * In previous versions, this default permission was also used to
184   * create files, so files created end up with ugo+x permission.
185   * See HADOOP-9155 for detail. 
186   * Two new constants are added to solve this, please use 
187   * {@link FileContext#DIR_DEFAULT_PERM} for directory, and use
188   * {@link FileContext#FILE_DEFAULT_PERM} for file.
189   * This constant is kept for compatibility.
190   */
191  public static final FsPermission DEFAULT_PERM = FsPermission.getDefault();
192  /**
193   * Default permission for directory
194   */
195  public static final FsPermission DIR_DEFAULT_PERM = FsPermission.getDirDefault();
196  /**
197   * Default permission for file
198   */
199  public static final FsPermission FILE_DEFAULT_PERM = FsPermission.getFileDefault();
200
201  /**
202   * Priority of the FileContext shutdown hook.
203   */
204  public static final int SHUTDOWN_HOOK_PRIORITY = 20;
205
206  /**
207   * List of files that should be deleted on JVM shutdown.
208   */
209  static final Map<FileContext, Set<Path>> DELETE_ON_EXIT = 
210    new IdentityHashMap<FileContext, Set<Path>>();
211
212  /** JVM shutdown hook thread. */
213  static final FileContextFinalizer FINALIZER = 
214    new FileContextFinalizer();
215  
216  private static final PathFilter DEFAULT_FILTER = new PathFilter() {
217    @Override
218    public boolean accept(final Path file) {
219      return true;
220    }
221  };
222  
223  /**
224   * The FileContext is defined by.
225   *  1) defaultFS (slash)
226   *  2) wd
227   *  3) umask
228   */   
229  private final AbstractFileSystem defaultFS; //default FS for this FileContext.
230  private Path workingDir;          // Fully qualified
231  private FsPermission umask;
232  private final Configuration conf;
233  private final UserGroupInformation ugi;
234  final boolean resolveSymlinks;
235
236  private FileContext(final AbstractFileSystem defFs,
237    final FsPermission theUmask, final Configuration aConf) {
238    defaultFS = defFs;
239    umask = FsPermission.getUMask(aConf);
240    conf = aConf;
241    try {
242      ugi = UserGroupInformation.getCurrentUser();
243    } catch (IOException e) {
244      LOG.error("Exception in getCurrentUser: ",e);
245      throw new RuntimeException("Failed to get the current user " +
246                "while creating a FileContext", e);
247    }
248    /*
249     * Init the wd.
250     * WorkingDir is implemented at the FileContext layer 
251     * NOT at the AbstractFileSystem layer. 
252     * If the DefaultFS, such as localFilesystem has a notion of
253     *  builtin WD, we use that as the initial WD.
254     *  Otherwise the WD is initialized to the home directory.
255     */
256    workingDir = defaultFS.getInitialWorkingDirectory();
257    if (workingDir == null) {
258      workingDir = defaultFS.getHomeDirectory();
259    }
260    resolveSymlinks = conf.getBoolean(
261        CommonConfigurationKeys.FS_CLIENT_RESOLVE_REMOTE_SYMLINKS_KEY,
262        CommonConfigurationKeys.FS_CLIENT_RESOLVE_REMOTE_SYMLINKS_DEFAULT);
263    util = new Util(); // for the inner class
264  }
265
266  /* 
267   * Remove relative part - return "absolute":
268   * If input is relative path ("foo/bar") add wd: ie "/<workingDir>/foo/bar"
269   * A fully qualified uri ("hdfs://nn:p/foo/bar") or a slash-relative path
270   * ("/foo/bar") are returned unchanged.
271   * 
272   * Applications that use FileContext should use #makeQualified() since
273   * they really want a fully qualified URI.
274   * Hence this method is not called makeAbsolute() and 
275   * has been deliberately declared private.
276   */
277  Path fixRelativePart(Path p) {
278    Preconditions.checkNotNull(p, "path cannot be null");
279    if (p.isUriPathAbsolute()) {
280      return p;
281    } else {
282      return new Path(workingDir, p);
283    }
284  }
285
286  /**
287   * Delete all the paths that were marked as delete-on-exit.
288   */
289  static void processDeleteOnExit() {
290    synchronized (DELETE_ON_EXIT) {
291      Set<Entry<FileContext, Set<Path>>> set = DELETE_ON_EXIT.entrySet();
292      for (Entry<FileContext, Set<Path>> entry : set) {
293        FileContext fc = entry.getKey();
294        Set<Path> paths = entry.getValue();
295        for (Path path : paths) {
296          try {
297            fc.delete(path, true);
298          } catch (IOException e) {
299            LOG.warn("Ignoring failure to deleteOnExit for path " + path);
300          }
301        }
302      }
303      DELETE_ON_EXIT.clear();
304    }
305  }
306
307  /**
308   * Get the file system of supplied path.
309   * 
310   * @param absOrFqPath - absolute or fully qualified path
311   * @return the file system of the path
312   * 
313   * @throws UnsupportedFileSystemException If the file system for
314   *           <code>absOrFqPath</code> is not supported.
315   * @throws IOExcepton If the file system for <code>absOrFqPath</code> could
316   *         not be instantiated.
317   */
318  protected AbstractFileSystem getFSofPath(final Path absOrFqPath)
319      throws UnsupportedFileSystemException, IOException {
320    absOrFqPath.checkNotSchemeWithRelative();
321    absOrFqPath.checkNotRelative();
322
323    try { 
324      // Is it the default FS for this FileContext?
325      defaultFS.checkPath(absOrFqPath);
326      return defaultFS;
327    } catch (Exception e) { // it is different FileSystem
328      return getAbstractFileSystem(ugi, absOrFqPath.toUri(), conf);
329    }
330  }
331  
332  private static AbstractFileSystem getAbstractFileSystem(
333      UserGroupInformation user, final URI uri, final Configuration conf)
334      throws UnsupportedFileSystemException, IOException {
335    try {
336      return user.doAs(new PrivilegedExceptionAction<AbstractFileSystem>() {
337        @Override
338        public AbstractFileSystem run() throws UnsupportedFileSystemException {
339          return AbstractFileSystem.get(uri, conf);
340        }
341      });
342    } catch (InterruptedException ex) {
343      LOG.error(ex);
344      throw new IOException("Failed to get the AbstractFileSystem for path: "
345          + uri, ex);
346    }
347  }
348  
349  /**
350   * Protected Static Factory methods for getting a FileContexts
351   * that take a AbstractFileSystem as input. To be used for testing.
352   */
353
354  /**
355   * Create a FileContext with specified FS as default using the specified
356   * config.
357   * 
358   * @param defFS
359   * @param aConf
360   * @return new FileContext with specifed FS as default.
361   */
362  public static FileContext getFileContext(final AbstractFileSystem defFS,
363                    final Configuration aConf) {
364    return new FileContext(defFS, FsPermission.getUMask(aConf), aConf);
365  }
366  
367  /**
368   * Create a FileContext for specified file system using the default config.
369   * 
370   * @param defaultFS
371   * @return a FileContext with the specified AbstractFileSystem
372   *                 as the default FS.
373   */
374  protected static FileContext getFileContext(
375    final AbstractFileSystem defaultFS) {
376    return getFileContext(defaultFS, new Configuration());
377  }
378 
379  /**
380   * Static Factory methods for getting a FileContext.
381   * Note new file contexts are created for each call.
382   * The only singleton is the local FS context using the default config.
383   * 
384   * Methods that use the default config: the default config read from the
385   * $HADOOP_CONFIG/core.xml,
386   * Unspecified key-values for config are defaulted from core-defaults.xml
387   * in the release jar.
388   * 
389   * The keys relevant to the FileContext layer are extracted at time of
390   * construction. Changes to the config after the call are ignore
391   * by the FileContext layer. 
392   * The conf is passed to lower layers like AbstractFileSystem and HDFS which
393   * pick up their own config variables.
394   */
395
396  /**
397   * Create a FileContext using the default config read from the
398   * $HADOOP_CONFIG/core.xml, Unspecified key-values for config are defaulted
399   * from core-defaults.xml in the release jar.
400   * 
401   * @throws UnsupportedFileSystemException If the file system from the default
402   *           configuration is not supported
403   */
404  public static FileContext getFileContext()
405      throws UnsupportedFileSystemException {
406    return getFileContext(new Configuration());
407  }
408
409  /**
410   * @return a FileContext for the local file system using the default config.
411   * @throws UnsupportedFileSystemException If the file system for
412   *           {@link FsConstants#LOCAL_FS_URI} is not supported.
413   */
414  public static FileContext getLocalFSFileContext()
415      throws UnsupportedFileSystemException {
416    return getFileContext(FsConstants.LOCAL_FS_URI);
417  }
418
419  /**
420   * Create a FileContext for specified URI using the default config.
421   * 
422   * @param defaultFsUri
423   * @return a FileContext with the specified URI as the default FS.
424   * 
425   * @throws UnsupportedFileSystemException If the file system for
426   *           <code>defaultFsUri</code> is not supported
427   */
428  public static FileContext getFileContext(final URI defaultFsUri)
429      throws UnsupportedFileSystemException {
430    return getFileContext(defaultFsUri, new Configuration());
431  }
432
433  /**
434   * Create a FileContext for specified default URI using the specified config.
435   * 
436   * @param defaultFsUri
437   * @param aConf
438   * @return new FileContext for specified uri
439   * @throws UnsupportedFileSystemException If the file system with specified is
440   *           not supported
441   * @throws RuntimeException If the file system specified is supported but
442   *         could not be instantiated, or if login fails.
443   */
444  public static FileContext getFileContext(final URI defaultFsUri,
445      final Configuration aConf) throws UnsupportedFileSystemException {
446    UserGroupInformation currentUser = null;
447    AbstractFileSystem defaultAfs = null;
448    if (defaultFsUri.getScheme() == null) {
449      return getFileContext(aConf);
450    }
451    try {
452      currentUser = UserGroupInformation.getCurrentUser();
453      defaultAfs = getAbstractFileSystem(currentUser, defaultFsUri, aConf);
454    } catch (UnsupportedFileSystemException ex) {
455      throw ex;
456    } catch (IOException ex) {
457      LOG.error(ex);
458      throw new RuntimeException(ex);
459    }
460    return getFileContext(defaultAfs, aConf);
461  }
462
463  /**
464   * Create a FileContext using the passed config. Generally it is better to use
465   * {@link #getFileContext(URI, Configuration)} instead of this one.
466   * 
467   * 
468   * @param aConf
469   * @return new FileContext
470   * @throws UnsupportedFileSystemException If file system in the config
471   *           is not supported
472   */
473  public static FileContext getFileContext(final Configuration aConf)
474      throws UnsupportedFileSystemException {
475    final URI defaultFsUri = URI.create(aConf.get(FS_DEFAULT_NAME_KEY,
476        FS_DEFAULT_NAME_DEFAULT));
477    if (   defaultFsUri.getScheme() != null
478        && !defaultFsUri.getScheme().trim().isEmpty()) {
479      return getFileContext(defaultFsUri, aConf);
480    }
481    throw new UnsupportedFileSystemException(String.format(
482        "%s: URI configured via %s carries no scheme",
483        defaultFsUri, FS_DEFAULT_NAME_KEY));
484  }
485
486  /**
487   * @param aConf - from which the FileContext is configured
488   * @return a FileContext for the local file system using the specified config.
489   * 
490   * @throws UnsupportedFileSystemException If default file system in the config
491   *           is not supported
492   * 
493   */
494  public static FileContext getLocalFSFileContext(final Configuration aConf)
495      throws UnsupportedFileSystemException {
496    return getFileContext(FsConstants.LOCAL_FS_URI, aConf);
497  }
498
499  /* This method is needed for tests. */
500  @InterfaceAudience.Private
501  @InterfaceStability.Unstable /* return type will change to AFS once
502                                  HADOOP-6223 is completed */
503  public AbstractFileSystem getDefaultFileSystem() {
504    return defaultFS;
505  }
506  
507  /**
508   * Set the working directory for wd-relative names (such a "foo/bar"). Working
509   * directory feature is provided by simply prefixing relative names with the
510   * working dir. Note this is different from Unix where the wd is actually set
511   * to the inode. Hence setWorkingDir does not follow symlinks etc. This works
512   * better in a distributed environment that has multiple independent roots.
513   * {@link #getWorkingDirectory()} should return what setWorkingDir() set.
514   * 
515   * @param newWDir new working directory
516   * @throws IOException 
517   * <br>
518   *           NewWdir can be one of:
519   *           <ul>
520   *           <li>relative path: "foo/bar";</li>
521   *           <li>absolute without scheme: "/foo/bar"</li>
522   *           <li>fully qualified with scheme: "xx://auth/foo/bar"</li>
523   *           </ul>
524   * <br>
525   *           Illegal WDs:
526   *           <ul>
527   *           <li>relative with scheme: "xx:foo/bar"</li>
528   *           <li>non existent directory</li>
529   *           </ul>
530   */
531  public void setWorkingDirectory(final Path newWDir) throws IOException {
532    newWDir.checkNotSchemeWithRelative();
533    /* wd is stored as a fully qualified path. We check if the given 
534     * path is not relative first since resolve requires and returns 
535     * an absolute path.
536     */  
537    final Path newWorkingDir = new Path(workingDir, newWDir);
538    FileStatus status = getFileStatus(newWorkingDir);
539    if (status.isFile()) {
540      throw new FileNotFoundException("Cannot setWD to a file");
541    }
542    workingDir = newWorkingDir;
543  }
544  
545  /**
546   * Gets the working directory for wd-relative names (such a "foo/bar").
547   */
548  public Path getWorkingDirectory() {
549    return workingDir;
550  }
551  
552  /**
553   * Gets the ugi in the file-context
554   * @return UserGroupInformation
555   */
556  public UserGroupInformation getUgi() {
557    return ugi;
558  }
559  
560  /**
561   * Return the current user's home directory in this file system.
562   * The default implementation returns "/user/$USER/".
563   * @return the home directory
564   */
565  public Path getHomeDirectory() {
566    return defaultFS.getHomeDirectory();
567  }
568  
569  /**
570   * 
571   * @return the umask of this FileContext
572   */
573  public FsPermission getUMask() {
574    return umask;
575  }
576  
577  /**
578   * Set umask to the supplied parameter.
579   * @param newUmask  the new umask
580   */
581  public void setUMask(final FsPermission newUmask) {
582    umask = newUmask;
583  }
584  
585  
586  /**
587   * Resolve the path following any symlinks or mount points
588   * @param f to be resolved
589   * @return fully qualified resolved path
590   * 
591   * @throws FileNotFoundException  If <code>f</code> does not exist
592   * @throws AccessControlException if access denied
593   * @throws IOException If an IO Error occurred
594   * 
595   * Exceptions applicable to file systems accessed over RPC:
596   * @throws RpcClientException If an exception occurred in the RPC client
597   * @throws RpcServerException If an exception occurred in the RPC server
598   * @throws UnexpectedServerException If server implementation throws
599   *           undeclared exception to RPC server
600   * 
601   * RuntimeExceptions:
602   * @throws InvalidPathException If path <code>f</code> is not valid
603   */
604  public Path resolvePath(final Path f) throws FileNotFoundException,
605      UnresolvedLinkException, AccessControlException, IOException {
606    return resolve(f);
607  }
608  
609  /**
610   * Make the path fully qualified if it is isn't. 
611   * A Fully-qualified path has scheme and authority specified and an absolute
612   * path.
613   * Use the default file system and working dir in this FileContext to qualify.
614   * @param path
615   * @return qualified path
616   */
617  public Path makeQualified(final Path path) {
618    return path.makeQualified(defaultFS.getUri(), getWorkingDirectory());
619  }
620
621  /**
622   * Create or overwrite file on indicated path and returns an output stream for
623   * writing into the file.
624   * 
625   * @param f the file name to open
626   * @param createFlag gives the semantics of create; see {@link CreateFlag}
627   * @param opts file creation options; see {@link Options.CreateOpts}.
628   *          <ul>
629   *          <li>Progress - to report progress on the operation - default null
630   *          <li>Permission - umask is applied against permisssion: default is
631   *          FsPermissions:getDefault()
632   * 
633   *          <li>CreateParent - create missing parent path; default is to not
634   *          to create parents
635   *          <li>The defaults for the following are SS defaults of the file
636   *          server implementing the target path. Not all parameters make sense
637   *          for all kinds of file system - eg. localFS ignores Blocksize,
638   *          replication, checksum
639   *          <ul>
640   *          <li>BufferSize - buffersize used in FSDataOutputStream
641   *          <li>Blocksize - block size for file blocks
642   *          <li>ReplicationFactor - replication for blocks
643   *          <li>ChecksumParam - Checksum parameters. server default is used
644   *          if not specified.
645   *          </ul>
646   *          </ul>
647   * 
648   * @return {@link FSDataOutputStream} for created file
649   * 
650   * @throws AccessControlException If access is denied
651   * @throws FileAlreadyExistsException If file <code>f</code> already exists
652   * @throws FileNotFoundException If parent of <code>f</code> does not exist
653   *           and <code>createParent</code> is false
654   * @throws ParentNotDirectoryException If parent of <code>f</code> is not a
655   *           directory.
656   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
657   *           not supported
658   * @throws IOException If an I/O error occurred
659   * 
660   * Exceptions applicable to file systems accessed over RPC:
661   * @throws RpcClientException If an exception occurred in the RPC client
662   * @throws RpcServerException If an exception occurred in the RPC server
663   * @throws UnexpectedServerException If server implementation throws
664   *           undeclared exception to RPC server
665   * 
666   * RuntimeExceptions:
667   * @throws InvalidPathException If path <code>f</code> is not valid
668   */
669  public FSDataOutputStream create(final Path f,
670      final EnumSet<CreateFlag> createFlag, Options.CreateOpts... opts)
671      throws AccessControlException, FileAlreadyExistsException,
672      FileNotFoundException, ParentNotDirectoryException,
673      UnsupportedFileSystemException, IOException {
674    Path absF = fixRelativePart(f);
675
676    // If one of the options is a permission, extract it & apply umask
677    // If not, add a default Perms and apply umask;
678    // AbstractFileSystem#create
679
680    CreateOpts.Perms permOpt = CreateOpts.getOpt(CreateOpts.Perms.class, opts);
681    FsPermission permission = (permOpt != null) ? permOpt.getValue() :
682                                      FILE_DEFAULT_PERM;
683    permission = permission.applyUMask(umask);
684
685    final CreateOpts[] updatedOpts = 
686                      CreateOpts.setOpt(CreateOpts.perms(permission), opts);
687    return new FSLinkResolver<FSDataOutputStream>() {
688      @Override
689      public FSDataOutputStream next(final AbstractFileSystem fs, final Path p) 
690        throws IOException {
691        return fs.create(p, createFlag, updatedOpts);
692      }
693    }.resolve(this, absF);
694  }
695
696  /**
697   * Make(create) a directory and all the non-existent parents.
698   * 
699   * @param dir - the dir to make
700   * @param permission - permissions is set permission&~umask
701   * @param createParent - if true then missing parent dirs are created if false
702   *          then parent must exist
703   * 
704   * @throws AccessControlException If access is denied
705   * @throws FileAlreadyExistsException If directory <code>dir</code> already
706   *           exists
707   * @throws FileNotFoundException If parent of <code>dir</code> does not exist
708   *           and <code>createParent</code> is false
709   * @throws ParentNotDirectoryException If parent of <code>dir</code> is not a
710   *           directory
711   * @throws UnsupportedFileSystemException If file system for <code>dir</code>
712   *         is not supported
713   * @throws IOException If an I/O error occurred
714   * 
715   * Exceptions applicable to file systems accessed over RPC:
716   * @throws RpcClientException If an exception occurred in the RPC client
717   * @throws UnexpectedServerException If server implementation throws 
718   *           undeclared exception to RPC server
719   * 
720   * RuntimeExceptions:
721   * @throws InvalidPathException If path <code>dir</code> is not valid
722   */
723  public void mkdir(final Path dir, final FsPermission permission,
724      final boolean createParent) throws AccessControlException,
725      FileAlreadyExistsException, FileNotFoundException,
726      ParentNotDirectoryException, UnsupportedFileSystemException, 
727      IOException {
728    final Path absDir = fixRelativePart(dir);
729    final FsPermission absFerms = (permission == null ? 
730          FsPermission.getDirDefault() : permission).applyUMask(umask);
731    new FSLinkResolver<Void>() {
732      @Override
733      public Void next(final AbstractFileSystem fs, final Path p) 
734        throws IOException, UnresolvedLinkException {
735        fs.mkdir(p, absFerms, createParent);
736        return null;
737      }
738    }.resolve(this, absDir);
739  }
740
741  /**
742   * Delete a file.
743   * @param f the path to delete.
744   * @param recursive if path is a directory and set to 
745   * true, the directory is deleted else throws an exception. In
746   * case of a file the recursive can be set to either true or false.
747   *
748   * @throws AccessControlException If access is denied
749   * @throws FileNotFoundException If <code>f</code> does not exist
750   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
751   *           not supported
752   * @throws IOException If an I/O error occurred
753   * 
754   * Exceptions applicable to file systems accessed over RPC:
755   * @throws RpcClientException If an exception occurred in the RPC client
756   * @throws RpcServerException If an exception occurred in the RPC server
757   * @throws UnexpectedServerException If server implementation throws 
758   *           undeclared exception to RPC server
759   * 
760   * RuntimeExceptions:
761   * @throws InvalidPathException If path <code>f</code> is invalid
762   */
763  public boolean delete(final Path f, final boolean recursive)
764      throws AccessControlException, FileNotFoundException,
765      UnsupportedFileSystemException, IOException {
766    Path absF = fixRelativePart(f);
767    return new FSLinkResolver<Boolean>() {
768      @Override
769      public Boolean next(final AbstractFileSystem fs, final Path p) 
770        throws IOException, UnresolvedLinkException {
771        return Boolean.valueOf(fs.delete(p, recursive));
772      }
773    }.resolve(this, absF);
774  }
775 
776  /**
777   * Opens an FSDataInputStream at the indicated Path using
778   * default buffersize.
779   * @param f the file name to open
780   *
781   * @throws AccessControlException If access is denied
782   * @throws FileNotFoundException If file <code>f</code> does not exist
783   * @throws UnsupportedFileSystemException If file system for <code>f</code>
784   *         is not supported
785   * @throws IOException If an I/O error occurred
786   * 
787   * Exceptions applicable to file systems accessed over RPC:
788   * @throws RpcClientException If an exception occurred in the RPC client
789   * @throws RpcServerException If an exception occurred in the RPC server
790   * @throws UnexpectedServerException If server implementation throws 
791   *           undeclared exception to RPC server
792   */
793  public FSDataInputStream open(final Path f) throws AccessControlException,
794      FileNotFoundException, UnsupportedFileSystemException, IOException {
795    final Path absF = fixRelativePart(f);
796    return new FSLinkResolver<FSDataInputStream>() {
797      @Override
798      public FSDataInputStream next(final AbstractFileSystem fs, final Path p) 
799        throws IOException, UnresolvedLinkException {
800        return fs.open(p);
801      }
802    }.resolve(this, absF);
803  }
804
805  /**
806   * Opens an FSDataInputStream at the indicated Path.
807   * 
808   * @param f the file name to open
809   * @param bufferSize the size of the buffer to be used.
810   * 
811   * @throws AccessControlException If access is denied
812   * @throws FileNotFoundException If file <code>f</code> does not exist
813   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
814   *           not supported
815   * @throws IOException If an I/O error occurred
816   * 
817   * Exceptions applicable to file systems accessed over RPC:
818   * @throws RpcClientException If an exception occurred in the RPC client
819   * @throws RpcServerException If an exception occurred in the RPC server
820   * @throws UnexpectedServerException If server implementation throws 
821   *           undeclared exception to RPC server
822   */
823  public FSDataInputStream open(final Path f, final int bufferSize)
824      throws AccessControlException, FileNotFoundException,
825      UnsupportedFileSystemException, IOException {
826    final Path absF = fixRelativePart(f);
827    return new FSLinkResolver<FSDataInputStream>() {
828      @Override
829      public FSDataInputStream next(final AbstractFileSystem fs, final Path p) 
830        throws IOException, UnresolvedLinkException {
831        return fs.open(p, bufferSize);
832      }
833    }.resolve(this, absF);
834  }
835
836  /**
837   * Truncate the file in the indicated path to the indicated size.
838   * <ul>
839   * <li>Fails if path is a directory.
840   * <li>Fails if path does not exist.
841   * <li>Fails if path is not closed.
842   * <li>Fails if new size is greater than current size.
843   * </ul>
844   * @param f The path to the file to be truncated
845   * @param newLength The size the file is to be truncated to
846   *
847   * @return <code>true</code> if the file has been truncated to the desired
848   * <code>newLength</code> and is immediately available to be reused for
849   * write operations such as <code>append</code>, or
850   * <code>false</code> if a background process of adjusting the length of
851   * the last block has been started, and clients should wait for it to
852   * complete before proceeding with further file updates.
853   *
854   * @throws AccessControlException If access is denied
855   * @throws FileNotFoundException If file <code>f</code> does not exist
856   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
857   *           not supported
858   * @throws IOException If an I/O error occurred
859   *
860   * Exceptions applicable to file systems accessed over RPC:
861   * @throws RpcClientException If an exception occurred in the RPC client
862   * @throws RpcServerException If an exception occurred in the RPC server
863   * @throws UnexpectedServerException If server implementation throws
864   *           undeclared exception to RPC server
865   */
866  public boolean truncate(final Path f, final long newLength)
867      throws AccessControlException, FileNotFoundException,
868      UnsupportedFileSystemException, IOException {
869    final Path absF = fixRelativePart(f);
870    return new FSLinkResolver<Boolean>() {
871      @Override
872      public Boolean next(final AbstractFileSystem fs, final Path p)
873          throws IOException, UnresolvedLinkException {
874        return fs.truncate(p, newLength);
875      }
876    }.resolve(this, absF);
877  }
878
879  /**
880   * Set replication for an existing file.
881   * 
882   * @param f file name
883   * @param replication new replication
884   *
885   * @return true if successful
886   *
887   * @throws AccessControlException If access is denied
888   * @throws FileNotFoundException If file <code>f</code> does not exist
889   * @throws IOException If an I/O error occurred
890   * 
891   * Exceptions applicable to file systems accessed over RPC:
892   * @throws RpcClientException If an exception occurred in the RPC client
893   * @throws RpcServerException If an exception occurred in the RPC server
894   * @throws UnexpectedServerException If server implementation throws 
895   *           undeclared exception to RPC server
896   */
897  public boolean setReplication(final Path f, final short replication)
898      throws AccessControlException, FileNotFoundException,
899      IOException {
900    final Path absF = fixRelativePart(f);
901    return new FSLinkResolver<Boolean>() {
902      @Override
903      public Boolean next(final AbstractFileSystem fs, final Path p) 
904        throws IOException, UnresolvedLinkException {
905        return Boolean.valueOf(fs.setReplication(p, replication));
906      }
907    }.resolve(this, absF);
908  }
909
910  /**
911   * Renames Path src to Path dst
912   * <ul>
913   * <li
914   * <li>Fails if src is a file and dst is a directory.
915   * <li>Fails if src is a directory and dst is a file.
916   * <li>Fails if the parent of dst does not exist or is a file.
917   * </ul>
918   * <p>
919   * If OVERWRITE option is not passed as an argument, rename fails if the dst
920   * already exists.
921   * <p>
922   * If OVERWRITE option is passed as an argument, rename overwrites the dst if
923   * it is a file or an empty directory. Rename fails if dst is a non-empty
924   * directory.
925   * <p>
926   * Note that atomicity of rename is dependent on the file system
927   * implementation. Please refer to the file system documentation for details
928   * <p>
929   * 
930   * @param src path to be renamed
931   * @param dst new path after rename
932   * 
933   * @throws AccessControlException If access is denied
934   * @throws FileAlreadyExistsException If <code>dst</code> already exists and
935   *           <code>options</options> has {@link Options.Rename#OVERWRITE} 
936   *           option false.
937   * @throws FileNotFoundException If <code>src</code> does not exist
938   * @throws ParentNotDirectoryException If parent of <code>dst</code> is not a
939   *           directory
940   * @throws UnsupportedFileSystemException If file system for <code>src</code>
941   *           and <code>dst</code> is not supported
942   * @throws IOException If an I/O error occurred
943   * 
944   * Exceptions applicable to file systems accessed over RPC:
945   * @throws RpcClientException If an exception occurred in the RPC client
946   * @throws RpcServerException If an exception occurred in the RPC server
947   * @throws UnexpectedServerException If server implementation throws
948   *           undeclared exception to RPC server
949   */
950  public void rename(final Path src, final Path dst,
951      final Options.Rename... options) throws AccessControlException,
952      FileAlreadyExistsException, FileNotFoundException,
953      ParentNotDirectoryException, UnsupportedFileSystemException,
954      IOException {
955    final Path absSrc = fixRelativePart(src);
956    final Path absDst = fixRelativePart(dst);
957    AbstractFileSystem srcFS = getFSofPath(absSrc);
958    AbstractFileSystem dstFS = getFSofPath(absDst);
959    if(!srcFS.getUri().equals(dstFS.getUri())) {
960      throw new IOException("Renames across AbstractFileSystems not supported");
961    }
962    try {
963      srcFS.rename(absSrc, absDst, options);
964    } catch (UnresolvedLinkException e) {
965      /* We do not know whether the source or the destination path
966       * was unresolved. Resolve the source path up until the final
967       * path component, then fully resolve the destination. 
968       */
969      final Path source = resolveIntermediate(absSrc);    
970      new FSLinkResolver<Void>() {
971        @Override
972        public Void next(final AbstractFileSystem fs, final Path p) 
973          throws IOException, UnresolvedLinkException {
974          fs.rename(source, p, options);
975          return null;
976        }
977      }.resolve(this, absDst);
978    }
979  }
980  
981  /**
982   * Set permission of a path.
983   * @param f
984   * @param permission - the new absolute permission (umask is not applied)
985   *
986   * @throws AccessControlException If access is denied
987   * @throws FileNotFoundException If <code>f</code> does not exist
988   * @throws UnsupportedFileSystemException If file system for <code>f</code>
989   *         is not supported
990   * @throws IOException If an I/O error occurred
991   * 
992   * Exceptions applicable to file systems accessed over RPC:
993   * @throws RpcClientException If an exception occurred in the RPC client
994   * @throws RpcServerException If an exception occurred in the RPC server
995   * @throws UnexpectedServerException If server implementation throws 
996   *           undeclared exception to RPC server
997   */
998  public void setPermission(final Path f, final FsPermission permission)
999      throws AccessControlException, FileNotFoundException,
1000      UnsupportedFileSystemException, IOException {
1001    final Path absF = fixRelativePart(f);
1002    new FSLinkResolver<Void>() {
1003      @Override
1004      public Void next(final AbstractFileSystem fs, final Path p) 
1005        throws IOException, UnresolvedLinkException {
1006        fs.setPermission(p, permission);
1007        return null;
1008      }
1009    }.resolve(this, absF);
1010  }
1011
1012  /**
1013   * Set owner of a path (i.e. a file or a directory). The parameters username
1014   * and groupname cannot both be null.
1015   * 
1016   * @param f The path
1017   * @param username If it is null, the original username remains unchanged.
1018   * @param groupname If it is null, the original groupname remains unchanged.
1019   * 
1020   * @throws AccessControlException If access is denied
1021   * @throws FileNotFoundException If <code>f</code> does not exist
1022   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1023   *           not supported
1024   * @throws IOException If an I/O error occurred
1025   * 
1026   * Exceptions applicable to file systems accessed over RPC:
1027   * @throws RpcClientException If an exception occurred in the RPC client
1028   * @throws RpcServerException If an exception occurred in the RPC server
1029   * @throws UnexpectedServerException If server implementation throws 
1030   *           undeclared exception to RPC server
1031   * 
1032   * RuntimeExceptions:
1033   * @throws HadoopIllegalArgumentException If <code>username</code> or
1034   *           <code>groupname</code> is invalid.
1035   */
1036  public void setOwner(final Path f, final String username,
1037      final String groupname) throws AccessControlException,
1038      UnsupportedFileSystemException, FileNotFoundException,
1039      IOException {
1040    if ((username == null) && (groupname == null)) {
1041      throw new HadoopIllegalArgumentException(
1042          "username and groupname cannot both be null");
1043    }
1044    final Path absF = fixRelativePart(f);
1045    new FSLinkResolver<Void>() {
1046      @Override
1047      public Void next(final AbstractFileSystem fs, final Path p) 
1048        throws IOException, UnresolvedLinkException {
1049        fs.setOwner(p, username, groupname);
1050        return null;
1051      }
1052    }.resolve(this, absF);
1053  }
1054
1055  /**
1056   * Set access time of a file.
1057   * @param f The path
1058   * @param mtime Set the modification time of this file.
1059   *        The number of milliseconds since epoch (Jan 1, 1970). 
1060   *        A value of -1 means that this call should not set modification time.
1061   * @param atime Set the access time of this file.
1062   *        The number of milliseconds since Jan 1, 1970. 
1063   *        A value of -1 means that this call should not set access time.
1064   *
1065   * @throws AccessControlException If access is denied
1066   * @throws FileNotFoundException If <code>f</code> does not exist
1067   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1068   *           not supported
1069   * @throws IOException If an I/O error occurred
1070   * 
1071   * Exceptions applicable to file systems accessed over RPC:
1072   * @throws RpcClientException If an exception occurred in the RPC client
1073   * @throws RpcServerException If an exception occurred in the RPC server
1074   * @throws UnexpectedServerException If server implementation throws 
1075   *           undeclared exception to RPC server
1076   */
1077  public void setTimes(final Path f, final long mtime, final long atime)
1078      throws AccessControlException, FileNotFoundException,
1079      UnsupportedFileSystemException, IOException {
1080    final Path absF = fixRelativePart(f);
1081    new FSLinkResolver<Void>() {
1082      @Override
1083      public Void next(final AbstractFileSystem fs, final Path p) 
1084        throws IOException, UnresolvedLinkException {
1085        fs.setTimes(p, mtime, atime);
1086        return null;
1087      }
1088    }.resolve(this, absF);
1089  }
1090
1091  /**
1092   * Get the checksum of a file.
1093   *
1094   * @param f file path
1095   *
1096   * @return The file checksum.  The default return value is null,
1097   *  which indicates that no checksum algorithm is implemented
1098   *  in the corresponding FileSystem.
1099   *
1100   * @throws AccessControlException If access is denied
1101   * @throws FileNotFoundException If <code>f</code> does not exist
1102   * @throws IOException If an I/O error occurred
1103   * 
1104   * Exceptions applicable to file systems accessed over RPC:
1105   * @throws RpcClientException If an exception occurred in the RPC client
1106   * @throws RpcServerException If an exception occurred in the RPC server
1107   * @throws UnexpectedServerException If server implementation throws 
1108   *           undeclared exception to RPC server
1109   */
1110  public FileChecksum getFileChecksum(final Path f)
1111      throws AccessControlException, FileNotFoundException,
1112      IOException {
1113    final Path absF = fixRelativePart(f);
1114    return new FSLinkResolver<FileChecksum>() {
1115      @Override
1116      public FileChecksum next(final AbstractFileSystem fs, final Path p) 
1117        throws IOException, UnresolvedLinkException {
1118        return fs.getFileChecksum(p);
1119      }
1120    }.resolve(this, absF);
1121  }
1122
1123  /**
1124   * Set the verify checksum flag for the  file system denoted by the path.
1125   * This is only applicable if the 
1126   * corresponding FileSystem supports checksum. By default doesn't do anything.
1127   * @param verifyChecksum
1128   * @param f set the verifyChecksum for the Filesystem containing this path
1129   *
1130   * @throws AccessControlException If access is denied
1131   * @throws FileNotFoundException If <code>f</code> does not exist
1132   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1133   *           not supported
1134   * @throws IOException If an I/O error occurred
1135   * 
1136   * Exceptions applicable to file systems accessed over RPC:
1137   * @throws RpcClientException If an exception occurred in the RPC client
1138   * @throws RpcServerException If an exception occurred in the RPC server
1139   * @throws UnexpectedServerException If server implementation throws 
1140   *           undeclared exception to RPC server
1141   */
1142  public void setVerifyChecksum(final boolean verifyChecksum, final Path f)
1143      throws AccessControlException, FileNotFoundException,
1144      UnsupportedFileSystemException, IOException {
1145    final Path absF = resolve(fixRelativePart(f));
1146    getFSofPath(absF).setVerifyChecksum(verifyChecksum);
1147  }
1148
1149  /**
1150   * Return a file status object that represents the path.
1151   * @param f The path we want information from
1152   *
1153   * @return a FileStatus object
1154   *
1155   * @throws AccessControlException If access is denied
1156   * @throws FileNotFoundException If <code>f</code> does not exist
1157   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1158   *           not supported
1159   * @throws IOException If an I/O error occurred
1160   * 
1161   * Exceptions applicable to file systems accessed over RPC:
1162   * @throws RpcClientException If an exception occurred in the RPC client
1163   * @throws RpcServerException If an exception occurred in the RPC server
1164   * @throws UnexpectedServerException If server implementation throws 
1165   *           undeclared exception to RPC server
1166   */
1167  public FileStatus getFileStatus(final Path f) throws AccessControlException,
1168      FileNotFoundException, UnsupportedFileSystemException, IOException {
1169    final Path absF = fixRelativePart(f);
1170    return new FSLinkResolver<FileStatus>() {
1171      @Override
1172      public FileStatus next(final AbstractFileSystem fs, final Path p) 
1173        throws IOException, UnresolvedLinkException {
1174        return fs.getFileStatus(p);
1175      }
1176    }.resolve(this, absF);
1177  }
1178
1179  /**
1180   * Checks if the user can access a path.  The mode specifies which access
1181   * checks to perform.  If the requested permissions are granted, then the
1182   * method returns normally.  If access is denied, then the method throws an
1183   * {@link AccessControlException}.
1184   * <p/>
1185   * The default implementation of this method calls {@link #getFileStatus(Path)}
1186   * and checks the returned permissions against the requested permissions.
1187   * Note that the getFileStatus call will be subject to authorization checks.
1188   * Typically, this requires search (execute) permissions on each directory in
1189   * the path's prefix, but this is implementation-defined.  Any file system
1190   * that provides a richer authorization model (such as ACLs) may override the
1191   * default implementation so that it checks against that model instead.
1192   * <p>
1193   * In general, applications should avoid using this method, due to the risk of
1194   * time-of-check/time-of-use race conditions.  The permissions on a file may
1195   * change immediately after the access call returns.  Most applications should
1196   * prefer running specific file system actions as the desired user represented
1197   * by a {@link UserGroupInformation}.
1198   *
1199   * @param path Path to check
1200   * @param mode type of access to check
1201   * @throws AccessControlException if access is denied
1202   * @throws FileNotFoundException if the path does not exist
1203   * @throws UnsupportedFileSystemException if file system for <code>path</code>
1204   *   is not supported
1205   * @throws IOException see specific implementation
1206   * 
1207   * Exceptions applicable to file systems accessed over RPC:
1208   * @throws RpcClientException If an exception occurred in the RPC client
1209   * @throws RpcServerException If an exception occurred in the RPC server
1210   * @throws UnexpectedServerException If server implementation throws 
1211   *           undeclared exception to RPC server
1212   */
1213  @InterfaceAudience.LimitedPrivate({"HDFS", "Hive"})
1214  public void access(final Path path, final FsAction mode)
1215      throws AccessControlException, FileNotFoundException,
1216      UnsupportedFileSystemException, IOException {
1217    final Path absPath = fixRelativePart(path);
1218    new FSLinkResolver<Void>() {
1219      @Override
1220      public Void next(AbstractFileSystem fs, Path p) throws IOException,
1221          UnresolvedLinkException {
1222        fs.access(p, mode);
1223        return null;
1224      }
1225    }.resolve(this, absPath);
1226  }
1227
1228  /**
1229   * Return a file status object that represents the path. If the path 
1230   * refers to a symlink then the FileStatus of the symlink is returned.
1231   * The behavior is equivalent to #getFileStatus() if the underlying
1232   * file system does not support symbolic links.
1233   * @param  f The path we want information from.
1234   * @return A FileStatus object
1235   * 
1236   * @throws AccessControlException If access is denied
1237   * @throws FileNotFoundException If <code>f</code> does not exist
1238   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1239   *           not supported
1240   * @throws IOException If an I/O error occurred
1241   */
1242  public FileStatus getFileLinkStatus(final Path f)
1243      throws AccessControlException, FileNotFoundException,
1244      UnsupportedFileSystemException, IOException {
1245    final Path absF = fixRelativePart(f);
1246    return new FSLinkResolver<FileStatus>() {
1247      @Override
1248      public FileStatus next(final AbstractFileSystem fs, final Path p) 
1249        throws IOException, UnresolvedLinkException {
1250        FileStatus fi = fs.getFileLinkStatus(p);
1251        if (fi.isSymlink()) {
1252          fi.setSymlink(FSLinkResolver.qualifySymlinkTarget(fs.getUri(), p,
1253              fi.getSymlink()));
1254        }
1255        return fi;
1256      }
1257    }.resolve(this, absF);
1258  }
1259  
1260  /**
1261   * Returns the target of the given symbolic link as it was specified
1262   * when the link was created.  Links in the path leading up to the
1263   * final path component are resolved transparently.
1264   *
1265   * @param f the path to return the target of
1266   * @return The un-interpreted target of the symbolic link.
1267   * 
1268   * @throws AccessControlException If access is denied
1269   * @throws FileNotFoundException If path <code>f</code> does not exist
1270   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1271   *           not supported
1272   * @throws IOException If the given path does not refer to a symlink
1273   *           or an I/O error occurred
1274   */
1275  public Path getLinkTarget(final Path f) throws AccessControlException,
1276      FileNotFoundException, UnsupportedFileSystemException, IOException {
1277    final Path absF = fixRelativePart(f);
1278    return new FSLinkResolver<Path>() {
1279      @Override
1280      public Path next(final AbstractFileSystem fs, final Path p) 
1281        throws IOException, UnresolvedLinkException {
1282        FileStatus fi = fs.getFileLinkStatus(p);
1283        return fi.getSymlink();
1284      }
1285    }.resolve(this, absF);
1286  }
1287  
1288  /**
1289   * Return blockLocation of the given file for the given offset and len.
1290   *  For a nonexistent file or regions, null will be returned.
1291   *
1292   * This call is most helpful with DFS, where it returns 
1293   * hostnames of machines that contain the given file.
1294   * 
1295   * @param f - get blocklocations of this file
1296   * @param start position (byte offset)
1297   * @param len (in bytes)
1298   *
1299   * @return block locations for given file at specified offset of len
1300   *
1301   * @throws AccessControlException If access is denied
1302   * @throws FileNotFoundException If <code>f</code> does not exist
1303   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1304   *           not supported
1305   * @throws IOException If an I/O error occurred
1306   * 
1307   * Exceptions applicable to file systems accessed over RPC:
1308   * @throws RpcClientException If an exception occurred in the RPC client
1309   * @throws RpcServerException If an exception occurred in the RPC server
1310   * @throws UnexpectedServerException If server implementation throws 
1311   *           undeclared exception to RPC server
1312   * 
1313   * RuntimeExceptions:
1314   * @throws InvalidPathException If path <code>f</code> is invalid
1315   */
1316  @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
1317  @InterfaceStability.Evolving
1318  public BlockLocation[] getFileBlockLocations(final Path f, final long start,
1319      final long len) throws AccessControlException, FileNotFoundException,
1320      UnsupportedFileSystemException, IOException {
1321    final Path absF = fixRelativePart(f);
1322    return new FSLinkResolver<BlockLocation[]>() {
1323      @Override
1324      public BlockLocation[] next(final AbstractFileSystem fs, final Path p) 
1325        throws IOException, UnresolvedLinkException {
1326        return fs.getFileBlockLocations(p, start, len);
1327      }
1328    }.resolve(this, absF);
1329  }
1330  
1331  /**
1332   * Returns a status object describing the use and capacity of the
1333   * file system denoted by the Parh argument p.
1334   * If the file system has multiple partitions, the
1335   * use and capacity of the partition pointed to by the specified
1336   * path is reflected.
1337   * 
1338   * @param f Path for which status should be obtained. null means the
1339   * root partition of the default file system. 
1340   *
1341   * @return a FsStatus object
1342   *
1343   * @throws AccessControlException If access is denied
1344   * @throws FileNotFoundException If <code>f</code> does not exist
1345   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1346   *           not supported
1347   * @throws IOException If an I/O error occurred
1348   * 
1349   * Exceptions applicable to file systems accessed over RPC:
1350   * @throws RpcClientException If an exception occurred in the RPC client
1351   * @throws RpcServerException If an exception occurred in the RPC server
1352   * @throws UnexpectedServerException If server implementation throws 
1353   *           undeclared exception to RPC server
1354   */
1355  public FsStatus getFsStatus(final Path f) throws AccessControlException,
1356      FileNotFoundException, UnsupportedFileSystemException, IOException {
1357    if (f == null) {
1358      return defaultFS.getFsStatus();
1359    }
1360    final Path absF = fixRelativePart(f);
1361    return new FSLinkResolver<FsStatus>() {
1362      @Override
1363      public FsStatus next(final AbstractFileSystem fs, final Path p) 
1364        throws IOException, UnresolvedLinkException {
1365        return fs.getFsStatus(p);
1366      }
1367    }.resolve(this, absF);
1368  }
1369
1370  /**
1371   * Creates a symbolic link to an existing file. An exception is thrown if 
1372   * the symlink exits, the user does not have permission to create symlink,
1373   * or the underlying file system does not support symlinks.
1374   * 
1375   * Symlink permissions are ignored, access to a symlink is determined by
1376   * the permissions of the symlink target.
1377   * 
1378   * Symlinks in paths leading up to the final path component are resolved 
1379   * transparently. If the final path component refers to a symlink some 
1380   * functions operate on the symlink itself, these are:
1381   * - delete(f) and deleteOnExit(f) - Deletes the symlink.
1382   * - rename(src, dst) - If src refers to a symlink, the symlink is 
1383   *   renamed. If dst refers to a symlink, the symlink is over-written.
1384   * - getLinkTarget(f) - Returns the target of the symlink. 
1385   * - getFileLinkStatus(f) - Returns a FileStatus object describing
1386   *   the symlink.
1387   * Some functions, create() and mkdir(), expect the final path component
1388   * does not exist. If they are given a path that refers to a symlink that 
1389   * does exist they behave as if the path referred to an existing file or 
1390   * directory. All other functions fully resolve, ie follow, the symlink. 
1391   * These are: open, setReplication, setOwner, setTimes, setWorkingDirectory,
1392   * setPermission, getFileChecksum, setVerifyChecksum, getFileBlockLocations,
1393   * getFsStatus, getFileStatus, exists, and listStatus.
1394   * 
1395   * Symlink targets are stored as given to createSymlink, assuming the 
1396   * underlying file system is capable of storing a fully qualified URI.
1397   * Dangling symlinks are permitted. FileContext supports four types of 
1398   * symlink targets, and resolves them as follows
1399   * <pre>
1400   * Given a path referring to a symlink of form:
1401   * 
1402   *   <---X---> 
1403   *   fs://host/A/B/link 
1404   *   <-----Y----->
1405   * 
1406   * In this path X is the scheme and authority that identify the file system,
1407   * and Y is the path leading up to the final path component "link". If Y is
1408   * a symlink  itself then let Y' be the target of Y and X' be the scheme and
1409   * authority of Y'. Symlink targets may:
1410   * 
1411   * 1. Fully qualified URIs
1412   * 
1413   * fs://hostX/A/B/file  Resolved according to the target file system.
1414   * 
1415   * 2. Partially qualified URIs (eg scheme but no host)
1416   * 
1417   * fs:///A/B/file  Resolved according to the target file system. Eg resolving
1418   *                 a symlink to hdfs:///A results in an exception because
1419   *                 HDFS URIs must be fully qualified, while a symlink to 
1420   *                 file:///A will not since Hadoop's local file systems 
1421   *                 require partially qualified URIs.
1422   * 
1423   * 3. Relative paths
1424   * 
1425   * path  Resolves to [Y'][path]. Eg if Y resolves to hdfs://host/A and path 
1426   *       is "../B/file" then [Y'][path] is hdfs://host/B/file
1427   * 
1428   * 4. Absolute paths
1429   * 
1430   * path  Resolves to [X'][path]. Eg if Y resolves hdfs://host/A/B and path
1431   *       is "/file" then [X][path] is hdfs://host/file
1432   * </pre>
1433   * 
1434   * @param target the target of the symbolic link
1435   * @param link the path to be created that points to target
1436   * @param createParent if true then missing parent dirs are created if 
1437   *                     false then parent must exist
1438   *
1439   *
1440   * @throws AccessControlException If access is denied
1441   * @throws FileAlreadyExistsException If file <code>linkcode> already exists
1442   * @throws FileNotFoundException If <code>target</code> does not exist
1443   * @throws ParentNotDirectoryException If parent of <code>link</code> is not a
1444   *           directory.
1445   * @throws UnsupportedFileSystemException If file system for 
1446   *           <code>target</code> or <code>link</code> is not supported
1447   * @throws IOException If an I/O error occurred
1448   */
1449  @SuppressWarnings("deprecation")
1450  public void createSymlink(final Path target, final Path link,
1451      final boolean createParent) throws AccessControlException,
1452      FileAlreadyExistsException, FileNotFoundException,
1453      ParentNotDirectoryException, UnsupportedFileSystemException, 
1454      IOException { 
1455    if (!FileSystem.areSymlinksEnabled()) {
1456      throw new UnsupportedOperationException("Symlinks not supported");
1457    }
1458    final Path nonRelLink = fixRelativePart(link);
1459    new FSLinkResolver<Void>() {
1460      @Override
1461      public Void next(final AbstractFileSystem fs, final Path p) 
1462        throws IOException, UnresolvedLinkException {
1463        fs.createSymlink(target, p, createParent);
1464        return null;
1465      }
1466    }.resolve(this, nonRelLink);
1467  }
1468  
1469  /**
1470   * List the statuses of the files/directories in the given path if the path is
1471   * a directory.
1472   * 
1473   * @param f is the path
1474   *
1475   * @return an iterator that traverses statuses of the files/directories 
1476   *         in the given path
1477   *
1478   * @throws AccessControlException If access is denied
1479   * @throws FileNotFoundException If <code>f</code> does not exist
1480   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1481   *           not supported
1482   * @throws IOException If an I/O error occurred
1483   * 
1484   * Exceptions applicable to file systems accessed over RPC:
1485   * @throws RpcClientException If an exception occurred in the RPC client
1486   * @throws RpcServerException If an exception occurred in the RPC server
1487   * @throws UnexpectedServerException If server implementation throws 
1488   *           undeclared exception to RPC server
1489   */
1490  public RemoteIterator<FileStatus> listStatus(final Path f) throws
1491      AccessControlException, FileNotFoundException,
1492      UnsupportedFileSystemException, IOException {
1493    final Path absF = fixRelativePart(f);
1494    return new FSLinkResolver<RemoteIterator<FileStatus>>() {
1495      @Override
1496      public RemoteIterator<FileStatus> next(
1497          final AbstractFileSystem fs, final Path p) 
1498        throws IOException, UnresolvedLinkException {
1499        return fs.listStatusIterator(p);
1500      }
1501    }.resolve(this, absF);
1502  }
1503
1504  /**
1505   * @return an iterator over the corrupt files under the given path
1506   * (may contain duplicates if a file has more than one corrupt block)
1507   * @throws IOException
1508   */
1509  public RemoteIterator<Path> listCorruptFileBlocks(Path path)
1510    throws IOException {
1511    final Path absF = fixRelativePart(path);
1512    return new FSLinkResolver<RemoteIterator<Path>>() {
1513      @Override
1514      public RemoteIterator<Path> next(final AbstractFileSystem fs,
1515                                       final Path p) 
1516        throws IOException, UnresolvedLinkException {
1517        return fs.listCorruptFileBlocks(p);
1518      }
1519    }.resolve(this, absF);
1520  }
1521  
1522  /**
1523   * List the statuses of the files/directories in the given path if the path is
1524   * a directory. 
1525   * Return the file's status and block locations If the path is a file.
1526   * 
1527   * If a returned status is a file, it contains the file's block locations.
1528   * 
1529   * @param f is the path
1530   *
1531   * @return an iterator that traverses statuses of the files/directories 
1532   *         in the given path
1533   * If any IO exception (for example the input directory gets deleted while
1534   * listing is being executed), next() or hasNext() of the returned iterator
1535   * may throw a RuntimeException with the io exception as the cause.
1536   *
1537   * @throws AccessControlException If access is denied
1538   * @throws FileNotFoundException If <code>f</code> does not exist
1539   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1540   *           not supported
1541   * @throws IOException If an I/O error occurred
1542   * 
1543   * Exceptions applicable to file systems accessed over RPC:
1544   * @throws RpcClientException If an exception occurred in the RPC client
1545   * @throws RpcServerException If an exception occurred in the RPC server
1546   * @throws UnexpectedServerException If server implementation throws 
1547   *           undeclared exception to RPC server
1548   */
1549  public RemoteIterator<LocatedFileStatus> listLocatedStatus(
1550      final Path f) throws
1551      AccessControlException, FileNotFoundException,
1552      UnsupportedFileSystemException, IOException {
1553    final Path absF = fixRelativePart(f);
1554    return new FSLinkResolver<RemoteIterator<LocatedFileStatus>>() {
1555      @Override
1556      public RemoteIterator<LocatedFileStatus> next(
1557          final AbstractFileSystem fs, final Path p) 
1558        throws IOException, UnresolvedLinkException {
1559        return fs.listLocatedStatus(p);
1560      }
1561    }.resolve(this, absF);
1562  }
1563
1564  /**
1565   * Mark a path to be deleted on JVM shutdown.
1566   * 
1567   * @param f the existing path to delete.
1568   *
1569   * @return  true if deleteOnExit is successful, otherwise false.
1570   *
1571   * @throws AccessControlException If access is denied
1572   * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1573   *           not supported
1574   * @throws IOException If an I/O error occurred
1575   * 
1576   * Exceptions applicable to file systems accessed over RPC:
1577   * @throws RpcClientException If an exception occurred in the RPC client
1578   * @throws RpcServerException If an exception occurred in the RPC server
1579   * @throws UnexpectedServerException If server implementation throws 
1580   *           undeclared exception to RPC server
1581   */
1582  public boolean deleteOnExit(Path f) throws AccessControlException,
1583      IOException {
1584    if (!this.util().exists(f)) {
1585      return false;
1586    }
1587    synchronized (DELETE_ON_EXIT) {
1588      if (DELETE_ON_EXIT.isEmpty()) {
1589        ShutdownHookManager.get().addShutdownHook(FINALIZER, SHUTDOWN_HOOK_PRIORITY);
1590      }
1591      
1592      Set<Path> set = DELETE_ON_EXIT.get(this);
1593      if (set == null) {
1594        set = new TreeSet<Path>();
1595        DELETE_ON_EXIT.put(this, set);
1596      }
1597      set.add(f);
1598    }
1599    return true;
1600  }
1601  
1602  private final Util util;
1603  public Util util() {
1604    return util;
1605  }
1606  
1607  
1608  /**
1609   * Utility/library methods built over the basic FileContext methods.
1610   * Since this are library functions, the oprtation are not atomic
1611   * and some of them may partially complete if other threads are making
1612   * changes to the same part of the name space.
1613   */
1614  public class Util {
1615    /**
1616     * Does the file exist?
1617     * Note: Avoid using this method if you already have FileStatus in hand.
1618     * Instead reuse the FileStatus 
1619     * @param f the  file or dir to be checked
1620     *
1621     * @throws AccessControlException If access is denied
1622     * @throws IOException If an I/O error occurred
1623     * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1624     *           not supported
1625     * 
1626     * Exceptions applicable to file systems accessed over RPC:
1627     * @throws RpcClientException If an exception occurred in the RPC client
1628     * @throws RpcServerException If an exception occurred in the RPC server
1629     * @throws UnexpectedServerException If server implementation throws 
1630     *           undeclared exception to RPC server
1631     */
1632    public boolean exists(final Path f) throws AccessControlException,
1633      UnsupportedFileSystemException, IOException {
1634      try {
1635        FileStatus fs = FileContext.this.getFileStatus(f);
1636        assert fs != null;
1637        return true;
1638      } catch (FileNotFoundException e) {
1639        return false;
1640      }
1641    }
1642    
1643    /**
1644     * Return the {@link ContentSummary} of path f.
1645     * @param f path
1646     *
1647     * @return the {@link ContentSummary} of path f.
1648     *
1649     * @throws AccessControlException If access is denied
1650     * @throws FileNotFoundException If <code>f</code> does not exist
1651     * @throws UnsupportedFileSystemException If file system for 
1652     *         <code>f</code> is not supported
1653     * @throws IOException If an I/O error occurred
1654     * 
1655     * Exceptions applicable to file systems accessed over RPC:
1656     * @throws RpcClientException If an exception occurred in the RPC client
1657     * @throws RpcServerException If an exception occurred in the RPC server
1658     * @throws UnexpectedServerException If server implementation throws 
1659     *           undeclared exception to RPC server
1660     */
1661    public ContentSummary getContentSummary(Path f)
1662        throws AccessControlException, FileNotFoundException,
1663        UnsupportedFileSystemException, IOException {
1664      FileStatus status = FileContext.this.getFileStatus(f);
1665      if (status.isFile()) {
1666        long length = status.getLen();
1667        return new ContentSummary.Builder().length(length).
1668            fileCount(1).directoryCount(0).spaceConsumed(length).
1669            build();
1670      }
1671      long[] summary = {0, 0, 1};
1672      RemoteIterator<FileStatus> statusIterator =
1673        FileContext.this.listStatus(f);
1674      while(statusIterator.hasNext()) {
1675        FileStatus s = statusIterator.next();
1676        long length = s.getLen();
1677        ContentSummary c = s.isDirectory() ? getContentSummary(s.getPath()) :
1678            new ContentSummary.Builder().length(length).fileCount(1).
1679            directoryCount(0).spaceConsumed(length).build();
1680        summary[0] += c.getLength();
1681        summary[1] += c.getFileCount();
1682        summary[2] += c.getDirectoryCount();
1683      }
1684      return new ContentSummary.Builder().length(summary[0]).
1685          fileCount(summary[1]).directoryCount(summary[2]).
1686          spaceConsumed(summary[0]).build();
1687    }
1688    
1689    /**
1690     * See {@link #listStatus(Path[], PathFilter)}
1691     */
1692    public FileStatus[] listStatus(Path[] files) throws AccessControlException,
1693        FileNotFoundException, IOException {
1694      return listStatus(files, DEFAULT_FILTER);
1695    }
1696     
1697    /**
1698     * Filter files/directories in the given path using the user-supplied path
1699     * filter.
1700     * 
1701     * @param f is the path name
1702     * @param filter is the user-supplied path filter
1703     *
1704     * @return an array of FileStatus objects for the files under the given path
1705     *         after applying the filter
1706     *
1707     * @throws AccessControlException If access is denied
1708     * @throws FileNotFoundException If <code>f</code> does not exist
1709     * @throws UnsupportedFileSystemException If file system for 
1710     *         <code>pathPattern</code> is not supported
1711     * @throws IOException If an I/O error occurred
1712     * 
1713     * Exceptions applicable to file systems accessed over RPC:
1714     * @throws RpcClientException If an exception occurred in the RPC client
1715     * @throws RpcServerException If an exception occurred in the RPC server
1716     * @throws UnexpectedServerException If server implementation throws 
1717     *           undeclared exception to RPC server
1718     */
1719    public FileStatus[] listStatus(Path f, PathFilter filter)
1720        throws AccessControlException, FileNotFoundException,
1721        UnsupportedFileSystemException, IOException {
1722      ArrayList<FileStatus> results = new ArrayList<FileStatus>();
1723      listStatus(results, f, filter);
1724      return results.toArray(new FileStatus[results.size()]);
1725    }
1726    
1727    /**
1728     * Filter files/directories in the given list of paths using user-supplied
1729     * path filter.
1730     * 
1731     * @param files is a list of paths
1732     * @param filter is the filter
1733     *
1734     * @return a list of statuses for the files under the given paths after
1735     *         applying the filter
1736     *
1737     * @throws AccessControlException If access is denied
1738     * @throws FileNotFoundException If a file in <code>files</code> does not 
1739     *           exist
1740     * @throws IOException If an I/O error occurred
1741     * 
1742     * Exceptions applicable to file systems accessed over RPC:
1743     * @throws RpcClientException If an exception occurred in the RPC client
1744     * @throws RpcServerException If an exception occurred in the RPC server
1745     * @throws UnexpectedServerException If server implementation throws 
1746     *           undeclared exception to RPC server
1747     */
1748    public FileStatus[] listStatus(Path[] files, PathFilter filter)
1749        throws AccessControlException, FileNotFoundException, IOException {
1750      ArrayList<FileStatus> results = new ArrayList<FileStatus>();
1751      for (int i = 0; i < files.length; i++) {
1752        listStatus(results, files[i], filter);
1753      }
1754      return results.toArray(new FileStatus[results.size()]);
1755    }
1756  
1757    /*
1758     * Filter files/directories in the given path using the user-supplied path
1759     * filter. Results are added to the given array <code>results</code>.
1760     */
1761    private void listStatus(ArrayList<FileStatus> results, Path f,
1762        PathFilter filter) throws AccessControlException,
1763        FileNotFoundException, IOException {
1764      FileStatus[] listing = listStatus(f);
1765      if (listing != null) {
1766        for (int i = 0; i < listing.length; i++) {
1767          if (filter.accept(listing[i].getPath())) {
1768            results.add(listing[i]);
1769          }
1770        }
1771      }
1772    }
1773
1774    /**
1775     * List the statuses of the files/directories in the given path 
1776     * if the path is a directory.
1777     * 
1778     * @param f is the path
1779     *
1780     * @return an array that contains statuses of the files/directories 
1781     *         in the given path
1782     *
1783     * @throws AccessControlException If access is denied
1784     * @throws FileNotFoundException If <code>f</code> does not exist
1785     * @throws UnsupportedFileSystemException If file system for <code>f</code> is
1786     *           not supported
1787     * @throws IOException If an I/O error occurred
1788     * 
1789     * Exceptions applicable to file systems accessed over RPC:
1790     * @throws RpcClientException If an exception occurred in the RPC client
1791     * @throws RpcServerException If an exception occurred in the RPC server
1792     * @throws UnexpectedServerException If server implementation throws 
1793     *           undeclared exception to RPC server
1794     */
1795    public FileStatus[] listStatus(final Path f) throws AccessControlException,
1796        FileNotFoundException, UnsupportedFileSystemException,
1797        IOException {
1798      final Path absF = fixRelativePart(f);
1799      return new FSLinkResolver<FileStatus[]>() {
1800        @Override
1801        public FileStatus[] next(final AbstractFileSystem fs, final Path p) 
1802          throws IOException, UnresolvedLinkException {
1803          return fs.listStatus(p);
1804        }
1805      }.resolve(FileContext.this, absF);
1806    }
1807
1808    /**
1809     * List the statuses and block locations of the files in the given path.
1810     * 
1811     * If the path is a directory, 
1812     *   if recursive is false, returns files in the directory;
1813     *   if recursive is true, return files in the subtree rooted at the path.
1814     *   The subtree is traversed in the depth-first order.
1815     * If the path is a file, return the file's status and block locations.
1816     * Files across symbolic links are also returned.
1817     * 
1818     * @param f is the path
1819     * @param recursive if the subdirectories need to be traversed recursively
1820     *
1821     * @return an iterator that traverses statuses of the files
1822     * If any IO exception (for example a sub-directory gets deleted while
1823     * listing is being executed), next() or hasNext() of the returned iterator
1824     * may throw a RuntimeException with the IO exception as the cause.
1825     *
1826     * @throws AccessControlException If access is denied
1827     * @throws FileNotFoundException If <code>f</code> does not exist
1828     * @throws UnsupportedFileSystemException If file system for <code>f</code>
1829     *         is not supported
1830     * @throws IOException If an I/O error occurred
1831     * 
1832     * Exceptions applicable to file systems accessed over RPC:
1833     * @throws RpcClientException If an exception occurred in the RPC client
1834     * @throws RpcServerException If an exception occurred in the RPC server
1835     * @throws UnexpectedServerException If server implementation throws 
1836     *           undeclared exception to RPC server
1837     */
1838    public RemoteIterator<LocatedFileStatus> listFiles(
1839        final Path f, final boolean recursive) throws AccessControlException,
1840        FileNotFoundException, UnsupportedFileSystemException, 
1841        IOException {
1842      return new RemoteIterator<LocatedFileStatus>() {
1843        private Stack<RemoteIterator<LocatedFileStatus>> itors = 
1844          new Stack<RemoteIterator<LocatedFileStatus>>();
1845        RemoteIterator<LocatedFileStatus> curItor = listLocatedStatus(f);
1846        LocatedFileStatus curFile;
1847
1848        /**
1849         * Returns <tt>true</tt> if the iterator has more files.
1850         *
1851         * @return <tt>true</tt> if the iterator has more files.
1852         * @throws AccessControlException if not allowed to access next
1853         *                                file's status or locations
1854         * @throws FileNotFoundException if next file does not exist any more
1855         * @throws UnsupportedFileSystemException if next file's 
1856         *                                        fs is unsupported
1857         * @throws IOException for all other IO errors
1858         *                     for example, NameNode is not avaialbe or
1859         *                     NameNode throws IOException due to an error
1860         *                     while getting the status or block locations
1861         */
1862        @Override
1863        public boolean hasNext() throws IOException {
1864          while (curFile == null) {
1865            if (curItor.hasNext()) {
1866              handleFileStat(curItor.next());
1867            } else if (!itors.empty()) {
1868              curItor = itors.pop();
1869            } else {
1870              return false;
1871            }
1872          }
1873          return true;
1874        }
1875
1876        /**
1877         * Process the input stat.
1878         * If it is a file, return the file stat.
1879         * If it is a directory, traverse the directory if recursive is true;
1880         * ignore it if recursive is false.
1881         * If it is a symlink, resolve the symlink first and then process it
1882         * depending on if it is a file or directory.
1883         * @param stat input status
1884         * @throws AccessControlException if access is denied
1885         * @throws FileNotFoundException if file is not found
1886         * @throws UnsupportedFileSystemException if fs is not supported
1887         * @throws IOException for all other IO errors
1888         */
1889        private void handleFileStat(LocatedFileStatus stat)
1890        throws IOException {
1891          if (stat.isFile()) { // file
1892            curFile = stat;
1893          } else if (stat.isSymlink()) { // symbolic link
1894            // resolve symbolic link
1895            FileStatus symstat = FileContext.this.getFileStatus(
1896                stat.getSymlink());
1897            if (symstat.isFile() || (recursive && symstat.isDirectory())) {
1898              itors.push(curItor);
1899              curItor = listLocatedStatus(stat.getPath());
1900            }
1901          } else if (recursive) { // directory
1902            itors.push(curItor);
1903            curItor = listLocatedStatus(stat.getPath());
1904          }
1905        }
1906
1907        /**
1908         * Returns the next file's status with its block locations
1909         *
1910         * @throws AccessControlException if not allowed to access next
1911         *                                file's status or locations
1912         * @throws FileNotFoundException if next file does not exist any more
1913         * @throws UnsupportedFileSystemException if next file's 
1914         *                                        fs is unsupported
1915         * @throws IOException for all other IO errors
1916         *                     for example, NameNode is not avaialbe or
1917         *                     NameNode throws IOException due to an error
1918         *                     while getting the status or block locations
1919         */
1920        @Override
1921        public LocatedFileStatus next() throws IOException {
1922          if (hasNext()) {
1923            LocatedFileStatus result = curFile;
1924            curFile = null;
1925            return result;
1926          } 
1927          throw new java.util.NoSuchElementException("No more entry in " + f);
1928        }
1929      };
1930    }
1931
1932    /**
1933     * <p>Return all the files that match filePattern and are not checksum
1934     * files. Results are sorted by their names.
1935     * 
1936     * <p>
1937     * A filename pattern is composed of <i>regular</i> characters and
1938     * <i>special pattern matching</i> characters, which are:
1939     *
1940     * <dl>
1941     *  <dd>
1942     *   <dl>
1943     *    <p>
1944     *    <dt> <tt> ? </tt>
1945     *    <dd> Matches any single character.
1946     *
1947     *    <p>
1948     *    <dt> <tt> * </tt>
1949     *    <dd> Matches zero or more characters.
1950     *
1951     *    <p>
1952     *    <dt> <tt> [<i>abc</i>] </tt>
1953     *    <dd> Matches a single character from character set
1954     *     <tt>{<i>a,b,c</i>}</tt>.
1955     *
1956     *    <p>
1957     *    <dt> <tt> [<i>a</i>-<i>b</i>] </tt>
1958     *    <dd> Matches a single character from the character range
1959     *     <tt>{<i>a...b</i>}</tt>. Note: character <tt><i>a</i></tt> must be
1960     *     lexicographically less than or equal to character <tt><i>b</i></tt>.
1961     *
1962     *    <p>
1963     *    <dt> <tt> [^<i>a</i>] </tt>
1964     *    <dd> Matches a single char that is not from character set or range
1965     *     <tt>{<i>a</i>}</tt>.  Note that the <tt>^</tt> character must occur
1966     *     immediately to the right of the opening bracket.
1967     *
1968     *    <p>
1969     *    <dt> <tt> \<i>c</i> </tt>
1970     *    <dd> Removes (escapes) any special meaning of character <i>c</i>.
1971     *
1972     *    <p>
1973     *    <dt> <tt> {ab,cd} </tt>
1974     *    <dd> Matches a string from the string set <tt>{<i>ab, cd</i>} </tt>
1975     *    
1976     *    <p>
1977     *    <dt> <tt> {ab,c{de,fh}} </tt>
1978     *    <dd> Matches a string from string set <tt>{<i>ab, cde, cfh</i>}</tt>
1979     *
1980     *   </dl>
1981     *  </dd>
1982     * </dl>
1983     *
1984     * @param pathPattern a regular expression specifying a pth pattern
1985     *
1986     * @return an array of paths that match the path pattern
1987     *
1988     * @throws AccessControlException If access is denied
1989     * @throws UnsupportedFileSystemException If file system for 
1990     *         <code>pathPattern</code> is not supported
1991     * @throws IOException If an I/O error occurred
1992     * 
1993     * Exceptions applicable to file systems accessed over RPC:
1994     * @throws RpcClientException If an exception occurred in the RPC client
1995     * @throws RpcServerException If an exception occurred in the RPC server
1996     * @throws UnexpectedServerException If server implementation throws 
1997     *           undeclared exception to RPC server
1998     */
1999    public FileStatus[] globStatus(Path pathPattern)
2000        throws AccessControlException, UnsupportedFileSystemException,
2001        IOException {
2002      return new Globber(FileContext.this, pathPattern, DEFAULT_FILTER).glob();
2003    }
2004    
2005    /**
2006     * Return an array of FileStatus objects whose path names match pathPattern
2007     * and is accepted by the user-supplied path filter. Results are sorted by
2008     * their path names.
2009     * Return null if pathPattern has no glob and the path does not exist.
2010     * Return an empty array if pathPattern has a glob and no path matches it. 
2011     * 
2012     * @param pathPattern regular expression specifying the path pattern
2013     * @param filter user-supplied path filter
2014     *
2015     * @return an array of FileStatus objects
2016     *
2017     * @throws AccessControlException If access is denied
2018     * @throws UnsupportedFileSystemException If file system for 
2019     *         <code>pathPattern</code> is not supported
2020     * @throws IOException If an I/O error occurred
2021     * 
2022     * Exceptions applicable to file systems accessed over RPC:
2023     * @throws RpcClientException If an exception occurred in the RPC client
2024     * @throws RpcServerException If an exception occurred in the RPC server
2025     * @throws UnexpectedServerException If server implementation throws 
2026     *           undeclared exception to RPC server
2027     */
2028    public FileStatus[] globStatus(final Path pathPattern,
2029        final PathFilter filter) throws AccessControlException,
2030        UnsupportedFileSystemException, IOException {
2031      return new Globber(FileContext.this, pathPattern, filter).glob();
2032    }
2033
2034    /**
2035     * Copy file from src to dest. See
2036     * {@link #copy(Path, Path, boolean, boolean)}
2037     */
2038    public boolean copy(final Path src, final Path dst)
2039        throws AccessControlException, FileAlreadyExistsException,
2040        FileNotFoundException, ParentNotDirectoryException,
2041        UnsupportedFileSystemException, IOException {
2042      return copy(src, dst, false, false);
2043    }
2044    
2045    /**
2046     * Copy from src to dst, optionally deleting src and overwriting dst.
2047     * @param src
2048     * @param dst
2049     * @param deleteSource - delete src if true
2050     * @param overwrite  overwrite dst if true; throw IOException if dst exists
2051     *         and overwrite is false.
2052     *
2053     * @return true if copy is successful
2054     *
2055     * @throws AccessControlException If access is denied
2056     * @throws FileAlreadyExistsException If <code>dst</code> already exists
2057     * @throws FileNotFoundException If <code>src</code> does not exist
2058     * @throws ParentNotDirectoryException If parent of <code>dst</code> is not
2059     *           a directory
2060     * @throws UnsupportedFileSystemException If file system for 
2061     *         <code>src</code> or <code>dst</code> is not supported
2062     * @throws IOException If an I/O error occurred
2063     * 
2064     * Exceptions applicable to file systems accessed over RPC:
2065     * @throws RpcClientException If an exception occurred in the RPC client
2066     * @throws RpcServerException If an exception occurred in the RPC server
2067     * @throws UnexpectedServerException If server implementation throws 
2068     *           undeclared exception to RPC server
2069     * 
2070     * RuntimeExceptions:
2071     * @throws InvalidPathException If path <code>dst</code> is invalid
2072     */
2073    public boolean copy(final Path src, final Path dst, boolean deleteSource,
2074        boolean overwrite) throws AccessControlException,
2075        FileAlreadyExistsException, FileNotFoundException,
2076        ParentNotDirectoryException, UnsupportedFileSystemException, 
2077        IOException {
2078      src.checkNotSchemeWithRelative();
2079      dst.checkNotSchemeWithRelative();
2080      Path qSrc = makeQualified(src);
2081      Path qDst = makeQualified(dst);
2082      checkDest(qSrc.getName(), qDst, overwrite);
2083      FileStatus fs = FileContext.this.getFileStatus(qSrc);
2084      if (fs.isDirectory()) {
2085        checkDependencies(qSrc, qDst);
2086        mkdir(qDst, FsPermission.getDirDefault(), true);
2087        FileStatus[] contents = listStatus(qSrc);
2088        for (FileStatus content : contents) {
2089          copy(makeQualified(content.getPath()), makeQualified(new Path(qDst,
2090              content.getPath().getName())), deleteSource, overwrite);
2091        }
2092      } else {
2093        InputStream in=null;
2094        OutputStream out = null;
2095        try {
2096          in = open(qSrc);
2097          EnumSet<CreateFlag> createFlag = overwrite ? EnumSet.of(
2098              CreateFlag.CREATE, CreateFlag.OVERWRITE) : 
2099                EnumSet.of(CreateFlag.CREATE);
2100          out = create(qDst, createFlag);
2101          IOUtils.copyBytes(in, out, conf, true);
2102        } finally {
2103          IOUtils.closeStream(out);
2104          IOUtils.closeStream(in);
2105        }
2106      }
2107      if (deleteSource) {
2108        return delete(qSrc, true);
2109      } else {
2110        return true;
2111      }
2112    }
2113  }
2114
2115  /**
2116   * Check if copying srcName to dst would overwrite an existing 
2117   * file or directory.
2118   * @param srcName File or directory to be copied.
2119   * @param dst Destination to copy srcName to.
2120   * @param overwrite Whether it's ok to overwrite an existing file. 
2121   * @throws AccessControlException If access is denied.
2122   * @throws IOException If dst is an existing directory, or dst is an 
2123   * existing file and the overwrite option is not passed.
2124   */
2125  private void checkDest(String srcName, Path dst, boolean overwrite)
2126      throws AccessControlException, IOException {
2127    try {
2128      FileStatus dstFs = getFileStatus(dst);
2129      if (dstFs.isDirectory()) {
2130        if (null == srcName) {
2131          throw new IOException("Target " + dst + " is a directory");
2132        }
2133        // Recurse to check if dst/srcName exists.
2134        checkDest(null, new Path(dst, srcName), overwrite);
2135      } else if (!overwrite) {
2136        throw new IOException("Target " + new Path(dst, srcName)
2137            + " already exists");
2138      }
2139    } catch (FileNotFoundException e) {
2140      // dst does not exist - OK to copy.
2141    }
2142  }
2143   
2144  //
2145  // If the destination is a subdirectory of the source, then
2146  // generate exception
2147  //
2148  private static void checkDependencies(Path qualSrc, Path qualDst)
2149    throws IOException {
2150    if (isSameFS(qualSrc, qualDst)) {
2151      String srcq = qualSrc.toString() + Path.SEPARATOR;
2152      String dstq = qualDst.toString() + Path.SEPARATOR;
2153      if (dstq.startsWith(srcq)) {
2154        if (srcq.length() == dstq.length()) {
2155          throw new IOException("Cannot copy " + qualSrc + " to itself.");
2156        } else {
2157          throw new IOException("Cannot copy " + qualSrc +
2158                             " to its subdirectory " + qualDst);
2159        }
2160      }
2161    }
2162  }
2163  
2164  /**
2165   * Are qualSrc and qualDst of the same file system?
2166   * @param qualPath1 - fully qualified path
2167   * @param qualPath2 - fully qualified path
2168   * @return
2169   */
2170  private static boolean isSameFS(Path qualPath1, Path qualPath2) {
2171    URI srcUri = qualPath1.toUri();
2172    URI dstUri = qualPath2.toUri();
2173    return (srcUri.getScheme().equals(dstUri.getScheme()) && 
2174        !(srcUri.getAuthority() != null && dstUri.getAuthority() != null && srcUri
2175        .getAuthority().equals(dstUri.getAuthority())));
2176  }
2177
2178  /**
2179   * Deletes all the paths in deleteOnExit on JVM shutdown.
2180   */
2181  static class FileContextFinalizer implements Runnable {
2182    @Override
2183    public synchronized void run() {
2184      processDeleteOnExit();
2185    }
2186  }
2187
2188  /**
2189   * Resolves all symbolic links in the specified path.
2190   * Returns the new path object.
2191   */
2192  protected Path resolve(final Path f) throws FileNotFoundException,
2193      UnresolvedLinkException, AccessControlException, IOException {
2194    return new FSLinkResolver<Path>() {
2195      @Override
2196      public Path next(final AbstractFileSystem fs, final Path p) 
2197        throws IOException, UnresolvedLinkException {
2198        return fs.resolvePath(p);
2199      }
2200    }.resolve(this, f);
2201  }
2202
2203  /**
2204   * Resolves all symbolic links in the specified path leading up 
2205   * to, but not including the final path component.
2206   * @param f path to resolve
2207   * @return the new path object.
2208   */
2209  protected Path resolveIntermediate(final Path f) throws IOException {
2210    return new FSLinkResolver<FileStatus>() {
2211      @Override
2212      public FileStatus next(final AbstractFileSystem fs, final Path p) 
2213        throws IOException, UnresolvedLinkException {
2214        return fs.getFileLinkStatus(p);
2215      }
2216    }.resolve(this, f).getPath();
2217  }
2218
2219  /**
2220   * Returns the list of AbstractFileSystems accessed in the path. The list may
2221   * contain more than one AbstractFileSystems objects in case of symlinks.
2222   * 
2223   * @param f
2224   *          Path which needs to be resolved
2225   * @return List of AbstractFileSystems accessed in the path
2226   * @throws IOException
2227   */
2228  Set<AbstractFileSystem> resolveAbstractFileSystems(final Path f)
2229      throws IOException {
2230    final Path absF = fixRelativePart(f);
2231    final HashSet<AbstractFileSystem> result 
2232      = new HashSet<AbstractFileSystem>();
2233    new FSLinkResolver<Void>() {
2234      @Override
2235      public Void next(final AbstractFileSystem fs, final Path p)
2236          throws IOException, UnresolvedLinkException {
2237        result.add(fs);
2238        fs.getFileStatus(p);
2239        return null;
2240      }
2241    }.resolve(this, absF);
2242    return result;
2243  }
2244
2245  /**
2246   * Get the statistics for a particular file system
2247   * 
2248   * @param uri
2249   *          the uri to lookup the statistics. Only scheme and authority part
2250   *          of the uri are used as the key to store and lookup.
2251   * @return a statistics object
2252   */
2253  public static Statistics getStatistics(URI uri) {
2254    return AbstractFileSystem.getStatistics(uri);
2255  }
2256
2257  /**
2258   * Clears all the statistics stored in AbstractFileSystem, for all the file
2259   * systems.
2260   */
2261  public static void clearStatistics() {
2262    AbstractFileSystem.clearStatistics();
2263  }
2264
2265  /**
2266   * Prints the statistics to standard output. File System is identified by the
2267   * scheme and authority.
2268   */
2269  public static void printStatistics() {
2270    AbstractFileSystem.printStatistics();
2271  }
2272
2273  /**
2274   * @return Map of uri and statistics for each filesystem instantiated. The uri
2275   *         consists of scheme and authority for the filesystem.
2276   */
2277  public static Map<URI, Statistics> getAllStatistics() {
2278    return AbstractFileSystem.getAllStatistics();
2279  }
2280  
2281  /**
2282   * Get delegation tokens for the file systems accessed for a given
2283   * path.
2284   * @param p Path for which delegations tokens are requested.
2285   * @param renewer the account name that is allowed to renew the token.
2286   * @return List of delegation tokens.
2287   * @throws IOException
2288   */
2289  @InterfaceAudience.LimitedPrivate( { "HDFS", "MapReduce" })
2290  public List<Token<?>> getDelegationTokens(
2291      Path p, String renewer) throws IOException {
2292    Set<AbstractFileSystem> afsSet = resolveAbstractFileSystems(p);
2293    List<Token<?>> tokenList = 
2294        new ArrayList<Token<?>>();
2295    for (AbstractFileSystem afs : afsSet) {
2296      List<Token<?>> afsTokens = afs.getDelegationTokens(renewer);
2297      tokenList.addAll(afsTokens);
2298    }
2299    return tokenList;
2300  }
2301
2302  /**
2303   * Modifies ACL entries of files and directories.  This method can add new ACL
2304   * entries or modify the permissions on existing ACL entries.  All existing
2305   * ACL entries that are not specified in this call are retained without
2306   * changes.  (Modifications are merged into the current ACL.)
2307   *
2308   * @param path Path to modify
2309   * @param aclSpec List<AclEntry> describing modifications
2310   * @throws IOException if an ACL could not be modified
2311   */
2312  public void modifyAclEntries(final Path path, final List<AclEntry> aclSpec)
2313      throws IOException {
2314    Path absF = fixRelativePart(path);
2315    new FSLinkResolver<Void>() {
2316      @Override
2317      public Void next(final AbstractFileSystem fs, final Path p)
2318          throws IOException {
2319        fs.modifyAclEntries(p, aclSpec);
2320        return null;
2321      }
2322    }.resolve(this, absF);
2323  }
2324
2325  /**
2326   * Removes ACL entries from files and directories.  Other ACL entries are
2327   * retained.
2328   *
2329   * @param path Path to modify
2330   * @param aclSpec List<AclEntry> describing entries to remove
2331   * @throws IOException if an ACL could not be modified
2332   */
2333  public void removeAclEntries(final Path path, final List<AclEntry> aclSpec)
2334      throws IOException {
2335    Path absF = fixRelativePart(path);
2336    new FSLinkResolver<Void>() {
2337      @Override
2338      public Void next(final AbstractFileSystem fs, final Path p)
2339          throws IOException {
2340        fs.removeAclEntries(p, aclSpec);
2341        return null;
2342      }
2343    }.resolve(this, absF);
2344  }
2345
2346  /**
2347   * Removes all default ACL entries from files and directories.
2348   *
2349   * @param path Path to modify
2350   * @throws IOException if an ACL could not be modified
2351   */
2352  public void removeDefaultAcl(Path path)
2353      throws IOException {
2354    Path absF = fixRelativePart(path);
2355    new FSLinkResolver<Void>() {
2356      @Override
2357      public Void next(final AbstractFileSystem fs, final Path p)
2358          throws IOException {
2359        fs.removeDefaultAcl(p);
2360        return null;
2361      }
2362    }.resolve(this, absF);
2363  }
2364
2365  /**
2366   * Removes all but the base ACL entries of files and directories.  The entries
2367   * for user, group, and others are retained for compatibility with permission
2368   * bits.
2369   *
2370   * @param path Path to modify
2371   * @throws IOException if an ACL could not be removed
2372   */
2373  public void removeAcl(Path path) throws IOException {
2374    Path absF = fixRelativePart(path);
2375    new FSLinkResolver<Void>() {
2376      @Override
2377      public Void next(final AbstractFileSystem fs, final Path p)
2378          throws IOException {
2379        fs.removeAcl(p);
2380        return null;
2381      }
2382    }.resolve(this, absF);
2383  }
2384
2385  /**
2386   * Fully replaces ACL of files and directories, discarding all existing
2387   * entries.
2388   *
2389   * @param path Path to modify
2390   * @param aclSpec List<AclEntry> describing modifications, must include entries
2391   *   for user, group, and others for compatibility with permission bits.
2392   * @throws IOException if an ACL could not be modified
2393   */
2394  public void setAcl(Path path, final List<AclEntry> aclSpec)
2395      throws IOException {
2396    Path absF = fixRelativePart(path);
2397    new FSLinkResolver<Void>() {
2398      @Override
2399      public Void next(final AbstractFileSystem fs, final Path p)
2400          throws IOException {
2401        fs.setAcl(p, aclSpec);
2402        return null;
2403      }
2404    }.resolve(this, absF);
2405  }
2406
2407  /**
2408   * Gets the ACLs of files and directories.
2409   *
2410   * @param path Path to get
2411   * @return RemoteIterator<AclStatus> which returns each AclStatus
2412   * @throws IOException if an ACL could not be read
2413   */
2414  public AclStatus getAclStatus(Path path) throws IOException {
2415    Path absF = fixRelativePart(path);
2416    return new FSLinkResolver<AclStatus>() {
2417      @Override
2418      public AclStatus next(final AbstractFileSystem fs, final Path p)
2419          throws IOException {
2420        return fs.getAclStatus(p);
2421      }
2422    }.resolve(this, absF);
2423  }
2424
2425  /**
2426   * Set an xattr of a file or directory.
2427   * The name must be prefixed with the namespace followed by ".". For example,
2428   * "user.attr".
2429   * <p/>
2430   * Refer to the HDFS extended attributes user documentation for details.
2431   *
2432   * @param path Path to modify
2433   * @param name xattr name.
2434   * @param value xattr value.
2435   * @throws IOException
2436   */
2437  public void setXAttr(Path path, String name, byte[] value)
2438      throws IOException {
2439    setXAttr(path, name, value, EnumSet.of(XAttrSetFlag.CREATE,
2440        XAttrSetFlag.REPLACE));
2441  }
2442
2443  /**
2444   * Set an xattr of a file or directory.
2445   * The name must be prefixed with the namespace followed by ".". For example,
2446   * "user.attr".
2447   * <p/>
2448   * Refer to the HDFS extended attributes user documentation for details.
2449   *
2450   * @param path Path to modify
2451   * @param name xattr name.
2452   * @param value xattr value.
2453   * @param flag xattr set flag
2454   * @throws IOException
2455   */
2456  public void setXAttr(Path path, final String name, final byte[] value,
2457      final EnumSet<XAttrSetFlag> flag) throws IOException {
2458    final Path absF = fixRelativePart(path);
2459    new FSLinkResolver<Void>() {
2460      @Override
2461      public Void next(final AbstractFileSystem fs, final Path p)
2462          throws IOException {
2463        fs.setXAttr(p, name, value, flag);
2464        return null;
2465      }
2466    }.resolve(this, absF);
2467  }
2468
2469  /**
2470   * Get an xattr for a file or directory.
2471   * The name must be prefixed with the namespace followed by ".". For example,
2472   * "user.attr".
2473   * <p/>
2474   * Refer to the HDFS extended attributes user documentation for details.
2475   *
2476   * @param path Path to get extended attribute
2477   * @param name xattr name.
2478   * @return byte[] xattr value.
2479   * @throws IOException
2480   */
2481  public byte[] getXAttr(Path path, final String name) throws IOException {
2482    final Path absF = fixRelativePart(path);
2483    return new FSLinkResolver<byte[]>() {
2484      @Override
2485      public byte[] next(final AbstractFileSystem fs, final Path p)
2486          throws IOException {
2487        return fs.getXAttr(p, name);
2488      }
2489    }.resolve(this, absF);
2490  }
2491
2492  /**
2493   * Get all of the xattrs for a file or directory.
2494   * Only those xattrs for which the logged-in user has permissions to view
2495   * are returned.
2496   * <p/>
2497   * Refer to the HDFS extended attributes user documentation for details.
2498   *
2499   * @param path Path to get extended attributes
2500   * @return Map<String, byte[]> describing the XAttrs of the file or directory
2501   * @throws IOException
2502   */
2503  public Map<String, byte[]> getXAttrs(Path path) throws IOException {
2504    final Path absF = fixRelativePart(path);
2505    return new FSLinkResolver<Map<String, byte[]>>() {
2506      @Override
2507      public Map<String, byte[]> next(final AbstractFileSystem fs, final Path p)
2508          throws IOException {
2509        return fs.getXAttrs(p);
2510      }
2511    }.resolve(this, absF);
2512  }
2513
2514  /**
2515   * Get all of the xattrs for a file or directory.
2516   * Only those xattrs for which the logged-in user has permissions to view
2517   * are returned.
2518   * <p/>
2519   * Refer to the HDFS extended attributes user documentation for details.
2520   *
2521   * @param path Path to get extended attributes
2522   * @param names XAttr names.
2523   * @return Map<String, byte[]> describing the XAttrs of the file or directory
2524   * @throws IOException
2525   */
2526  public Map<String, byte[]> getXAttrs(Path path, final List<String> names)
2527      throws IOException {
2528    final Path absF = fixRelativePart(path);
2529    return new FSLinkResolver<Map<String, byte[]>>() {
2530      @Override
2531      public Map<String, byte[]> next(final AbstractFileSystem fs, final Path p)
2532          throws IOException {
2533        return fs.getXAttrs(p, names);
2534      }
2535    }.resolve(this, absF);
2536  }
2537
2538  /**
2539   * Remove an xattr of a file or directory.
2540   * The name must be prefixed with the namespace followed by ".". For example,
2541   * "user.attr".
2542   * <p/>
2543   * Refer to the HDFS extended attributes user documentation for details.
2544   *
2545   * @param path Path to remove extended attribute
2546   * @param name xattr name
2547   * @throws IOException
2548   */
2549  public void removeXAttr(Path path, final String name) throws IOException {
2550    final Path absF = fixRelativePart(path);
2551    new FSLinkResolver<Void>() {
2552      @Override
2553      public Void next(final AbstractFileSystem fs, final Path p)
2554          throws IOException {
2555        fs.removeXAttr(p, name);
2556        return null;
2557      }
2558    }.resolve(this, absF);
2559  }
2560
2561  /**
2562   * Get all of the xattr names for a file or directory.
2563   * Only those xattr names which the logged-in user has permissions to view
2564   * are returned.
2565   * <p/>
2566   * Refer to the HDFS extended attributes user documentation for details.
2567   *
2568   * @param path Path to get extended attributes
2569   * @return List<String> of the XAttr names of the file or directory
2570   * @throws IOException
2571   */
2572  public List<String> listXAttrs(Path path) throws IOException {
2573    final Path absF = fixRelativePart(path);
2574    return new FSLinkResolver<List<String>>() {
2575      @Override
2576      public List<String> next(final AbstractFileSystem fs, final Path p)
2577          throws IOException {
2578        return fs.listXAttrs(p);
2579      }
2580    }.resolve(this, absF);
2581  }
2582
2583  /**
2584   * Create a snapshot with a default name.
2585   *
2586   * @param path The directory where snapshots will be taken.
2587   * @return the snapshot path.
2588   *
2589   * @throws IOException If an I/O error occurred
2590   *
2591   * <p>Exceptions applicable to file systems accessed over RPC:
2592   * @throws RpcClientException If an exception occurred in the RPC client
2593   * @throws RpcServerException If an exception occurred in the RPC server
2594   * @throws UnexpectedServerException If server implementation throws
2595   *           undeclared exception to RPC server
2596   */
2597  public final Path createSnapshot(Path path) throws IOException {
2598    return createSnapshot(path, null);
2599  }
2600
2601  /**
2602   * Create a snapshot.
2603   *
2604   * @param path The directory where snapshots will be taken.
2605   * @param snapshotName The name of the snapshot
2606   * @return the snapshot path.
2607   *
2608   * @throws IOException If an I/O error occurred
2609   *
2610   * <p>Exceptions applicable to file systems accessed over RPC:
2611   * @throws RpcClientException If an exception occurred in the RPC client
2612   * @throws RpcServerException If an exception occurred in the RPC server
2613   * @throws UnexpectedServerException If server implementation throws
2614   *           undeclared exception to RPC server
2615   */
2616  public Path createSnapshot(final Path path, final String snapshotName)
2617      throws IOException {
2618    final Path absF = fixRelativePart(path);
2619    return new FSLinkResolver<Path>() {
2620
2621      @Override
2622      public Path next(final AbstractFileSystem fs, final Path p)
2623          throws IOException {
2624        return fs.createSnapshot(p, snapshotName);
2625      }
2626    }.resolve(this, absF);
2627  }
2628
2629  /**
2630   * Rename a snapshot.
2631   *
2632   * @param path The directory path where the snapshot was taken
2633   * @param snapshotOldName Old name of the snapshot
2634   * @param snapshotNewName New name of the snapshot
2635   *
2636   * @throws IOException If an I/O error occurred
2637   *
2638   * <p>Exceptions applicable to file systems accessed over RPC:
2639   * @throws RpcClientException If an exception occurred in the RPC client
2640   * @throws RpcServerException If an exception occurred in the RPC server
2641   * @throws UnexpectedServerException If server implementation throws
2642   *           undeclared exception to RPC server
2643   */
2644  public void renameSnapshot(final Path path, final String snapshotOldName,
2645      final String snapshotNewName) throws IOException {
2646    final Path absF = fixRelativePart(path);
2647    new FSLinkResolver<Void>() {
2648      @Override
2649      public Void next(final AbstractFileSystem fs, final Path p)
2650          throws IOException {
2651        fs.renameSnapshot(p, snapshotOldName, snapshotNewName);
2652        return null;
2653      }
2654    }.resolve(this, absF);
2655  }
2656
2657  /**
2658   * Delete a snapshot of a directory.
2659   *
2660   * @param path The directory that the to-be-deleted snapshot belongs to
2661   * @param snapshotName The name of the snapshot
2662   *
2663   * @throws IOException If an I/O error occurred
2664   *
2665   * <p>Exceptions applicable to file systems accessed over RPC:
2666   * @throws RpcClientException If an exception occurred in the RPC client
2667   * @throws RpcServerException If an exception occurred in the RPC server
2668   * @throws UnexpectedServerException If server implementation throws
2669   *           undeclared exception to RPC server
2670   */
2671  public void deleteSnapshot(final Path path, final String snapshotName)
2672      throws IOException {
2673    final Path absF = fixRelativePart(path);
2674    new FSLinkResolver<Void>() {
2675      @Override
2676      public Void next(final AbstractFileSystem fs, final Path p)
2677          throws IOException {
2678        fs.deleteSnapshot(p, snapshotName);
2679        return null;
2680      }
2681    }.resolve(this, absF);
2682  }
2683
2684  /**
2685   * Set the storage policy for a given file or directory.
2686   *
2687   * @param path file or directory path.
2688   * @param policyName the name of the target storage policy. The list
2689   *                   of supported Storage policies can be retrieved
2690   *                   via {@link #getAllStoragePolicies}.
2691   */
2692  public void setStoragePolicy(final Path path, final String policyName)
2693      throws IOException {
2694    final Path absF = fixRelativePart(path);
2695    new FSLinkResolver<Void>() {
2696      @Override
2697      public Void next(final AbstractFileSystem fs, final Path p)
2698          throws IOException {
2699        fs.setStoragePolicy(path, policyName);
2700        return null;
2701      }
2702    }.resolve(this, absF);
2703  }
2704
2705  /**
2706   * Query the effective storage policy ID for the given file or directory.
2707   *
2708   * @param src file or directory path.
2709   * @return storage policy for give file.
2710   * @throws IOException
2711   */
2712  public BlockStoragePolicySpi getStoragePolicy(Path path) throws IOException {
2713    final Path absF = fixRelativePart(path);
2714    return new FSLinkResolver<BlockStoragePolicySpi>() {
2715      @Override
2716      public BlockStoragePolicySpi next(final AbstractFileSystem fs,
2717          final Path p)
2718          throws IOException {
2719        return fs.getStoragePolicy(p);
2720      }
2721    }.resolve(this, absF);
2722  }
2723
2724  /**
2725   * Retrieve all the storage policies supported by this file system.
2726   *
2727   * @return all storage policies supported by this filesystem.
2728   * @throws IOException
2729   */
2730  public Collection<? extends BlockStoragePolicySpi> getAllStoragePolicies()
2731      throws IOException {
2732    return defaultFS.getAllStoragePolicies();
2733  }
2734}