org.apache.hadoop.fs
Class FilterFileSystem

java.lang.Object
  extended by org.apache.hadoop.conf.Configured
      extended by org.apache.hadoop.fs.FileSystem
          extended by org.apache.hadoop.fs.FilterFileSystem
All Implemented Interfaces:
Closeable, Configurable
Direct Known Subclasses:
ChecksumFileSystem

@InterfaceAudience.Public
@InterfaceStability.Stable
public class FilterFileSystem
extends FileSystem

A FilterFileSystem contains some other file system, which it uses as its basic file system, possibly transforming the data along the way or providing additional functionality. The class FilterFileSystem itself simply overrides all methods of FileSystem with versions that pass all requests to the contained file system. Subclasses of FilterFileSystem may further override some of these methods and may also provide additional methods and fields.


Field Summary
protected  FileSystem fs
           
protected  String swapScheme
           
 
Fields inherited from class org.apache.hadoop.fs.FileSystem
DEFAULT_FS, FS_DEFAULT_NAME_KEY, LOG, SHUTDOWN_HOOK_PRIORITY, statistics
 
Constructor Summary
FilterFileSystem()
           
FilterFileSystem(FileSystem fs)
           
 
Method Summary
 FSDataOutputStream append(Path f, int bufferSize, Progressable progress)
          Append to an existing file (optional operation).
protected  URI canonicalizeUri(URI uri)
          Canonicalize the given URI.
protected  void checkPath(Path path)
          Check that a Path belongs to this FileSystem.
 void close()
          No more filesystem operations are needed.
 void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
          Called when we're all done writing to the target.
 void concat(Path f, 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 copyToLocalFile(boolean delSrc, Path src, Path dst)
          The src file is under FS, and the dst is on the local disk.
 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 createNonRecursive(Path f, FsPermission permission, EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize, Progressable progress)
          Deprecated. 
 Path createSnapshot(Path path, String snapshotName)
          Create a snapshot
 void createSymlink(Path target, Path link, boolean createParent)
          See FileContext.createSymlink(Path, Path, boolean)
 boolean delete(Path f, boolean recursive)
          Delete a file
 void deleteSnapshot(Path path, String snapshotName)
          Delete a snapshot of a directory
 AclStatus getAclStatus(Path path)
          Gets the ACL of a file or directory.
protected  URI getCanonicalUri()
          Return a canonicalized form of this FileSystem's URI.
 FileSystem[] getChildFileSystems()
          Get all the immediate child FileSystems embedded in this FileSystem.
 Configuration getConf()
          Return the configuration used by this object.
 long getDefaultBlockSize()
          Return the number of bytes that large input files should be optimally be split into to minimize i/o time.
 long getDefaultBlockSize(Path f)
          Return the number of bytes that large input files should be optimally be split into to minimize i/o time.
 short getDefaultReplication()
          Get the default replication.
 short getDefaultReplication(Path f)
          Get the default replication for a path.
 BlockLocation[] getFileBlockLocations(FileStatus file, 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.
 FileStatus getFileLinkStatus(Path f)
          See FileContext.getFileLinkStatus(Path)
 FileStatus getFileStatus(Path f)
          Get file status.
 Path getHomeDirectory()
          Return the current user's home directory in this filesystem.
protected  Path getInitialWorkingDirectory()
          Note: with the new FilesContext class, getWorkingDirectory() will be removed.
 Path getLinkTarget(Path f)
          See FileContext.getLinkTarget(Path)
 FileSystem getRawFileSystem()
          Get the raw file system
 FsServerDefaults getServerDefaults()
          Return a set of server default configuration values
 FsServerDefaults getServerDefaults(Path f)
          Return a set of server default configuration values
 FsStatus getStatus(Path p)
          Returns a status object describing the use and capacity of the file system.
 URI getUri()
          Returns a URI whose scheme and authority identify this FileSystem.
 long getUsed()
          Return the total size of all files in the filesystem.
 Path getWorkingDirectory()
          Get the current working directory for the given file system
 void initialize(URI name, Configuration conf)
          Called after a new FileSystem instance is constructed.
 org.apache.hadoop.fs.RemoteIterator<Path> listCorruptFileBlocks(Path path)
           
 org.apache.hadoop.fs.RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
          List files and its block locations in a directory.
 FileStatus[] listStatus(Path f)
          List files in a directory.
 Path makeQualified(Path path)
          Make sure that a path specifies a FileSystem.
 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.
 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)
           
protected  boolean primitiveMkdir(Path f, FsPermission abdolutePermission)
          This version of the mkdirs method assumes that the permission is absolute.
 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.
 boolean rename(Path src, Path dst)
          Renames Path src to Path dst.
 void renameSnapshot(Path path, String snapshotOldName, String snapshotNewName)
          Rename a snapshot
protected  Path resolveLink(Path f)
          See AbstractFileSystem.getLinkTarget(Path)
 Path resolvePath(Path p)
          Return the fully-qualified path of path f 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.
 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 replication for an existing file.
 void setTimes(Path p, long mtime, long atime)
          Set access time of a file
 void setVerifyChecksum(boolean verifyChecksum)
          Set the verify checksum flag.
 void setWorkingDirectory(Path newDir)
          Set the current working directory for the given file system.
 void setWriteChecksum(boolean writeChecksum)
          Set the write checksum flag.
 Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
          Returns a local File that the user can write output to.
 boolean supportsSymlinks()
          See AbstractFileSystem.supportsSymlinks()
 
Methods inherited from class org.apache.hadoop.fs.FileSystem
append, append, areSymlinksEnabled, cancelDeleteOnExit, clearStatistics, closeAll, closeAllForUGI, copyFromLocalFile, copyToLocalFile, copyToLocalFile, create, create, create, create, create, create, create, create, create, create, create, create, createNewFile, createNonRecursive, createNonRecursive, createSnapshot, delete, deleteOnExit, enableSymlinks, exists, fixRelativePart, get, get, get, getAllStatistics, getBlockSize, getContentSummary, getDefaultPort, getDefaultUri, getFileBlockLocations, getFileSystemClass, getFSofPath, getLength, getLocal, getName, getNamed, getReplication, getScheme, getStatistics, getStatistics, getStatus, globStatus, globStatus, isDirectory, isFile, listFiles, listLocatedStatus, listStatus, listStatus, listStatus, mkdirs, mkdirs, moveFromLocalFile, moveFromLocalFile, moveToLocalFile, newInstance, newInstance, newInstance, newInstanceLocal, open, primitiveMkdir, printStatistics, processDeleteOnExit, rename, setDefaultUri, setDefaultUri
 
Methods inherited from class org.apache.hadoop.conf.Configured
setConf
 
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
 

Field Detail

fs

protected FileSystem fs

swapScheme

protected String swapScheme
Constructor Detail

FilterFileSystem

public FilterFileSystem()

FilterFileSystem

public FilterFileSystem(FileSystem fs)
Method Detail

getRawFileSystem

public FileSystem getRawFileSystem()
Get the raw file system

Returns:
FileSystem being filtered

initialize

public void initialize(URI name,
                       Configuration conf)
                throws IOException
Called after a new FileSystem instance is constructed.

Overrides:
initialize in class FileSystem
Parameters:
name - a uri whose authority section names the host, port, etc. for this FileSystem
conf - the configuration
Throws:
IOException

getUri

public URI getUri()
Returns a URI whose scheme and authority identify this FileSystem.

Specified by:
getUri in class FileSystem

getCanonicalUri

protected URI getCanonicalUri()
Description copied from class: FileSystem
Return a canonicalized form of this FileSystem's URI. The default implementation simply calls FileSystem.canonicalizeUri(URI) on the filesystem's own URI, so subclasses typically only need to implement that method.

Overrides:
getCanonicalUri in class FileSystem
See Also:
FileSystem.canonicalizeUri(URI)

canonicalizeUri

protected URI canonicalizeUri(URI uri)
Description copied from class: FileSystem
Canonicalize the given URI. This is filesystem-dependent, but may for example consist of canonicalizing the hostname using DNS and adding the default port if not specified. The default implementation simply fills in the default port if not specified and if the filesystem has a default port.

Overrides:
canonicalizeUri in class FileSystem
Returns:
URI
See Also:
NetUtils.getCanonicalUri(URI, int)

makeQualified

public Path makeQualified(Path path)
Make sure that a path specifies a FileSystem.

Overrides:
makeQualified in class FileSystem
Parameters:
path - to use

checkPath

protected void checkPath(Path path)
Check that a Path belongs to this FileSystem.

Overrides:
checkPath in class FileSystem
Parameters:
path - to check

getFileBlockLocations

public BlockLocation[] getFileBlockLocations(FileStatus file,
                                             long start,
                                             long len)
                                      throws IOException
Description copied from class: FileSystem
Return an array containing hostnames, offset and size of portions of the given file. For a nonexistent file or regions, null will be returned. This call is most helpful with DFS, where it returns hostnames of machines that contain the given file. The FileSystem will simply return an elt containing 'localhost'.

Overrides:
getFileBlockLocations in class FileSystem
Parameters:
file - FilesStatus to get data from
start - offset into the given file
len - length for which to get locations for
Throws:
IOException

resolvePath

public Path resolvePath(Path p)
                 throws IOException
Description copied from class: FileSystem
Return the fully-qualified path of path f resolving the path through any symlinks or mount point

Overrides:
resolvePath in class FileSystem
Parameters:
p - path to be resolved
Returns:
fully qualified path
Throws:
FileNotFoundException
IOException

open

public FSDataInputStream open(Path f,
                              int bufferSize)
                       throws IOException
Opens an FSDataInputStream at the indicated Path.

Specified by:
open in class FileSystem
Parameters:
f - the file name to open
bufferSize - the size of the buffer to be used.
Throws:
IOException

append

public FSDataOutputStream append(Path f,
                                 int bufferSize,
                                 Progressable progress)
                          throws IOException
Description copied from class: FileSystem
Append to an existing file (optional operation).

Specified by:
append in class FileSystem
Parameters:
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.
Throws:
IOException

concat

public void concat(Path f,
                   Path[] psrcs)
            throws IOException
Description copied from class: FileSystem
Concat existing files together.

Overrides:
concat in class FileSystem
Parameters:
f - the path to the target destination.
psrcs - the paths to the sources to use for the concatenation.
Throws:
IOException

create

public FSDataOutputStream create(Path f,
                                 FsPermission permission,
                                 boolean overwrite,
                                 int bufferSize,
                                 short replication,
                                 long blockSize,
                                 Progressable progress)
                          throws IOException
Description copied from class: FileSystem
Create an FSDataOutputStream at the indicated Path with write-progress reporting.

Specified by:
create in class FileSystem
Parameters:
f - the file name to open
overwrite - 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.
Throws:
IOException
See Also:
FileSystem.setPermission(Path, FsPermission)

createNonRecursive

@Deprecated
public FSDataOutputStream createNonRecursive(Path f,
                                                        FsPermission permission,
                                                        EnumSet<CreateFlag> flags,
                                                        int bufferSize,
                                                        short replication,
                                                        long blockSize,
                                                        Progressable progress)
                                      throws IOException
Deprecated. 

Description copied from class: FileSystem
Opens an FSDataOutputStream at the indicated Path with write-progress reporting. Same as create(), except fails if parent directory doesn't already exist.

Overrides:
createNonRecursive in class FileSystem
Parameters:
f - the file name to open
flags - CreateFlags to use for this stream.
bufferSize - the size of the buffer to be used.
replication - required block replication for the file.
Throws:
IOException
See Also:
FileSystem.setPermission(Path, FsPermission)

setReplication

public boolean setReplication(Path src,
                              short replication)
                       throws IOException
Set replication for an existing file.

Overrides:
setReplication in class FileSystem
Parameters:
src - file name
replication - new replication
Returns:
true if successful; false if file does not exist or is a directory
Throws:
IOException

rename

public boolean rename(Path src,
                      Path dst)
               throws IOException
Renames Path src to Path dst. Can take place on local fs or remote DFS.

Specified by:
rename in class FileSystem
Parameters:
src - path to be renamed
dst - new path after rename
Returns:
true if rename is successful
Throws:
IOException - on failure

delete

public boolean delete(Path f,
                      boolean recursive)
               throws IOException
Delete a file

Specified by:
delete in class FileSystem
Parameters:
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.
Returns:
true if delete is successful else false.
Throws:
IOException

listStatus

public FileStatus[] listStatus(Path f)
                        throws IOException
List files in a directory.

Specified by:
listStatus in class FileSystem
Parameters:
f - given path
Returns:
the statuses of the files/directories in the given patch
Throws:
FileNotFoundException - when the path does not exist; IOException see specific implementation
IOException

listCorruptFileBlocks

public org.apache.hadoop.fs.RemoteIterator<Path> listCorruptFileBlocks(Path path)
                                                                throws IOException
Overrides:
listCorruptFileBlocks in class FileSystem
Returns:
an iterator over the corrupt files under the given path (may contain duplicates if a file has more than one corrupt block)
Throws:
IOException

listLocatedStatus

public org.apache.hadoop.fs.RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
                                                                         throws IOException
List files and its block locations in a directory.

Overrides:
listLocatedStatus in class FileSystem
Parameters:
f - is the path
Returns:
an iterator that traverses statuses of the files/directories in the given path
Throws:
FileNotFoundException - If f does not exist
IOException - If an I/O error occurred

getHomeDirectory

public Path getHomeDirectory()
Description copied from class: FileSystem
Return the current user's home directory in this filesystem. The default implementation returns "/user/$USER/".

Overrides:
getHomeDirectory in class FileSystem

setWorkingDirectory

public void setWorkingDirectory(Path newDir)
Set the current working directory for the given file system. All relative paths will be resolved relative to it.

Specified by:
setWorkingDirectory in class FileSystem
Parameters:
newDir -

getWorkingDirectory

public Path getWorkingDirectory()
Get the current working directory for the given file system

Specified by:
getWorkingDirectory in class FileSystem
Returns:
the directory pathname

getInitialWorkingDirectory

protected Path getInitialWorkingDirectory()
Description copied from class: FileSystem
Note: with the new FilesContext class, getWorkingDirectory() will be removed. The working directory is implemented in FilesContext. Some file systems like LocalFileSystem have an initial workingDir that we use as the starting workingDir. For other file systems like HDFS there is no built in notion of an initial workingDir.

Overrides:
getInitialWorkingDirectory in class FileSystem
Returns:
if there is built in notion of workingDir then it is returned; else a null is returned.

getStatus

public FsStatus getStatus(Path p)
                   throws IOException
Description copied from class: FileSystem
Returns a status object describing the use and capacity of the file system. If the file system has multiple partitions, the use and capacity of the partition pointed to by the specified path is reflected.

Overrides:
getStatus in class FileSystem
Parameters:
p - Path for which status should be obtained. null means the default partition.
Returns:
a FsStatus object
Throws:
IOException - see specific implementation

mkdirs

public boolean mkdirs(Path f,
                      FsPermission permission)
               throws IOException
Description copied from class: FileSystem
Make the given file and all non-existent parents into directories. Has the semantics of Unix 'mkdir -p'. Existence of the directory hierarchy is not an error.

Specified by:
mkdirs in class FileSystem
Parameters:
f - path to create
permission - to apply to f
Throws:
IOException

copyFromLocalFile

public void copyFromLocalFile(boolean delSrc,
                              Path src,
                              Path dst)
                       throws IOException
The src file is on the local disk. Add it to FS at the given dst name. delSrc indicates if the source should be removed

Overrides:
copyFromLocalFile in class FileSystem
Parameters:
delSrc - whether to delete the src
src - path
dst - path
Throws:
IOException

copyFromLocalFile

public void copyFromLocalFile(boolean delSrc,
                              boolean overwrite,
                              Path[] srcs,
                              Path dst)
                       throws IOException
The src files are on the local disk. Add it to FS at the given dst name. delSrc indicates if the source should be removed

Overrides:
copyFromLocalFile in class FileSystem
Parameters:
delSrc - whether to delete the src
overwrite - whether to overwrite an existing file
srcs - array of paths which are source
dst - path
Throws:
IOException

copyFromLocalFile

public void copyFromLocalFile(boolean delSrc,
                              boolean overwrite,
                              Path src,
                              Path dst)
                       throws IOException
The src file is on the local disk. Add it to FS at the given dst name. delSrc indicates if the source should be removed

Overrides:
copyFromLocalFile in class FileSystem
Parameters:
delSrc - whether to delete the src
overwrite - whether to overwrite an existing file
src - path
dst - path
Throws:
IOException

copyToLocalFile

public void copyToLocalFile(boolean delSrc,
                            Path src,
                            Path dst)
                     throws IOException
The src file is under FS, and the dst is on the local disk. Copy it from FS control to the local dst name. delSrc indicates if the src will be removed or not.

Overrides:
copyToLocalFile in class FileSystem
Parameters:
delSrc - whether to delete the src
src - path
dst - path
Throws:
IOException

startLocalOutput

public Path startLocalOutput(Path fsOutputFile,
                             Path tmpLocalFile)
                      throws IOException
Returns a local File that the user can write output to. The caller provides both the eventual FS target name and the local working file. If the FS is local, we write directly into the target. If the FS is remote, we write into the tmp local area.

Overrides:
startLocalOutput in class FileSystem
Parameters:
fsOutputFile - path of output file
tmpLocalFile - path of local tmp file
Throws:
IOException

completeLocalOutput

public void completeLocalOutput(Path fsOutputFile,
                                Path tmpLocalFile)
                         throws IOException
Called when we're all done writing to the target. A local FS will do nothing, because we've written to exactly the right place. A remote FS will copy the contents of tmpLocalFile to the correct target at fsOutputFile.

Overrides:
completeLocalOutput in class FileSystem
Parameters:
fsOutputFile - path of output file
tmpLocalFile - path to local tmp file
Throws:
IOException

getUsed

public long getUsed()
             throws IOException
Return the total size of all files in the filesystem.

Overrides:
getUsed in class FileSystem
Throws:
IOException

getDefaultBlockSize

public long getDefaultBlockSize()
Description copied from class: FileSystem
Return the number of bytes that large input files should be optimally be split into to minimize i/o time.

Overrides:
getDefaultBlockSize in class FileSystem

getDefaultReplication

public short getDefaultReplication()
Description copied from class: FileSystem
Get the default replication.

Overrides:
getDefaultReplication in class FileSystem

getServerDefaults

public FsServerDefaults getServerDefaults()
                                   throws IOException
Description copied from class: FileSystem
Return a set of server default configuration values

Overrides:
getServerDefaults in class FileSystem
Returns:
server default configuration values
Throws:
IOException

getDefaultBlockSize

public long getDefaultBlockSize(Path f)
Description copied from class: FileSystem
Return the number of bytes that large input files should be optimally be split into to minimize i/o time. The given path will be used to locate the actual filesystem. The full path does not have to exist.

Overrides:
getDefaultBlockSize in class FileSystem
Parameters:
f - path of file
Returns:
the default block size for the path's filesystem

getDefaultReplication

public short getDefaultReplication(Path f)
Description copied from class: FileSystem
Get the default replication for a path. The given path will be used to locate the actual filesystem. The full path does not have to exist.

Overrides:
getDefaultReplication in class FileSystem
Parameters:
f - of the file
Returns:
default replication for the path's filesystem

getServerDefaults

public FsServerDefaults getServerDefaults(Path f)
                                   throws IOException
Description copied from class: FileSystem
Return a set of server default configuration values

Overrides:
getServerDefaults in class FileSystem
Parameters:
f - path is used to identify an FS since an FS could have another FS that it could be delegating the call to
Returns:
server default configuration values
Throws:
IOException

getFileStatus

public FileStatus getFileStatus(Path f)
                         throws IOException
Get file status.

Specified by:
getFileStatus in class FileSystem
Parameters:
f - The path we want information from
Returns:
a FileStatus object
Throws:
FileNotFoundException - when the path does not exist; IOException see specific implementation
IOException

createSymlink

public void createSymlink(Path target,
                          Path link,
                          boolean createParent)
                   throws org.apache.hadoop.security.AccessControlException,
                          FileAlreadyExistsException,
                          FileNotFoundException,
                          ParentNotDirectoryException,
                          UnsupportedFileSystemException,
                          IOException
Description copied from class: FileSystem
See FileContext.createSymlink(Path, Path, boolean)

Overrides:
createSymlink in class FileSystem
Throws:
org.apache.hadoop.security.AccessControlException
FileAlreadyExistsException
FileNotFoundException
ParentNotDirectoryException
UnsupportedFileSystemException
IOException

getFileLinkStatus

public FileStatus getFileLinkStatus(Path f)
                             throws org.apache.hadoop.security.AccessControlException,
                                    FileNotFoundException,
                                    UnsupportedFileSystemException,
                                    IOException
Description copied from class: FileSystem
See FileContext.getFileLinkStatus(Path)

Overrides:
getFileLinkStatus in class FileSystem
Throws:
org.apache.hadoop.security.AccessControlException
FileNotFoundException
UnsupportedFileSystemException
IOException

supportsSymlinks

public boolean supportsSymlinks()
Description copied from class: FileSystem
See AbstractFileSystem.supportsSymlinks()

Overrides:
supportsSymlinks in class FileSystem

getLinkTarget

public Path getLinkTarget(Path f)
                   throws IOException
Description copied from class: FileSystem
See FileContext.getLinkTarget(Path)

Overrides:
getLinkTarget in class FileSystem
Throws:
IOException

resolveLink

protected Path resolveLink(Path f)
                    throws IOException
Description copied from class: FileSystem
See AbstractFileSystem.getLinkTarget(Path)

Overrides:
resolveLink in class FileSystem
Throws:
IOException

getFileChecksum

public FileChecksum getFileChecksum(Path f)
                             throws IOException
Description copied from class: FileSystem
Get the checksum of a file.

Overrides:
getFileChecksum in class FileSystem
Parameters:
f - The file path
Returns:
The file checksum. The default return value is null, which indicates that no checksum algorithm is implemented in the corresponding FileSystem.
Throws:
IOException

setVerifyChecksum

public void setVerifyChecksum(boolean verifyChecksum)
Description copied from class: FileSystem
Set the verify checksum flag. This is only applicable if the corresponding FileSystem supports checksum. By default doesn't do anything.

Overrides:
setVerifyChecksum in class FileSystem

setWriteChecksum

public void setWriteChecksum(boolean writeChecksum)
Description copied from class: FileSystem
Set the write checksum flag. This is only applicable if the corresponding FileSystem supports checksum. By default doesn't do anything.

Overrides:
setWriteChecksum in class FileSystem

getConf

public Configuration getConf()
Description copied from interface: Configurable
Return the configuration used by this object.

Specified by:
getConf in interface Configurable
Overrides:
getConf in class Configured

close

public void close()
           throws IOException
Description copied from class: FileSystem
No more filesystem operations are needed. Will release any held locks.

Specified by:
close in interface Closeable
Overrides:
close in class FileSystem
Throws:
IOException

setOwner

public void setOwner(Path p,
                     String username,
                     String groupname)
              throws IOException
Description copied from class: FileSystem
Set owner of a path (i.e. a file or a directory). The parameters username and groupname cannot both be null.

Overrides:
setOwner in class FileSystem
Parameters:
p - The path
username - If it is null, the original username remains unchanged.
groupname - If it is null, the original groupname remains unchanged.
Throws:
IOException

setTimes

public void setTimes(Path p,
                     long mtime,
                     long atime)
              throws IOException
Description copied from class: FileSystem
Set access time of a file

Overrides:
setTimes in class FileSystem
Parameters:
p - The path
mtime - 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.
Throws:
IOException

setPermission

public void setPermission(Path p,
                          FsPermission permission)
                   throws IOException
Description copied from class: FileSystem
Set permission of a path.

Overrides:
setPermission in class FileSystem
Throws:
IOException

primitiveCreate

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
Overrides:
primitiveCreate in class FileSystem
Throws:
IOException

primitiveMkdir

protected boolean primitiveMkdir(Path f,
                                 FsPermission abdolutePermission)
                          throws IOException
Description copied from class: FileSystem
This version of the mkdirs method assumes that the permission is absolute. It has been added to support the FileContext that processes the permission with umask before calling this method. This a temporary method added to support the transition from FileSystem to FileContext for user applications.

Overrides:
primitiveMkdir in class FileSystem
Throws:
IOException

getChildFileSystems

public FileSystem[] getChildFileSystems()
Description copied from class: FileSystem
Get all the immediate child FileSystems embedded in this FileSystem. It does not recurse and get grand children. If a FileSystem has multiple child FileSystems, then it should return a unique list of those FileSystems. Default is to return null to signify no children.

Returns:
FileSystems used by this FileSystem

createSnapshot

public Path createSnapshot(Path path,
                           String snapshotName)
                    throws IOException
Description copied from class: FileSystem
Create a snapshot

Overrides:
createSnapshot in class FileSystem
Parameters:
path - The directory where snapshots will be taken.
snapshotName - The name of the snapshot
Returns:
the snapshot path.
Throws:
IOException

renameSnapshot

public void renameSnapshot(Path path,
                           String snapshotOldName,
                           String snapshotNewName)
                    throws IOException
Description copied from class: FileSystem
Rename a snapshot

Overrides:
renameSnapshot in class FileSystem
Parameters:
path - The directory path where the snapshot was taken
snapshotOldName - Old name of the snapshot
snapshotNewName - New name of the snapshot
Throws:
IOException

deleteSnapshot

public void deleteSnapshot(Path path,
                           String snapshotName)
                    throws IOException
Description copied from class: FileSystem
Delete a snapshot of a directory

Overrides:
deleteSnapshot in class FileSystem
Parameters:
path - The directory that the to-be-deleted snapshot belongs to
snapshotName - The name of the snapshot
Throws:
IOException

modifyAclEntries

public void modifyAclEntries(Path path,
                             List<AclEntry> aclSpec)
                      throws IOException
Description copied from class: FileSystem
Modifies ACL entries of files and directories. This method can add new ACL entries or modify the permissions on existing ACL entries. All existing ACL entries that are not specified in this call are retained without changes. (Modifications are merged into the current ACL.)

Overrides:
modifyAclEntries in class FileSystem
Parameters:
path - Path to modify
aclSpec - List describing modifications
Throws:
IOException - if an ACL could not be modified

removeAclEntries

public void removeAclEntries(Path path,
                             List<AclEntry> aclSpec)
                      throws IOException
Description copied from class: FileSystem
Removes ACL entries from files and directories. Other ACL entries are retained.

Overrides:
removeAclEntries in class FileSystem
Parameters:
path - Path to modify
aclSpec - List describing entries to remove
Throws:
IOException - if an ACL could not be modified

removeDefaultAcl

public void removeDefaultAcl(Path path)
                      throws IOException
Description copied from class: FileSystem
Removes all default ACL entries from files and directories.

Overrides:
removeDefaultAcl in class FileSystem
Parameters:
path - Path to modify
Throws:
IOException - if an ACL could not be modified

removeAcl

public void removeAcl(Path path)
               throws IOException
Description copied from class: FileSystem
Removes all but the base ACL entries of files and directories. The entries for user, group, and others are retained for compatibility with permission bits.

Overrides:
removeAcl in class FileSystem
Parameters:
path - Path to modify
Throws:
IOException - if an ACL could not be removed

setAcl

public void setAcl(Path path,
                   List<AclEntry> aclSpec)
            throws IOException
Description copied from class: FileSystem
Fully replaces ACL of files and directories, discarding all existing entries.

Overrides:
setAcl in class FileSystem
Parameters:
path - Path to modify
aclSpec - List describing modifications, must include entries for user, group, and others for compatibility with permission bits.
Throws:
IOException - if an ACL could not be modified

getAclStatus

public AclStatus getAclStatus(Path path)
                       throws IOException
Description copied from class: FileSystem
Gets the ACL of a file or directory.

Overrides:
getAclStatus in class FileSystem
Parameters:
path - Path to get
Returns:
AclStatus describing the ACL of the file or directory
Throws:
IOException - if an ACL could not be read


Copyright © 2014 Apache Software Foundation. All Rights Reserved.