HDFS-12877. Add open(PathHandle) with default buffersize

This commit is contained in:
Chris Douglas 2017-11-30 15:13:16 -08:00
parent a409425986
commit 0780fdb1eb
3 changed files with 17 additions and 0 deletions

View File

@ -951,6 +951,21 @@ public FSDataInputStream open(Path f) throws IOException {
IO_FILE_BUFFER_SIZE_DEFAULT)); IO_FILE_BUFFER_SIZE_DEFAULT));
} }
/**
* Open an FSDataInputStream matching the PathHandle instance. The
* implementation may encode metadata in PathHandle to address the
* resource directly and verify that the resource referenced
* satisfies constraints specified at its construciton.
* @param fd PathHandle object returned by the FS authority.
* @throws IOException IO failure
* @throws UnsupportedOperationException If {@link #open(PathHandle, int)}
* not overridden by subclass
*/
public FSDataInputStream open(PathHandle fd) throws IOException {
return open(fd, getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
IO_FILE_BUFFER_SIZE_DEFAULT));
}
/** /**
* Open an FSDataInputStream matching the PathHandle instance. The * Open an FSDataInputStream matching the PathHandle instance. The
* implementation may encode metadata in PathHandle to address the * implementation may encode metadata in PathHandle to address the

View File

@ -79,6 +79,7 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
public boolean mkdirs(Path f); public boolean mkdirs(Path f);
public FSDataInputStream open(Path f); public FSDataInputStream open(Path f);
public FSDataInputStream open(PathHandle f);
public FSDataOutputStream create(Path f); public FSDataOutputStream create(Path f);
public FSDataOutputStream create(Path f, boolean overwrite); public FSDataOutputStream create(Path f, boolean overwrite);
public FSDataOutputStream create(Path f, Progressable progress); public FSDataOutputStream create(Path f, Progressable progress);

View File

@ -80,6 +80,7 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
public boolean mkdirs(Path f); public boolean mkdirs(Path f);
public FSDataInputStream open(Path f); public FSDataInputStream open(Path f);
public FSDataInputStream open(PathHandle f);
public FSDataOutputStream create(Path f); public FSDataOutputStream create(Path f);
public FSDataOutputStream create(Path f, boolean overwrite); public FSDataOutputStream create(Path f, boolean overwrite);
public FSDataOutputStream create(Path f, Progressable progress); public FSDataOutputStream create(Path f, Progressable progress);