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 ed6939f94a
commit ff49338d20
1 changed files with 10 additions and 1 deletions

View File

@ -29,6 +29,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.fs.CanUnbuffer;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileStatus;
@ -100,7 +101,7 @@ public class FileLink {
* and the alternative locations, when the file is moved.
*/
private static class FileLinkInputStream extends InputStream
implements Seekable, PositionedReadable {
implements Seekable, PositionedReadable, CanUnbuffer {
private FSDataInputStream in = null;
private Path currentPath = null;
private long pos = 0;
@ -279,6 +280,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.
*