accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ctubb...@apache.org
Subject [09/45] accumulo git commit: ACCUMULO-4391 Use a closed flag
Date Fri, 09 Sep 2016 23:37:28 GMT
ACCUMULO-4391 Use a closed flag


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/9038d8b3
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/9038d8b3
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/9038d8b3

Branch: refs/heads/1.8
Commit: 9038d8b3a36e7189fbdedfbbb950c7b1f43b2b7d
Parents: 6ff35f0
Author: Ivan Bella <ivan@bella.name>
Authored: Thu Aug 4 00:04:26 2016 -0400
Committer: Ivan Bella <ivan@bella.name>
Committed: Thu Aug 4 00:04:26 2016 -0400

----------------------------------------------------------------------
 .../bcfile/BoundedRangeFileInputStream.java     | 31 ++++++++------------
 1 file changed, 12 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/9038d8b3/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BoundedRangeFileInputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BoundedRangeFileInputStream.java
b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BoundedRangeFileInputStream.java
index b182b26..876b585 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BoundedRangeFileInputStream.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BoundedRangeFileInputStream.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
  */
 class BoundedRangeFileInputStream extends InputStream {
 
+  private volatile boolean closed = false;
   private FSDataInputStream in;
   private long pos;
   private long end;
@@ -60,12 +61,6 @@ class BoundedRangeFileInputStream extends InputStream {
     this.mark = -1;
   }
 
-  private void check() throws IOException {
-    if (in == null) {
-      throw new IOException("Stream closed");
-    }
-  }
-
   @Override
   public int available() throws IOException {
     return (int) (end - pos);
@@ -94,17 +89,18 @@ class BoundedRangeFileInputStream extends InputStream {
     if (n == 0)
       return -1;
     Integer ret = 0;
-    final FSDataInputStream inLocal = in;
-    check(); // ensuring inLocal is not null
-    synchronized (inLocal) {
-      check(); // ensuring in is not null in which case we were closed which would be followed
by someone else reusing the decompressor
-      inLocal.seek(pos);
+    synchronized (in) {
+      // ensuring we are not closed which would be followed by someone else reusing the decompressor
+      if (closed) {
+        throw new IOException("Stream closed");
+      }
+      in.seek(pos);
       try {
         ret = AccessController.doPrivileged(new PrivilegedExceptionAction<Integer>()
{
           @Override
           public Integer run() throws IOException {
             int ret = 0;
-            ret = inLocal.read(b, off, n);
+            ret = in.read(b, off, n);
             return ret;
           }
         });
@@ -149,13 +145,10 @@ class BoundedRangeFileInputStream extends InputStream {
 
   @Override
   public void close() {
-    final FSDataInputStream inLocal = in;
-    if (inLocal != null) {
-      // synchronize on the FSDataInputStream to ensure we block closing if in the read method
-      synchronized (inLocal) {
-        // Invalidate the state of the stream.
-        in = null;
-      }
+    // synchronize on the FSDataInputStream to ensure we are blocked if in the read method
+    synchronized (in) {
+      // Invalidate the state of the stream.
+      closed = true;
     }
   }
 }


Mime
View raw message