You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2017/04/26 22:52:25 UTC

[18/40] hbase git commit: HBASE-17937 Memstore size becomes negative in case of expensive postPut/Delete Coprocessor call

HBASE-17937 Memstore size becomes negative in case of expensive postPut/Delete Coprocessor call

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 49cba2c237ecc1b3285d942f1ad176ea50c44cd1
Parents: 40cc666
Author: Abhishek Singh Chouhan <ab...@gmail.com>
Authored: Wed Apr 19 11:22:23 2017 +0530
Committer: zhangduo <zh...@apache.org>
Committed: Fri Apr 21 08:50:09 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      |   4 +-
 ...NegativeMemstoreSizeWithSlowCoprocessor.java | 104 +++++++++++++++++++
 2 files changed, 107 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/49cba2c2/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 b21a84d..4836dc8 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
@@ -3392,6 +3392,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         applyFamilyMapToMemstore(familyMaps[i], memstoreSize);
       }
 
+      // update memstore size
+      this.addAndGetMemstoreSize(memstoreSize);
+
       // calling the post CP hook for batch mutation
       if (!replay && coprocessorHost != null) {
         MiniBatchOperationInProgress<Mutation> miniBatchOp =
@@ -3444,7 +3447,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     } finally {
       // Call complete rather than completeAndWait because we probably had error if walKey != null
       if (writeEntry != null) mvcc.complete(writeEntry);
-      this.addAndGetMemstoreSize(memstoreSize);
       if (locked) {
         this.updatesLock.readLock().unlock();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/49cba2c2/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemstoreSizeWithSlowCoprocessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemstoreSizeWithSlowCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemstoreSizeWithSlowCoprocessor.java
new file mode 100644
index 0000000..ae2f055
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemstoreSizeWithSlowCoprocessor.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
+ * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
+ * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
+ * for the specific language governing permissions and limitations under the License.
+ */
+package org.apache.hadoop.hbase.coprocessor;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemstoreSize;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test that verifies we do not have memstore size negative when a postPut/Delete hook is
+ * slow/expensive and a flush is triggered at the same time the coprocessow is doing its work. To
+ * simulate this we call flush from the coprocessor itself
+ */
+@Category(LargeTests.class)
+public class TestNegativeMemstoreSizeWithSlowCoprocessor {
+
+  static final Log LOG = LogFactory.getLog(TestNegativeMemstoreSizeWithSlowCoprocessor.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final byte[] tableName = Bytes.toBytes("test_table");
+  private static final byte[] family = Bytes.toBytes("f");
+  private static final byte[] qualifier = Bytes.toBytes("q");
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
+      FlushingRegionObserver.class.getName());
+    conf.setBoolean(CoprocessorHost.ABORT_ON_ERROR_KEY, true);
+    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2); // Let's fail fast.
+    TEST_UTIL.startMiniCluster(1);
+    TEST_UTIL.createTable(TableName.valueOf(tableName), family);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testNegativeMemstoreSize() throws IOException, InterruptedException {
+    boolean IOEthrown = false;
+    Table table = null;
+    try {
+      table = TEST_UTIL.getConnection().getTable(TableName.valueOf(tableName));
+
+      // Adding data
+      Put put1 = new Put(Bytes.toBytes("row1"));
+      put1.addColumn(family, qualifier, Bytes.toBytes("Value1"));
+      table.put(put1);
+      Put put2 = new Put(Bytes.toBytes("row2"));
+      put2.addColumn(family, qualifier, Bytes.toBytes("Value2"));
+      table.put(put2);
+      table.put(put2);
+    } catch (IOException e) {
+      IOEthrown = true;
+    } finally {
+      Assert.assertFalse("Shouldn't have thrown an exception", IOEthrown);
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  public static class FlushingRegionObserver extends SimpleRegionObserver {
+
+    @Override
+    public void postPut(final ObserverContext<RegionCoprocessorEnvironment> c, final Put put,
+        final WALEdit edit, final Durability durability) throws IOException {
+      HRegion region = (HRegion) c.getEnvironment().getRegion();
+      super.postPut(c, put, edit, durability);
+
+      if (Bytes.equals(put.getRow(), Bytes.toBytes("row2"))) {
+        region.flush(false);
+        Assert.assertTrue(region.addAndGetMemstoreSize(new MemstoreSize()) >= 0);
+      }
+    }
+  }
+}
\ No newline at end of file