@InterfaceAudience.Public @InterfaceStability.Stable public abstract class FileSystem extends Configured implements Closeable
 All user code that may potentially use the Hadoop Distributed
 File System should be written to use a FileSystem object or its
 successor, FileContext.
 
 The local implementation is LocalFileSystem and distributed
 implementation is DistributedFileSystem. There are other implementations
 for object stores and (outside the Apache Hadoop codebase),
 third party filesystems.
 
Notes
FileSystem refers to an instance of this class.filesystem refers to the distributed/local filesystem
 itself, rather than the class used to interact with it.java.io.File.| Modifier and Type | Field and Description | 
|---|---|
| static String | DEFAULT_FS | 
| static String | FS_DEFAULT_NAME_KEY | 
| static int | SHUTDOWN_HOOK_PRIORITYPriority of the FileSystem shutdown hook: 10. | 
| protected org.apache.hadoop.fs.FileSystem.Statistics | statisticsThe statistics for this file system. | 
| static String | TRASH_PREFIXPrefix for trash directory: ".Trash". | 
| static String | USER_HOME_PREFIX | 
| Modifier | Constructor and Description | 
|---|---|
| protected  | FileSystem() | 
| Modifier and Type | Method and Description | 
|---|---|
| Token<?>[] | addDelegationTokens(String renewer,
                   Credentials credentials)Obtain all delegation tokens used by this FileSystem that are not
 already present in the given Credentials. | 
| FSDataOutputStream | append(Path f)Append to an existing file (optional operation). | 
| FSDataOutputStream | append(Path f,
      int bufferSize)Append to an existing file (optional operation). | 
| abstract FSDataOutputStream | append(Path f,
      int bufferSize,
      Progressable progress)Append to an existing file (optional operation). | 
| FSDataOutputStreamBuilder | appendFile(Path path)Create a Builder to append a file. | 
| static boolean | areSymlinksEnabled() | 
| boolean | cancelDeleteOnExit(Path f)Cancel the scheduled deletion of the path when the FileSystem is closed. | 
| protected URI | canonicalizeUri(URI uri)Canonicalize the given URI. | 
| protected void | checkPath(Path path)Check that a Path belongs to this FileSystem. | 
| static void | clearStatistics()Reset all statistics for all file systems. | 
| void | close()Close this FileSystem instance. | 
| static void | closeAll()Close all cached FileSystem instances. | 
| static void | closeAllForUGI(UserGroupInformation ugi)Close all cached FileSystem instances for a given UGI. | 
| void | completeLocalOutput(Path fsOutputFile,
                   Path tmpLocalFile)Called when we're all done writing to the target. | 
| void | concat(Path trg,
      Path[] psrcs)Concat existing files together. | 
| void | copyFromLocalFile(boolean delSrc,
                 boolean overwrite,
                 Path[] srcs,
                 Path dst)The src files are on the local disk. | 
| void | copyFromLocalFile(boolean delSrc,
                 boolean overwrite,
                 Path src,
                 Path dst)The src file is on the local disk. | 
| void | copyFromLocalFile(boolean delSrc,
                 Path src,
                 Path dst)The src file is on the local disk. | 
| void | copyFromLocalFile(Path src,
                 Path dst)The src file is on the local disk. | 
| void | copyToLocalFile(boolean delSrc,
               Path src,
               Path dst)Copy it a file from a remote filesystem to the local one. | 
| void | copyToLocalFile(boolean delSrc,
               Path src,
               Path dst,
               boolean useRawLocalFileSystem)The src file is under this filesystem, and the dst is on the local disk. | 
| void | copyToLocalFile(Path src,
               Path dst)Copy it a file from the remote filesystem to the local one. | 
| static FSDataOutputStream | create(FileSystem fs,
      Path file,
      FsPermission permission)Create a file with the provided permission. | 
| FSDataOutputStream | create(Path f)Create an FSDataOutputStream at the indicated Path. | 
| FSDataOutputStream | create(Path f,
      boolean overwrite)Create an FSDataOutputStream at the indicated Path. | 
| FSDataOutputStream | create(Path f,
      boolean overwrite,
      int bufferSize)Create an FSDataOutputStream at the indicated Path. | 
| FSDataOutputStream | create(Path f,
      boolean overwrite,
      int bufferSize,
      Progressable progress)Create an  FSDataOutputStreamat the indicated Path
 with write-progress reporting. | 
| FSDataOutputStream | create(Path f,
      boolean overwrite,
      int bufferSize,
      short replication,
      long blockSize)Create an FSDataOutputStream at the indicated Path. | 
| FSDataOutputStream | create(Path f,
      boolean overwrite,
      int bufferSize,
      short replication,
      long blockSize,
      Progressable progress)Create an FSDataOutputStream at the indicated Path with write-progress
 reporting. | 
| abstract FSDataOutputStream | create(Path f,
      FsPermission permission,
      boolean overwrite,
      int bufferSize,
      short replication,
      long blockSize,
      Progressable progress)Create an FSDataOutputStream at the indicated Path with write-progress
 reporting. | 
| FSDataOutputStream | create(Path f,
      FsPermission permission,
      EnumSet<CreateFlag> flags,
      int bufferSize,
      short replication,
      long blockSize,
      Progressable progress)Create an FSDataOutputStream at the indicated Path with write-progress
 reporting. | 
| FSDataOutputStream | create(Path f,
      FsPermission permission,
      EnumSet<CreateFlag> flags,
      int bufferSize,
      short replication,
      long blockSize,
      Progressable progress,
      org.apache.hadoop.fs.Options.ChecksumOpt checksumOpt)Create an FSDataOutputStream at the indicated Path with a custom
 checksum option. | 
| FSDataOutputStream | create(Path f,
      Progressable progress)Create an FSDataOutputStream at the indicated Path with write-progress
 reporting. | 
| FSDataOutputStream | create(Path f,
      short replication)Create an FSDataOutputStream at the indicated Path. | 
| FSDataOutputStream | create(Path f,
      short replication,
      Progressable progress)Create an FSDataOutputStream at the indicated Path with write-progress
 reporting. | 
| FSDataOutputStreamBuilder | createFile(Path path)Create a new FSDataOutputStreamBuilder for the file with path. | 
| boolean | createNewFile(Path f)Creates the given Path as a brand-new zero-length file. | 
| FSDataOutputStream | createNonRecursive(Path f,
                  boolean overwrite,
                  int bufferSize,
                  short replication,
                  long blockSize,
                  Progressable progress)Opens an FSDataOutputStream at the indicated Path with write-progress
 reporting. | 
| FSDataOutputStream | createNonRecursive(Path f,
                  FsPermission permission,
                  boolean overwrite,
                  int bufferSize,
                  short replication,
                  long blockSize,
                  Progressable progress)Opens an FSDataOutputStream at the indicated Path with write-progress
 reporting. | 
| FSDataOutputStream | createNonRecursive(Path f,
                  FsPermission permission,
                  EnumSet<CreateFlag> flags,
                  int bufferSize,
                  short replication,
                  long blockSize,
                  Progressable progress)Opens an FSDataOutputStream at the indicated Path with write-progress
 reporting. | 
| Path | createSnapshot(Path path)Create a snapshot with a default name. | 
| Path | createSnapshot(Path path,
              String snapshotName)Create a snapshot. | 
| void | createSymlink(Path target,
             Path link,
             boolean createParent) | 
| boolean | delete(Path f)Deprecated. 
 Use  delete(Path, boolean)instead. | 
| abstract boolean | delete(Path f,
      boolean recursive)Delete a file. | 
| boolean | deleteOnExit(Path f)Mark a path to be deleted when its FileSystem is closed. | 
| void | deleteSnapshot(Path path,
              String snapshotName)Delete a snapshot of a directory. | 
| static void | enableSymlinks() | 
| boolean | exists(Path f)Check if a path exists. | 
| protected Path | fixRelativePart(Path p) | 
| static FileSystem | get(Configuration conf)Returns the configured FileSystem implementation. | 
| static FileSystem | get(URI uri,
   Configuration conf)Get a FileSystem for this URI's scheme and authority. | 
| static FileSystem | get(URI uri,
   Configuration conf,
   String user)Get a FileSystem instance based on the uri, the passed in
 configuration and the user. | 
| AclStatus | getAclStatus(Path path)Gets the ACL of a file or directory. | 
| static List<org.apache.hadoop.fs.FileSystem.Statistics> | getAllStatistics()Deprecated. 
 | 
| Collection<? extends BlockStoragePolicySpi> | getAllStoragePolicies()Retrieve all the storage policies supported by this file system. | 
| long | getBlockSize(Path f)Deprecated. 
 Use  getFileStatus(Path)instead | 
| String | getCanonicalServiceName()Get a canonical service name for this FileSystem. | 
| protected URI | getCanonicalUri()Return a canonicalized form of this FileSystem's URI. | 
| ContentSummary | getContentSummary(Path f)Return the  ContentSummaryof a givenPath. | 
| long | getDefaultBlockSize()Deprecated. 
 use  getDefaultBlockSize(Path)instead | 
| long | getDefaultBlockSize(Path f)Return the number of bytes that large input files should be optimally
 be split into to minimize I/O time. | 
| protected int | getDefaultPort()Get the default port for this FileSystem. | 
| short | getDefaultReplication()Deprecated. 
 use  getDefaultReplication(Path)instead | 
| short | getDefaultReplication(Path path)Get the default replication for a path. | 
| static URI | getDefaultUri(Configuration conf)Get the default FileSystem URI from a configuration. | 
| BlockLocation[] | getFileBlockLocations(FileStatus file,
                     long start,
                     long len)Return an array containing hostnames, offset and size of
 portions of the given file. | 
| BlockLocation[] | getFileBlockLocations(Path p,
                     long start,
                     long len)Return an array containing hostnames, offset and size of
 portions of the given file. | 
| FileChecksum | getFileChecksum(Path f)Get the checksum of a file, if the FS supports checksums. | 
| FileChecksum | getFileChecksum(Path f,
               long length)Get the checksum of a file, from the beginning of the file till the
 specific length. | 
| FileStatus | getFileLinkStatus(Path f) | 
| abstract FileStatus | getFileStatus(Path f)Return a file status object that represents the path. | 
| static Class<? extends FileSystem> | getFileSystemClass(String scheme,
                  Configuration conf)Get the FileSystem implementation class of a filesystem. | 
| protected static FileSystem | getFSofPath(Path absOrFqPath,
           Configuration conf) | 
| static GlobalStorageStatistics | getGlobalStorageStatistics()Get the global storage statistics. | 
| Path | getHomeDirectory()Return the current user's home directory in this FileSystem. | 
| protected Path | getInitialWorkingDirectory()Note: with the new FileContext class, getWorkingDirectory()
 will be removed. | 
| long | getLength(Path f)Deprecated. 
 Use  getFileStatus(Path)instead. | 
| Path | getLinkTarget(Path f) | 
| static LocalFileSystem | getLocal(Configuration conf)Get the local FileSystem. | 
| String | getName()Deprecated. 
 call  getUri()instead. | 
| static FileSystem | getNamed(String name,
        Configuration conf)Deprecated. 
 call  get(URI, Configuration)instead. | 
| QuotaUsage | getQuotaUsage(Path f)Return the  QuotaUsageof a givenPath. | 
| short | getReplication(Path src)Deprecated. 
 Use  getFileStatus(Path)instead | 
| String | getScheme()Return the protocol scheme for this FileSystem. | 
| FsServerDefaults | getServerDefaults()Deprecated. 
 use  getServerDefaults(Path)instead | 
| FsServerDefaults | getServerDefaults(Path p)Return a set of server default configuration values. | 
| static Map<String,org.apache.hadoop.fs.FileSystem.Statistics> | getStatistics()Deprecated. 
 | 
| static org.apache.hadoop.fs.FileSystem.Statistics | getStatistics(String scheme,
             Class<? extends FileSystem> cls)Deprecated. 
 | 
| FsStatus | getStatus()Returns a status object describing the use and capacity of the
 filesystem. | 
| FsStatus | getStatus(Path p)Returns a status object describing the use and capacity of the
 filesystem. | 
| BlockStoragePolicySpi | getStoragePolicy(Path src)Query the effective storage policy ID for the given file or directory. | 
| StorageStatistics | getStorageStatistics()Get the StorageStatistics for this FileSystem object. | 
| Path | getTrashRoot(Path path)Get the root directory of Trash for current user when the path specified
 is deleted. | 
| Collection<FileStatus> | getTrashRoots(boolean allUsers)Get all the trash roots for current user or all users. | 
| abstract URI | getUri()Returns a URI which identifies this FileSystem. | 
| long | getUsed()Return the total size of all files in the filesystem. | 
| long | getUsed(Path path)Return the total size of all files from a specified path. | 
| abstract Path | getWorkingDirectory()Get the current working directory for the given FileSystem | 
| byte[] | getXAttr(Path path,
        String name)Get an xattr name and value for a file or directory. | 
| Map<String,byte[]> | getXAttrs(Path path)Get all of the xattr name/value pairs for a file or directory. | 
| Map<String,byte[]> | getXAttrs(Path path,
         List<String> names)Get all of the xattrs name/value pairs for a file or directory. | 
| FileStatus[] | globStatus(Path pathPattern)Return all the files that match filePattern and are not checksum
 files. | 
| FileStatus[] | globStatus(Path pathPattern,
          PathFilter filter)Return an array of  FileStatusobjects whose path names matchpathPatternand is accepted by the user-supplied path filter. | 
| void | initialize(URI name,
          Configuration conf)Initialize a FileSystem. | 
| boolean | isDirectory(Path f)Deprecated. 
 Use  getFileStatus(Path)instead | 
| boolean | isFile(Path f)Deprecated. 
 Use  getFileStatus(Path)instead | 
| org.apache.hadoop.fs.RemoteIterator<Path> | listCorruptFileBlocks(Path path)List corrupted file blocks. | 
| org.apache.hadoop.fs.RemoteIterator<LocatedFileStatus> | listFiles(Path f,
         boolean recursive)List the statuses and block locations of the files in the given path. | 
| org.apache.hadoop.fs.RemoteIterator<LocatedFileStatus> | listLocatedStatus(Path f)List the statuses of the files/directories in the given path if the path is
 a directory. | 
| protected org.apache.hadoop.fs.RemoteIterator<LocatedFileStatus> | listLocatedStatus(Path f,
                 PathFilter filter)List a directory. | 
| abstract FileStatus[] | listStatus(Path f)List the statuses of the files/directories in the given path if the path is
 a directory. | 
| FileStatus[] | listStatus(Path[] files)Filter files/directories in the given list of paths using default
 path filter. | 
| FileStatus[] | listStatus(Path[] files,
          PathFilter filter)Filter files/directories in the given list of paths using user-supplied
 path filter. | 
| FileStatus[] | listStatus(Path f,
          PathFilter filter)Filter files/directories in the given path using the user-supplied path
 filter. | 
| org.apache.hadoop.fs.RemoteIterator<FileStatus> | listStatusIterator(Path p)Returns a remote iterator so that followup calls are made on demand
 while consuming the entries. | 
| List<String> | listXAttrs(Path path)Get all of the xattr names for a file or directory. | 
| Path | makeQualified(Path path)Qualify a path to one which uses this FileSystem and, if relative,
 made absolute. | 
| static boolean | mkdirs(FileSystem fs,
      Path dir,
      FsPermission permission)Create a directory with the provided permission. | 
| boolean | mkdirs(Path f)Call  mkdirs(Path, FsPermission)with default permission. | 
| abstract boolean | mkdirs(Path f,
      FsPermission permission)Make the given file and all non-existent parents into
 directories. | 
| void | modifyAclEntries(Path path,
                List<AclEntry> aclSpec)Modifies ACL entries of files and directories. | 
| void | moveFromLocalFile(Path[] srcs,
                 Path dst)The src files is on the local disk. | 
| void | moveFromLocalFile(Path src,
                 Path dst)The src file is on the local disk. | 
| void | moveToLocalFile(Path src,
               Path dst)Copy a file to the local filesystem, then delete it from the
 remote filesystem (if successfully copied). | 
| static FileSystem | newInstance(Configuration conf)Returns a unique configured FileSystem implementation for the default
 filesystem of the supplied configuration. | 
| static FileSystem | newInstance(URI uri,
           Configuration config)Returns the FileSystem for this URI's scheme and authority. | 
| static FileSystem | newInstance(URI uri,
           Configuration conf,
           String user)Returns the FileSystem for this URI's scheme and authority and the
 given user. | 
| static LocalFileSystem | newInstanceLocal(Configuration conf)Get a unique local FileSystem object. | 
| FSDataInputStream | open(Path f)Opens an FSDataInputStream at the indicated Path. | 
| abstract FSDataInputStream | open(Path f,
    int bufferSize)Opens an FSDataInputStream at the indicated Path. | 
| protected FSDataOutputStream | primitiveCreate(Path f,
               FsPermission absolutePermission,
               EnumSet<CreateFlag> flag,
               int bufferSize,
               short replication,
               long blockSize,
               Progressable progress,
               org.apache.hadoop.fs.Options.ChecksumOpt checksumOpt)Deprecated.  | 
| protected boolean | primitiveMkdir(Path f,
              FsPermission absolutePermission)Deprecated.  | 
| protected void | primitiveMkdir(Path f,
              FsPermission absolutePermission,
              boolean createParent)Deprecated.  | 
| static void | printStatistics()Print all statistics for all file systems to  System.out | 
| protected void | processDeleteOnExit()Delete all paths that were marked as delete-on-exit. | 
| void | removeAcl(Path path)Removes all but the base ACL entries of files and directories. | 
| void | removeAclEntries(Path path,
                List<AclEntry> aclSpec)Removes ACL entries from files and directories. | 
| void | removeDefaultAcl(Path path)Removes all default ACL entries from files and directories. | 
| void | removeXAttr(Path path,
           String name)Remove an xattr of a file or directory. | 
| abstract boolean | rename(Path src,
      Path dst)Renames Path src to Path dst. | 
| protected void | rename(Path src,
      Path dst,
      org.apache.hadoop.fs.Options.Rename... options)Deprecated.  | 
| void | renameSnapshot(Path path,
              String snapshotOldName,
              String snapshotNewName)Rename a snapshot. | 
| protected Path | resolveLink(Path f) | 
| Path | resolvePath(Path p)Return the fully-qualified path of path, resolving the path
 through any symlinks or mount point. | 
| void | setAcl(Path path,
      List<AclEntry> aclSpec)Fully replaces ACL of files and directories, discarding all existing
 entries. | 
| static void | setDefaultUri(Configuration conf,
             String uri)Set the default FileSystem URI in a configuration. | 
| static void | setDefaultUri(Configuration conf,
             URI uri)Set the default FileSystem URI in a configuration. | 
| void | setOwner(Path p,
        String username,
        String groupname)Set owner of a path (i.e. | 
| void | setPermission(Path p,
             FsPermission permission)Set permission of a path. | 
| boolean | setReplication(Path src,
              short replication)Set the replication for an existing file. | 
| void | setStoragePolicy(Path src,
                String policyName)Set the storage policy for a given file or directory. | 
| void | setTimes(Path p,
        long mtime,
        long atime)Set access time of a file. | 
| void | setVerifyChecksum(boolean verifyChecksum)Set the verify checksum flag. | 
| abstract void | setWorkingDirectory(Path new_dir)Set the current working directory for the given FileSystem. | 
| void | setWriteChecksum(boolean writeChecksum)Set the write checksum flag. | 
| void | setXAttr(Path path,
        String name,
        byte[] value)Set an xattr of a file or directory. | 
| void | setXAttr(Path path,
        String name,
        byte[] value,
        EnumSet<XAttrSetFlag> flag)Set an xattr of a file or directory. | 
| Path | startLocalOutput(Path fsOutputFile,
                Path tmpLocalFile)Returns a local file that the user can write output to. | 
| boolean | supportsSymlinks() | 
| boolean | truncate(Path f,
        long newLength)Truncate the file in the indicated path to the indicated size. | 
| void | unsetStoragePolicy(Path src)Unset the storage policy set for a given file or directory. | 
getConf, setConfpublic static final String FS_DEFAULT_NAME_KEY
public static final String DEFAULT_FS
public static final int SHUTDOWN_HOOK_PRIORITY
public static final String TRASH_PREFIX
public static final String USER_HOME_PREFIX
protected org.apache.hadoop.fs.FileSystem.Statistics statistics
public static FileSystem get(URI uri, Configuration conf, String user) throws IOException, InterruptedException
uri - of the filesystemconf - the configuration to useuser - to perform the get asIOException - failure to loadInterruptedException - If the UGI.doAs() call was
 somehow interrupted.public static FileSystem get(Configuration conf) throws IOException
conf - the configuration to useIOExceptionpublic static URI getDefaultUri(Configuration conf)
conf - the configuration to usepublic static void setDefaultUri(Configuration conf, URI uri)
conf - the configuration to alteruri - the new default filesystem uripublic static void setDefaultUri(Configuration conf, String uri)
conf - the configuration to alteruri - the new default filesystem uripublic void initialize(URI name, Configuration conf) throws IOException
name - a URI whose authority section names the host, port, etc.
   for this FileSystemconf - the configurationIOException - on any failure to initialize this instance.IllegalArgumentException - if the URI is considered invalid.public String getScheme()
 This implementation throws an UnsupportedOperationException.
UnsupportedOperationException - if the operation is unsupported
         (default).public abstract URI getUri()
protected URI getCanonicalUri()
canonicalizeUri(URI)
 on the filesystem's own URI, so subclasses typically only need to
 implement that method.canonicalizeUri(URI)protected URI canonicalizeUri(URI uri)
getDefaultPort() returns a
 default port.NetUtils.getCanonicalUri(URI, int)protected int getDefaultPort()
protected static FileSystem getFSofPath(Path absOrFqPath, Configuration conf) throws UnsupportedFileSystemException, IOException
@InterfaceAudience.Public @InterfaceStability.Evolving public String getCanonicalServiceName()
SecurityUtil.buildDTServiceName(URI, int)@Deprecated public String getName()
getUri() instead.@Deprecated public static FileSystem getNamed(String name, Configuration conf) throws IOException
get(URI, Configuration) instead.IOExceptionpublic static LocalFileSystem getLocal(Configuration conf) throws IOException
conf - the configuration to configure the FileSystem with
 if it is newly instantiated.IOException - if somehow the local FS cannot be instantiated.public static FileSystem get(URI uri, Configuration conf) throws IOException
"fs.$SCHEME.impl.disable.cache" set to true,
   a new instance will be created, initialized with the supplied URI and
   configuration, then returned without being cached.
 IOException - if the FileSystem cannot be instantiated.public static FileSystem newInstance(URI uri, Configuration conf, String user) throws IOException, InterruptedException
newInstance(URI, Configuration)uri - of the filesystemconf - the configuration to useuser - to perform the get asIOException - if the FileSystem cannot be instantiated.InterruptedException - If the UGI.doAs() call was
         somehow interrupted.public static FileSystem newInstance(URI uri, Configuration config) throws IOException
uri - FS URIconfig - configuration to useIOException - FS creation or initialization failure.public static FileSystem newInstance(Configuration conf) throws IOException
conf - the configuration to useIOException - FS creation or initialization failure.public static LocalFileSystem newInstanceLocal(Configuration conf) throws IOException
conf - the configuration to configure the FileSystem withIOException - FS creation or initialization failure.public static void closeAll()
                     throws IOException
IOException - a problem arose closing one or more filesystem.public static void closeAllForUGI(UserGroupInformation ugi) throws IOException
ugi - user group info to closeIOException - a problem arose closing one or more filesystem.public Path makeQualified(Path path)
path - to qualify.IllegalArgumentException - if the path has a schema/URI different
 from this FileSystem.Path.makeQualified(URI, Path)@InterfaceAudience.Public @InterfaceStability.Evolving public Token<?>[] addDelegationTokens(String renewer, Credentials credentials) throws IOException
renewer - the user allowed to renew the delegation tokenscredentials - cache in which to add new delegation tokensIOException - problems obtaining a tokenpublic static FSDataOutputStream create(FileSystem fs, Path file, FsPermission permission) throws IOException
fs - FileSystemfile - the name of the file to be createdpermission - the permission of the fileIOException - IO failurepublic static boolean mkdirs(FileSystem fs, Path dir, FsPermission permission) throws IOException
fs - FileSystem handledir - the name of the directory to be createdpermission - the permission of the directoryIOException - A problem creating the directories.create(FileSystem, Path, FsPermission)protected void checkPath(Path path)
path - to checkIllegalArgumentException - if the path is not considered to be
 part of this FileSystem.public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len) throws IOException
null is returned.
 
   if f == null :
     result = null
   elif f.getLen() <= start:
     result = []
   else result = [ locations(FS, b) for b in blocks(FS, p, s, s+l)]
 
 This call is most helpful with and distributed filesystem
 where the hostnames of machines that contain blocks of the given file
 can be determined.
 The default implementation returns an array containing one element:
 
 BlockLocation( { "localhost:9866" },  { "localhost" }, 0, file.getLen())
 
 In HDFS, if file is three-replicated, the returned array contains
 elements like:
 
 BlockLocation(offset: 0, length: BLOCK_SIZE,
   hosts: {"host1:9866", "host2:9866, host3:9866"})
 BlockLocation(offset: BLOCK_SIZE, length: BLOCK_SIZE,
   hosts: {"host2:9866", "host3:9866, host4:9866"})
 
 And if a file is erasure-coded, the returned BlockLocation are logical
 block groups.
 Suppose we have a RS_3_2 coded file (3 data units and 2 parity units).
 1. If the file size is less than one stripe size, say 2 * CELL_SIZE, then
 there will be one BlockLocation returned, with 0 offset, actual file size
 and 4 hosts (2 data blocks and 2 parity blocks) hosting the actual blocks.
 3. If the file size is less than one group size but greater than one
 stripe size, then there will be one BlockLocation returned, with 0 offset,
 actual file size with 5 hosts (3 data blocks and 2 parity blocks) hosting
 the actual blocks.
 4. If the file size is greater than one group size, 3 * BLOCK_SIZE + 123
 for example, then the result will be like:
 
 BlockLocation(offset: 0, length: 3 * BLOCK_SIZE, hosts: {"host1:9866",
   "host2:9866","host3:9866","host4:9866","host5:9866"})
 BlockLocation(offset: 3 * BLOCK_SIZE, length: 123, hosts: {"host1:9866",
   "host4:9866", "host5:9866"})
 file - FilesStatus to get data fromstart - offset into the given filelen - length for which to get locations forIOException - IO failurepublic BlockLocation[] getFileBlockLocations(Path p, long start, long len) throws IOException
null is returned.
 This call is most helpful with location-aware distributed
 filesystems, where it returns hostnames of machines that
 contain the given file.
 A FileSystem will normally return the equivalent result
 of passing the FileStatus of the path to
 getFileBlockLocations(FileStatus, long, long)p - path is used to identify an FS since an FS could have
          another FS that it could be delegating the call tostart - offset into the given filelen - length for which to get locations forFileNotFoundException - when the path does not existIOException - IO failure@Deprecated public FsServerDefaults getServerDefaults() throws IOException
getServerDefaults(Path) insteadIOException - IO failurepublic FsServerDefaults getServerDefaults(Path p) throws IOException
p - path is used to identify an FS since an FS could have
          another FS that it could be delegating the call toIOException - IO failurepublic Path resolvePath(Path p) throws IOException
p - path to be resolvedFileNotFoundException - if the path is not presentIOException - for any other errorpublic abstract FSDataInputStream open(Path f, int bufferSize) throws IOException
f - the file name to openbufferSize - the size of the buffer to be used.IOException - IO failurepublic FSDataInputStream open(Path f) throws IOException
f - the file to openIOException - IO failurepublic FSDataOutputStream create(Path f) throws IOException
f - the file to createIOException - IO failurepublic FSDataOutputStream create(Path f, boolean overwrite) throws IOException
f - the file to createoverwrite - if a file with this name already exists, then if true,
   the file will be overwritten, and if false an exception will be thrown.IOException - IO failurepublic FSDataOutputStream create(Path f, Progressable progress) throws IOException
f - the file to createprogress - to report progressIOException - IO failurepublic FSDataOutputStream create(Path f, short replication) throws IOException
f - the file to createreplication - the replication factorIOException - IO failurepublic FSDataOutputStream create(Path f, short replication, Progressable progress) throws IOException
f - the file to createreplication - the replication factorprogress - to report progressIOException - IO failurepublic FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException
f - the file to createoverwrite - if a path with this name already exists, then if true,
   the file will be overwritten, and if false an error will be thrown.bufferSize - the size of the buffer to be used.IOException - IO failurepublic FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress) throws IOException
FSDataOutputStream at the indicated Path
 with write-progress reporting.
 The frequency of callbacks is implementation-specific; it may be "none".f - the path of the file to openoverwrite - if a file with this name already exists, then if true,
   the file will be overwritten, and if false an error will be thrown.bufferSize - the size of the buffer to be used.IOException - IO failurepublic FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize) throws IOException
f - the file name to openoverwrite - if a file with this name already exists, then if true,
   the file will be overwritten, and if false an error will be thrown.bufferSize - the size of the buffer to be used.replication - required block replication for the file.IOException - IO failurepublic FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException
f - the file name to openoverwrite - if a file with this name already exists, then if true,
   the file will be overwritten, and if false an error will be thrown.bufferSize - the size of the buffer to be used.replication - required block replication for the file.IOException - IO failurepublic abstract FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException
f - the file name to openpermission - file permissionoverwrite - if a file with this name already exists, then if true,
   the file will be overwritten, and if false an error will be thrown.bufferSize - the size of the buffer to be used.replication - required block replication for the file.blockSize - block sizeprogress - the progress reporterIOException - IO failuresetPermission(Path, FsPermission)public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException
f - the file name to openpermission - file permissionflags - CreateFlags to use for this stream.bufferSize - the size of the buffer to be used.replication - required block replication for the file.blockSize - block sizeprogress - the progress reporterIOException - IO failuresetPermission(Path, FsPermission)public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize, Progressable progress, org.apache.hadoop.fs.Options.ChecksumOpt checksumOpt) throws IOException
f - the file name to openpermission - file permissionflags - CreateFlags to use for this stream.bufferSize - the size of the buffer to be used.replication - required block replication for the file.blockSize - block sizeprogress - the progress reporterchecksumOpt - checksum parameter. If null, the values
        found in conf will be used.IOException - IO failuresetPermission(Path, FsPermission)@Deprecated protected FSDataOutputStream primitiveCreate(Path f, FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize, Progressable progress, org.apache.hadoop.fs.Options.ChecksumOpt checksumOpt) throws IOException
IOException - IO failure@Deprecated protected boolean primitiveMkdir(Path f, FsPermission absolutePermission) throws IOException
f - pathabsolutePermission - permissionsIOException - IO failuremkdirs(Path, FsPermission)@Deprecated protected void primitiveMkdir(Path f, FsPermission absolutePermission, boolean createParent) throws IOException
IOExceptionpublic FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException
f - the file name to openoverwrite - if a file with this name already exists, then if true,
 the file will be overwritten, and if false an error will be thrown.bufferSize - the size of the buffer to be used.replication - required block replication for the file.blockSize - block sizeprogress - the progress reporterIOException - IO failuresetPermission(Path, FsPermission)public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException
f - the file name to openpermission - file permissionoverwrite - if a file with this name already exists, then if true,
 the file will be overwritten, and if false an error will be thrown.bufferSize - the size of the buffer to be used.replication - required block replication for the file.blockSize - block sizeprogress - the progress reporterIOException - IO failuresetPermission(Path, FsPermission)public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException
f - the file name to openpermission - file permissionflags - CreateFlags to use for this stream.bufferSize - the size of the buffer to be used.replication - required block replication for the file.blockSize - block sizeprogress - the progress reporterIOException - IO failuresetPermission(Path, FsPermission)public boolean createNewFile(Path f) throws IOException
f - path to use for createIOException - IO failurepublic FSDataOutputStream append(Path f) throws IOException
append(f, getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
     IO_FILE_BUFFER_SIZE_DEFAULT), null)f - the existing file to be appended.IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default).public FSDataOutputStream append(Path f, int bufferSize) throws IOException
f - the existing file to be appended.bufferSize - the size of the buffer to be used.IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default).public abstract FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException
f - the existing file to be appended.bufferSize - the size of the buffer to be used.progress - for reporting progress if it is not null.IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default).public void concat(Path trg, Path[] psrcs) throws IOException
trg - the path to the target destination.psrcs - the paths to the sources to use for the concatenation.IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default).@Deprecated public short getReplication(Path src) throws IOException
getFileStatus(Path) insteadsrc - file nameFileNotFoundException - if the path does not resolve.IOException - an IO failurepublic boolean setReplication(Path src, short replication) throws IOException
src - file namereplication - new replicationIOExceptionpublic abstract boolean rename(Path src, Path dst) throws IOException
src - path to be renameddst - new path after renameIOException - on failure@Deprecated protected void rename(Path src, Path dst, org.apache.hadoop.fs.Options.Rename... options) throws IOException
If OVERWRITE option is not passed as an argument, rename fails if the dst already exists.
If OVERWRITE option is passed as an argument, rename overwrites the dst if it is a file or an empty directory. Rename fails if dst is a non-empty directory.
Note that atomicity of rename is dependent on the file system implementation. Please refer to the file system documentation for details. This default implementation is non atomic.
This method is deprecated since it is a temporary method added to support the transition from FileSystem to FileContext for user applications.
src - path to be renameddst - new path after renameFileNotFoundException - src path does not exist, or the parent
 path of dst does not exist.FileAlreadyExistsException - dest path exists and is a fileParentNotDirectoryException - if the parent path of dest is not
 a directoryIOException - on failurepublic boolean truncate(Path f, long newLength) throws IOException
f - The path to the file to be truncatednewLength - The size the file is to be truncated totrue if the file has been truncated to the desired
 newLength and is immediately available to be reused for
 write operations such as append, or
 false if a background process of adjusting the length of
 the last block has been started, and clients should wait for it to
 complete before proceeding with further file updates.IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default).@Deprecated public boolean delete(Path f) throws IOException
delete(Path, boolean) instead.IOExceptionpublic abstract boolean delete(Path f, boolean recursive) throws IOException
f - the path to delete.recursive - if path is a directory and set to
 true, the directory is deleted else throws an exception. In
 case of a file the recursive can be set to either true or false.IOException - IO failurepublic boolean deleteOnExit(Path f) throws IOException
createFileSystem(URI, Configuration), then the paths will
 be deleted in when close() is called on that instance.
 The path must exist in the filesystem at the time of the method call;
 it does not have to exist at the time of JVM shutdown.
 Notes
 f - the path to delete.IOException - IO failurepublic boolean cancelDeleteOnExit(Path f)
f - the path to cancel deletionprotected void processDeleteOnExit()
O(paths), with the actual
 time dependent on the time for existence and deletion operations to
 complete, successfully or not.public boolean exists(Path f) throws IOException
getFileStatus(Path) calls, as this will involve multiple redundant
 RPC calls in HDFS.f - source pathIOException - IO failure@Deprecated public boolean isDirectory(Path f) throws IOException
getFileStatus(Path) insteadf - path to checkIOException - IO failure@Deprecated public boolean isFile(Path f) throws IOException
getFileStatus(Path) insteadgetFileStatus(Path) or listStatus() methods.f - path to checkIOException - IO failure@Deprecated public long getLength(Path f) throws IOException
getFileStatus(Path) instead.FileNotFoundException - if the path does not resolveIOException - IO failurepublic ContentSummary getContentSummary(Path f) throws IOException
ContentSummary of a given Path.f - path to useFileNotFoundException - if the path does not resolveIOException - IO failurepublic QuotaUsage getQuotaUsage(Path f) throws IOException
QuotaUsage of a given Path.f - path to useIOException - IO failurepublic abstract FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException
Does not guarantee to return the List of files/directories status in a sorted order.
Will not return null. Expect IOException upon access error.
f - given pathFileNotFoundException - when the path does not existIOException - see specific implementationpublic org.apache.hadoop.fs.RemoteIterator<Path> listCorruptFileBlocks(Path path) throws IOException
UnsupportedOperationException - if the operation is unsupported
         (default).IOException - IO failurepublic FileStatus[] listStatus(Path f, PathFilter filter) throws FileNotFoundException, IOException
Does not guarantee to return the List of files/directories status in a sorted order.
f - a path namefilter - the user-supplied path filterFileNotFoundException - when the path does not existIOException - see specific implementationpublic FileStatus[] listStatus(Path[] files) throws FileNotFoundException, IOException
Does not guarantee to return the List of files/directories status in a sorted order.
files - a list of pathsFileNotFoundException - when the path does not existIOException - see specific implementationpublic FileStatus[] listStatus(Path[] files, PathFilter filter) throws FileNotFoundException, IOException
Does not guarantee to return the List of files/directories status in a sorted order.
files - a list of pathsfilter - the user-supplied path filterFileNotFoundException - when the path does not existIOException - see specific implementationpublic FileStatus[] globStatus(Path pathPattern) throws IOException
Return all the files that match filePattern and are not checksum files. Results are sorted by their names.
A filename pattern is composed of regular characters and special pattern matching characters, which are:
pathPattern - a regular expression specifying a pth patternIOException - IO failurepublic FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException
FileStatus objects whose path names match
 pathPattern and is accepted by the user-supplied path filter.
 Results are sorted by their path names.pathPattern - a regular expression specifying the path patternfilter - a user-supplied path filterpathPattern has no glob and the path does not exist
         an empty array if pathPattern has a glob and no path
         matches it else an array of FileStatus objects matching the
         patternIOException - if any I/O error occurs when fetching file statuspublic org.apache.hadoop.fs.RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f) throws FileNotFoundException, IOException
f - is the pathFileNotFoundException - If f does not existIOException - If an I/O error occurredprotected org.apache.hadoop.fs.RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f, PathFilter filter) throws FileNotFoundException, IOException
f - a pathfilter - a path filterFileNotFoundException - if f does not existIOException - if any I/O error occurredpublic org.apache.hadoop.fs.RemoteIterator<FileStatus> listStatusIterator(Path p) throws FileNotFoundException, IOException
p - target pathFileNotFoundException - if p does not existIOException - if any I/O error occurredpublic org.apache.hadoop.fs.RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive) throws FileNotFoundException, IOException
If the path is a directory, if recursive is false, returns files in the directory; if recursive is true, return files in the subtree rooted at the path. If the path is a file, return the file's status and block locations.
f - is the pathrecursive - if the subdirectories need to be traversed recursivelyFileNotFoundException - when the path does not exist;IOException - see specific implementationpublic Path getHomeDirectory()
"/user/$USER/".public abstract void setWorkingDirectory(Path new_dir)
new_dir - Path of new working directorypublic abstract Path getWorkingDirectory()
protected Path getInitialWorkingDirectory()
public boolean mkdirs(Path f) throws IOException
mkdirs(Path, FsPermission) with default permission.f - pathIOException - IO failurepublic abstract boolean mkdirs(Path f, FsPermission permission) throws IOException
f - path to createpermission - to apply to fIOException - IO failurepublic void copyFromLocalFile(Path src, Path dst) throws IOException
src - pathdst - pathIOException - IO failurepublic void moveFromLocalFile(Path[] srcs, Path dst) throws IOException
srcs - source pathsdst - pathIOException - IO failurepublic void moveFromLocalFile(Path src, Path dst) throws IOException
src - local pathdst - pathIOException - IO failurepublic void copyFromLocalFile(boolean delSrc,
                              Path src,
                              Path dst)
                       throws IOException
delSrc - whether to delete the srcsrc - pathdst - pathIOExceptionpublic void copyFromLocalFile(boolean delSrc,
                              boolean overwrite,
                              Path[] srcs,
                              Path dst)
                       throws IOException
delSrc - whether to delete the srcoverwrite - whether to overwrite an existing filesrcs - array of paths which are sourcedst - pathIOException - IO failurepublic void copyFromLocalFile(boolean delSrc,
                              boolean overwrite,
                              Path src,
                              Path dst)
                       throws IOException
delSrc - whether to delete the srcoverwrite - whether to overwrite an existing filesrc - pathdst - pathIOException - IO failurepublic void copyToLocalFile(Path src, Path dst) throws IOException
src - path src file in the remote filesystemdst - path local destinationIOException - IO failurepublic void moveToLocalFile(Path src, Path dst) throws IOException
src - path src file in the remote filesystemdst - path local destinationIOException - IO failurepublic void copyToLocalFile(boolean delSrc,
                            Path src,
                            Path dst)
                     throws IOException
delSrc - whether to delete the srcsrc - path src file in the remote filesystemdst - path local destinationIOException - IO failurepublic void copyToLocalFile(boolean delSrc,
                            Path src,
                            Path dst,
                            boolean useRawLocalFileSystem)
                     throws IOException
delSrc - whether to delete the srcsrc - pathdst - pathuseRawLocalFileSystem - whether to use RawLocalFileSystem as local file system or not.IOException - for any IO errorpublic Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException
fsOutputFile - path of output filetmpLocalFile - path of local tmp fileIOException - IO failurepublic void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException
fsOutputFile - path of output filetmpLocalFile - path to local tmp fileIOException - IO failurepublic void close()
           throws IOException
deleteOnExit(Path), and remove this FS instance
 from the cache, if cached.
 After this operation, the outcome of any method call on this FileSystem
 instance, or any input/output stream created by it is undefined.close in interface Closeableclose in interface AutoCloseableIOException - IO failurepublic long getUsed()
             throws IOException
IOException - IO failurepublic long getUsed(Path path) throws IOException
IOException - IO failure@Deprecated public long getBlockSize(Path f) throws IOException
getFileStatus(Path) insteadf - the filenameFileNotFoundException - if the path is not presentIOException - IO failure@Deprecated public long getDefaultBlockSize()
getDefaultBlockSize(Path) insteadpublic long getDefaultBlockSize(Path f)
f - path of file@Deprecated public short getDefaultReplication()
getDefaultReplication(Path) insteadpublic short getDefaultReplication(Path path)
path - of the filepublic abstract FileStatus getFileStatus(Path f) throws IOException
f - The path we want information fromFileNotFoundException - when the path does not existIOException - see specific implementationpublic void createSymlink(Path target, Path link, boolean createParent) throws AccessControlException, FileAlreadyExistsException, FileNotFoundException, ParentNotDirectoryException, UnsupportedFileSystemException, IOException
public FileStatus getFileLinkStatus(Path f) throws AccessControlException, FileNotFoundException, UnsupportedFileSystemException, IOException
FileNotFoundException - when the path does not existIOException - see specific implementationAccessControlExceptionUnsupportedFileSystemExceptionpublic boolean supportsSymlinks()
public Path getLinkTarget(Path f) throws IOException
UnsupportedOperationException - if the operation is unsupported
         (default outcome).IOExceptionprotected Path resolveLink(Path f) throws IOException
UnsupportedOperationException - if the operation is unsupported
         (default outcome).IOExceptionpublic FileChecksum getFileChecksum(Path f) throws IOException
f - The file pathIOException - IO failurepublic FileChecksum getFileChecksum(Path f, long length) throws IOException
f - The file pathlength - The length of the file range for checksum calculationIOException - IO failurepublic void setVerifyChecksum(boolean verifyChecksum)
verifyChecksum - Verify checksum flagpublic void setWriteChecksum(boolean writeChecksum)
writeChecksum - Write checksum flagpublic FsStatus getStatus() throws IOException
IOException - see specific implementationpublic FsStatus getStatus(Path p) throws IOException
p - Path for which status should be obtained. null means
 the default partition.IOException - see specific implementationpublic void setPermission(Path p, FsPermission permission) throws IOException
p - The pathpermission - permissionIOException - IO failurepublic void setOwner(Path p, String username, String groupname) throws IOException
p - The pathusername - If it is null, the original username remains unchanged.groupname - If it is null, the original groupname remains unchanged.IOException - IO failurepublic void setTimes(Path p, long mtime, long atime) throws IOException
p - The pathmtime - Set the modification time of this file.
              The number of milliseconds since Jan 1, 1970.
              A value of -1 means that this call should not set modification time.atime - Set the access time of this file.
              The number of milliseconds since Jan 1, 1970.
              A value of -1 means that this call should not set access time.IOException - IO failurepublic final Path createSnapshot(Path path) throws IOException
path - The directory where snapshots will be taken.IOException - IO failureUnsupportedOperationException - if the operation is unsupportedpublic Path createSnapshot(Path path, String snapshotName) throws IOException
path - The directory where snapshots will be taken.snapshotName - The name of the snapshotIOException - IO failureUnsupportedOperationException - if the operation is unsupportedpublic void renameSnapshot(Path path, String snapshotOldName, String snapshotNewName) throws IOException
path - The directory path where the snapshot was takensnapshotOldName - Old name of the snapshotsnapshotNewName - New name of the snapshotIOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public void deleteSnapshot(Path path, String snapshotName) throws IOException
path - The directory that the to-be-deleted snapshot belongs tosnapshotName - The name of the snapshotIOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public void modifyAclEntries(Path path, List<AclEntry> aclSpec) throws IOException
path - Path to modifyaclSpec - ListIOException - if an ACL could not be modifiedUnsupportedOperationException - if the operation is unsupported
         (default outcome).public void removeAclEntries(Path path, List<AclEntry> aclSpec) throws IOException
path - Path to modifyaclSpec - List describing entries to removeIOException - if an ACL could not be modifiedUnsupportedOperationException - if the operation is unsupported
         (default outcome).public void removeDefaultAcl(Path path) throws IOException
path - Path to modifyIOException - if an ACL could not be modifiedUnsupportedOperationException - if the operation is unsupported
         (default outcome).public void removeAcl(Path path) throws IOException
path - Path to modifyIOException - if an ACL could not be removedUnsupportedOperationException - if the operation is unsupported
         (default outcome).public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException
path - Path to modifyaclSpec - List describing modifications, which must include entries
   for user, group, and others for compatibility with permission bits.IOException - if an ACL could not be modifiedUnsupportedOperationException - if the operation is unsupported
         (default outcome).public AclStatus getAclStatus(Path path) throws IOException
path - Path to getIOException - if an ACL could not be readUnsupportedOperationException - if the operation is unsupported
         (default outcome).public void setXAttr(Path path, String name, byte[] value) throws IOException
Refer to the HDFS extended attributes user documentation for details.
path - Path to modifyname - xattr name.value - xattr value.IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public void setXAttr(Path path, String name, byte[] value, EnumSet<XAttrSetFlag> flag) throws IOException
Refer to the HDFS extended attributes user documentation for details.
path - Path to modifyname - xattr name.value - xattr value.flag - xattr set flagIOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public byte[] getXAttr(Path path, String name) throws IOException
Refer to the HDFS extended attributes user documentation for details.
path - Path to get extended attributename - xattr name.IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public Map<String,byte[]> getXAttrs(Path path) throws IOException
Refer to the HDFS extended attributes user documentation for details.
path - Path to get extended attributesIOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public Map<String,byte[]> getXAttrs(Path path, List<String> names) throws IOException
Refer to the HDFS extended attributes user documentation for details.
path - Path to get extended attributesnames - XAttr names.IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public List<String> listXAttrs(Path path) throws IOException
Refer to the HDFS extended attributes user documentation for details.
path - Path to get extended attributesIOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public void removeXAttr(Path path, String name) throws IOException
Refer to the HDFS extended attributes user documentation for details.
path - Path to remove extended attributename - xattr nameIOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public void setStoragePolicy(Path src, String policyName) throws IOException
src - file or directory path.policyName - the name of the target storage policy. The list
                   of supported Storage policies can be retrieved
                   via getAllStoragePolicies().IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public void unsetStoragePolicy(Path src) throws IOException
src - file or directory path.IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public BlockStoragePolicySpi getStoragePolicy(Path src) throws IOException
src - file or directory path.IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public Collection<? extends BlockStoragePolicySpi> getAllStoragePolicies() throws IOException
IOException - IO failureUnsupportedOperationException - if the operation is unsupported
         (default outcome).public Path getTrashRoot(Path path)
path - the trash root of the path to be determined./user/$USER/.Trashpublic Collection<FileStatus> getTrashRoots(boolean allUsers)
allUsers - return trash roots for all users if true./user/$USER/.Trash exists.public static Class<? extends FileSystem> getFileSystemClass(String scheme, Configuration conf) throws IOException
ServiceLoaderscheme - URL scheme of FSconf - configuration: can be null, in which case the check for
 a filesystem binding declaration in the configuration is skipped.UnsupportedFileSystemException - if there was no known implementation
         for the scheme.IOException - if the filesystem could not be loaded@Deprecated public static Map<String,org.apache.hadoop.fs.FileSystem.Statistics> getStatistics()
getGlobalStorageStatistics()@Deprecated public static List<org.apache.hadoop.fs.FileSystem.Statistics> getAllStatistics()
getGlobalStorageStatistics()@Deprecated public static org.apache.hadoop.fs.FileSystem.Statistics getStatistics(String scheme, Class<? extends FileSystem> cls)
getGlobalStorageStatistics()cls - the class to lookuppublic static void clearStatistics()
public static void printStatistics()
                            throws IOException
System.outIOExceptionpublic static boolean areSymlinksEnabled()
public static void enableSymlinks()
public StorageStatistics getStorageStatistics()
This is a default method which is intended to be overridden by subclasses. The default implementation returns an empty storage statistics object.
public static GlobalStorageStatistics getGlobalStorageStatistics()
public FSDataOutputStreamBuilder createFile(Path path)
path - file pathpublic FSDataOutputStreamBuilder appendFile(Path path)
path - file path.FSDataOutputStreamBuilder to build file append request.Copyright © 2018 Apache Software Foundation. All rights reserved.