You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2019/02/19 23:10:21 UTC

[hbase] branch branch-1.4 updated: HBASE-21915 Make FileLinkInputStream implement CanUnbuffer

This is an automated email from the ASF dual-hosted git repository.

elserj pushed a commit to branch branch-1.4
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-1.4 by this push:
     new 06aa0ec  HBASE-21915 Make FileLinkInputStream implement CanUnbuffer
06aa0ec is described below

commit 06aa0ec5a59981b2e6c404557f57b04478a0467b
Author: Josh Elser <el...@apache.org>
AuthorDate: Fri Feb 15 15:59:30 2019 -0500

    HBASE-21915 Make FileLinkInputStream implement CanUnbuffer
    
    Signed-off-by: Wellington Chevreuil <we...@gmail.com>
    Signed-off-by: Esteban Gutierrez <es...@apache.org>
---
 .../src/main/java/org/apache/hadoop/hbase/io/FileLink.java    | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
index beeb8af..39aecec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
@@ -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.
      *