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