001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.fs;
019
020import java.io.Closeable;
021import java.io.FileNotFoundException;
022import java.io.IOException;
023import java.lang.ref.WeakReference;
024import java.lang.ref.ReferenceQueue;
025import java.net.URI;
026import java.net.URISyntaxException;
027import java.security.PrivilegedExceptionAction;
028import java.util.ArrayList;
029import java.util.Collection;
030import java.util.EnumSet;
031import java.util.HashMap;
032import java.util.HashSet;
033import java.util.IdentityHashMap;
034import java.util.Iterator;
035import java.util.List;
036import java.util.Map;
037import java.util.NoSuchElementException;
038import java.util.ServiceConfigurationError;
039import java.util.ServiceLoader;
040import java.util.Set;
041import java.util.Stack;
042import java.util.TreeSet;
043import java.util.concurrent.atomic.AtomicLong;
044
045import org.apache.commons.logging.Log;
046import org.apache.commons.logging.LogFactory;
047import org.apache.hadoop.classification.InterfaceAudience;
048import org.apache.hadoop.classification.InterfaceStability;
049import org.apache.hadoop.conf.Configuration;
050import org.apache.hadoop.conf.Configured;
051import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
052import org.apache.hadoop.fs.Options.ChecksumOpt;
053import org.apache.hadoop.fs.Options.Rename;
054import org.apache.hadoop.fs.permission.AclEntry;
055import org.apache.hadoop.fs.permission.AclStatus;
056import org.apache.hadoop.fs.permission.FsAction;
057import org.apache.hadoop.fs.permission.FsPermission;
058import org.apache.hadoop.io.MultipleIOException;
059import org.apache.hadoop.io.Text;
060import org.apache.hadoop.net.NetUtils;
061import org.apache.hadoop.security.AccessControlException;
062import org.apache.hadoop.security.Credentials;
063import org.apache.hadoop.security.SecurityUtil;
064import org.apache.hadoop.security.UserGroupInformation;
065import org.apache.hadoop.security.token.Token;
066import org.apache.hadoop.util.ClassUtil;
067import org.apache.hadoop.util.DataChecksum;
068import org.apache.hadoop.util.Progressable;
069import org.apache.hadoop.util.ReflectionUtils;
070import org.apache.hadoop.util.ShutdownHookManager;
071import org.apache.hadoop.util.StringUtils;
072import org.apache.htrace.core.Tracer;
073import org.apache.htrace.core.TraceScope;
074
075import com.google.common.annotations.VisibleForTesting;
076
077import static com.google.common.base.Preconditions.checkArgument;
078import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
079
080/****************************************************************
081 * An abstract base class for a fairly generic filesystem.  It
082 * may be implemented as a distributed filesystem, or as a "local"
083 * one that reflects the locally-connected disk.  The local version
084 * exists for small Hadoop instances and for testing.
085 *
086 * <p>
087 *
088 * All user code that may potentially use the Hadoop Distributed
089 * File System should be written to use a FileSystem object.  The
090 * Hadoop DFS is a multi-machine system that appears as a single
091 * disk.  It's useful because of its fault tolerance and potentially
092 * very large capacity.
093 * 
094 * <p>
095 * The local implementation is {@link LocalFileSystem} and distributed
096 * implementation is DistributedFileSystem.
097 *****************************************************************/
098@InterfaceAudience.Public
099@InterfaceStability.Stable
100public abstract class FileSystem extends Configured implements Closeable {
101  public static final String FS_DEFAULT_NAME_KEY = 
102                   CommonConfigurationKeys.FS_DEFAULT_NAME_KEY;
103  public static final String DEFAULT_FS = 
104                   CommonConfigurationKeys.FS_DEFAULT_NAME_DEFAULT;
105
106  public static final Log LOG = LogFactory.getLog(FileSystem.class);
107
108  /**
109   * Priority of the FileSystem shutdown hook.
110   */
111  public static final int SHUTDOWN_HOOK_PRIORITY = 10;
112
113  public static final String TRASH_PREFIX = ".Trash";
114
115  /** FileSystem cache */
116  static final Cache CACHE = new Cache();
117
118  /** The key this instance is stored under in the cache. */
119  private Cache.Key key;
120
121  /** Recording statistics per a FileSystem class */
122  private static final Map<Class<? extends FileSystem>, Statistics> 
123    statisticsTable =
124      new IdentityHashMap<Class<? extends FileSystem>, Statistics>();
125  
126  /**
127   * The statistics for this file system.
128   */
129  protected Statistics statistics;
130
131  /**
132   * A cache of files that should be deleted when filsystem is closed
133   * or the JVM is exited.
134   */
135  private Set<Path> deleteOnExit = new TreeSet<Path>();
136  
137  boolean resolveSymlinks;
138
139  /**
140   * This method adds a file system for testing so that we can find it later. It
141   * is only for testing.
142   * @param uri the uri to store it under
143   * @param conf the configuration to store it under
144   * @param fs the file system to store
145   * @throws IOException
146   */
147  static void addFileSystemForTesting(URI uri, Configuration conf,
148      FileSystem fs) throws IOException {
149    CACHE.map.put(new Cache.Key(uri, conf), fs);
150  }
151
152  /**
153   * Get a filesystem instance based on the uri, the passed
154   * configuration and the user
155   * @param uri of the filesystem
156   * @param conf the configuration to use
157   * @param user to perform the get as
158   * @return the filesystem instance
159   * @throws IOException
160   * @throws InterruptedException
161   */
162  public static FileSystem get(final URI uri, final Configuration conf,
163        final String user) throws IOException, InterruptedException {
164    String ticketCachePath =
165      conf.get(CommonConfigurationKeys.KERBEROS_TICKET_CACHE_PATH);
166    UserGroupInformation ugi =
167        UserGroupInformation.getBestUGI(ticketCachePath, user);
168    return ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
169      @Override
170      public FileSystem run() throws IOException {
171        return get(uri, conf);
172      }
173    });
174  }
175
176  /**
177   * Returns the configured filesystem implementation.
178   * @param conf the configuration to use
179   */
180  public static FileSystem get(Configuration conf) throws IOException {
181    return get(getDefaultUri(conf), conf);
182  }
183  
184  /** Get the default filesystem URI from a configuration.
185   * @param conf the configuration to use
186   * @return the uri of the default filesystem
187   */
188  public static URI getDefaultUri(Configuration conf) {
189    return URI.create(fixName(conf.get(FS_DEFAULT_NAME_KEY, DEFAULT_FS)));
190  }
191
192  /** Set the default filesystem URI in a configuration.
193   * @param conf the configuration to alter
194   * @param uri the new default filesystem uri
195   */
196  public static void setDefaultUri(Configuration conf, URI uri) {
197    conf.set(FS_DEFAULT_NAME_KEY, uri.toString());
198  }
199
200  /** Set the default filesystem URI in a configuration.
201   * @param conf the configuration to alter
202   * @param uri the new default filesystem uri
203   */
204  public static void setDefaultUri(Configuration conf, String uri) {
205    setDefaultUri(conf, URI.create(fixName(uri)));
206  }
207
208  /** Called after a new FileSystem instance is constructed.
209   * @param name a uri whose authority section names the host, port, etc.
210   *   for this FileSystem
211   * @param conf the configuration
212   */
213  public void initialize(URI name, Configuration conf) throws IOException {
214    final String scheme;
215    if (name.getScheme() == null || name.getScheme().isEmpty()) {
216      scheme = getDefaultUri(conf).getScheme();
217    } else {
218      scheme = name.getScheme();
219    }
220    statistics = getStatistics(scheme, getClass());
221    resolveSymlinks = conf.getBoolean(
222        CommonConfigurationKeys.FS_CLIENT_RESOLVE_REMOTE_SYMLINKS_KEY,
223        CommonConfigurationKeys.FS_CLIENT_RESOLVE_REMOTE_SYMLINKS_DEFAULT);
224  }
225
226  /**
227   * Return the protocol scheme for the FileSystem.
228   * <p/>
229   * This implementation throws an <code>UnsupportedOperationException</code>.
230   *
231   * @return the protocol scheme for the FileSystem.
232   */
233  public String getScheme() {
234    throw new UnsupportedOperationException("Not implemented by the " + getClass().getSimpleName() + " FileSystem implementation");
235  }
236
237  /** Returns a URI whose scheme and authority identify this FileSystem.*/
238  public abstract URI getUri();
239  
240  /**
241   * Return a canonicalized form of this FileSystem's URI.
242   * 
243   * The default implementation simply calls {@link #canonicalizeUri(URI)}
244   * on the filesystem's own URI, so subclasses typically only need to
245   * implement that method.
246   *
247   * @see #canonicalizeUri(URI)
248   */
249  protected URI getCanonicalUri() {
250    return canonicalizeUri(getUri());
251  }
252  
253  /**
254   * Canonicalize the given URI.
255   * 
256   * This is filesystem-dependent, but may for example consist of
257   * canonicalizing the hostname using DNS and adding the default
258   * port if not specified.
259   * 
260   * The default implementation simply fills in the default port if
261   * not specified and if the filesystem has a default port.
262   *
263   * @return URI
264   * @see NetUtils#getCanonicalUri(URI, int)
265   */
266  protected URI canonicalizeUri(URI uri) {
267    if (uri.getPort() == -1 && getDefaultPort() > 0) {
268      // reconstruct the uri with the default port set
269      try {
270        uri = new URI(uri.getScheme(), uri.getUserInfo(),
271            uri.getHost(), getDefaultPort(),
272            uri.getPath(), uri.getQuery(), uri.getFragment());
273      } catch (URISyntaxException e) {
274        // Should never happen!
275        throw new AssertionError("Valid URI became unparseable: " +
276            uri);
277      }
278    }
279    
280    return uri;
281  }
282  
283  /**
284   * Get the default port for this file system.
285   * @return the default port or 0 if there isn't one
286   */
287  protected int getDefaultPort() {
288    return 0;
289  }
290
291  protected static FileSystem getFSofPath(final Path absOrFqPath,
292      final Configuration conf)
293      throws UnsupportedFileSystemException, IOException {
294    absOrFqPath.checkNotSchemeWithRelative();
295    absOrFqPath.checkNotRelative();
296
297    // Uses the default file system if not fully qualified
298    return get(absOrFqPath.toUri(), conf);
299  }
300
301  /**
302   * Get a canonical service name for this file system.  The token cache is
303   * the only user of the canonical service name, and uses it to lookup this
304   * filesystem's service tokens.
305   * If file system provides a token of its own then it must have a canonical
306   * name, otherwise canonical name can be null.
307   * 
308   * Default Impl: If the file system has child file systems 
309   * (such as an embedded file system) then it is assumed that the fs has no
310   * tokens of its own and hence returns a null name; otherwise a service
311   * name is built using Uri and port.
312   * 
313   * @return a service string that uniquely identifies this file system, null
314   *         if the filesystem does not implement tokens
315   * @see SecurityUtil#buildDTServiceName(URI, int)
316   */
317  @InterfaceAudience.Public
318  @InterfaceStability.Evolving
319  public String getCanonicalServiceName() {
320    return (getChildFileSystems() == null)
321      ? SecurityUtil.buildDTServiceName(getUri(), getDefaultPort())
322      : null;
323  }
324
325  /** @deprecated call #getUri() instead.*/
326  @Deprecated
327  public String getName() { return getUri().toString(); }
328
329  /** @deprecated call #get(URI,Configuration) instead. */
330  @Deprecated
331  public static FileSystem getNamed(String name, Configuration conf)
332    throws IOException {
333    return get(URI.create(fixName(name)), conf);
334  }
335  
336  /** Update old-format filesystem names, for back-compatibility.  This should
337   * eventually be replaced with a checkName() method that throws an exception
338   * for old-format names. */ 
339  private static String fixName(String name) {
340    // convert old-format name to new-format name
341    if (name.equals("local")) {         // "local" is now "file:///".
342      LOG.warn("\"local\" is a deprecated filesystem name."
343               +" Use \"file:///\" instead.");
344      name = "file:///";
345    } else if (name.indexOf('/')==-1) {   // unqualified is "hdfs://"
346      LOG.warn("\""+name+"\" is a deprecated filesystem name."
347               +" Use \"hdfs://"+name+"/\" instead.");
348      name = "hdfs://"+name;
349    }
350    return name;
351  }
352
353  /**
354   * Get the local file system.
355   * @param conf the configuration to configure the file system with
356   * @return a LocalFileSystem
357   */
358  public static LocalFileSystem getLocal(Configuration conf)
359    throws IOException {
360    return (LocalFileSystem)get(LocalFileSystem.NAME, conf);
361  }
362
363  /** Returns the FileSystem for this URI's scheme and authority.  The scheme
364   * of the URI determines a configuration property name,
365   * <tt>fs.<i>scheme</i>.class</tt> whose value names the FileSystem class.
366   * The entire URI is passed to the FileSystem instance's initialize method.
367   */
368  public static FileSystem get(URI uri, Configuration conf) throws IOException {
369    String scheme = uri.getScheme();
370    String authority = uri.getAuthority();
371
372    if (scheme == null && authority == null) {     // use default FS
373      return get(conf);
374    }
375
376    if (scheme != null && authority == null) {     // no authority
377      URI defaultUri = getDefaultUri(conf);
378      if (scheme.equals(defaultUri.getScheme())    // if scheme matches default
379          && defaultUri.getAuthority() != null) {  // & default has authority
380        return get(defaultUri, conf);              // return default
381      }
382    }
383    
384    String disableCacheName = String.format("fs.%s.impl.disable.cache", scheme);
385    if (conf.getBoolean(disableCacheName, false)) {
386      return createFileSystem(uri, conf);
387    }
388
389    return CACHE.get(uri, conf);
390  }
391
392  /**
393   * Returns the FileSystem for this URI's scheme and authority and the 
394   * passed user. Internally invokes {@link #newInstance(URI, Configuration)}
395   * @param uri of the filesystem
396   * @param conf the configuration to use
397   * @param user to perform the get as
398   * @return filesystem instance
399   * @throws IOException
400   * @throws InterruptedException
401   */
402  public static FileSystem newInstance(final URI uri, final Configuration conf,
403      final String user) throws IOException, InterruptedException {
404    String ticketCachePath =
405      conf.get(CommonConfigurationKeys.KERBEROS_TICKET_CACHE_PATH);
406    UserGroupInformation ugi =
407        UserGroupInformation.getBestUGI(ticketCachePath, user);
408    return ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
409      @Override
410      public FileSystem run() throws IOException {
411        return newInstance(uri,conf); 
412      }
413    });
414  }
415  /** Returns the FileSystem for this URI's scheme and authority.  The scheme
416   * of the URI determines a configuration property name,
417   * <tt>fs.<i>scheme</i>.class</tt> whose value names the FileSystem class.
418   * The entire URI is passed to the FileSystem instance's initialize method.
419   * This always returns a new FileSystem object.
420   */
421  public static FileSystem newInstance(URI uri, Configuration conf) throws IOException {
422    String scheme = uri.getScheme();
423    String authority = uri.getAuthority();
424
425    if (scheme == null) {                       // no scheme: use default FS
426      return newInstance(conf);
427    }
428
429    if (authority == null) {                       // no authority
430      URI defaultUri = getDefaultUri(conf);
431      if (scheme.equals(defaultUri.getScheme())    // if scheme matches default
432          && defaultUri.getAuthority() != null) {  // & default has authority
433        return newInstance(defaultUri, conf);              // return default
434      }
435    }
436    return CACHE.getUnique(uri, conf);
437  }
438
439  /** Returns a unique configured filesystem implementation.
440   * This always returns a new FileSystem object.
441   * @param conf the configuration to use
442   */
443  public static FileSystem newInstance(Configuration conf) throws IOException {
444    return newInstance(getDefaultUri(conf), conf);
445  }
446
447  /**
448   * Get a unique local file system object
449   * @param conf the configuration to configure the file system with
450   * @return a LocalFileSystem
451   * This always returns a new FileSystem object.
452   */
453  public static LocalFileSystem newInstanceLocal(Configuration conf)
454    throws IOException {
455    return (LocalFileSystem)newInstance(LocalFileSystem.NAME, conf);
456  }
457
458  /**
459   * Close all cached filesystems. Be sure those filesystems are not
460   * used anymore.
461   * 
462   * @throws IOException
463   */
464  public static void closeAll() throws IOException {
465    CACHE.closeAll();
466  }
467
468  /**
469   * Close all cached filesystems for a given UGI. Be sure those filesystems 
470   * are not used anymore.
471   * @param ugi user group info to close
472   * @throws IOException
473   */
474  public static void closeAllForUGI(UserGroupInformation ugi) 
475  throws IOException {
476    CACHE.closeAll(ugi);
477  }
478
479  /** 
480   * Make sure that a path specifies a FileSystem.
481   * @param path to use
482   */
483  public Path makeQualified(Path path) {
484    checkPath(path);
485    return path.makeQualified(this.getUri(), this.getWorkingDirectory());
486  }
487    
488  /**
489   * Get a new delegation token for this file system.
490   * This is an internal method that should have been declared protected
491   * but wasn't historically.
492   * Callers should use {@link #addDelegationTokens(String, Credentials)}
493   * 
494   * @param renewer the account name that is allowed to renew the token.
495   * @return a new delegation token
496   * @throws IOException
497   */
498  @InterfaceAudience.Private()
499  public Token<?> getDelegationToken(String renewer) throws IOException {
500    return null;
501  }
502  
503  /**
504   * Obtain all delegation tokens used by this FileSystem that are not
505   * already present in the given Credentials.  Existing tokens will neither
506   * be verified as valid nor having the given renewer.  Missing tokens will
507   * be acquired and added to the given Credentials.
508   * 
509   * Default Impl: works for simple fs with its own token
510   * and also for an embedded fs whose tokens are those of its
511   * children file system (i.e. the embedded fs has not tokens of its
512   * own).
513   * 
514   * @param renewer the user allowed to renew the delegation tokens
515   * @param credentials cache in which to add new delegation tokens
516   * @return list of new delegation tokens
517   * @throws IOException
518   */
519  @InterfaceAudience.Public
520  @InterfaceStability.Evolving
521  public Token<?>[] addDelegationTokens(
522      final String renewer, Credentials credentials) throws IOException {
523    if (credentials == null) {
524      credentials = new Credentials();
525    }
526    final List<Token<?>> tokens = new ArrayList<Token<?>>();
527    collectDelegationTokens(renewer, credentials, tokens);
528    return tokens.toArray(new Token<?>[tokens.size()]);
529  }
530  
531  /**
532   * Recursively obtain the tokens for this FileSystem and all descended
533   * FileSystems as determined by getChildFileSystems().
534   * @param renewer the user allowed to renew the delegation tokens
535   * @param credentials cache in which to add the new delegation tokens
536   * @param tokens list in which to add acquired tokens
537   * @throws IOException
538   */
539  private void collectDelegationTokens(final String renewer,
540                                       final Credentials credentials,
541                                       final List<Token<?>> tokens)
542                                           throws IOException {
543    final String serviceName = getCanonicalServiceName();
544    // Collect token of the this filesystem and then of its embedded children
545    if (serviceName != null) { // fs has token, grab it
546      final Text service = new Text(serviceName);
547      Token<?> token = credentials.getToken(service);
548      if (token == null) {
549        token = getDelegationToken(renewer);
550        if (token != null) {
551          tokens.add(token);
552          credentials.addToken(service, token);
553        }
554      }
555    }
556    // Now collect the tokens from the children
557    final FileSystem[] children = getChildFileSystems();
558    if (children != null) {
559      for (final FileSystem fs : children) {
560        fs.collectDelegationTokens(renewer, credentials, tokens);
561      }
562    }
563  }
564
565  /**
566   * Get all the immediate child FileSystems embedded in this FileSystem.
567   * It does not recurse and get grand children.  If a FileSystem
568   * has multiple child FileSystems, then it should return a unique list
569   * of those FileSystems.  Default is to return null to signify no children.
570   * 
571   * @return FileSystems used by this FileSystem
572   */
573  @InterfaceAudience.LimitedPrivate({ "HDFS" })
574  @VisibleForTesting
575  public FileSystem[] getChildFileSystems() {
576    return null;
577  }
578  
579  /** create a file with the provided permission
580   * The permission of the file is set to be the provided permission as in
581   * setPermission, not permission&~umask
582   * 
583   * It is implemented using two RPCs. It is understood that it is inefficient,
584   * but the implementation is thread-safe. The other option is to change the
585   * value of umask in configuration to be 0, but it is not thread-safe.
586   * 
587   * @param fs file system handle
588   * @param file the name of the file to be created
589   * @param permission the permission of the file
590   * @return an output stream
591   * @throws IOException
592   */
593  public static FSDataOutputStream create(FileSystem fs,
594      Path file, FsPermission permission) throws IOException {
595    // create the file with default permission
596    FSDataOutputStream out = fs.create(file);
597    // set its permission to the supplied one
598    fs.setPermission(file, permission);
599    return out;
600  }
601
602  /** create a directory with the provided permission
603   * The permission of the directory is set to be the provided permission as in
604   * setPermission, not permission&~umask
605   * 
606   * @see #create(FileSystem, Path, FsPermission)
607   * 
608   * @param fs file system handle
609   * @param dir the name of the directory to be created
610   * @param permission the permission of the directory
611   * @return true if the directory creation succeeds; false otherwise
612   * @throws IOException
613   */
614  public static boolean mkdirs(FileSystem fs, Path dir, FsPermission permission)
615  throws IOException {
616    // create the directory using the default permission
617    boolean result = fs.mkdirs(dir);
618    // set its permission to be the supplied one
619    fs.setPermission(dir, permission);
620    return result;
621  }
622
623  ///////////////////////////////////////////////////////////////
624  // FileSystem
625  ///////////////////////////////////////////////////////////////
626
627  protected FileSystem() {
628    super(null);
629  }
630
631  /** 
632   * Check that a Path belongs to this FileSystem.
633   * @param path to check
634   */
635  protected void checkPath(Path path) {
636    URI uri = path.toUri();
637    String thatScheme = uri.getScheme();
638    if (thatScheme == null)                // fs is relative
639      return;
640    URI thisUri = getCanonicalUri();
641    String thisScheme = thisUri.getScheme();
642    //authority and scheme are not case sensitive
643    if (thisScheme.equalsIgnoreCase(thatScheme)) {// schemes match
644      String thisAuthority = thisUri.getAuthority();
645      String thatAuthority = uri.getAuthority();
646      if (thatAuthority == null &&                // path's authority is null
647          thisAuthority != null) {                // fs has an authority
648        URI defaultUri = getDefaultUri(getConf());
649        if (thisScheme.equalsIgnoreCase(defaultUri.getScheme())) {
650          uri = defaultUri; // schemes match, so use this uri instead
651        } else {
652          uri = null; // can't determine auth of the path
653        }
654      }
655      if (uri != null) {
656        // canonicalize uri before comparing with this fs
657        uri = canonicalizeUri(uri);
658        thatAuthority = uri.getAuthority();
659        if (thisAuthority == thatAuthority ||       // authorities match
660            (thisAuthority != null &&
661             thisAuthority.equalsIgnoreCase(thatAuthority)))
662          return;
663      }
664    }
665    throw new IllegalArgumentException("Wrong FS: "+path+
666                                       ", expected: "+this.getUri());
667  }
668
669  /**
670   * Return an array containing hostnames, offset and size of 
671   * portions of the given file.  For a nonexistent 
672   * file or regions, null will be returned.
673   *
674   * This call is most helpful with DFS, where it returns 
675   * hostnames of machines that contain the given file.
676   *
677   * The FileSystem will simply return an elt containing 'localhost'.
678   *
679   * @param file FilesStatus to get data from
680   * @param start offset into the given file
681   * @param len length for which to get locations for
682   */
683  public BlockLocation[] getFileBlockLocations(FileStatus file, 
684      long start, long len) throws IOException {
685    if (file == null) {
686      return null;
687    }
688
689    if (start < 0 || len < 0) {
690      throw new IllegalArgumentException("Invalid start or len parameter");
691    }
692
693    if (file.getLen() <= start) {
694      return new BlockLocation[0];
695
696    }
697    String[] name = { "localhost:50010" };
698    String[] host = { "localhost" };
699    return new BlockLocation[] {
700      new BlockLocation(name, host, 0, file.getLen()) };
701  }
702 
703
704  /**
705   * Return an array containing hostnames, offset and size of 
706   * portions of the given file.  For a nonexistent 
707   * file or regions, null will be returned.
708   *
709   * This call is most helpful with DFS, where it returns 
710   * hostnames of machines that contain the given file.
711   *
712   * The FileSystem will simply return an elt containing 'localhost'.
713   *
714   * @param p path is used to identify an FS since an FS could have
715   *          another FS that it could be delegating the call to
716   * @param start offset into the given file
717   * @param len length for which to get locations for
718   */
719  public BlockLocation[] getFileBlockLocations(Path p, 
720      long start, long len) throws IOException {
721    if (p == null) {
722      throw new NullPointerException();
723    }
724    FileStatus file = getFileStatus(p);
725    return getFileBlockLocations(file, start, len);
726  }
727  
728  /**
729   * Return a set of server default configuration values
730   * @return server default configuration values
731   * @throws IOException
732   * @deprecated use {@link #getServerDefaults(Path)} instead
733   */
734  @Deprecated
735  public FsServerDefaults getServerDefaults() throws IOException {
736    Configuration conf = getConf();
737    // CRC32 is chosen as default as it is available in all 
738    // releases that support checksum.
739    // The client trash configuration is ignored.
740    return new FsServerDefaults(getDefaultBlockSize(), 
741        conf.getInt("io.bytes.per.checksum", 512), 
742        64 * 1024, 
743        getDefaultReplication(),
744        conf.getInt("io.file.buffer.size", 4096),
745        false,
746        CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT,
747        DataChecksum.Type.CRC32);
748  }
749
750  /**
751   * Return a set of server default configuration values
752   * @param p path is used to identify an FS since an FS could have
753   *          another FS that it could be delegating the call to
754   * @return server default configuration values
755   * @throws IOException
756   */
757  public FsServerDefaults getServerDefaults(Path p) throws IOException {
758    return getServerDefaults();
759  }
760
761  /**
762   * Return the fully-qualified path of path f resolving the path
763   * through any symlinks or mount point
764   * @param p path to be resolved
765   * @return fully qualified path 
766   * @throws FileNotFoundException
767   */
768   public Path resolvePath(final Path p) throws IOException {
769     checkPath(p);
770     return getFileStatus(p).getPath();
771   }
772
773  /**
774   * Opens an FSDataInputStream at the indicated Path.
775   * @param f the file name to open
776   * @param bufferSize the size of the buffer to be used.
777   */
778  public abstract FSDataInputStream open(Path f, int bufferSize)
779    throws IOException;
780    
781  /**
782   * Opens an FSDataInputStream at the indicated Path.
783   * @param f the file to open
784   */
785  public FSDataInputStream open(Path f) throws IOException {
786    return open(f, getConf().getInt("io.file.buffer.size", 4096));
787  }
788
789  /**
790   * Create an FSDataOutputStream at the indicated Path.
791   * Files are overwritten by default.
792   * @param f the file to create
793   */
794  public FSDataOutputStream create(Path f) throws IOException {
795    return create(f, true);
796  }
797
798  /**
799   * Create an FSDataOutputStream at the indicated Path.
800   * @param f the file to create
801   * @param overwrite if a file with this name already exists, then if true,
802   *   the file will be overwritten, and if false an exception will be thrown.
803   */
804  public FSDataOutputStream create(Path f, boolean overwrite)
805      throws IOException {
806    return create(f, overwrite, 
807                  getConf().getInt("io.file.buffer.size", 4096),
808                  getDefaultReplication(f),
809                  getDefaultBlockSize(f));
810  }
811
812  /**
813   * Create an FSDataOutputStream at the indicated Path with write-progress
814   * reporting.
815   * Files are overwritten by default.
816   * @param f the file to create
817   * @param progress to report progress
818   */
819  public FSDataOutputStream create(Path f, Progressable progress) 
820      throws IOException {
821    return create(f, true, 
822                  getConf().getInt("io.file.buffer.size", 4096),
823                  getDefaultReplication(f),
824                  getDefaultBlockSize(f), progress);
825  }
826
827  /**
828   * Create an FSDataOutputStream at the indicated Path.
829   * Files are overwritten by default.
830   * @param f the file to create
831   * @param replication the replication factor
832   */
833  public FSDataOutputStream create(Path f, short replication)
834      throws IOException {
835    return create(f, true, 
836                  getConf().getInt("io.file.buffer.size", 4096),
837                  replication,
838                  getDefaultBlockSize(f));
839  }
840
841  /**
842   * Create an FSDataOutputStream at the indicated Path with write-progress
843   * reporting.
844   * Files are overwritten by default.
845   * @param f the file to create
846   * @param replication the replication factor
847   * @param progress to report progress
848   */
849  public FSDataOutputStream create(Path f, short replication, 
850      Progressable progress) throws IOException {
851    return create(f, true, 
852                  getConf().getInt(
853                      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
854                      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT),
855                  replication,
856                  getDefaultBlockSize(f), progress);
857  }
858
859    
860  /**
861   * Create an FSDataOutputStream at the indicated Path.
862   * @param f the file name to create
863   * @param overwrite if a file with this name already exists, then if true,
864   *   the file will be overwritten, and if false an error will be thrown.
865   * @param bufferSize the size of the buffer to be used.
866   */
867  public FSDataOutputStream create(Path f, 
868                                   boolean overwrite,
869                                   int bufferSize
870                                   ) throws IOException {
871    return create(f, overwrite, bufferSize, 
872                  getDefaultReplication(f),
873                  getDefaultBlockSize(f));
874  }
875    
876  /**
877   * Create an FSDataOutputStream at the indicated Path with write-progress
878   * reporting.
879   * @param f the path of the file to open
880   * @param overwrite if a file with this name already exists, then if true,
881   *   the file will be overwritten, and if false an error will be thrown.
882   * @param bufferSize the size of the buffer to be used.
883   */
884  public FSDataOutputStream create(Path f, 
885                                   boolean overwrite,
886                                   int bufferSize,
887                                   Progressable progress
888                                   ) throws IOException {
889    return create(f, overwrite, bufferSize, 
890                  getDefaultReplication(f),
891                  getDefaultBlockSize(f), progress);
892  }
893    
894    
895  /**
896   * Create an FSDataOutputStream at the indicated Path.
897   * @param f the file name to open
898   * @param overwrite if a file with this name already exists, then if true,
899   *   the file will be overwritten, and if false an error will be thrown.
900   * @param bufferSize the size of the buffer to be used.
901   * @param replication required block replication for the file. 
902   */
903  public FSDataOutputStream create(Path f, 
904                                   boolean overwrite,
905                                   int bufferSize,
906                                   short replication,
907                                   long blockSize
908                                   ) throws IOException {
909    return create(f, overwrite, bufferSize, replication, blockSize, null);
910  }
911
912  /**
913   * Create an FSDataOutputStream at the indicated Path with write-progress
914   * reporting.
915   * @param f the file name to open
916   * @param overwrite if a file with this name already exists, then if true,
917   *   the file will be overwritten, and if false an error will be thrown.
918   * @param bufferSize the size of the buffer to be used.
919   * @param replication required block replication for the file. 
920   */
921  public FSDataOutputStream create(Path f,
922                                            boolean overwrite,
923                                            int bufferSize,
924                                            short replication,
925                                            long blockSize,
926                                            Progressable progress
927                                            ) throws IOException {
928    return this.create(f, FsPermission.getFileDefault().applyUMask(
929        FsPermission.getUMask(getConf())), overwrite, bufferSize,
930        replication, blockSize, progress);
931  }
932
933  /**
934   * Create an FSDataOutputStream at the indicated Path with write-progress
935   * reporting.
936   * @param f the file name to open
937   * @param permission
938   * @param overwrite if a file with this name already exists, then if true,
939   *   the file will be overwritten, and if false an error will be thrown.
940   * @param bufferSize the size of the buffer to be used.
941   * @param replication required block replication for the file.
942   * @param blockSize
943   * @param progress
944   * @throws IOException
945   * @see #setPermission(Path, FsPermission)
946   */
947  public abstract FSDataOutputStream create(Path f,
948      FsPermission permission,
949      boolean overwrite,
950      int bufferSize,
951      short replication,
952      long blockSize,
953      Progressable progress) throws IOException;
954  
955  /**
956   * Create an FSDataOutputStream at the indicated Path with write-progress
957   * reporting.
958   * @param f the file name to open
959   * @param permission
960   * @param flags {@link CreateFlag}s to use for this stream.
961   * @param bufferSize the size of the buffer to be used.
962   * @param replication required block replication for the file.
963   * @param blockSize
964   * @param progress
965   * @throws IOException
966   * @see #setPermission(Path, FsPermission)
967   */
968  public FSDataOutputStream create(Path f,
969      FsPermission permission,
970      EnumSet<CreateFlag> flags,
971      int bufferSize,
972      short replication,
973      long blockSize,
974      Progressable progress) throws IOException {
975    return create(f, permission, flags, bufferSize, replication,
976        blockSize, progress, null);
977  }
978  
979  /**
980   * Create an FSDataOutputStream at the indicated Path with a custom
981   * checksum option
982   * @param f the file name to open
983   * @param permission
984   * @param flags {@link CreateFlag}s to use for this stream.
985   * @param bufferSize the size of the buffer to be used.
986   * @param replication required block replication for the file.
987   * @param blockSize
988   * @param progress
989   * @param checksumOpt checksum parameter. If null, the values
990   *        found in conf will be used.
991   * @throws IOException
992   * @see #setPermission(Path, FsPermission)
993   */
994  public FSDataOutputStream create(Path f,
995      FsPermission permission,
996      EnumSet<CreateFlag> flags,
997      int bufferSize,
998      short replication,
999      long blockSize,
1000      Progressable progress,
1001      ChecksumOpt checksumOpt) throws IOException {
1002    // Checksum options are ignored by default. The file systems that
1003    // implement checksum need to override this method. The full
1004    // support is currently only available in DFS.
1005    return create(f, permission, flags.contains(CreateFlag.OVERWRITE), 
1006        bufferSize, replication, blockSize, progress);
1007  }
1008
1009  /*.
1010   * This create has been added to support the FileContext that processes
1011   * the permission
1012   * with umask before calling this method.
1013   * This a temporary method added to support the transition from FileSystem
1014   * to FileContext for user applications.
1015   */
1016  @Deprecated
1017  protected FSDataOutputStream primitiveCreate(Path f,
1018     FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
1019     short replication, long blockSize, Progressable progress,
1020     ChecksumOpt checksumOpt) throws IOException {
1021
1022    boolean pathExists = exists(f);
1023    CreateFlag.validate(f, pathExists, flag);
1024    
1025    // Default impl  assumes that permissions do not matter and 
1026    // nor does the bytesPerChecksum  hence
1027    // calling the regular create is good enough.
1028    // FSs that implement permissions should override this.
1029
1030    if (pathExists && flag.contains(CreateFlag.APPEND)) {
1031      return append(f, bufferSize, progress);
1032    }
1033    
1034    return this.create(f, absolutePermission,
1035        flag.contains(CreateFlag.OVERWRITE), bufferSize, replication,
1036        blockSize, progress);
1037  }
1038  
1039  /**
1040   * This version of the mkdirs method assumes that the permission is absolute.
1041   * It has been added to support the FileContext that processes the permission
1042   * with umask before calling this method.
1043   * This a temporary method added to support the transition from FileSystem
1044   * to FileContext for user applications.
1045   */
1046  @Deprecated
1047  protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
1048    throws IOException {
1049    // Default impl is to assume that permissions do not matter and hence
1050    // calling the regular mkdirs is good enough.
1051    // FSs that implement permissions should override this.
1052   return this.mkdirs(f, absolutePermission);
1053  }
1054
1055
1056  /**
1057   * This version of the mkdirs method assumes that the permission is absolute.
1058   * It has been added to support the FileContext that processes the permission
1059   * with umask before calling this method.
1060   * This a temporary method added to support the transition from FileSystem
1061   * to FileContext for user applications.
1062   */
1063  @Deprecated
1064  protected void primitiveMkdir(Path f, FsPermission absolutePermission, 
1065                    boolean createParent)
1066    throws IOException {
1067    
1068    if (!createParent) { // parent must exist.
1069      // since the this.mkdirs makes parent dirs automatically
1070      // we must throw exception if parent does not exist.
1071      final FileStatus stat = getFileStatus(f.getParent());
1072      if (stat == null) {
1073        throw new FileNotFoundException("Missing parent:" + f);
1074      }
1075      if (!stat.isDirectory()) {
1076        throw new ParentNotDirectoryException("parent is not a dir");
1077      }
1078      // parent does exist - go ahead with mkdir of leaf
1079    }
1080    // Default impl is to assume that permissions do not matter and hence
1081    // calling the regular mkdirs is good enough.
1082    // FSs that implement permissions should override this.
1083    if (!this.mkdirs(f, absolutePermission)) {
1084      throw new IOException("mkdir of "+ f + " failed");
1085    }
1086  }
1087
1088  /**
1089   * Opens an FSDataOutputStream at the indicated Path with write-progress
1090   * reporting. Same as create(), except fails if parent directory doesn't
1091   * already exist.
1092   * @param f the file name to open
1093   * @param overwrite if a file with this name already exists, then if true,
1094   * the file will be overwritten, and if false an error will be thrown.
1095   * @param bufferSize the size of the buffer to be used.
1096   * @param replication required block replication for the file.
1097   * @param blockSize
1098   * @param progress
1099   * @throws IOException
1100   * @see #setPermission(Path, FsPermission)
1101   */
1102  public FSDataOutputStream createNonRecursive(Path f,
1103      boolean overwrite,
1104      int bufferSize, short replication, long blockSize,
1105      Progressable progress) throws IOException {
1106    return this.createNonRecursive(f, FsPermission.getFileDefault(),
1107        overwrite, bufferSize, replication, blockSize, progress);
1108  }
1109
1110  /**
1111   * Opens an FSDataOutputStream at the indicated Path with write-progress
1112   * reporting. Same as create(), except fails if parent directory doesn't
1113   * already exist.
1114   * @param f the file name to open
1115   * @param permission
1116   * @param overwrite if a file with this name already exists, then if true,
1117   * the file will be overwritten, and if false an error will be thrown.
1118   * @param bufferSize the size of the buffer to be used.
1119   * @param replication required block replication for the file.
1120   * @param blockSize
1121   * @param progress
1122   * @throws IOException
1123   * @see #setPermission(Path, FsPermission)
1124   */
1125   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
1126       boolean overwrite, int bufferSize, short replication, long blockSize,
1127       Progressable progress) throws IOException {
1128     return createNonRecursive(f, permission,
1129         overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
1130             : EnumSet.of(CreateFlag.CREATE), bufferSize,
1131             replication, blockSize, progress);
1132   }
1133
1134   /**
1135    * Opens an FSDataOutputStream at the indicated Path with write-progress
1136    * reporting. Same as create(), except fails if parent directory doesn't
1137    * already exist.
1138    * @param f the file name to open
1139    * @param permission
1140    * @param flags {@link CreateFlag}s to use for this stream.
1141    * @param bufferSize the size of the buffer to be used.
1142    * @param replication required block replication for the file.
1143    * @param blockSize
1144    * @param progress
1145    * @throws IOException
1146    * @see #setPermission(Path, FsPermission)
1147    */
1148    public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
1149        EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
1150        Progressable progress) throws IOException {
1151      throw new IOException("createNonRecursive unsupported for this filesystem "
1152          + this.getClass());
1153    }
1154
1155  /**
1156   * Creates the given Path as a brand-new zero-length file.  If
1157   * create fails, or if it already existed, return false.
1158   *
1159   * @param f path to use for create
1160   */
1161  public boolean createNewFile(Path f) throws IOException {
1162    if (exists(f)) {
1163      return false;
1164    } else {
1165      create(f, false, getConf().getInt("io.file.buffer.size", 4096)).close();
1166      return true;
1167    }
1168  }
1169
1170  /**
1171   * Append to an existing file (optional operation).
1172   * Same as append(f, getConf().getInt("io.file.buffer.size", 4096), null)
1173   * @param f the existing file to be appended.
1174   * @throws IOException
1175   */
1176  public FSDataOutputStream append(Path f) throws IOException {
1177    return append(f, getConf().getInt("io.file.buffer.size", 4096), null);
1178  }
1179  /**
1180   * Append to an existing file (optional operation).
1181   * Same as append(f, bufferSize, null).
1182   * @param f the existing file to be appended.
1183   * @param bufferSize the size of the buffer to be used.
1184   * @throws IOException
1185   */
1186  public FSDataOutputStream append(Path f, int bufferSize) throws IOException {
1187    return append(f, bufferSize, null);
1188  }
1189
1190  /**
1191   * Append to an existing file (optional operation).
1192   * @param f the existing file to be appended.
1193   * @param bufferSize the size of the buffer to be used.
1194   * @param progress for reporting progress if it is not null.
1195   * @throws IOException
1196   */
1197  public abstract FSDataOutputStream append(Path f, int bufferSize,
1198      Progressable progress) throws IOException;
1199
1200  /**
1201   * Concat existing files together.
1202   * @param trg the path to the target destination.
1203   * @param psrcs the paths to the sources to use for the concatenation.
1204   * @throws IOException
1205   */
1206  public void concat(final Path trg, final Path [] psrcs) throws IOException {
1207    throw new UnsupportedOperationException("Not implemented by the " + 
1208        getClass().getSimpleName() + " FileSystem implementation");
1209  }
1210
1211 /**
1212   * Get replication.
1213   * 
1214   * @deprecated Use getFileStatus() instead
1215   * @param src file name
1216   * @return file replication
1217   * @throws IOException
1218   */ 
1219  @Deprecated
1220  public short getReplication(Path src) throws IOException {
1221    return getFileStatus(src).getReplication();
1222  }
1223
1224  /**
1225   * Set replication for an existing file.
1226   * 
1227   * @param src file name
1228   * @param replication new replication
1229   * @throws IOException
1230   * @return true if successful;
1231   *         false if file does not exist or is a directory
1232   */
1233  public boolean setReplication(Path src, short replication)
1234    throws IOException {
1235    return true;
1236  }
1237
1238  /**
1239   * Renames Path src to Path dst.  Can take place on local fs
1240   * or remote DFS.
1241   * @param src path to be renamed
1242   * @param dst new path after rename
1243   * @throws IOException on failure
1244   * @return true if rename is successful
1245   */
1246  public abstract boolean rename(Path src, Path dst) throws IOException;
1247
1248  /**
1249   * Renames Path src to Path dst
1250   * <ul>
1251   * <li>Fails if src is a file and dst is a directory.
1252   * <li>Fails if src is a directory and dst is a file.
1253   * <li>Fails if the parent of dst does not exist or is a file.
1254   * </ul>
1255   * <p>
1256   * If OVERWRITE option is not passed as an argument, rename fails
1257   * if the dst already exists.
1258   * <p>
1259   * If OVERWRITE option is passed as an argument, rename overwrites
1260   * the dst if it is a file or an empty directory. Rename fails if dst is
1261   * a non-empty directory.
1262   * <p>
1263   * Note that atomicity of rename is dependent on the file system
1264   * implementation. Please refer to the file system documentation for
1265   * details. This default implementation is non atomic.
1266   * <p>
1267   * This method is deprecated since it is a temporary method added to 
1268   * support the transition from FileSystem to FileContext for user 
1269   * applications.
1270   * 
1271   * @param src path to be renamed
1272   * @param dst new path after rename
1273   * @throws IOException on failure
1274   */
1275  @Deprecated
1276  protected void rename(final Path src, final Path dst,
1277      final Rename... options) throws IOException {
1278    // Default implementation
1279    final FileStatus srcStatus = getFileLinkStatus(src);
1280    if (srcStatus == null) {
1281      throw new FileNotFoundException("rename source " + src + " not found.");
1282    }
1283
1284    boolean overwrite = false;
1285    if (null != options) {
1286      for (Rename option : options) {
1287        if (option == Rename.OVERWRITE) {
1288          overwrite = true;
1289        }
1290      }
1291    }
1292
1293    FileStatus dstStatus;
1294    try {
1295      dstStatus = getFileLinkStatus(dst);
1296    } catch (IOException e) {
1297      dstStatus = null;
1298    }
1299    if (dstStatus != null) {
1300      if (srcStatus.isDirectory() != dstStatus.isDirectory()) {
1301        throw new IOException("Source " + src + " Destination " + dst
1302            + " both should be either file or directory");
1303      }
1304      if (!overwrite) {
1305        throw new FileAlreadyExistsException("rename destination " + dst
1306            + " already exists.");
1307      }
1308      // Delete the destination that is a file or an empty directory
1309      if (dstStatus.isDirectory()) {
1310        FileStatus[] list = listStatus(dst);
1311        if (list != null && list.length != 0) {
1312          throw new IOException(
1313              "rename cannot overwrite non empty destination directory " + dst);
1314        }
1315      }
1316      delete(dst, false);
1317    } else {
1318      final Path parent = dst.getParent();
1319      final FileStatus parentStatus = getFileStatus(parent);
1320      if (parentStatus == null) {
1321        throw new FileNotFoundException("rename destination parent " + parent
1322            + " not found.");
1323      }
1324      if (!parentStatus.isDirectory()) {
1325        throw new ParentNotDirectoryException("rename destination parent " + parent
1326            + " is a file.");
1327      }
1328    }
1329    if (!rename(src, dst)) {
1330      throw new IOException("rename from " + src + " to " + dst + " failed.");
1331    }
1332  }
1333
1334  /**
1335   * Truncate the file in the indicated path to the indicated size.
1336   * <ul>
1337   * <li>Fails if path is a directory.
1338   * <li>Fails if path does not exist.
1339   * <li>Fails if path is not closed.
1340   * <li>Fails if new size is greater than current size.
1341   * </ul>
1342   * @param f The path to the file to be truncated
1343   * @param newLength The size the file is to be truncated to
1344   *
1345   * @return <code>true</code> if the file has been truncated to the desired
1346   * <code>newLength</code> and is immediately available to be reused for
1347   * write operations such as <code>append</code>, or
1348   * <code>false</code> if a background process of adjusting the length of
1349   * the last block has been started, and clients should wait for it to
1350   * complete before proceeding with further file updates.
1351   */
1352  public boolean truncate(Path f, long newLength) throws IOException {
1353    throw new UnsupportedOperationException("Not implemented by the " +
1354        getClass().getSimpleName() + " FileSystem implementation");
1355  }
1356  
1357  /**
1358   * Delete a file 
1359   * @deprecated Use {@link #delete(Path, boolean)} instead.
1360   */
1361  @Deprecated
1362  public boolean delete(Path f) throws IOException {
1363    return delete(f, true);
1364  }
1365  
1366  /** Delete a file.
1367   *
1368   * @param f the path to delete.
1369   * @param recursive if path is a directory and set to 
1370   * true, the directory is deleted else throws an exception. In
1371   * case of a file the recursive can be set to either true or false. 
1372   * @return  true if delete is successful else false. 
1373   * @throws IOException
1374   */
1375  public abstract boolean delete(Path f, boolean recursive) throws IOException;
1376
1377  /**
1378   * Mark a path to be deleted when FileSystem is closed.
1379   * When the JVM shuts down,
1380   * all FileSystem objects will be closed automatically.
1381   * Then,
1382   * the marked path will be deleted as a result of closing the FileSystem.
1383   *
1384   * The path has to exist in the file system.
1385   * 
1386   * @param f the path to delete.
1387   * @return  true if deleteOnExit is successful, otherwise false.
1388   * @throws IOException
1389   */
1390  public boolean deleteOnExit(Path f) throws IOException {
1391    if (!exists(f)) {
1392      return false;
1393    }
1394    synchronized (deleteOnExit) {
1395      deleteOnExit.add(f);
1396    }
1397    return true;
1398  }
1399  
1400  /**
1401   * Cancel the deletion of the path when the FileSystem is closed
1402   * @param f the path to cancel deletion
1403   */
1404  public boolean cancelDeleteOnExit(Path f) {
1405    synchronized (deleteOnExit) {
1406      return deleteOnExit.remove(f);
1407    }
1408  }
1409
1410  /**
1411   * Delete all files that were marked as delete-on-exit. This recursively
1412   * deletes all files in the specified paths.
1413   */
1414  protected void processDeleteOnExit() {
1415    synchronized (deleteOnExit) {
1416      for (Iterator<Path> iter = deleteOnExit.iterator(); iter.hasNext();) {
1417        Path path = iter.next();
1418        try {
1419          if (exists(path)) {
1420            delete(path, true);
1421          }
1422        }
1423        catch (IOException e) {
1424          LOG.info("Ignoring failure to deleteOnExit for path " + path);
1425        }
1426        iter.remove();
1427      }
1428    }
1429  }
1430  
1431  /** Check if exists.
1432   * @param f source file
1433   */
1434  public boolean exists(Path f) throws IOException {
1435    try {
1436      return getFileStatus(f) != null;
1437    } catch (FileNotFoundException e) {
1438      return false;
1439    }
1440  }
1441
1442  /** True iff the named path is a directory.
1443   * Note: Avoid using this method. Instead reuse the FileStatus 
1444   * returned by getFileStatus() or listStatus() methods.
1445   * @param f path to check
1446   */
1447  public boolean isDirectory(Path f) throws IOException {
1448    try {
1449      return getFileStatus(f).isDirectory();
1450    } catch (FileNotFoundException e) {
1451      return false;               // f does not exist
1452    }
1453  }
1454
1455  /** True iff the named path is a regular file.
1456   * Note: Avoid using this method. Instead reuse the FileStatus 
1457   * returned by getFileStatus() or listStatus() methods.
1458   * @param f path to check
1459   */
1460  public boolean isFile(Path f) throws IOException {
1461    try {
1462      return getFileStatus(f).isFile();
1463    } catch (FileNotFoundException e) {
1464      return false;               // f does not exist
1465    }
1466  }
1467  
1468  /** The number of bytes in a file. */
1469  /** @deprecated Use getFileStatus() instead */
1470  @Deprecated
1471  public long getLength(Path f) throws IOException {
1472    return getFileStatus(f).getLen();
1473  }
1474    
1475  /** Return the {@link ContentSummary} of a given {@link Path}.
1476  * @param f path to use
1477  */
1478  public ContentSummary getContentSummary(Path f) throws IOException {
1479    FileStatus status = getFileStatus(f);
1480    if (status.isFile()) {
1481      // f is a file
1482      long length = status.getLen();
1483      return new ContentSummary.Builder().length(length).
1484          fileCount(1).directoryCount(0).spaceConsumed(length).build();
1485    }
1486    // f is a directory
1487    long[] summary = {0, 0, 1};
1488    for(FileStatus s : listStatus(f)) {
1489      long length = s.getLen();
1490      ContentSummary c = s.isDirectory() ? getContentSummary(s.getPath()) :
1491          new ContentSummary.Builder().length(length).
1492          fileCount(1).directoryCount(0).spaceConsumed(length).build();
1493      summary[0] += c.getLength();
1494      summary[1] += c.getFileCount();
1495      summary[2] += c.getDirectoryCount();
1496    }
1497    return new ContentSummary.Builder().length(summary[0]).
1498        fileCount(summary[1]).directoryCount(summary[2]).
1499        spaceConsumed(summary[0]).build();
1500  }
1501
1502  /** Return the {@link QuotaUsage} of a given {@link Path}.
1503   * @param f path to use
1504   */
1505  public QuotaUsage getQuotaUsage(Path f) throws IOException {
1506    return getContentSummary(f);
1507  }
1508
1509  final private static PathFilter DEFAULT_FILTER = new PathFilter() {
1510      @Override
1511      public boolean accept(Path file) {
1512        return true;
1513      }     
1514    };
1515    
1516  /**
1517   * List the statuses of the files/directories in the given path if the path is
1518   * a directory.
1519   * <p>
1520   * Does not guarantee to return the List of files/directories status in a
1521   * sorted order.
1522   * @param f given path
1523   * @return the statuses of the files/directories in the given patch
1524   * @throws FileNotFoundException when the path does not exist;
1525   *         IOException see specific implementation
1526   */
1527  public abstract FileStatus[] listStatus(Path f) throws FileNotFoundException, 
1528                                                         IOException;
1529    
1530  /*
1531   * Filter files/directories in the given path using the user-supplied path
1532   * filter. Results are added to the given array <code>results</code>.
1533   */
1534  private void listStatus(ArrayList<FileStatus> results, Path f,
1535      PathFilter filter) throws FileNotFoundException, IOException {
1536    FileStatus listing[] = listStatus(f);
1537    if (listing == null) {
1538      throw new IOException("Error accessing " + f);
1539    }
1540
1541    for (int i = 0; i < listing.length; i++) {
1542      if (filter.accept(listing[i].getPath())) {
1543        results.add(listing[i]);
1544      }
1545    }
1546  }
1547
1548  /**
1549   * @return an iterator over the corrupt files under the given path
1550   * (may contain duplicates if a file has more than one corrupt block)
1551   * @throws IOException
1552   */
1553  public RemoteIterator<Path> listCorruptFileBlocks(Path path)
1554    throws IOException {
1555    throw new UnsupportedOperationException(getClass().getCanonicalName() +
1556                                            " does not support" +
1557                                            " listCorruptFileBlocks");
1558  }
1559
1560  /**
1561   * Filter files/directories in the given path using the user-supplied path
1562   * filter.
1563   * <p>
1564   * Does not guarantee to return the List of files/directories status in a
1565   * sorted order.
1566   * 
1567   * @param f
1568   *          a path name
1569   * @param filter
1570   *          the user-supplied path filter
1571   * @return an array of FileStatus objects for the files under the given path
1572   *         after applying the filter
1573   * @throws FileNotFoundException when the path does not exist;
1574   *         IOException see specific implementation   
1575   */
1576  public FileStatus[] listStatus(Path f, PathFilter filter) 
1577                                   throws FileNotFoundException, IOException {
1578    ArrayList<FileStatus> results = new ArrayList<FileStatus>();
1579    listStatus(results, f, filter);
1580    return results.toArray(new FileStatus[results.size()]);
1581  }
1582
1583  /**
1584   * Filter files/directories in the given list of paths using default
1585   * path filter.
1586   * <p>
1587   * Does not guarantee to return the List of files/directories status in a
1588   * sorted order.
1589   * 
1590   * @param files
1591   *          a list of paths
1592   * @return a list of statuses for the files under the given paths after
1593   *         applying the filter default Path filter
1594   * @throws FileNotFoundException when the path does not exist;
1595   *         IOException see specific implementation
1596   */
1597  public FileStatus[] listStatus(Path[] files)
1598      throws FileNotFoundException, IOException {
1599    return listStatus(files, DEFAULT_FILTER);
1600  }
1601
1602  /**
1603   * Filter files/directories in the given list of paths using user-supplied
1604   * path filter.
1605   * <p>
1606   * Does not guarantee to return the List of files/directories status in a
1607   * sorted order.
1608   * 
1609   * @param files
1610   *          a list of paths
1611   * @param filter
1612   *          the user-supplied path filter
1613   * @return a list of statuses for the files under the given paths after
1614   *         applying the filter
1615   * @throws FileNotFoundException when the path does not exist;
1616   *         IOException see specific implementation
1617   */
1618  public FileStatus[] listStatus(Path[] files, PathFilter filter)
1619      throws FileNotFoundException, IOException {
1620    ArrayList<FileStatus> results = new ArrayList<FileStatus>();
1621    for (int i = 0; i < files.length; i++) {
1622      listStatus(results, files[i], filter);
1623    }
1624    return results.toArray(new FileStatus[results.size()]);
1625  }
1626
1627  /**
1628   * <p>Return all the files that match filePattern and are not checksum
1629   * files. Results are sorted by their names.
1630   * 
1631   * <p>
1632   * A filename pattern is composed of <i>regular</i> characters and
1633   * <i>special pattern matching</i> characters, which are:
1634   *
1635   * <dl>
1636   *  <dd>
1637   *   <dl>
1638   *    <p>
1639   *    <dt> <tt> ? </tt>
1640   *    <dd> Matches any single character.
1641   *
1642   *    <p>
1643   *    <dt> <tt> * </tt>
1644   *    <dd> Matches zero or more characters.
1645   *
1646   *    <p>
1647   *    <dt> <tt> [<i>abc</i>] </tt>
1648   *    <dd> Matches a single character from character set
1649   *     <tt>{<i>a,b,c</i>}</tt>.
1650   *
1651   *    <p>
1652   *    <dt> <tt> [<i>a</i>-<i>b</i>] </tt>
1653   *    <dd> Matches a single character from the character range
1654   *     <tt>{<i>a...b</i>}</tt>.  Note that character <tt><i>a</i></tt> must be
1655   *     lexicographically less than or equal to character <tt><i>b</i></tt>.
1656   *
1657   *    <p>
1658   *    <dt> <tt> [^<i>a</i>] </tt>
1659   *    <dd> Matches a single character that is not from character set or range
1660   *     <tt>{<i>a</i>}</tt>.  Note that the <tt>^</tt> character must occur
1661   *     immediately to the right of the opening bracket.
1662   *
1663   *    <p>
1664   *    <dt> <tt> \<i>c</i> </tt>
1665   *    <dd> Removes (escapes) any special meaning of character <i>c</i>.
1666   *
1667   *    <p>
1668   *    <dt> <tt> {ab,cd} </tt>
1669   *    <dd> Matches a string from the string set <tt>{<i>ab, cd</i>} </tt>
1670   *    
1671   *    <p>
1672   *    <dt> <tt> {ab,c{de,fh}} </tt>
1673   *    <dd> Matches a string from the string set <tt>{<i>ab, cde, cfh</i>}</tt>
1674   *
1675   *   </dl>
1676   *  </dd>
1677   * </dl>
1678   *
1679   * @param pathPattern a regular expression specifying a pth pattern
1680
1681   * @return an array of paths that match the path pattern
1682   * @throws IOException
1683   */
1684  public FileStatus[] globStatus(Path pathPattern) throws IOException {
1685    return new Globber(this, pathPattern, DEFAULT_FILTER).glob();
1686  }
1687  
1688  /**
1689   * Return an array of FileStatus objects whose path names match pathPattern
1690   * and is accepted by the user-supplied path filter. Results are sorted by
1691   * their path names.
1692   * Return null if pathPattern has no glob and the path does not exist.
1693   * Return an empty array if pathPattern has a glob and no path matches it. 
1694   * 
1695   * @param pathPattern
1696   *          a regular expression specifying the path pattern
1697   * @param filter
1698   *          a user-supplied path filter
1699   * @return an array of FileStatus objects
1700   * @throws IOException if any I/O error occurs when fetching file status
1701   */
1702  public FileStatus[] globStatus(Path pathPattern, PathFilter filter)
1703      throws IOException {
1704    return new Globber(this, pathPattern, filter).glob();
1705  }
1706  
1707  /**
1708   * List the statuses of the files/directories in the given path if the path is
1709   * a directory. 
1710   * Return the file's status and block locations If the path is a file.
1711   * 
1712   * If a returned status is a file, it contains the file's block locations.
1713   * 
1714   * @param f is the path
1715   *
1716   * @return an iterator that traverses statuses of the files/directories 
1717   *         in the given path
1718   *
1719   * @throws FileNotFoundException If <code>f</code> does not exist
1720   * @throws IOException If an I/O error occurred
1721   */
1722  public RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path f)
1723  throws FileNotFoundException, IOException {
1724    return listLocatedStatus(f, DEFAULT_FILTER);
1725  }
1726
1727  /**
1728   * Listing a directory
1729   * The returned results include its block location if it is a file
1730   * The results are filtered by the given path filter
1731   * @param f a path
1732   * @param filter a path filter
1733   * @return an iterator that traverses statuses of the files/directories 
1734   *         in the given path
1735   * @throws FileNotFoundException if <code>f</code> does not exist
1736   * @throws IOException if any I/O error occurred
1737   */
1738  protected RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path f,
1739      final PathFilter filter)
1740  throws FileNotFoundException, IOException {
1741    return new RemoteIterator<LocatedFileStatus>() {
1742      private final FileStatus[] stats = listStatus(f, filter);
1743      private int i = 0;
1744
1745      @Override
1746      public boolean hasNext() {
1747        return i<stats.length;
1748      }
1749
1750      @Override
1751      public LocatedFileStatus next() throws IOException {
1752        if (!hasNext()) {
1753          throw new NoSuchElementException("No more entry in " + f);
1754        }
1755        FileStatus result = stats[i++];
1756        // for files, use getBlockLocations(FileStatus, int, int) to avoid
1757        // calling getFileStatus(Path) to load the FileStatus again
1758        BlockLocation[] locs = result.isFile() ?
1759            getFileBlockLocations(result, 0, result.getLen()) :
1760            null;
1761        return new LocatedFileStatus(result, locs);
1762      }
1763    };
1764  }
1765
1766  /**
1767   * Returns a remote iterator so that followup calls are made on demand
1768   * while consuming the entries. Each file system implementation should
1769   * override this method and provide a more efficient implementation, if
1770   * possible. 
1771   * Does not guarantee to return the iterator that traverses statuses
1772   * of the files in a sorted order.
1773   *
1774   * @param p target path
1775   * @return remote iterator
1776   */
1777  public RemoteIterator<FileStatus> listStatusIterator(final Path p)
1778  throws FileNotFoundException, IOException {
1779    return new RemoteIterator<FileStatus>() {
1780      private final FileStatus[] stats = listStatus(p);
1781      private int i = 0;
1782
1783      @Override
1784      public boolean hasNext() {
1785        return i<stats.length;
1786      }
1787
1788      @Override
1789      public FileStatus next() throws IOException {
1790        if (!hasNext()) {
1791          throw new NoSuchElementException("No more entry in " + p);
1792        }
1793        return stats[i++];
1794      }
1795    };
1796  }
1797
1798  /**
1799   * List the statuses and block locations of the files in the given path.
1800   * Does not guarantee to return the iterator that traverses statuses
1801   * of the files in a sorted order.
1802   * 
1803   * If the path is a directory, 
1804   *   if recursive is false, returns files in the directory;
1805   *   if recursive is true, return files in the subtree rooted at the path.
1806   * If the path is a file, return the file's status and block locations.
1807   * 
1808   * @param f is the path
1809   * @param recursive if the subdirectories need to be traversed recursively
1810   *
1811   * @return an iterator that traverses statuses of the files
1812   *
1813   * @throws FileNotFoundException when the path does not exist;
1814   *         IOException see specific implementation
1815   */
1816  public RemoteIterator<LocatedFileStatus> listFiles(
1817      final Path f, final boolean recursive)
1818  throws FileNotFoundException, IOException {
1819    return new RemoteIterator<LocatedFileStatus>() {
1820      private Stack<RemoteIterator<LocatedFileStatus>> itors = 
1821        new Stack<RemoteIterator<LocatedFileStatus>>();
1822      private RemoteIterator<LocatedFileStatus> curItor =
1823        listLocatedStatus(f);
1824      private LocatedFileStatus curFile;
1825     
1826      @Override
1827      public boolean hasNext() throws IOException {
1828        while (curFile == null) {
1829          if (curItor.hasNext()) {
1830            handleFileStat(curItor.next());
1831          } else if (!itors.empty()) {
1832            curItor = itors.pop();
1833          } else {
1834            return false;
1835          }
1836        }
1837        return true;
1838      }
1839
1840      /**
1841       * Process the input stat.
1842       * If it is a file, return the file stat.
1843       * If it is a directory, traverse the directory if recursive is true;
1844       * ignore it if recursive is false.
1845       * @param stat input status
1846       * @throws IOException if any IO error occurs
1847       */
1848      private void handleFileStat(LocatedFileStatus stat) throws IOException {
1849        if (stat.isFile()) { // file
1850          curFile = stat;
1851        } else if (recursive) { // directory
1852          itors.push(curItor);
1853          curItor = listLocatedStatus(stat.getPath());
1854        }
1855      }
1856
1857      @Override
1858      public LocatedFileStatus next() throws IOException {
1859        if (hasNext()) {
1860          LocatedFileStatus result = curFile;
1861          curFile = null;
1862          return result;
1863        } 
1864        throw new java.util.NoSuchElementException("No more entry in " + f);
1865      }
1866    };
1867  }
1868  
1869  /** Return the current user's home directory in this filesystem.
1870   * The default implementation returns "/user/$USER/".
1871   */
1872  public Path getHomeDirectory() {
1873    return this.makeQualified(
1874        new Path("/user/"+System.getProperty("user.name")));
1875  }
1876
1877
1878  /**
1879   * Set the current working directory for the given file system. All relative
1880   * paths will be resolved relative to it.
1881   * 
1882   * @param new_dir
1883   */
1884  public abstract void setWorkingDirectory(Path new_dir);
1885    
1886  /**
1887   * Get the current working directory for the given file system
1888   * @return the directory pathname
1889   */
1890  public abstract Path getWorkingDirectory();
1891  
1892  
1893  /**
1894   * Note: with the new FilesContext class, getWorkingDirectory()
1895   * will be removed. 
1896   * The working directory is implemented in FilesContext.
1897   * 
1898   * Some file systems like LocalFileSystem have an initial workingDir
1899   * that we use as the starting workingDir. For other file systems
1900   * like HDFS there is no built in notion of an initial workingDir.
1901   * 
1902   * @return if there is built in notion of workingDir then it
1903   * is returned; else a null is returned.
1904   */
1905  protected Path getInitialWorkingDirectory() {
1906    return null;
1907  }
1908
1909  /**
1910   * Call {@link #mkdirs(Path, FsPermission)} with default permission.
1911   */
1912  public boolean mkdirs(Path f) throws IOException {
1913    return mkdirs(f, FsPermission.getDirDefault());
1914  }
1915
1916  /**
1917   * Make the given file and all non-existent parents into
1918   * directories. Has the semantics of Unix 'mkdir -p'.
1919   * Existence of the directory hierarchy is not an error.
1920   * @param f path to create
1921   * @param permission to apply to f
1922   */
1923  public abstract boolean mkdirs(Path f, FsPermission permission
1924      ) throws IOException;
1925
1926  /**
1927   * The src file is on the local disk.  Add it to FS at
1928   * the given dst name and the source is kept intact afterwards
1929   * @param src path
1930   * @param dst path
1931   */
1932  public void copyFromLocalFile(Path src, Path dst)
1933    throws IOException {
1934    copyFromLocalFile(false, src, dst);
1935  }
1936
1937  /**
1938   * The src files is on the local disk.  Add it to FS at
1939   * the given dst name, removing the source afterwards.
1940   * @param srcs path
1941   * @param dst path
1942   */
1943  public void moveFromLocalFile(Path[] srcs, Path dst)
1944    throws IOException {
1945    copyFromLocalFile(true, true, srcs, dst);
1946  }
1947
1948  /**
1949   * The src file is on the local disk.  Add it to FS at
1950   * the given dst name, removing the source afterwards.
1951   * @param src path
1952   * @param dst path
1953   */
1954  public void moveFromLocalFile(Path src, Path dst)
1955    throws IOException {
1956    copyFromLocalFile(true, src, dst);
1957  }
1958
1959  /**
1960   * The src file is on the local disk.  Add it to FS at
1961   * the given dst name.
1962   * delSrc indicates if the source should be removed
1963   * @param delSrc whether to delete the src
1964   * @param src path
1965   * @param dst path
1966   */
1967  public void copyFromLocalFile(boolean delSrc, Path src, Path dst)
1968    throws IOException {
1969    copyFromLocalFile(delSrc, true, src, dst);
1970  }
1971  
1972  /**
1973   * The src files are on the local disk.  Add it to FS at
1974   * the given dst name.
1975   * delSrc indicates if the source should be removed
1976   * @param delSrc whether to delete the src
1977   * @param overwrite whether to overwrite an existing file
1978   * @param srcs array of paths which are source
1979   * @param dst path
1980   */
1981  public void copyFromLocalFile(boolean delSrc, boolean overwrite, 
1982                                Path[] srcs, Path dst)
1983    throws IOException {
1984    Configuration conf = getConf();
1985    FileUtil.copy(getLocal(conf), srcs, this, dst, delSrc, overwrite, conf);
1986  }
1987  
1988  /**
1989   * The src file is on the local disk.  Add it to FS at
1990   * the given dst name.
1991   * delSrc indicates if the source should be removed
1992   * @param delSrc whether to delete the src
1993   * @param overwrite whether to overwrite an existing file
1994   * @param src path
1995   * @param dst path
1996   */
1997  public void copyFromLocalFile(boolean delSrc, boolean overwrite, 
1998                                Path src, Path dst)
1999    throws IOException {
2000    Configuration conf = getConf();
2001    FileUtil.copy(getLocal(conf), src, this, dst, delSrc, overwrite, conf);
2002  }
2003    
2004  /**
2005   * The src file is under FS, and the dst is on the local disk.
2006   * Copy it from FS control to the local dst name.
2007   * @param src path
2008   * @param dst path
2009   */
2010  public void copyToLocalFile(Path src, Path dst) throws IOException {
2011    copyToLocalFile(false, src, dst);
2012  }
2013    
2014  /**
2015   * The src file is under FS, and the dst is on the local disk.
2016   * Copy it from FS control to the local dst name.
2017   * Remove the source afterwards
2018   * @param src path
2019   * @param dst path
2020   */
2021  public void moveToLocalFile(Path src, Path dst) throws IOException {
2022    copyToLocalFile(true, src, dst);
2023  }
2024
2025  /**
2026   * The src file is under FS, and the dst is on the local disk.
2027   * Copy it from FS control to the local dst name.
2028   * delSrc indicates if the src will be removed or not.
2029   * @param delSrc whether to delete the src
2030   * @param src path
2031   * @param dst path
2032   */   
2033  public void copyToLocalFile(boolean delSrc, Path src, Path dst)
2034    throws IOException {
2035    copyToLocalFile(delSrc, src, dst, false);
2036  }
2037  
2038    /**
2039   * The src file is under FS, and the dst is on the local disk. Copy it from FS
2040   * control to the local dst name. delSrc indicates if the src will be removed
2041   * or not. useRawLocalFileSystem indicates whether to use RawLocalFileSystem
2042   * as local file system or not. RawLocalFileSystem is non crc file system.So,
2043   * It will not create any crc files at local.
2044   * 
2045   * @param delSrc
2046   *          whether to delete the src
2047   * @param src
2048   *          path
2049   * @param dst
2050   *          path
2051   * @param useRawLocalFileSystem
2052   *          whether to use RawLocalFileSystem as local file system or not.
2053   * 
2054   * @throws IOException
2055   *           - if any IO error
2056   */
2057  public void copyToLocalFile(boolean delSrc, Path src, Path dst,
2058      boolean useRawLocalFileSystem) throws IOException {
2059    Configuration conf = getConf();
2060    FileSystem local = null;
2061    if (useRawLocalFileSystem) {
2062      local = getLocal(conf).getRawFileSystem();
2063    } else {
2064      local = getLocal(conf);
2065    }
2066    FileUtil.copy(this, src, local, dst, delSrc, conf);
2067  }
2068
2069  /**
2070   * Returns a local File that the user can write output to.  The caller
2071   * provides both the eventual FS target name and the local working
2072   * file.  If the FS is local, we write directly into the target.  If
2073   * the FS is remote, we write into the tmp local area.
2074   * @param fsOutputFile path of output file
2075   * @param tmpLocalFile path of local tmp file
2076   */
2077  public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
2078    throws IOException {
2079    return tmpLocalFile;
2080  }
2081
2082  /**
2083   * Called when we're all done writing to the target.  A local FS will
2084   * do nothing, because we've written to exactly the right place.  A remote
2085   * FS will copy the contents of tmpLocalFile to the correct target at
2086   * fsOutputFile.
2087   * @param fsOutputFile path of output file
2088   * @param tmpLocalFile path to local tmp file
2089   */
2090  public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
2091    throws IOException {
2092    moveFromLocalFile(tmpLocalFile, fsOutputFile);
2093  }
2094
2095  /**
2096   * No more filesystem operations are needed.  Will
2097   * release any held locks.
2098   */
2099  @Override
2100  public void close() throws IOException {
2101    // delete all files that were marked as delete-on-exit.
2102    processDeleteOnExit();
2103    CACHE.remove(this.key, this);
2104  }
2105
2106  /** Return the total size of all files in the filesystem. */
2107  public long getUsed() throws IOException {
2108    Path path = new Path("/");
2109    return getUsed(path);
2110  }
2111
2112  /** Return the total size of all files from a specified path. */
2113  public long getUsed(Path path) throws IOException {
2114    return getContentSummary(path).getLength();
2115  }
2116
2117  /**
2118   * Get the block size for a particular file.
2119   * @param f the filename
2120   * @return the number of bytes in a block
2121   */
2122  /** @deprecated Use getFileStatus() instead */
2123  @Deprecated
2124  public long getBlockSize(Path f) throws IOException {
2125    return getFileStatus(f).getBlockSize();
2126  }
2127
2128  /**
2129   * Return the number of bytes that large input files should be optimally
2130   * be split into to minimize i/o time.
2131   * @deprecated use {@link #getDefaultBlockSize(Path)} instead
2132   */
2133  @Deprecated
2134  public long getDefaultBlockSize() {
2135    // default to 32MB: large enough to minimize the impact of seeks
2136    return getConf().getLong("fs.local.block.size", 32 * 1024 * 1024);
2137  }
2138    
2139  /** Return the number of bytes that large input files should be optimally
2140   * be split into to minimize i/o time.  The given path will be used to
2141   * locate the actual filesystem.  The full path does not have to exist.
2142   * @param f path of file
2143   * @return the default block size for the path's filesystem
2144   */
2145  public long getDefaultBlockSize(Path f) {
2146    return getDefaultBlockSize();
2147  }
2148
2149  /**
2150   * Get the default replication.
2151   * @deprecated use {@link #getDefaultReplication(Path)} instead
2152   */
2153  @Deprecated
2154  public short getDefaultReplication() { return 1; }
2155
2156  /**
2157   * Get the default replication for a path.   The given path will be used to
2158   * locate the actual filesystem.  The full path does not have to exist.
2159   * @param path of the file
2160   * @return default replication for the path's filesystem 
2161   */
2162  public short getDefaultReplication(Path path) {
2163    return getDefaultReplication();
2164  }
2165  
2166  /**
2167   * Return a file status object that represents the path.
2168   * @param f The path we want information from
2169   * @return a FileStatus object
2170   * @throws FileNotFoundException when the path does not exist;
2171   *         IOException see specific implementation
2172   */
2173  public abstract FileStatus getFileStatus(Path f) throws IOException;
2174
2175  /**
2176   * Checks if the user can access a path.  The mode specifies which access
2177   * checks to perform.  If the requested permissions are granted, then the
2178   * method returns normally.  If access is denied, then the method throws an
2179   * {@link AccessControlException}.
2180   * <p/>
2181   * The default implementation of this method calls {@link #getFileStatus(Path)}
2182   * and checks the returned permissions against the requested permissions.
2183   * Note that the getFileStatus call will be subject to authorization checks.
2184   * Typically, this requires search (execute) permissions on each directory in
2185   * the path's prefix, but this is implementation-defined.  Any file system
2186   * that provides a richer authorization model (such as ACLs) may override the
2187   * default implementation so that it checks against that model instead.
2188   * <p>
2189   * In general, applications should avoid using this method, due to the risk of
2190   * time-of-check/time-of-use race conditions.  The permissions on a file may
2191   * change immediately after the access call returns.  Most applications should
2192   * prefer running specific file system actions as the desired user represented
2193   * by a {@link UserGroupInformation}.
2194   *
2195   * @param path Path to check
2196   * @param mode type of access to check
2197   * @throws AccessControlException if access is denied
2198   * @throws FileNotFoundException if the path does not exist
2199   * @throws IOException see specific implementation
2200   */
2201  @InterfaceAudience.LimitedPrivate({"HDFS", "Hive"})
2202  public void access(Path path, FsAction mode) throws AccessControlException,
2203      FileNotFoundException, IOException {
2204    checkAccessPermissions(this.getFileStatus(path), mode);
2205  }
2206
2207  /**
2208   * This method provides the default implementation of
2209   * {@link #access(Path, FsAction)}.
2210   *
2211   * @param stat FileStatus to check
2212   * @param mode type of access to check
2213   * @throws IOException for any error
2214   */
2215  @InterfaceAudience.Private
2216  static void checkAccessPermissions(FileStatus stat, FsAction mode)
2217      throws IOException {
2218    FsPermission perm = stat.getPermission();
2219    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
2220    String user = ugi.getShortUserName();
2221    if (user.equals(stat.getOwner())) {
2222      if (perm.getUserAction().implies(mode)) {
2223        return;
2224      }
2225    } else if (ugi.getGroups().contains(stat.getGroup())) {
2226      if (perm.getGroupAction().implies(mode)) {
2227        return;
2228      }
2229    } else {
2230      if (perm.getOtherAction().implies(mode)) {
2231        return;
2232      }
2233    }
2234    throw new AccessControlException(String.format(
2235      "Permission denied: user=%s, path=\"%s\":%s:%s:%s%s", user, stat.getPath(),
2236      stat.getOwner(), stat.getGroup(), stat.isDirectory() ? "d" : "-", perm));
2237  }
2238
2239  /**
2240   * See {@link FileContext#fixRelativePart}
2241   */
2242  protected Path fixRelativePart(Path p) {
2243    if (p.isUriPathAbsolute()) {
2244      return p;
2245    } else {
2246      return new Path(getWorkingDirectory(), p);
2247    }
2248  }
2249
2250  /**
2251   * See {@link FileContext#createSymlink(Path, Path, boolean)}
2252   */
2253  public void createSymlink(final Path target, final Path link,
2254      final boolean createParent) throws AccessControlException,
2255      FileAlreadyExistsException, FileNotFoundException,
2256      ParentNotDirectoryException, UnsupportedFileSystemException, 
2257      IOException {
2258    // Supporting filesystems should override this method
2259    throw new UnsupportedOperationException(
2260        "Filesystem does not support symlinks!");
2261  }
2262
2263  /**
2264   * See {@link FileContext#getFileLinkStatus(Path)}
2265   */
2266  public FileStatus getFileLinkStatus(final Path f)
2267      throws AccessControlException, FileNotFoundException,
2268      UnsupportedFileSystemException, IOException {
2269    // Supporting filesystems should override this method
2270    return getFileStatus(f);
2271  }
2272
2273  /**
2274   * See {@link AbstractFileSystem#supportsSymlinks()}
2275   */
2276  public boolean supportsSymlinks() {
2277    return false;
2278  }
2279
2280  /**
2281   * See {@link FileContext#getLinkTarget(Path)}
2282   */
2283  public Path getLinkTarget(Path f) throws IOException {
2284    // Supporting filesystems should override this method
2285    throw new UnsupportedOperationException(
2286        "Filesystem does not support symlinks!");
2287  }
2288
2289  /**
2290   * See {@link AbstractFileSystem#getLinkTarget(Path)}
2291   */
2292  protected Path resolveLink(Path f) throws IOException {
2293    // Supporting filesystems should override this method
2294    throw new UnsupportedOperationException(
2295        "Filesystem does not support symlinks!");
2296  }
2297
2298  /**
2299   * Get the checksum of a file.
2300   *
2301   * @param f The file path
2302   * @return The file checksum.  The default return value is null,
2303   *  which indicates that no checksum algorithm is implemented
2304   *  in the corresponding FileSystem.
2305   */
2306  public FileChecksum getFileChecksum(Path f) throws IOException {
2307    return getFileChecksum(f, Long.MAX_VALUE);
2308  }
2309
2310  /**
2311   * Get the checksum of a file, from the beginning of the file till the
2312   * specific length.
2313   * @param f The file path
2314   * @param length The length of the file range for checksum calculation
2315   * @return The file checksum.
2316   */
2317  public FileChecksum getFileChecksum(Path f, final long length)
2318      throws IOException {
2319    return null;
2320  }
2321
2322  /**
2323   * Set the verify checksum flag. This is only applicable if the 
2324   * corresponding FileSystem supports checksum. By default doesn't do anything.
2325   * @param verifyChecksum
2326   */
2327  public void setVerifyChecksum(boolean verifyChecksum) {
2328    //doesn't do anything
2329  }
2330
2331  /**
2332   * Set the write checksum flag. This is only applicable if the 
2333   * corresponding FileSystem supports checksum. By default doesn't do anything.
2334   * @param writeChecksum
2335   */
2336  public void setWriteChecksum(boolean writeChecksum) {
2337    //doesn't do anything
2338  }
2339
2340  /**
2341   * Returns a status object describing the use and capacity of the
2342   * file system. If the file system has multiple partitions, the
2343   * use and capacity of the root partition is reflected.
2344   * 
2345   * @return a FsStatus object
2346   * @throws IOException
2347   *           see specific implementation
2348   */
2349  public FsStatus getStatus() throws IOException {
2350    return getStatus(null);
2351  }
2352
2353  /**
2354   * Returns a status object describing the use and capacity of the
2355   * file system. If the file system has multiple partitions, the
2356   * use and capacity of the partition pointed to by the specified
2357   * path is reflected.
2358   * @param p Path for which status should be obtained. null means
2359   * the default partition. 
2360   * @return a FsStatus object
2361   * @throws IOException
2362   *           see specific implementation
2363   */
2364  public FsStatus getStatus(Path p) throws IOException {
2365    return new FsStatus(Long.MAX_VALUE, 0, Long.MAX_VALUE);
2366  }
2367
2368  /**
2369   * Set permission of a path.
2370   * @param p
2371   * @param permission
2372   */
2373  public void setPermission(Path p, FsPermission permission
2374      ) throws IOException {
2375  }
2376
2377  /**
2378   * Set owner of a path (i.e. a file or a directory).
2379   * The parameters username and groupname cannot both be null.
2380   * @param p The path
2381   * @param username If it is null, the original username remains unchanged.
2382   * @param groupname If it is null, the original groupname remains unchanged.
2383   */
2384  public void setOwner(Path p, String username, String groupname
2385      ) throws IOException {
2386  }
2387
2388  /**
2389   * Set access time of a file
2390   * @param p The path
2391   * @param mtime Set the modification time of this file.
2392   *              The number of milliseconds since Jan 1, 1970. 
2393   *              A value of -1 means that this call should not set modification time.
2394   * @param atime Set the access time of this file.
2395   *              The number of milliseconds since Jan 1, 1970. 
2396   *              A value of -1 means that this call should not set access time.
2397   */
2398  public void setTimes(Path p, long mtime, long atime
2399      ) throws IOException {
2400  }
2401
2402  /**
2403   * Create a snapshot with a default name.
2404   * @param path The directory where snapshots will be taken.
2405   * @return the snapshot path.
2406   */
2407  public final Path createSnapshot(Path path) throws IOException {
2408    return createSnapshot(path, null);
2409  }
2410
2411  /**
2412   * Create a snapshot
2413   * @param path The directory where snapshots will be taken.
2414   * @param snapshotName The name of the snapshot
2415   * @return the snapshot path.
2416   */
2417  public Path createSnapshot(Path path, String snapshotName)
2418      throws IOException {
2419    throw new UnsupportedOperationException(getClass().getSimpleName()
2420        + " doesn't support createSnapshot");
2421  }
2422  
2423  /**
2424   * Rename a snapshot
2425   * @param path The directory path where the snapshot was taken
2426   * @param snapshotOldName Old name of the snapshot
2427   * @param snapshotNewName New name of the snapshot
2428   * @throws IOException
2429   */
2430  public void renameSnapshot(Path path, String snapshotOldName,
2431      String snapshotNewName) throws IOException {
2432    throw new UnsupportedOperationException(getClass().getSimpleName()
2433        + " doesn't support renameSnapshot");
2434  }
2435  
2436  /**
2437   * Delete a snapshot of a directory
2438   * @param path  The directory that the to-be-deleted snapshot belongs to
2439   * @param snapshotName The name of the snapshot
2440   */
2441  public void deleteSnapshot(Path path, String snapshotName)
2442      throws IOException {
2443    throw new UnsupportedOperationException(getClass().getSimpleName()
2444        + " doesn't support deleteSnapshot");
2445  }
2446  
2447  /**
2448   * Modifies ACL entries of files and directories.  This method can add new ACL
2449   * entries or modify the permissions on existing ACL entries.  All existing
2450   * ACL entries that are not specified in this call are retained without
2451   * changes.  (Modifications are merged into the current ACL.)
2452   *
2453   * @param path Path to modify
2454   * @param aclSpec List<AclEntry> describing modifications
2455   * @throws IOException if an ACL could not be modified
2456   */
2457  public void modifyAclEntries(Path path, List<AclEntry> aclSpec)
2458      throws IOException {
2459    throw new UnsupportedOperationException(getClass().getSimpleName()
2460        + " doesn't support modifyAclEntries");
2461  }
2462
2463  /**
2464   * Removes ACL entries from files and directories.  Other ACL entries are
2465   * retained.
2466   *
2467   * @param path Path to modify
2468   * @param aclSpec List<AclEntry> describing entries to remove
2469   * @throws IOException if an ACL could not be modified
2470   */
2471  public void removeAclEntries(Path path, List<AclEntry> aclSpec)
2472      throws IOException {
2473    throw new UnsupportedOperationException(getClass().getSimpleName()
2474        + " doesn't support removeAclEntries");
2475  }
2476
2477  /**
2478   * Removes all default ACL entries from files and directories.
2479   *
2480   * @param path Path to modify
2481   * @throws IOException if an ACL could not be modified
2482   */
2483  public void removeDefaultAcl(Path path)
2484      throws IOException {
2485    throw new UnsupportedOperationException(getClass().getSimpleName()
2486        + " doesn't support removeDefaultAcl");
2487  }
2488
2489  /**
2490   * Removes all but the base ACL entries of files and directories.  The entries
2491   * for user, group, and others are retained for compatibility with permission
2492   * bits.
2493   *
2494   * @param path Path to modify
2495   * @throws IOException if an ACL could not be removed
2496   */
2497  public void removeAcl(Path path)
2498      throws IOException {
2499    throw new UnsupportedOperationException(getClass().getSimpleName()
2500        + " doesn't support removeAcl");
2501  }
2502
2503  /**
2504   * Fully replaces ACL of files and directories, discarding all existing
2505   * entries.
2506   *
2507   * @param path Path to modify
2508   * @param aclSpec List<AclEntry> describing modifications, must include entries
2509   *   for user, group, and others for compatibility with permission bits.
2510   * @throws IOException if an ACL could not be modified
2511   */
2512  public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException {
2513    throw new UnsupportedOperationException(getClass().getSimpleName()
2514        + " doesn't support setAcl");
2515  }
2516
2517  /**
2518   * Gets the ACL of a file or directory.
2519   *
2520   * @param path Path to get
2521   * @return AclStatus describing the ACL of the file or directory
2522   * @throws IOException if an ACL could not be read
2523   */
2524  public AclStatus getAclStatus(Path path) throws IOException {
2525    throw new UnsupportedOperationException(getClass().getSimpleName()
2526        + " doesn't support getAclStatus");
2527  }
2528
2529  /**
2530   * Set an xattr of a file or directory.
2531   * The name must be prefixed with the namespace followed by ".". For example,
2532   * "user.attr".
2533   * <p/>
2534   * Refer to the HDFS extended attributes user documentation for details.
2535   *
2536   * @param path Path to modify
2537   * @param name xattr name.
2538   * @param value xattr value.
2539   * @throws IOException
2540   */
2541  public void setXAttr(Path path, String name, byte[] value)
2542      throws IOException {
2543    setXAttr(path, name, value, EnumSet.of(XAttrSetFlag.CREATE,
2544        XAttrSetFlag.REPLACE));
2545  }
2546
2547  /**
2548   * Set an xattr of a file or directory.
2549   * The name must be prefixed with the namespace followed by ".". For example,
2550   * "user.attr".
2551   * <p/>
2552   * Refer to the HDFS extended attributes user documentation for details.
2553   *
2554   * @param path Path to modify
2555   * @param name xattr name.
2556   * @param value xattr value.
2557   * @param flag xattr set flag
2558   * @throws IOException
2559   */
2560  public void setXAttr(Path path, String name, byte[] value,
2561      EnumSet<XAttrSetFlag> flag) throws IOException {
2562    throw new UnsupportedOperationException(getClass().getSimpleName()
2563        + " doesn't support setXAttr");
2564  }
2565
2566  /**
2567   * Get an xattr name and value for a file or directory.
2568   * The name must be prefixed with the namespace followed by ".". For example,
2569   * "user.attr".
2570   * <p/>
2571   * Refer to the HDFS extended attributes user documentation for details.
2572   *
2573   * @param path Path to get extended attribute
2574   * @param name xattr name.
2575   * @return byte[] xattr value.
2576   * @throws IOException
2577   */
2578  public byte[] getXAttr(Path path, String name) throws IOException {
2579    throw new UnsupportedOperationException(getClass().getSimpleName()
2580        + " doesn't support getXAttr");
2581  }
2582
2583  /**
2584   * Get all of the xattr name/value pairs for a file or directory.
2585   * Only those xattrs which the logged-in user has permissions to view
2586   * are returned.
2587   * <p/>
2588   * Refer to the HDFS extended attributes user documentation for details.
2589   *
2590   * @param path Path to get extended attributes
2591   * @return Map<String, byte[]> describing the XAttrs of the file or directory
2592   * @throws IOException
2593   */
2594  public Map<String, byte[]> getXAttrs(Path path) throws IOException {
2595    throw new UnsupportedOperationException(getClass().getSimpleName()
2596        + " doesn't support getXAttrs");
2597  }
2598
2599  /**
2600   * Get all of the xattrs name/value pairs for a file or directory.
2601   * Only those xattrs which the logged-in user has permissions to view
2602   * are returned.
2603   * <p/>
2604   * Refer to the HDFS extended attributes user documentation for details.
2605   *
2606   * @param path Path to get extended attributes
2607   * @param names XAttr names.
2608   * @return Map<String, byte[]> describing the XAttrs of the file or directory
2609   * @throws IOException
2610   */
2611  public Map<String, byte[]> getXAttrs(Path path, List<String> names)
2612      throws IOException {
2613    throw new UnsupportedOperationException(getClass().getSimpleName()
2614        + " doesn't support getXAttrs");
2615  }
2616
2617  /**
2618   * Get all of the xattr names for a file or directory.
2619   * Only those xattr names which the logged-in user has permissions to view
2620   * are returned.
2621   * <p/>
2622   * Refer to the HDFS extended attributes user documentation for details.
2623   *
2624   * @param path Path to get extended attributes
2625   * @return List<String> of the XAttr names of the file or directory
2626   * @throws IOException
2627   */
2628  public List<String> listXAttrs(Path path) throws IOException {
2629    throw new UnsupportedOperationException(getClass().getSimpleName()
2630            + " doesn't support listXAttrs");
2631  }
2632
2633  /**
2634   * Remove an xattr of a file or directory.
2635   * The name must be prefixed with the namespace followed by ".". For example,
2636   * "user.attr".
2637   * <p/>
2638   * Refer to the HDFS extended attributes user documentation for details.
2639   *
2640   * @param path Path to remove extended attribute
2641   * @param name xattr name
2642   * @throws IOException
2643   */
2644  public void removeXAttr(Path path, String name) throws IOException {
2645    throw new UnsupportedOperationException(getClass().getSimpleName()
2646        + " doesn't support removeXAttr");
2647  }
2648
2649  /**
2650   * Set the storage policy for a given file or directory.
2651   *
2652   * @param src file or directory path.
2653   * @param policyName the name of the target storage policy. The list
2654   *                   of supported Storage policies can be retrieved
2655   *                   via {@link #getAllStoragePolicies}.
2656   * @throws IOException
2657   */
2658  public void setStoragePolicy(final Path src, final String policyName)
2659      throws IOException {
2660    throw new UnsupportedOperationException(getClass().getSimpleName()
2661        + " doesn't support setStoragePolicy");
2662  }
2663
2664  /**
2665   * Unset the storage policy set for a given file or directory.
2666   * @param src file or directory path.
2667   * @throws IOException
2668   */
2669  public void unsetStoragePolicy(final Path src) throws IOException {
2670    throw new UnsupportedOperationException(getClass().getSimpleName()
2671        + " doesn't support unsetStoragePolicy");
2672  }
2673
2674  /**
2675   * Query the effective storage policy ID for the given file or directory.
2676   *
2677   * @param src file or directory path.
2678   * @return storage policy for give file.
2679   * @throws IOException
2680   */
2681  public BlockStoragePolicySpi getStoragePolicy(final Path src)
2682      throws IOException {
2683    throw new UnsupportedOperationException(getClass().getSimpleName()
2684        + " doesn't support getStoragePolicy");
2685  }
2686
2687  /**
2688   * Retrieve all the storage policies supported by this file system.
2689   *
2690   * @return all storage policies supported by this filesystem.
2691   * @throws IOException
2692   */
2693  public Collection<? extends BlockStoragePolicySpi> getAllStoragePolicies()
2694      throws IOException {
2695    throw new UnsupportedOperationException(getClass().getSimpleName()
2696        + " doesn't support getAllStoragePolicies");
2697  }
2698
2699  /**
2700   * Get the root directory of Trash for current user when the path specified
2701   * is deleted.
2702   *
2703   * @param path the trash root of the path to be determined.
2704   * @return the default implementation returns "/user/$USER/.Trash".
2705   */
2706  public Path getTrashRoot(Path path) {
2707    return this.makeQualified(new Path(getHomeDirectory().toUri().getPath(),
2708        TRASH_PREFIX));
2709  }
2710
2711  /**
2712   * Get all the trash roots for current user or all users.
2713   *
2714   * @param allUsers return trash roots for all users if true.
2715   * @return all the trash root directories.
2716   *         Default FileSystem returns .Trash under users' home directories if
2717   *         /user/$USER/.Trash exists.
2718   */
2719  public Collection<FileStatus> getTrashRoots(boolean allUsers) {
2720    Path userHome = new Path(getHomeDirectory().toUri().getPath());
2721    List<FileStatus> ret = new ArrayList<>();
2722    try {
2723      if (!allUsers) {
2724        Path userTrash = new Path(userHome, TRASH_PREFIX);
2725        if (exists(userTrash)) {
2726          ret.add(getFileStatus(userTrash));
2727        }
2728      } else {
2729        Path homeParent = userHome.getParent();
2730        if (exists(homeParent)) {
2731          FileStatus[] candidates = listStatus(homeParent);
2732          for (FileStatus candidate : candidates) {
2733            Path userTrash = new Path(candidate.getPath(), TRASH_PREFIX);
2734            if (exists(userTrash)) {
2735              candidate.setPath(userTrash);
2736              ret.add(candidate);
2737            }
2738          }
2739        }
2740      }
2741    } catch (IOException e) {
2742      LOG.warn("Cannot get all trash roots", e);
2743    }
2744    return ret;
2745  }
2746
2747  // making it volatile to be able to do a double checked locking
2748  private volatile static boolean FILE_SYSTEMS_LOADED = false;
2749
2750  private static final Map<String, Class<? extends FileSystem>>
2751    SERVICE_FILE_SYSTEMS = new HashMap<String, Class<? extends FileSystem>>();
2752
2753  private static void loadFileSystems() {
2754    synchronized (FileSystem.class) {
2755      if (!FILE_SYSTEMS_LOADED) {
2756        ServiceLoader<FileSystem> serviceLoader = ServiceLoader.load(FileSystem.class);
2757        Iterator<FileSystem> it = serviceLoader.iterator();
2758        while (it.hasNext()) {
2759          FileSystem fs = null;
2760          try {
2761            fs = it.next();
2762            try {
2763              SERVICE_FILE_SYSTEMS.put(fs.getScheme(), fs.getClass());
2764            } catch (Exception e) {
2765              LOG.warn("Cannot load: " + fs + " from " +
2766                  ClassUtil.findContainingJar(fs.getClass()), e);
2767            }
2768          } catch (ServiceConfigurationError ee) {
2769            LOG.warn("Cannot load filesystem: " + ee);
2770            Throwable cause = ee.getCause();
2771            // print all the nested exception messages
2772            while (cause != null) {
2773              LOG.warn(cause.toString());
2774              cause = cause.getCause();
2775            }
2776            // and at debug: the full stack
2777            LOG.debug("Stack Trace", ee);
2778          }
2779        }
2780        FILE_SYSTEMS_LOADED = true;
2781      }
2782    }
2783  }
2784
2785  public static Class<? extends FileSystem> getFileSystemClass(String scheme,
2786      Configuration conf) throws IOException {
2787    if (!FILE_SYSTEMS_LOADED) {
2788      loadFileSystems();
2789    }
2790    Class<? extends FileSystem> clazz = null;
2791    if (conf != null) {
2792      clazz = (Class<? extends FileSystem>) conf.getClass("fs." + scheme + ".impl", null);
2793    }
2794    if (clazz == null) {
2795      clazz = SERVICE_FILE_SYSTEMS.get(scheme);
2796    }
2797    if (clazz == null) {
2798      throw new IOException("No FileSystem for scheme: " + scheme);
2799    }
2800    return clazz;
2801  }
2802
2803  private static FileSystem createFileSystem(URI uri, Configuration conf
2804      ) throws IOException {
2805    Tracer tracer = FsTracer.get(conf);
2806    TraceScope scope = tracer.newScope("FileSystem#createFileSystem");
2807    scope.addKVAnnotation("scheme", uri.getScheme());
2808    try {
2809      Class<?> clazz = getFileSystemClass(uri.getScheme(), conf);
2810      FileSystem fs = (FileSystem)ReflectionUtils.newInstance(clazz, conf);
2811      fs.initialize(uri, conf);
2812      return fs;
2813    } finally {
2814      scope.close();
2815    }
2816  }
2817
2818  /** Caching FileSystem objects */
2819  static class Cache {
2820    private final ClientFinalizer clientFinalizer = new ClientFinalizer();
2821
2822    private final Map<Key, FileSystem> map = new HashMap<Key, FileSystem>();
2823    private final Set<Key> toAutoClose = new HashSet<Key>();
2824
2825    /** A variable that makes all objects in the cache unique */
2826    private static AtomicLong unique = new AtomicLong(1);
2827
2828    FileSystem get(URI uri, Configuration conf) throws IOException{
2829      Key key = new Key(uri, conf);
2830      return getInternal(uri, conf, key);
2831    }
2832
2833    /** The objects inserted into the cache using this method are all unique */
2834    FileSystem getUnique(URI uri, Configuration conf) throws IOException{
2835      Key key = new Key(uri, conf, unique.getAndIncrement());
2836      return getInternal(uri, conf, key);
2837    }
2838
2839    private FileSystem getInternal(URI uri, Configuration conf, Key key) throws IOException{
2840      FileSystem fs;
2841      synchronized (this) {
2842        fs = map.get(key);
2843      }
2844      if (fs != null) {
2845        return fs;
2846      }
2847
2848      fs = createFileSystem(uri, conf);
2849      synchronized (this) { // refetch the lock again
2850        FileSystem oldfs = map.get(key);
2851        if (oldfs != null) { // a file system is created while lock is releasing
2852          fs.close(); // close the new file system
2853          return oldfs;  // return the old file system
2854        }
2855        
2856        // now insert the new file system into the map
2857        if (map.isEmpty()
2858                && !ShutdownHookManager.get().isShutdownInProgress()) {
2859          ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY);
2860        }
2861        fs.key = key;
2862        map.put(key, fs);
2863        if (conf.getBoolean(
2864            FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) {
2865          toAutoClose.add(key);
2866        }
2867        return fs;
2868      }
2869    }
2870
2871    synchronized void remove(Key key, FileSystem fs) {
2872      FileSystem cachedFs = map.remove(key);
2873      if (fs == cachedFs) {
2874        toAutoClose.remove(key);
2875      } else if (cachedFs != null) {
2876        map.put(key, cachedFs);
2877      }
2878    }
2879
2880    synchronized void closeAll() throws IOException {
2881      closeAll(false);
2882    }
2883
2884    /**
2885     * Close all FileSystem instances in the Cache.
2886     * @param onlyAutomatic only close those that are marked for automatic closing
2887     */
2888    synchronized void closeAll(boolean onlyAutomatic) throws IOException {
2889      List<IOException> exceptions = new ArrayList<IOException>();
2890
2891      // Make a copy of the keys in the map since we'll be modifying
2892      // the map while iterating over it, which isn't safe.
2893      List<Key> keys = new ArrayList<Key>();
2894      keys.addAll(map.keySet());
2895
2896      for (Key key : keys) {
2897        final FileSystem fs = map.get(key);
2898
2899        if (onlyAutomatic && !toAutoClose.contains(key)) {
2900          continue;
2901        }
2902
2903        //remove from cache
2904        map.remove(key);
2905        toAutoClose.remove(key);
2906
2907        if (fs != null) {
2908          try {
2909            fs.close();
2910          }
2911          catch(IOException ioe) {
2912            exceptions.add(ioe);
2913          }
2914        }
2915      }
2916
2917      if (!exceptions.isEmpty()) {
2918        throw MultipleIOException.createIOException(exceptions);
2919      }
2920    }
2921
2922    private class ClientFinalizer implements Runnable {
2923      @Override
2924      public synchronized void run() {
2925        try {
2926          closeAll(true);
2927        } catch (IOException e) {
2928          LOG.info("FileSystem.Cache.closeAll() threw an exception:\n" + e);
2929        }
2930      }
2931    }
2932
2933    synchronized void closeAll(UserGroupInformation ugi) throws IOException {
2934      List<FileSystem> targetFSList = new ArrayList<FileSystem>();
2935      //Make a pass over the list and collect the filesystems to close
2936      //we cannot close inline since close() removes the entry from the Map
2937      for (Map.Entry<Key, FileSystem> entry : map.entrySet()) {
2938        final Key key = entry.getKey();
2939        final FileSystem fs = entry.getValue();
2940        if (ugi.equals(key.ugi) && fs != null) {
2941          targetFSList.add(fs);   
2942        }
2943      }
2944      List<IOException> exceptions = new ArrayList<IOException>();
2945      //now make a pass over the target list and close each
2946      for (FileSystem fs : targetFSList) {
2947        try {
2948          fs.close();
2949        }
2950        catch(IOException ioe) {
2951          exceptions.add(ioe);
2952        }
2953      }
2954      if (!exceptions.isEmpty()) {
2955        throw MultipleIOException.createIOException(exceptions);
2956      }
2957    }
2958
2959    /** FileSystem.Cache.Key */
2960    static class Key {
2961      final String scheme;
2962      final String authority;
2963      final UserGroupInformation ugi;
2964      final long unique;   // an artificial way to make a key unique
2965
2966      Key(URI uri, Configuration conf) throws IOException {
2967        this(uri, conf, 0);
2968      }
2969
2970      Key(URI uri, Configuration conf, long unique) throws IOException {
2971        scheme = uri.getScheme()==null ?
2972            "" : StringUtils.toLowerCase(uri.getScheme());
2973        authority = uri.getAuthority()==null ?
2974            "" : StringUtils.toLowerCase(uri.getAuthority());
2975        this.unique = unique;
2976        
2977        this.ugi = UserGroupInformation.getCurrentUser();
2978      }
2979
2980      @Override
2981      public int hashCode() {
2982        return (scheme + authority).hashCode() + ugi.hashCode() + (int)unique;
2983      }
2984
2985      static boolean isEqual(Object a, Object b) {
2986        return a == b || (a != null && a.equals(b));        
2987      }
2988
2989      @Override
2990      public boolean equals(Object obj) {
2991        if (obj == this) {
2992          return true;
2993        }
2994        if (obj != null && obj instanceof Key) {
2995          Key that = (Key)obj;
2996          return isEqual(this.scheme, that.scheme)
2997                 && isEqual(this.authority, that.authority)
2998                 && isEqual(this.ugi, that.ugi)
2999                 && (this.unique == that.unique);
3000        }
3001        return false;        
3002      }
3003
3004      @Override
3005      public String toString() {
3006        return "("+ugi.toString() + ")@" + scheme + "://" + authority;        
3007      }
3008    }
3009  }
3010  
3011  /**
3012   * Tracks statistics about how many reads, writes, and so forth have been
3013   * done in a FileSystem.
3014   * 
3015   * Since there is only one of these objects per FileSystem, there will 
3016   * typically be many threads writing to this object.  Almost every operation
3017   * on an open file will involve a write to this object.  In contrast, reading
3018   * statistics is done infrequently by most programs, and not at all by others.
3019   * Hence, this is optimized for writes.
3020   * 
3021   * Each thread writes to its own thread-local area of memory.  This removes 
3022   * contention and allows us to scale up to many, many threads.  To read
3023   * statistics, the reader thread totals up the contents of all of the 
3024   * thread-local data areas.
3025   */
3026  public static final class Statistics {
3027    /**
3028     * Statistics data.
3029     * 
3030     * There is only a single writer to thread-local StatisticsData objects.
3031     * Hence, volatile is adequate here-- we do not need AtomicLong or similar
3032     * to prevent lost updates.
3033     * The Java specification guarantees that updates to volatile longs will
3034     * be perceived as atomic with respect to other threads, which is all we
3035     * need.
3036     */
3037    public static class StatisticsData {
3038      volatile long bytesRead;
3039      volatile long bytesWritten;
3040      volatile int readOps;
3041      volatile int largeReadOps;
3042      volatile int writeOps;
3043
3044      /**
3045       * Add another StatisticsData object to this one.
3046       */
3047      void add(StatisticsData other) {
3048        this.bytesRead += other.bytesRead;
3049        this.bytesWritten += other.bytesWritten;
3050        this.readOps += other.readOps;
3051        this.largeReadOps += other.largeReadOps;
3052        this.writeOps += other.writeOps;
3053      }
3054
3055      /**
3056       * Negate the values of all statistics.
3057       */
3058      void negate() {
3059        this.bytesRead = -this.bytesRead;
3060        this.bytesWritten = -this.bytesWritten;
3061        this.readOps = -this.readOps;
3062        this.largeReadOps = -this.largeReadOps;
3063        this.writeOps = -this.writeOps;
3064      }
3065
3066      @Override
3067      public String toString() {
3068        return bytesRead + " bytes read, " + bytesWritten + " bytes written, "
3069            + readOps + " read ops, " + largeReadOps + " large read ops, "
3070            + writeOps + " write ops";
3071      }
3072      
3073      public long getBytesRead() {
3074        return bytesRead;
3075      }
3076      
3077      public long getBytesWritten() {
3078        return bytesWritten;
3079      }
3080      
3081      public int getReadOps() {
3082        return readOps;
3083      }
3084      
3085      public int getLargeReadOps() {
3086        return largeReadOps;
3087      }
3088      
3089      public int getWriteOps() {
3090        return writeOps;
3091      }
3092    }
3093
3094    private interface StatisticsAggregator<T> {
3095      void accept(StatisticsData data);
3096      T aggregate();
3097    }
3098
3099    private final String scheme;
3100
3101    /**
3102     * rootData is data that doesn't belong to any thread, but will be added
3103     * to the totals.  This is useful for making copies of Statistics objects,
3104     * and for storing data that pertains to threads that have been garbage
3105     * collected.  Protected by the Statistics lock.
3106     */
3107    private final StatisticsData rootData;
3108
3109    /**
3110     * Thread-local data.
3111     */
3112    private final ThreadLocal<StatisticsData> threadData;
3113
3114    /**
3115     * Set of all thread-local data areas.  Protected by the Statistics lock.
3116     * The references to the statistics data are kept using weak references
3117     * to the associated threads. Proper clean-up is performed by the cleaner
3118     * thread when the threads are garbage collected.
3119     */
3120    private final Set<StatisticsDataReference> allData;
3121
3122    /**
3123     * Global reference queue and a cleaner thread that manage statistics data
3124     * references from all filesystem instances.
3125     */
3126    private static final ReferenceQueue<Thread> STATS_DATA_REF_QUEUE;
3127    private static final Thread STATS_DATA_CLEANER;
3128
3129    static {
3130      STATS_DATA_REF_QUEUE = new ReferenceQueue<Thread>();
3131      // start a single daemon cleaner thread
3132      STATS_DATA_CLEANER = new Thread(new StatisticsDataReferenceCleaner());
3133      STATS_DATA_CLEANER.
3134          setName(StatisticsDataReferenceCleaner.class.getName());
3135      STATS_DATA_CLEANER.setDaemon(true);
3136      STATS_DATA_CLEANER.start();
3137    }
3138
3139    public Statistics(String scheme) {
3140      this.scheme = scheme;
3141      this.rootData = new StatisticsData();
3142      this.threadData = new ThreadLocal<StatisticsData>();
3143      this.allData = new HashSet<StatisticsDataReference>();
3144    }
3145
3146    /**
3147     * Copy constructor.
3148     * 
3149     * @param other    The input Statistics object which is cloned.
3150     */
3151    public Statistics(Statistics other) {
3152      this.scheme = other.scheme;
3153      this.rootData = new StatisticsData();
3154      other.visitAll(new StatisticsAggregator<Void>() {
3155        @Override
3156        public void accept(StatisticsData data) {
3157          rootData.add(data);
3158        }
3159
3160        public Void aggregate() {
3161          return null;
3162        }
3163      });
3164      this.threadData = new ThreadLocal<StatisticsData>();
3165      this.allData = new HashSet<StatisticsDataReference>();
3166    }
3167
3168    /**
3169     * A weak reference to a thread that also includes the data associated
3170     * with that thread. On the thread being garbage collected, it is enqueued
3171     * to the reference queue for clean-up.
3172     */
3173    private class StatisticsDataReference extends WeakReference<Thread> {
3174      private final StatisticsData data;
3175
3176      public StatisticsDataReference(StatisticsData data, Thread thread) {
3177        super(thread, STATS_DATA_REF_QUEUE);
3178        this.data = data;
3179      }
3180
3181      public StatisticsData getData() {
3182        return data;
3183      }
3184
3185      /**
3186       * Performs clean-up action when the associated thread is garbage
3187       * collected.
3188       */
3189      public void cleanUp() {
3190        // use the statistics lock for safety
3191        synchronized (Statistics.this) {
3192          /*
3193           * If the thread that created this thread-local data no longer exists,
3194           * remove the StatisticsData from our list and fold the values into
3195           * rootData.
3196           */
3197          rootData.add(data);
3198          allData.remove(this);
3199        }
3200      }
3201    }
3202
3203    /**
3204     * Background action to act on references being removed.
3205     */
3206    private static class StatisticsDataReferenceCleaner implements Runnable {
3207      @Override
3208      public void run() {
3209        while (true) {
3210          try {
3211            StatisticsDataReference ref =
3212                (StatisticsDataReference)STATS_DATA_REF_QUEUE.remove();
3213            ref.cleanUp();
3214          } catch (Throwable th) {
3215            // the cleaner thread should continue to run even if there are
3216            // exceptions, including InterruptedException
3217            LOG.warn("exception in the cleaner thread but it will continue to "
3218                + "run", th);
3219          }
3220        }
3221      }
3222    }
3223
3224    /**
3225     * Get or create the thread-local data associated with the current thread.
3226     */
3227    public StatisticsData getThreadStatistics() {
3228      StatisticsData data = threadData.get();
3229      if (data == null) {
3230        data = new StatisticsData();
3231        threadData.set(data);
3232        StatisticsDataReference ref =
3233            new StatisticsDataReference(data, Thread.currentThread());
3234        synchronized(this) {
3235          allData.add(ref);
3236        }
3237      }
3238      return data;
3239    }
3240
3241    /**
3242     * Increment the bytes read in the statistics
3243     * @param newBytes the additional bytes read
3244     */
3245    public void incrementBytesRead(long newBytes) {
3246      getThreadStatistics().bytesRead += newBytes;
3247    }
3248    
3249    /**
3250     * Increment the bytes written in the statistics
3251     * @param newBytes the additional bytes written
3252     */
3253    public void incrementBytesWritten(long newBytes) {
3254      getThreadStatistics().bytesWritten += newBytes;
3255    }
3256    
3257    /**
3258     * Increment the number of read operations
3259     * @param count number of read operations
3260     */
3261    public void incrementReadOps(int count) {
3262      getThreadStatistics().readOps += count;
3263    }
3264
3265    /**
3266     * Increment the number of large read operations
3267     * @param count number of large read operations
3268     */
3269    public void incrementLargeReadOps(int count) {
3270      getThreadStatistics().largeReadOps += count;
3271    }
3272
3273    /**
3274     * Increment the number of write operations
3275     * @param count number of write operations
3276     */
3277    public void incrementWriteOps(int count) {
3278      getThreadStatistics().writeOps += count;
3279    }
3280
3281    /**
3282     * Apply the given aggregator to all StatisticsData objects associated with
3283     * this Statistics object.
3284     *
3285     * For each StatisticsData object, we will call accept on the visitor.
3286     * Finally, at the end, we will call aggregate to get the final total. 
3287     *
3288     * @param         visitor to use.
3289     * @return        The total.
3290     */
3291    private synchronized <T> T visitAll(StatisticsAggregator<T> visitor) {
3292      visitor.accept(rootData);
3293      for (StatisticsDataReference ref: allData) {
3294        StatisticsData data = ref.getData();
3295        visitor.accept(data);
3296      }
3297      return visitor.aggregate();
3298    }
3299
3300    /**
3301     * Get the total number of bytes read
3302     * @return the number of bytes
3303     */
3304    public long getBytesRead() {
3305      return visitAll(new StatisticsAggregator<Long>() {
3306        private long bytesRead = 0;
3307
3308        @Override
3309        public void accept(StatisticsData data) {
3310          bytesRead += data.bytesRead;
3311        }
3312
3313        public Long aggregate() {
3314          return bytesRead;
3315        }
3316      });
3317    }
3318    
3319    /**
3320     * Get the total number of bytes written
3321     * @return the number of bytes
3322     */
3323    public long getBytesWritten() {
3324      return visitAll(new StatisticsAggregator<Long>() {
3325        private long bytesWritten = 0;
3326
3327        @Override
3328        public void accept(StatisticsData data) {
3329          bytesWritten += data.bytesWritten;
3330        }
3331
3332        public Long aggregate() {
3333          return bytesWritten;
3334        }
3335      });
3336    }
3337    
3338    /**
3339     * Get the number of file system read operations such as list files
3340     * @return number of read operations
3341     */
3342    public int getReadOps() {
3343      return visitAll(new StatisticsAggregator<Integer>() {
3344        private int readOps = 0;
3345
3346        @Override
3347        public void accept(StatisticsData data) {
3348          readOps += data.readOps;
3349          readOps += data.largeReadOps;
3350        }
3351
3352        public Integer aggregate() {
3353          return readOps;
3354        }
3355      });
3356    }
3357
3358    /**
3359     * Get the number of large file system read operations such as list files
3360     * under a large directory
3361     * @return number of large read operations
3362     */
3363    public int getLargeReadOps() {
3364      return visitAll(new StatisticsAggregator<Integer>() {
3365        private int largeReadOps = 0;
3366
3367        @Override
3368        public void accept(StatisticsData data) {
3369          largeReadOps += data.largeReadOps;
3370        }
3371
3372        public Integer aggregate() {
3373          return largeReadOps;
3374        }
3375      });
3376    }
3377
3378    /**
3379     * Get the number of file system write operations such as create, append 
3380     * rename etc.
3381     * @return number of write operations
3382     */
3383    public int getWriteOps() {
3384      return visitAll(new StatisticsAggregator<Integer>() {
3385        private int writeOps = 0;
3386
3387        @Override
3388        public void accept(StatisticsData data) {
3389          writeOps += data.writeOps;
3390        }
3391
3392        public Integer aggregate() {
3393          return writeOps;
3394        }
3395      });
3396    }
3397
3398    /**
3399     * Get all statistics data
3400     * MR or other frameworks can use the method to get all statistics at once.
3401     * @return the StatisticsData
3402     */
3403    public StatisticsData getData() {
3404      return visitAll(new StatisticsAggregator<StatisticsData>() {
3405        private StatisticsData all = new StatisticsData();
3406
3407        @Override
3408        public void accept(StatisticsData data) {
3409          all.add(data);
3410        }
3411
3412        public StatisticsData aggregate() {
3413          return all;
3414        }
3415      });
3416    }
3417
3418    @Override
3419    public String toString() {
3420      return visitAll(new StatisticsAggregator<String>() {
3421        private StatisticsData total = new StatisticsData();
3422
3423        @Override
3424        public void accept(StatisticsData data) {
3425          total.add(data);
3426        }
3427
3428        public String aggregate() {
3429          return total.toString();
3430        }
3431      });
3432    }
3433
3434    /**
3435     * Resets all statistics to 0.
3436     *
3437     * In order to reset, we add up all the thread-local statistics data, and
3438     * set rootData to the negative of that.
3439     *
3440     * This may seem like a counterintuitive way to reset the statsitics.  Why
3441     * can't we just zero out all the thread-local data?  Well, thread-local
3442     * data can only be modified by the thread that owns it.  If we tried to
3443     * modify the thread-local data from this thread, our modification might get
3444     * interleaved with a read-modify-write operation done by the thread that
3445     * owns the data.  That would result in our update getting lost.
3446     *
3447     * The approach used here avoids this problem because it only ever reads
3448     * (not writes) the thread-local data.  Both reads and writes to rootData
3449     * are done under the lock, so we're free to modify rootData from any thread
3450     * that holds the lock.
3451     */
3452    public void reset() {
3453      visitAll(new StatisticsAggregator<Void>() {
3454        private StatisticsData total = new StatisticsData();
3455
3456        @Override
3457        public void accept(StatisticsData data) {
3458          total.add(data);
3459        }
3460
3461        public Void aggregate() {
3462          total.negate();
3463          rootData.add(total);
3464          return null;
3465        }
3466      });
3467    }
3468    
3469    /**
3470     * Get the uri scheme associated with this statistics object.
3471     * @return the schema associated with this set of statistics
3472     */
3473    public String getScheme() {
3474      return scheme;
3475    }
3476
3477    @VisibleForTesting
3478    synchronized int getAllThreadLocalDataSize() {
3479      return allData.size();
3480    }
3481  }
3482  
3483  /**
3484   * Get the Map of Statistics object indexed by URI Scheme.
3485   * @return a Map having a key as URI scheme and value as Statistics object
3486   * @deprecated use {@link #getGlobalStorageStatistics()}
3487   */
3488  @Deprecated
3489  public static synchronized Map<String, Statistics> getStatistics() {
3490    Map<String, Statistics> result = new HashMap<String, Statistics>();
3491    for(Statistics stat: statisticsTable.values()) {
3492      result.put(stat.getScheme(), stat);
3493    }
3494    return result;
3495  }
3496
3497  /**
3498   * Return the FileSystem classes that have Statistics.
3499   * @deprecated use {@link #getGlobalStorageStatistics()}
3500   */
3501  @Deprecated
3502  public static synchronized List<Statistics> getAllStatistics() {
3503    return new ArrayList<Statistics>(statisticsTable.values());
3504  }
3505  
3506  /**
3507   * Get the statistics for a particular file system
3508   * @param cls the class to lookup
3509   * @return a statistics object
3510   * @deprecated use {@link #getGlobalStorageStatistics()}
3511   */
3512  @Deprecated
3513  public static synchronized Statistics getStatistics(final String scheme,
3514      Class<? extends FileSystem> cls) {
3515    checkArgument(scheme != null,
3516        "No statistics is allowed for a file system with null scheme!");
3517    Statistics result = statisticsTable.get(cls);
3518    if (result == null) {
3519      final Statistics newStats = new Statistics(scheme);
3520      statisticsTable.put(cls, newStats);
3521      result = newStats;
3522      GlobalStorageStatistics.INSTANCE.put(scheme,
3523          new StorageStatisticsProvider() {
3524            @Override
3525            public StorageStatistics provide() {
3526              return new FileSystemStorageStatistics(scheme, newStats);
3527            }
3528          });
3529    }
3530    return result;
3531  }
3532  
3533  /**
3534   * Reset all statistics for all file systems
3535   */
3536  public static synchronized void clearStatistics() {
3537    GlobalStorageStatistics.INSTANCE.reset();
3538  }
3539
3540  /**
3541   * Print all statistics for all file systems
3542   */
3543  public static synchronized
3544  void printStatistics() throws IOException {
3545    for (Map.Entry<Class<? extends FileSystem>, Statistics> pair: 
3546            statisticsTable.entrySet()) {
3547      System.out.println("  FileSystem " + pair.getKey().getName() + 
3548                         ": " + pair.getValue());
3549    }
3550  }
3551
3552  // Symlinks are temporarily disabled - see HADOOP-10020 and HADOOP-10052
3553  private static boolean symlinksEnabled = false;
3554
3555  private static Configuration conf = null;
3556
3557  @VisibleForTesting
3558  public static boolean areSymlinksEnabled() {
3559    return symlinksEnabled;
3560  }
3561
3562  @VisibleForTesting
3563  public static void enableSymlinks() {
3564    symlinksEnabled = true;
3565  }
3566
3567  /**
3568   * Get the StorageStatistics for this FileSystem object.  These statistics are
3569   * per-instance.  They are not shared with any other FileSystem object.
3570   *
3571   * <p>This is a default method which is intended to be overridden by
3572   * subclasses. The default implementation returns an empty storage statistics
3573   * object.</p>
3574   *
3575   * @return    The StorageStatistics for this FileSystem instance.
3576   *            Will never be null.
3577   */
3578  public StorageStatistics getStorageStatistics() {
3579    return new EmptyStorageStatistics(getUri().toString());
3580  }
3581
3582  /**
3583   * Get the global storage statistics.
3584   */
3585  public static GlobalStorageStatistics getGlobalStorageStatistics() {
3586    return GlobalStorageStatistics.INSTANCE;
3587  }
3588}