You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2015/10/26 20:09:21 UTC

hbase git commit: HBASE-14689 Addendum and unit test for HBASE-13471

Repository: hbase
Updated Branches:
  refs/heads/branch-1 d76dbb4f8 -> ec021a7b2


HBASE-14689 Addendum and unit test for HBASE-13471

Conflicts:
	hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java


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

Branch: refs/heads/branch-1
Commit: ec021a7b25ecef91685fdab70ebdd386414a677c
Parents: d76dbb4
Author: Enis Soztutar <en...@apache.org>
Authored: Mon Oct 26 11:56:51 2015 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Mon Oct 26 12:08:58 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 14 ++---
 .../hadoop/hbase/regionserver/TestHRegion.java  | 56 +++++++++++++++++++-
 2 files changed, 62 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ec021a7b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 8b8a108..a788696 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -2990,11 +2990,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         } catch (IOException ioe) {
           LOG.warn("Failed getting lock in batch put, row="
             + Bytes.toStringBinary(mutation.getRow()), ioe);
+          throw ioe;
         }
         if (rowLock == null) {
           // We failed to grab another lock
-          assert false: "Should never fail to get lock when blocking";
-          break; // stop acquiring more rows for this batch
+          throw new IOException("Failed getting lock in batch put, row=" +
+              Bytes.toStringBinary(mutation.getRow()));
         } else {
           acquiredRowLocks.add(rowLock);
         }
@@ -5068,6 +5069,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param readLock is the lock reader or writer. True indicates that a non-exlcusive
    *                 lock is requested
    */
+  @Override
   public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
     // Make sure the row is inside of this region before getting the lock for it.
     checkRow(row, "row lock");
@@ -7031,7 +7033,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   private static List<Tag> carryForwardTags(final Cell cell, final List<Tag> tags) {
     if (cell.getTagsLength() <= 0) return tags;
-    List<Tag> newTags = tags == null? new ArrayList<Tag>(): /*Append Tags*/tags; 
+    List<Tag> newTags = tags == null? new ArrayList<Tag>(): /*Append Tags*/tags;
     Iterator<Tag> i =
         CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
     while (i.hasNext()) newTags.add(i.next());
@@ -7322,7 +7324,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   // They are subtley different in quiet a few ways. This came out only
   // after study. I am not sure that many of the differences are intentional.
-  // TODO: St.Ack 20150907 
+  // TODO: St.Ack 20150907
 
   @Override
   public Result increment(Increment mutation, long nonceGroup, long nonce)
@@ -7336,7 +7338,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     boolean writeToWAL = durability != Durability.SKIP_WAL;
     WALEdit walEdits = null;
     List<Cell> allKVs = new ArrayList<Cell>(mutation.size());
-    
+
     Map<Store, List<Cell>> tempMemstore = new HashMap<Store, List<Cell>>();
     long size = 0;
     long txid = 0;
@@ -8138,7 +8140,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     WALKey key = new HLogKey(getRegionInfo().getEncodedNameAsBytes(),
       getRegionInfo().getTable(), WALKey.NO_SEQUENCE_ID, 0, null,
       HConstants.NO_NONCE, HConstants.NO_NONCE, getMVCC());
-    
+
     // Call append but with an empty WALEdit.  The returned sequence id will not be associated
     // with any edit and we can be sure it went in after all outstanding appends.
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ec021a7b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 9dd7b82..7a9f769 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -56,7 +56,11 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.UUID;
+import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -100,6 +104,7 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.RowMutations;
@@ -6131,7 +6136,7 @@ public class TestHRegion {
           key.setWriteEntry(we);
           return 1L;
         }
-      
+
     });
     return wal;
   }
@@ -6154,7 +6159,7 @@ public class TestHRegion {
     // capture append() calls
     WAL wal = mockWAL();
     when(rss.getWAL((HRegionInfo) any())).thenReturn(wal);
-    
+
 
     // open a region first so that it can be closed later
     region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
@@ -6494,6 +6499,53 @@ public class TestHRegion {
       qual2, 0, qual2.length));
   }
 
+  @Test(timeout = 30000)
+  public void testBatchMutateWithWrongRegionException() throws IOException, InterruptedException {
+    final byte[] a = Bytes.toBytes("a");
+    final byte[] b = Bytes.toBytes("b");
+    final byte[] c = Bytes.toBytes("c"); // exclusive
+
+    int prevLockTimeout = CONF.getInt("hbase.rowlock.wait.duration", 30000);
+    CONF.setInt("hbase.rowlock.wait.duration", 3000);
+    final HRegion region = initHRegion(tableName, a, c, name.getMethodName(), CONF, false, fam1);
+
+    Mutation[] mutations = new Mutation[] {
+        new Put(a).addImmutable(fam1, null, null),
+        new Put(c).addImmutable(fam1, null, null), // this is outside the region boundary
+        new Put(b).addImmutable(fam1, null, null),
+    };
+
+    OperationStatus[] status = region.batchMutate(mutations);
+    assertEquals(status[0].getOperationStatusCode(), OperationStatusCode.SUCCESS);
+    assertEquals(status[1].getOperationStatusCode(), OperationStatusCode.SANITY_CHECK_FAILURE);
+    assertEquals(status[2].getOperationStatusCode(), OperationStatusCode.SUCCESS);
+
+    // test with a leaked row lock
+    ExecutorService exec = Executors.newSingleThreadExecutor();
+    exec.submit(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        region.getRowLock(b);
+        return null;
+      }
+    });
+    exec.shutdown();
+    exec.awaitTermination(30, TimeUnit.SECONDS);
+
+    mutations = new Mutation[] {
+        new Put(a).addImmutable(fam1, null, null),
+        new Put(b).addImmutable(fam1, null, null),
+    };
+
+    try {
+      status = region.batchMutate(mutations);
+      fail("Failed to throw exception");
+    } catch (IOException expected) {
+    }
+
+    CONF.setInt("hbase.rowlock.wait.duration", prevLockTimeout);
+  }
+
   static HRegion initHRegion(byte[] tableName, String callingMethod,
       byte[]... families) throws IOException {
     return initHRegion(tableName, callingMethod, HBaseConfiguration.create(),