hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject svn commit: r920985 - in /hadoop/hbase/branches/0.20: CHANGES.txt src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
Date Tue, 09 Mar 2010 17:10:32 GMT
Author: stack
Date: Tue Mar  9 17:10:32 2010
New Revision: 920985

URL: http://svn.apache.org/viewvc?rev=920985&view=rev
Log:
HBASE-2295 Row locks may deadlock with themselves

Modified:
    hadoop/hbase/branches/0.20/CHANGES.txt
    hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java

Modified: hadoop/hbase/branches/0.20/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/CHANGES.txt?rev=920985&r1=920984&r2=920985&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.20/CHANGES.txt Tue Mar  9 17:10:32 2010
@@ -31,6 +31,8 @@ Release 0.20.4 - Unreleased
    HBASE-2261  The javadoc in WhileMatchFilter and it's tests in TestFilter are
                not accurate/wrong
    HBASE-2299  [EC2] mapreduce fixups for PE
+   HBASE-2295  Row locks may deadlock with themselves
+               (dhruba borthakur via Stack)
 
   IMPROVEMENTS
    HBASE-2180  Bad read performance from synchronizing hfile.fddatainputstream

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=920985&r1=920984&r2=920985&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
(original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
Tue Mar  9 17:10:32 2010
@@ -59,10 +59,13 @@ package org.apache.hadoop.hbase.regionse
  import java.util.Collection;
  import java.util.List;
  import java.util.Map;
+ import java.util.Set;
  import java.util.NavigableSet;
  import java.util.TreeMap;
  import java.util.TreeSet;
- import java.util.concurrent.ConcurrentHashMap;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Random;
  import java.util.concurrent.ConcurrentSkipListMap;
  import java.util.concurrent.atomic.AtomicBoolean;
  import java.util.concurrent.atomic.AtomicLong;
@@ -120,8 +123,13 @@ public class HRegion implements HConstan
   // Members
   //////////////////////////////////////////////////////////////////////////////
 
-  private final Map<Integer, byte []> locksToRows =
-    new ConcurrentHashMap<Integer, byte []>();
+  private final Set<byte[]> lockedRows =
+    new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+  private final Map<Integer, byte []> lockIds =
+    new HashMap<Integer, byte []>();
+  private int lockIdGenerator = 1;
+  static private Random rand = new Random();
+
   protected final Map<byte [], Store> stores =
     new ConcurrentSkipListMap<byte [], Store>(Bytes.BYTES_RAWCOMPARATOR);
   
@@ -1623,18 +1631,34 @@ public class HRegion implements HConstan
       if (this.closed.get()) {
         throw new NotServingRegionException("Region " + this + " closed");
       }
-      Integer key = Bytes.mapKey(row);
-      synchronized (locksToRows) {
-        while (locksToRows.containsKey(key)) {
+      synchronized (lockedRows) {
+        while (lockedRows.contains(row)) {
           try {
-            locksToRows.wait();
+            lockedRows.wait();
           } catch (InterruptedException ie) {
             // Empty
           }
         }
-        locksToRows.put(key, row);
-        locksToRows.notifyAll();
-        return key;
+        // generate a new lockid. Attempt to insert the new [lockid, row].
+        // if this lockid already exists in the map then revert and retry
+        // We could have first done a lockIds.get, and if it does not exist only
+        // then do a lockIds.put, but the hope is that the lockIds.put will 
+        // mostly return null the first time itself because there won't be
+        // too many lockId collisions.
+        byte [] prev = null;
+        Integer lockId = null;
+        do {
+          lockId = new Integer(lockIdGenerator++);
+          prev = lockIds.put(lockId, row);
+          if (prev != null) {
+            lockIds.put(lockId, prev);    // revert old value
+            lockIdGenerator = rand.nextInt(); // generate new start point
+          }
+        } while (prev != null);
+
+        lockedRows.add(row);
+        lockedRows.notifyAll();
+        return lockId;
       }
     } finally {
       splitsAndClosesLock.readLock().unlock();
@@ -1647,7 +1671,9 @@ public class HRegion implements HConstan
    * @return Row that goes with <code>lockid</code>
    */
   byte [] getRowFromLock(final Integer lockid) {
-    return locksToRows.get(lockid);
+    synchronized (lockedRows) {
+      return lockIds.get(lockid);
+    }
   }
   
   /** 
@@ -1655,9 +1681,10 @@ public class HRegion implements HConstan
    * @param lockid  The lock ID to release.
    */
   void releaseRowLock(final Integer lockid) {
-    synchronized (locksToRows) {
-      locksToRows.remove(lockid);
-      locksToRows.notifyAll();
+    synchronized (lockedRows) {
+      byte[] row = lockIds.remove(lockid);
+      lockedRows.remove(row);
+      lockedRows.notifyAll();
     }
   }
   
@@ -1667,8 +1694,8 @@ public class HRegion implements HConstan
    * @return boolean
    */
   private boolean isRowLocked(final Integer lockid) {
-    synchronized (locksToRows) {
-      if(locksToRows.containsKey(lockid)) {
+    synchronized (lockedRows) {
+      if (lockIds.get(lockid) != null) {
         return true;
       }
       return false;
@@ -1696,11 +1723,11 @@ public class HRegion implements HConstan
   }
   
   private void waitOnRowLocks() {
-    synchronized (locksToRows) {
-      while (this.locksToRows.size() > 0) {
-        LOG.debug("waiting for " + this.locksToRows.size() + " row locks");
+    synchronized (lockedRows) {
+      while (this.lockedRows.size() > 0) {
+        LOG.debug("waiting for " + this.lockedRows.size() + " row locks");
         try {
-          this.locksToRows.wait();
+          this.lockedRows.wait();
         } catch (InterruptedException e) {
           // Catch. Let while test determine loop-end.
         }



Mime
View raw message