You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2018/05/24 22:32:27 UTC

hbase git commit: HBASE-20588 Space quota change after quota violation doesn't seem to take in effect

Repository: hbase
Updated Branches:
  refs/heads/branch-2.0 603a0372a -> e3deb9156


HBASE-20588 Space quota change after quota violation doesn't seem to take in effect

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/branch-2.0
Commit: e3deb9156e787bea8de48d7f216bd9cc91e695c7
Parents: 603a037
Author: Nihal Jain <ni...@gmail.com>
Authored: Thu May 24 10:57:40 2018 +0530
Committer: Josh Elser <el...@apache.org>
Committed: Thu May 24 18:21:08 2018 -0400

----------------------------------------------------------------------
 .../hbase/quotas/SpaceQuotaRefresherChore.java  |  11 +
 .../hadoop/hbase/quotas/TestSpaceQuotas.java    | 203 ++++++++++++++++++-
 2 files changed, 212 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e3deb915/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
index d3be620..045a44b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
@@ -112,6 +112,17 @@ public class SpaceQuotaRefresherChore extends ScheduledChore {
         }
       }
 
+      // Disable violation policy for all such tables which have been removed in new snapshot
+      for (TableName tableName : currentSnapshots.keySet()) {
+        // check whether table was removed in new snapshot
+        if (!newSnapshots.containsKey(tableName)) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Removing quota violation policy on " + tableName);
+          }
+          getManager().disableViolationPolicyEnforcement(tableName);
+        }
+      }
+
       // We're intentionally ignoring anything extra with the currentSnapshots. If we were missing
       // information from the RegionServers to create an accurate SpaceQuotaSnapshot in the Master,
       // the Master will generate a new SpaceQuotaSnapshot which represents this state. This lets

http://git-wip-us.apache.org/repos/asf/hbase/blob/e3deb915/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
index ae71b31..7365f7b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.ClientServiceCallable;
@@ -355,6 +356,161 @@ public class TestSpaceQuotas {
     verifyViolation(policy, tn, p);
   }
 
+  @Test
+  public void testSetQuotaAndThenRemoveWithNoInserts() throws Exception {
+    setQuotaAndThenRemove(SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test
+  public void testSetQuotaAndThenRemoveWithNoWrite() throws Exception {
+    setQuotaAndThenRemove(SpaceViolationPolicy.NO_WRITES);
+  }
+
+  @Test
+  public void testSetQuotaAndThenRemoveWithNoWritesCompactions() throws Exception {
+    setQuotaAndThenRemove(SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
+  }
+
+  @Test
+  public void testSetQuotaAndThenRemoveWithDisable() throws Exception {
+    setQuotaAndThenRemove(SpaceViolationPolicy.DISABLE);
+  }
+
+  @Test
+  public void testSetQuotaAndThenDropTableWithNoInserts() throws Exception {
+    setQuotaAndThenDropTable(SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test
+  public void testSetQuotaAndThenDropTableWithNoWrite() throws Exception {
+    setQuotaAndThenDropTable(SpaceViolationPolicy.NO_WRITES);
+  }
+
+  @Test
+  public void testSetQuotaAndThenDropTableeWithNoWritesCompactions() throws Exception {
+    setQuotaAndThenDropTable(SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
+  }
+
+  @Test
+  public void testSetQuotaAndThenIncreaseQuotaWithNoInserts() throws Exception {
+    setQuotaAndThenIncreaseQuota(SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test
+  public void testSetQuotaAndThenIncreaseQuotaWithNoWrite() throws Exception {
+    setQuotaAndThenIncreaseQuota(SpaceViolationPolicy.NO_WRITES);
+  }
+
+  @Test
+  public void testSetQuotaAndThenIncreaseQuotaWithNoWritesCompactions() throws Exception {
+    setQuotaAndThenIncreaseQuota(SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
+  }
+
+  @Test
+  public void testSetQuotaAndThenRemoveInOneWithNoInserts() throws Exception {
+    setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test
+  public void testSetQuotaAndThenRemoveInOneWithNoWrite() throws Exception {
+    setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy.NO_WRITES);
+  }
+
+  @Test
+  public void testSetQuotaAndThenRemoveInOneWithNoWritesCompaction() throws Exception {
+    setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
+  }
+
+  @Test
+  public void testSetQuotaAndThenRemoveInOneWithDisable() throws Exception {
+    setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy.DISABLE);
+  }
+
+  private void setQuotaAndThenRemove(SpaceViolationPolicy policy) throws Exception {
+    Put put = new Put(Bytes.toBytes("to_reject"));
+    put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
+      Bytes.toBytes("reject"));
+
+    // Do puts until we violate space policy
+    final TableName tn = writeUntilViolationAndVerifyViolation(policy, put);
+
+    // Now, remove the quota
+    removeQuotaFromtable(tn);
+
+    // Put some rows now: should not violate as quota settings removed
+    verifyNoViolation(policy, tn, put);
+  }
+
+  private void setQuotaAndThenDropTable(SpaceViolationPolicy policy) throws Exception {
+    Put put = new Put(Bytes.toBytes("to_reject"));
+    put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
+      Bytes.toBytes("reject"));
+
+    // Do puts until we violate space policy
+    final TableName tn = writeUntilViolationAndVerifyViolation(policy, put);
+
+    // Now, drop the table
+    TEST_UTIL.deleteTable(tn);
+    LOG.debug("Successfully deleted table ", tn);
+
+    // Now re-create the table
+    TEST_UTIL.createTable(tn, Bytes.toBytes(SpaceQuotaHelperForTests.F1));
+    LOG.debug("Successfully re-created table ", tn);
+
+    // Put some rows now: should not violate as table/quota was dropped
+    verifyNoViolation(policy, tn, put);
+  }
+
+  private void setQuotaAndThenIncreaseQuota(SpaceViolationPolicy policy) throws Exception {
+    Put put = new Put(Bytes.toBytes("to_reject"));
+    put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
+      Bytes.toBytes("reject"));
+
+    // Do puts until we violate space policy
+    final TableName tn = writeUntilViolationAndVerifyViolation(policy, put);
+
+    // Now, increase limit and perform put
+    increaseQuotaLimit(tn, policy);
+
+    // Put some row now: should not violate as quota limit increased
+    verifyNoViolation(policy, tn, put);
+  }
+
+  public void setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy policy)
+      throws Exception {
+    Put put = new Put(Bytes.toBytes("to_reject"));
+    put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
+      Bytes.toBytes("reject"));
+
+    // Do puts until we violate space policy on table tn1
+    final TableName tn1 = writeUntilViolationAndVerifyViolation(policy, put);
+
+    // Do puts until we violate space policy on table tn2
+    final TableName tn2 = writeUntilViolationAndVerifyViolation(policy, put);
+
+    // Now, remove the quota from table tn1
+    removeQuotaFromtable(tn1);
+
+    // Put a new row now on tn1: should not violate as quota settings removed
+    verifyNoViolation(policy, tn1, put);
+    // Put a new row now on tn2: should violate as quota settings exists
+    verifyViolation(policy, tn2, put);
+  }
+
+  private void removeQuotaFromtable(final TableName tn) throws Exception {
+    QuotaSettings removeQuota = QuotaSettingsFactory.removeTableSpaceLimit(tn);
+    TEST_UTIL.getAdmin().setQuota(removeQuota);
+    LOG.debug("Space quota settings removed from the table ", tn);
+  }
+
+  private void increaseQuotaLimit(final TableName tn, SpaceViolationPolicy policy)
+      throws Exception {
+    final long sizeLimit = 4L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, policy);
+    TEST_UTIL.getAdmin().setQuota(settings);
+    LOG.debug("Quota limit increased for table ", tn);
+  }
+
   private Map<RegionInfo,Long> getReportedSizesForTable(TableName tn) {
     HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
     MasterQuotaManager quotaManager = master.getMasterQuotaManager();
@@ -413,8 +569,13 @@ public class TestSpaceQuotas {
         Thread.sleep(2000);
       } catch (Exception e) {
         String msg = StringUtils.stringifyException(e);
-        assertTrue("Expected exception message to contain the word '" + policyToViolate.name() +
-            "', but was " + msg, msg.contains(policyToViolate.name()));
+        if (policyToViolate.equals(SpaceViolationPolicy.DISABLE)) {
+          assertTrue(e instanceof TableNotEnabledException);
+        } else {
+          assertTrue("Expected exception message to contain the word '" + policyToViolate.name()
+              + "', but was " + msg,
+            msg.contains(policyToViolate.name()));
+        }
         sawError = true;
       }
     }
@@ -467,4 +628,42 @@ public class TestSpaceQuotas {
       }
     };
   }
+
+  private void verifyNoViolation(SpaceViolationPolicy policyToViolate, TableName tn, Mutation m)
+      throws Exception {
+    // But let's try a few times to write data before failing
+    boolean sawSuccess = false;
+    for (int i = 0; i < NUM_RETRIES && !sawSuccess; i++) {
+      try (Table table = TEST_UTIL.getConnection().getTable(tn)) {
+        if (m instanceof Put) {
+          table.put((Put) m);
+        } else if (m instanceof Delete) {
+          table.delete((Delete) m);
+        } else if (m instanceof Append) {
+          table.append((Append) m);
+        } else if (m instanceof Increment) {
+          table.increment((Increment) m);
+        } else {
+          fail(
+            "Failed to apply " + m.getClass().getSimpleName() + " to the table. Programming error");
+        }
+        sawSuccess = true;
+      } catch (Exception e) {
+        LOG.info("Rejected the " + m.getClass().getSimpleName() + ", will sleep and retry");
+        Thread.sleep(2000);
+      }
+    }
+    if (!sawSuccess) {
+      try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
+        ResultScanner scanner = quotaTable.getScanner(new Scan());
+        Result result = null;
+        LOG.info("Dumping contents of hbase:quota table");
+        while ((result = scanner.next()) != null) {
+          LOG.info(Bytes.toString(result.getRow()) + " => " + result.toString());
+        }
+        scanner.close();
+      }
+    }
+    assertTrue("Expected to succeed in writing data to a table not having quota ", sawSuccess);
+  }
 }