HDFS-12877. Add open(PathHandle) with default buffersize
This commit is contained in:
parent
a409425986
commit
0780fdb1eb
|
@ -951,6 +951,21 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
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
|
||||
* implementation may encode metadata in PathHandle to address the
|
||||
|
|
|
@ -79,6 +79,7 @@ public class TestFilterFileSystem {
|
|||
|
||||
public boolean mkdirs(Path f);
|
||||
public FSDataInputStream open(Path f);
|
||||
public FSDataInputStream open(PathHandle f);
|
||||
public FSDataOutputStream create(Path f);
|
||||
public FSDataOutputStream create(Path f, boolean overwrite);
|
||||
public FSDataOutputStream create(Path f, Progressable progress);
|
||||
|
|
|
@ -80,6 +80,7 @@ public class TestHarFileSystem {
|
|||
|
||||
public boolean mkdirs(Path f);
|
||||
public FSDataInputStream open(Path f);
|
||||
public FSDataInputStream open(PathHandle f);
|
||||
public FSDataOutputStream create(Path f);
|
||||
public FSDataOutputStream create(Path f, boolean overwrite);
|
||||
public FSDataOutputStream create(Path f, Progressable progress);
|
||||
|
|
Loading…
Reference in New Issue