@InterfaceAudience.Public @InterfaceStability.Stable public abstract class FileSystem extends Configured implements Closeable, org.apache.hadoop.security.token.DelegationTokenIssuer, org.apache.hadoop.fs.PathCapabilities
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.
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
.Important note for developers
If you are making changes here to the public API or protected methods, you must review the following subclasses and make sure that they are filtering/passing through new methods as appropriate.FilterFileSystem
: methods are passed through. If not,
then TestFilterFileSystem.MustNotImplement
must be
updated with the unsupported interface.
Furthermore, if the new API's support is probed for via
hasPathCapability(Path, String)
then
FilterFileSystem.hasPathCapability(Path, String)
must return false, always.
ChecksumFileSystem
: checksums are created and
verified.
TestHarFileSystem
will need its MustNotImplement
interface updated.
There are some external places your changes will break things. Do co-ordinate changes here.
HBase: HBossHive: HiveShim23
shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
Modifier and Type | Field and Description |
---|---|
static String |
DEFAULT_FS |
static String |
FS_DEFAULT_NAME_KEY |
static int |
SHUTDOWN_HOOK_PRIORITY
Priority of the FileSystem shutdown hook: 10.
|
protected org.apache.hadoop.fs.FileSystem.Statistics |
statistics
The statistics for this file system.
|
static String |
TRASH_PREFIX
Prefix for trash directory: ".Trash".
|
static String |
USER_HOME_PREFIX |
Modifier | Constructor and Description |
---|---|
protected |
FileSystem() |
Modifier and Type | Method and Description |
---|---|
FSDataOutputStream |
append(Path f)
Append to an existing file (optional operation).
|
FSDataOutputStream |
append(Path f,
boolean appendToNewBlock)
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).
|
FSDataOutputStream |
append(Path f,
int bufferSize,
Progressable progress,
boolean appendToNewBlock)
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
FSDataOutputStream at 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.
|
protected static FSDataOutputStreamBuilder |
createDataOutputStreamBuilder(FileSystem fileSystem,
Path path)
Create instance of the standard FSDataOutputStreamBuilder for the
given filesystem and path.
|
FSDataOutputStreamBuilder |
createFile(Path path)
Create a new FSDataOutputStreamBuilder for the file with path.
|
org.apache.hadoop.fs.MultipartUploaderBuilder |
createMultipartUploader(Path basePath)
Create a multipart uploader.
|
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.
|
protected PathHandle |
createPathHandle(FileStatus stat,
org.apache.hadoop.fs.Options.HandleOpt... opt)
Hook to implement support for
PathHandle operations. |
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
ContentSummary of a given Path . |
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. |
PathHandle |
getPathHandle(FileStatus stat,
org.apache.hadoop.fs.Options.HandleOpt... opt)
Create a durable, serializable handle to the referent of the given
entity.
|
QuotaUsage |
getQuotaUsage(Path f)
Return the
QuotaUsage of a given Path . |
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
FileStatus objects whose path names match
pathPattern and is accepted by the user-supplied path filter. |
boolean |
hasPathCapability(Path path,
String capability)
The base FileSystem implementation generally has no knowledge
of the capabilities of actual implementations.
|
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).
|
void |
msync()
Synchronize client metadata state.
|
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.
|
FSDataInputStream |
open(PathHandle fd)
Open an FSDataInputStream matching the PathHandle instance.
|
FSDataInputStream |
open(PathHandle fd,
int bufferSize)
Open an FSDataInputStream matching the PathHandle instance.
|
abstract FSDataInputStream |
open(Path f,
int bufferSize)
Opens an FSDataInputStream at the indicated Path.
|
FutureDataInputStreamBuilder |
openFile(Path path)
Open a file for reading through a builder API.
|
FutureDataInputStreamBuilder |
openFile(PathHandle pathHandle)
Open a file for reading through a builder API.
|
protected CompletableFuture<FSDataInputStream> |
openFileWithOptions(PathHandle pathHandle,
org.apache.hadoop.fs.impl.OpenFileParameters parameters)
Execute the actual open file operation.
|
protected CompletableFuture<FSDataInputStream> |
openFileWithOptions(Path path,
org.apache.hadoop.fs.impl.OpenFileParameters parameters)
Execute the actual open file operation.
|
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 |
satisfyStoragePolicy(Path path)
Set the source path to satisfy storage policy.
|
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.
|
void |
setQuota(Path src,
long namespaceQuota,
long storagespaceQuota)
Set quota for the given
Path . |
void |
setQuotaByStorageType(Path src,
StorageType type,
long quota)
Set per storage type quota for the given
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, setConf
public 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 useIOException
- If an I/O error occurred.public 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.uri
- url.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()
getCanonicalServiceName
in interface org.apache.hadoop.security.token.DelegationTokenIssuer
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.name
- name.conf
- configuration.IOException
- If an I/O error occurred.public 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.
uri
- uri of the filesystem.conf
- configrution.IOException
- if the FileSystem cannot be instantiated.public static FileSystem newInstance(URI uri, Configuration conf, String user) throws IOException, InterruptedException
newInstance(URI, Configuration)
uri
- uri of the filesystem.conf
- 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)
public 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 FSDataInputStream open(PathHandle fd) throws IOException
fd
- PathHandle object returned by the FS authority.InvalidPathHandleException
- If PathHandle
constraints are
not satisfiedIOException
- IO failureUnsupportedOperationException
- If open(PathHandle, int)
not overridden by subclasspublic FSDataInputStream open(PathHandle fd, int bufferSize) throws IOException
fd
- PathHandle object returned by the FS authority.bufferSize
- the size of the buffer to useInvalidPathHandleException
- If PathHandle
constraints are
not satisfiedIOException
- IO failureUnsupportedOperationException
- If not overridden by subclasspublic final PathHandle getPathHandle(FileStatus stat, org.apache.hadoop.fs.Options.HandleOpt... opt)
stat
- Referent in the target FileSystemopt
- If absent, assume Options.HandleOpt.path()
.IllegalArgumentException
- If the FileStatus does not belong to
this FileSystemUnsupportedOperationException
- If createPathHandle(org.apache.hadoop.fs.FileStatus, org.apache.hadoop.fs.Options.HandleOpt...)
not overridden by subclass.UnsupportedOperationException
- If this FileSystem cannot enforce
the specified constraints.protected PathHandle createPathHandle(FileStatus stat, org.apache.hadoop.fs.Options.HandleOpt... opt)
PathHandle
operations.stat
- Referent in the target FileSystemopt
- Constraints that determine the validity of the
PathHandle
reference.public 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.progress
- to report progress.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.blockSize
- the size of the buffer to be used.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.blockSize
- the size of the buffer to be used.progress
- to report progress.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
- CreateFlag
s 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
- CreateFlag
s 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
f
- path.absolutePermission
- permission.flag
- create flag.bufferSize
- buffer size.replication
- replication.blockSize
- block size.progress
- progress.checksumOpt
- check sum opt.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
f
- the path.absolutePermission
- permission.createParent
- create parent.IOException
- IO failure.public 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
- CreateFlag
s 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 FSDataOutputStream append(Path f, boolean appendToNewBlock) throws IOException
f
- the existing file to be appended.appendToNewBlock
- whether to append data to a new block
instead of the end of the last partial blockIOException
- IO failureUnsupportedOperationException
- if the operation is unsupported
(default).public FSDataOutputStream append(Path f, int bufferSize, Progressable progress, boolean appendToNewBlock) 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.appendToNewBlock
- whether to append data to a new block
instead of the end of the last partial blockIOException
- 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 replicationIOException
- an IO failure.public 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 renameoptions
- rename options.FileNotFoundException
- 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.f
- the path.IOException
- IO failure.public 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.f
- the path.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 void setQuota(Path src, long namespaceQuota, long storagespaceQuota) throws IOException
Path
.src
- the target path to set quota fornamespaceQuota
- the namespace quota (i.e., # of files/directories)
to setstoragespaceQuota
- the storage space quota to setIOException
- IO failurepublic void setQuotaByStorageType(Path src, StorageType type, long quota) throws IOException
Path
.src
- the target path to set storage type quota fortype
- the storage type to setquota
- the quota to set for the given storage typeIOException
- 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
path
- the path.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 glob specifying a path 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 glob 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
- pathIOException
- IO failure.public 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 Closeable
close
in interface AutoCloseable
IOException
- IO failurepublic long getUsed() throws IOException
IOException
- IO failurepublic long getUsed(Path path) throws IOException
path
- the path.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 msync() throws IOException, UnsupportedOperationException
In some FileSystem implementations such as HDFS metadata synchronization is essential to guarantee consistency of read requests particularly in HA setting.
IOException
- If an I/O error occurred.UnsupportedOperationException
- if the operation is unsupported.protected Path fixRelativePart(Path p)
p
- the path.public void createSymlink(Path target, Path link, boolean createParent) throws AccessControlException, FileAlreadyExistsException, FileNotFoundException, ParentNotDirectoryException, UnsupportedFileSystemException, IOException
target
- target path.link
- link.createParent
- create parent.AccessControlException
- if access is denied.FileAlreadyExistsException
- when the path does not exist.FileNotFoundException
- when the path does not exist.ParentNotDirectoryException
- if the parent path of dest is not
a directory.UnsupportedFileSystemException
- if there was no known implementation
for the scheme.IOException
- raised on errors performing I/O.public FileStatus getFileLinkStatus(Path f) throws AccessControlException, FileNotFoundException, UnsupportedFileSystemException, IOException
f
- the path.AccessControlException
- if access is denied.FileNotFoundException
- when the path does not exist.IOException
- raised on errors performing I/O.UnsupportedFileSystemException
- if there was no known implementation
for the scheme.public boolean supportsSymlinks()
public Path getLinkTarget(Path f) throws IOException
f
- the path.UnsupportedOperationException
- if the operation is unsupported
(default outcome).IOException
- IO failure.protected Path resolveLink(Path f) throws IOException
f
- the path.UnsupportedOperationException
- if the operation is unsupported
(default outcome).IOException
- IO failure.public 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
- List<AclEntry> describing modificationsIOException
- 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 satisfyStoragePolicy(Path path) throws IOException
path
- The source path referring to either a directory or a file.IOException
- If an I/O error occurred.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/.Trash
public Collection<FileStatus> getTrashRoots(boolean allUsers)
allUsers
- return trash roots for all users if true./user/$USER/.Trash
exists.public boolean hasPathCapability(Path path, String capability) throws IOException
true
, this instance is explicitly
declaring that the capability is available.
If the function returns false
, it can mean one of:
Implementors: PathCapabilitiesSupport
can be used to help implement this method.
hasPathCapability
in interface org.apache.hadoop.fs.PathCapabilities
path
- path to query the capability of.capability
- non-null, non-empty string to query the path for support.IOException
- this should not be raised, except on problems
resolving paths or relaying the call.public static Class<? extends FileSystem> getFileSystemClass(String scheme, Configuration conf) throws IOException
ServiceLoader
scheme
- 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()
scheme
- scheme.cls
- the class to lookuppublic static void clearStatistics()
public static void printStatistics() throws IOException
System.out
IOException
- If an I/O error occurred.public 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()
@InterfaceStability.Unstable protected static FSDataOutputStreamBuilder createDataOutputStreamBuilder(@Nonnull FileSystem fileSystem, @Nonnull Path path)
fileSystem
- ownerpath
- path to createpublic FSDataOutputStreamBuilder createFile(Path path)
path
- file pathpublic FSDataOutputStreamBuilder appendFile(Path path)
path
- file path.FSDataOutputStreamBuilder
to build file append request.@InterfaceStability.Unstable public FutureDataInputStreamBuilder openFile(Path path) throws IOException, UnsupportedOperationException
open(Path, int)
unless a subclass
executes the open command differently.
The semantics of this call are therefore the same as that of
open(Path, int)
with one special point: it is in
FSDataInputStreamBuilder.build()
in which the open operation
takes place -it is there where all preconditions to the operation
are checked.path
- file pathIOException
- if some early checks cause IO failures.UnsupportedOperationException
- if support is checked early.@InterfaceStability.Unstable public FutureDataInputStreamBuilder openFile(PathHandle pathHandle) throws IOException, UnsupportedOperationException
open(PathHandle, int)
unless a subclass
executes the open command differently.
If PathHandles are unsupported, this may fail in the
FSDataInputStreamBuilder.build()
command,
rather than in this openFile()
operation.pathHandle
- path handle.IOException
- if some early checks cause IO failures.UnsupportedOperationException
- if support is checked early.protected CompletableFuture<FSDataInputStream> openFileWithOptions(Path path, org.apache.hadoop.fs.impl.OpenFileParameters parameters) throws IOException
FSDataInputStreamBuilder.build()
and from DelegateToFileSystem
and is where
the action of opening the file should begin.
The base implementation performs a blocking
call to open(Path, int)
in this call;
the actual outcome is in the returned CompletableFuture
.
This avoids having to create some thread pool, while still
setting up the expectation that the get()
call
is needed to evaluate the result.path
- path to the fileparameters
- open file parameters from the builder.IOException
- failure to resolve the link.IllegalArgumentException
- unknown mandatory keyprotected CompletableFuture<FSDataInputStream> openFileWithOptions(PathHandle pathHandle, org.apache.hadoop.fs.impl.OpenFileParameters parameters) throws IOException
open(Path, int)
in this call;
the actual outcome is in the returned CompletableFuture
.
This avoids having to create some thread pool, while still
setting up the expectation that the get()
call
is needed to evaluate the result.pathHandle
- path to the fileparameters
- open file parameters from the builder.IOException
- failure to resolve the link.IllegalArgumentException
- unknown mandatory keyUnsupportedOperationException
- PathHandles are not supported.
This may be deferred until the future is evaluated.@InterfaceStability.Unstable public org.apache.hadoop.fs.MultipartUploaderBuilder createMultipartUploader(Path basePath) throws IOException
basePath
- file path under which all files are uploadedIOException
- if some early checks cause IO failures.UnsupportedOperationException
- if support is checked early.Copyright © 2023 Apache Software Foundation. All rights reserved.