HBASE-21915 Make FileLinkInputStream implement CanUnbuffer

Signed-off-by: Wellington Chevreuil <wellington.chevreuil@gmail.com>
Signed-off-by: Esteban Gutierrez <esteban@apache.org>
This commit is contained in:
Josh Elser 2019-02-15 15:59:30 -05:00
parent c578020588
commit b76b0191b1
1 changed files with 10 additions and 1 deletions

View File

@ -31,6 +31,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.CanSetDropBehind;
import org.apache.hadoop.fs.CanSetReadahead;
import org.apache.hadoop.fs.CanUnbuffer;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileStatus;
@ -102,7 +103,7 @@ public class FileLink {
* and the alternative locations, when the file is moved.
*/
private static class FileLinkInputStream extends InputStream
implements Seekable, PositionedReadable, CanSetDropBehind, CanSetReadahead {
implements Seekable, PositionedReadable, CanSetDropBehind, CanSetReadahead, CanUnbuffer {
private FSDataInputStream in = null;
private Path currentPath = null;
private long pos = 0;
@ -281,6 +282,14 @@ public class FileLink {
return false;
}
@Override
public void unbuffer() {
if (in == null) {
return;
}
in.unbuffer();
}
/**
* Try to open the file from one of the available locations.
*