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 2019/09/26 20:20:25 UTC

[hbase] branch branch-2 updated: HBASE-22012 Prevent DISABLE quota violation policy from disabling violation policy

This is an automated email from the ASF dual-hosted git repository.

elserj pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 43b4ad9  HBASE-22012 Prevent DISABLE quota violation policy from disabling violation policy
43b4ad9 is described below

commit 43b4ad9e8a50bf25a45bba4469f608c18521aeb0
Author: shardul-cr7 <sh...@gmail.com>
AuthorDate: Wed Sep 25 11:33:16 2019 -0400

    HBASE-22012 Prevent DISABLE quota violation policy from disabling violation policy
    
    Space quotas has a feature which intends to avoid enacting a space quota
    violation policy when only a subset of the Regions for that Table have
    reported their space usage (under the assumption that we cannot make an
    informed decision if we do not include all regions in our calculations).
    
    This had the unintended side-effect, when a table is disabled as a part
    of a violation policy, of causing the regions for that table to not be
    reported which disables the violation policy and enables the table.
    
    Need to make sure that when a table is disabled because of a violation
    policy that the code does not automatically move that table out of
    violation because region sizes are not being reported (because those
    regions are not open).
    
    Closes #572
    
    Signed-off-by: Josh Elser <el...@apache.org>
---
 .../hadoop/hbase/quotas/MasterQuotaManager.java    | 53 ++++++++++++++++++++--
 .../hadoop/hbase/quotas/QuotaObserverChore.java    |  2 +-
 .../hbase/quotas/TestMasterQuotaManager.java       |  9 ++--
 .../quotas/TestSpaceQuotaBasicFunctioning.java     | 36 +++++++++++++++
 4 files changed, 91 insertions(+), 9 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index c103c68..1489a2b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -25,6 +25,8 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.lang3.builder.HashCodeBuilder;
@@ -693,16 +695,22 @@ public class MasterQuotaManager implements RegionStateListener {
     return copy;
   }
 
-  int pruneEntriesOlderThan(long timeToPruneBefore) {
+  int pruneEntriesOlderThan(long timeToPruneBefore, QuotaObserverChore quotaObserverChore) {
     if (regionSizes == null) {
       return 0;
     }
     int numEntriesRemoved = 0;
-    Iterator<Entry<RegionInfo,SizeSnapshotWithTimestamp>> iterator =
+    Iterator<Entry<RegionInfo, SizeSnapshotWithTimestamp>> iterator =
         regionSizes.entrySet().iterator();
     while (iterator.hasNext()) {
-      long currentEntryTime = iterator.next().getValue().getTime();
-      if (currentEntryTime < timeToPruneBefore) {
+      RegionInfo regionInfo = iterator.next().getKey();
+      long currentEntryTime = regionSizes.get(regionInfo).getTime();
+      // do not prune the entries if table is in violation and
+      // violation policy is disable to avoid cycle of enable/disable.
+      // Please refer HBASE-22012 for more details.
+      // prune entries older than time.
+      if (currentEntryTime < timeToPruneBefore && !isInViolationAndPolicyDisable(
+          regionInfo.getTable(), quotaObserverChore)) {
         iterator.remove();
         numEntriesRemoved++;
       }
@@ -711,6 +719,43 @@ public class MasterQuotaManager implements RegionStateListener {
   }
 
   /**
+   * Method to check if a table is in violation and policy set on table is DISABLE.
+   *
+   * @param tableName          tableName to check.
+   * @param quotaObserverChore QuotaObserverChore instance
+   * @return returns true if table is in violation and policy is disable else false.
+   */
+  private boolean isInViolationAndPolicyDisable(TableName tableName,
+      QuotaObserverChore quotaObserverChore) {
+    boolean isInViolationAtTable = false;
+    boolean isInViolationAtNamespace = false;
+    SpaceViolationPolicy tablePolicy = null;
+    SpaceViolationPolicy namespacePolicy = null;
+    // Get Current Snapshot for the given table
+    SpaceQuotaSnapshot tableQuotaSnapshot = quotaObserverChore.getTableQuotaSnapshot(tableName);
+    SpaceQuotaSnapshot namespaceQuotaSnapshot =
+        quotaObserverChore.getNamespaceQuotaSnapshot(tableName.getNamespaceAsString());
+    if (tableQuotaSnapshot != null) {
+      // check if table in violation
+      isInViolationAtTable = tableQuotaSnapshot.getQuotaStatus().isInViolation();
+      Optional<SpaceViolationPolicy> policy = tableQuotaSnapshot.getQuotaStatus().getPolicy();
+      if (policy.isPresent()) {
+        tablePolicy = policy.get();
+      }
+    }
+    if (namespaceQuotaSnapshot != null) {
+      // check namespace in violation
+      isInViolationAtNamespace = namespaceQuotaSnapshot.getQuotaStatus().isInViolation();
+      Optional<SpaceViolationPolicy> policy = namespaceQuotaSnapshot.getQuotaStatus().getPolicy();
+      if (policy.isPresent()) {
+        namespacePolicy = policy.get();
+      }
+    }
+    return (tablePolicy == SpaceViolationPolicy.DISABLE && isInViolationAtTable) || (
+        namespacePolicy == SpaceViolationPolicy.DISABLE && isInViolationAtNamespace);
+  }
+
+  /**
    * Removes each region size entry where the RegionInfo references the provided TableName.
    *
    * @param tableName tableName.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
index f565235..425c240 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
@@ -472,7 +472,7 @@ public class QuotaObserverChore extends ScheduledChore {
   void pruneOldRegionReports() {
     final long now = EnvironmentEdgeManager.currentTime();
     final long pruneTime = now - regionReportLifetimeMillis;
-    final int numRemoved = quotaManager.pruneEntriesOlderThan(pruneTime);
+    final int numRemoved = quotaManager.pruneEntriesOlderThan(pruneTime,this);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Removed " + numRemoved + " old region size reports that were older than "
           + pruneTime + ".");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java
index 5c41c41..a9305a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java
@@ -73,18 +73,19 @@ public class TestMasterQuotaManager {
 
     assertEquals(5, manager.snapshotRegionSizes().size());
 
+    QuotaObserverChore chore = mock(QuotaObserverChore.class);
     // Prune nothing
-    assertEquals(0, manager.pruneEntriesOlderThan(0));
+    assertEquals(0, manager.pruneEntriesOlderThan(0, chore));
     assertEquals(5, manager.snapshotRegionSizes().size());
-    assertEquals(0, manager.pruneEntriesOlderThan(10));
+    assertEquals(0, manager.pruneEntriesOlderThan(10, chore));
     assertEquals(5, manager.snapshotRegionSizes().size());
 
     // Prune the elements at time1
-    assertEquals(2, manager.pruneEntriesOlderThan(15));
+    assertEquals(2, manager.pruneEntriesOlderThan(15, chore));
     assertEquals(3, manager.snapshotRegionSizes().size());
 
     // Prune the elements at time2
-    assertEquals(2, manager.pruneEntriesOlderThan(30));
+    assertEquals(2, manager.pruneEntriesOlderThan(30, chore));
     assertEquals(1, manager.snapshotRegionSizes().size());
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaBasicFunctioning.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaBasicFunctioning.java
index 754c96e..d5d4159 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaBasicFunctioning.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaBasicFunctioning.java
@@ -19,24 +19,31 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -221,4 +228,33 @@ public class TestSpaceQuotaBasicFunctioning {
         Bytes.toBytes("reject"));
     helper.verifyViolation(policy, tn, p);
   }
+
+  @Test
+  public void testDisablePolicyQuotaAndViolate() throws Exception {
+    TableName tableName = helper.createTable();
+    helper.setQuotaLimit(tableName, SpaceViolationPolicy.DISABLE, 1L);
+    helper.writeData(tableName, SpaceQuotaHelperForTests.ONE_MEGABYTE * 2L);
+    TEST_UTIL.getConfiguration()
+        .setLong("hbase.master.quotas.region.report.retention.millis", 100);
+
+    HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
+    MasterQuotaManager quotaManager = master.getMasterQuotaManager();
+
+    // Make sure the master has report for the table.
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        Map<RegionInfo, Long> regionSizes = quotaManager.snapshotRegionSizes();
+        List<RegionInfo> tableRegions =
+            MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tableName);
+        return regionSizes.containsKey(tableRegions.get(0));
+      }
+    });
+
+    // Check if disabled table region report present in the map after retention period expired.
+    // It should be present after retention period expired.
+    final long regionSizes = quotaManager.snapshotRegionSizes().keySet().stream()
+        .filter(k -> k.getTable().equals(tableName)).count();
+    Assert.assertTrue(regionSizes > 0);
+  }
 }