You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2013/11/26 22:08:21 UTC

svn commit: r1545840 - /hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java

Author: larsh
Date: Tue Nov 26 21:08:20 2013
New Revision: 1545840

URL: http://svn.apache.org/r1545840
Log:
HBASE-10015 Replace intrinsic locking with explicit locks in StoreScanner

Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java?rev=1545840&r1=1545839&r2=1545840&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java Tue Nov 26 21:08:20 2013
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.NavigableSet;
+import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -75,6 +76,7 @@ public class StoreScanner extends NonLaz
 
   // A flag whether use pread for scan
   private boolean scanUsePread = false;
+  private ReentrantLock lock = new ReentrantLock();
 
   /** An internal constructor. */
   private StoreScanner(Store store, boolean cacheBlocks, Scan scan,
@@ -267,11 +269,17 @@ public class StoreScanner extends NonLaz
   }
 
   @Override
-  public synchronized KeyValue peek() {
+  public KeyValue peek() {
+    lock.lock();
+    try {
+
     if (this.heap == null) {
       return this.lastTop;
     }
     return this.heap.peek();
+    } finally {
+      lock.unlock();
+    }
   }
 
   @Override
@@ -281,7 +289,9 @@ public class StoreScanner extends NonLaz
   }
 
   @Override
-  public synchronized void close() {
+  public void close() {
+    lock.lock();
+    try {
     if (this.closing) return;
     this.closing = true;
     // under test, we dont have a this.store
@@ -291,10 +301,15 @@ public class StoreScanner extends NonLaz
       this.heap.close();
     this.heap = null; // CLOSED!
     this.lastTop = null; // If both are null, we are closed.
+    } finally {
+      lock.unlock();
+    }
   }
 
   @Override
-  public synchronized boolean seek(KeyValue key) throws IOException {
+  public boolean seek(KeyValue key) throws IOException {
+    lock.lock();
+    try {
     if (this.heap == null) {
 
       List<KeyValueScanner> scanners = getScannersNoCompaction();
@@ -303,6 +318,9 @@ public class StoreScanner extends NonLaz
     }
 
     return this.heap.seek(key);
+  } finally {
+    lock.unlock();
+  }
   }
 
   /**
@@ -312,7 +330,7 @@ public class StoreScanner extends NonLaz
    * @return true if there are more rows, false if scanner is done
    */
   @Override
-  public synchronized boolean next(List<KeyValue> outResult, int limit) throws IOException {
+  public boolean next(List<KeyValue> outResult, int limit) throws IOException {
     return next(outResult, limit, null);
   }
 
@@ -323,8 +341,10 @@ public class StoreScanner extends NonLaz
    * @return true if there are more rows, false if scanner is done
    */
   @Override
-  public synchronized boolean next(List<KeyValue> outResult, int limit,
+  public boolean next(List<KeyValue> outResult, int limit,
       String metric) throws IOException {
+    lock.lock();
+    try {
 
     if (checkReseek()) {
       return true;
@@ -447,22 +467,27 @@ public class StoreScanner extends NonLaz
     // No more keys
     close();
     return false;
+    } finally {
+      lock.unlock();
+    }
   }
 
   @Override
-  public synchronized boolean next(List<KeyValue> outResult) throws IOException {
+  public boolean next(List<KeyValue> outResult) throws IOException {
     return next(outResult, -1, null);
   }
 
   @Override
-  public synchronized boolean next(List<KeyValue> outResult, String metric)
+  public boolean next(List<KeyValue> outResult, String metric)
       throws IOException {
     return next(outResult, -1, metric);
   }
 
   // Implementation of ChangedReadersObserver
   @Override
-  public synchronized void updateReaders() throws IOException {
+  public void updateReaders() throws IOException {
+    lock.lock();
+    try {
     if (this.closing) return;
 
     // All public synchronized API calls will call 'checkReseek' which will cause
@@ -482,6 +507,9 @@ public class StoreScanner extends NonLaz
     this.heap = null; // the re-seeks could be slow (access HDFS) free up memory ASAP
 
     // Let the next() call handle re-creating and seeking
+    } finally {
+      lock.unlock();
+    }
   }
 
   /**
@@ -539,7 +567,9 @@ public class StoreScanner extends NonLaz
   }
 
   @Override
-  public synchronized boolean reseek(KeyValue kv) throws IOException {
+  public boolean reseek(KeyValue kv) throws IOException {
+    lock.lock();
+    try {
     //Heap will not be null, if this is called from next() which.
     //If called from RegionScanner.reseek(...) make sure the scanner
     //stack is reset if needed.
@@ -549,6 +579,9 @@ public class StoreScanner extends NonLaz
     } else {
       return heap.reseek(kv);
     }
+    } finally {
+      lock.unlock();
+    }
   }
 
   @Override