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/20 17:51:35 UTC

[hbase] branch branch-2 updated (63593d6 -> f1d3b54)

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

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


    from 63593d6  HBASE-22649: Encode StoreFile path URLs in the UI to handle scenarios where CF contains special characters (like # etc.)
     new ffbf850  HBASE-22944 Check for hbase:quota table existence in SpaceQuotaRefresherChore
     new f1d3b54  HBASE-22142 Drop table RegionSizes with namespace quota

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../hadoop/hbase/quotas/MasterQuotaManager.java    |  9 ++++
 .../hadoop/hbase/quotas/MasterQuotasObserver.java  | 47 +++++++++++++------
 .../hbase/quotas/SpaceQuotaRefresherChore.java     | 19 ++++++++
 .../hbase/quotas/TestSpaceQuotaDropTable.java      | 52 ++++++++++++++++++++++
 ...estSpaceQuotaViolationPolicyRefresherChore.java |  1 +
 5 files changed, 115 insertions(+), 13 deletions(-)


[hbase] 01/02: HBASE-22944 Check for hbase:quota table existence in SpaceQuotaRefresherChore

Posted by el...@apache.org.
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

commit ffbf8503eaf171e4614f32f9d9a3c575e5edb85f
Author: shardul-cr7 <sh...@gmail.com>
AuthorDate: Thu Aug 29 11:41:04 2019 +0530

    HBASE-22944 Check for hbase:quota table existence in SpaceQuotaRefresherChore
    
    During startup, it's possible that quotas are enabled but the Master has
    not yet created the hbase:quotas table.
    
    Closes #559
    
    Signed-off-by: stack <st...@apache.org>
    Signed-off-by: Josh Elser <el...@apache.org>
---
 .../hadoop/hbase/quotas/SpaceQuotaRefresherChore.java | 19 +++++++++++++++++++
 .../TestSpaceQuotaViolationPolicyRefresherChore.java  |  1 +
 2 files changed, 20 insertions(+)

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 7ae7240..94f1bda 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
@@ -23,6 +23,7 @@ import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -60,6 +61,7 @@ public class SpaceQuotaRefresherChore extends ScheduledChore {
 
   private final RegionServerSpaceQuotaManager manager;
   private final Connection conn;
+  private boolean quotaTablePresent = false;
 
   public SpaceQuotaRefresherChore(RegionServerSpaceQuotaManager manager, Connection conn) {
     super(SpaceQuotaRefresherChore.class.getSimpleName(),
@@ -74,6 +76,13 @@ public class SpaceQuotaRefresherChore extends ScheduledChore {
   @Override
   protected void chore() {
     try {
+      // check whether quotaTable is present or not.
+      if (!quotaTablePresent && !checkQuotaTableExists()) {
+        LOG.info("Quota table not found, skipping quota manager cache refresh.");
+        return;
+      }
+      // since quotaTable is present so setting the flag as true.
+      quotaTablePresent = true;
       if (LOG.isTraceEnabled()) {
         LOG.trace("Reading current quota snapshots from hbase:quota.");
       }
@@ -145,6 +154,16 @@ public class SpaceQuotaRefresherChore extends ScheduledChore {
   }
 
   /**
+   * Checks if hbase:quota exists in hbase:meta
+   *
+   * @return true if hbase:quota table is in meta, else returns false.
+   * @throws IOException throws IOException
+   */
+  boolean checkQuotaTableExists() throws IOException {
+    return MetaTableAccessor.tableExists(getConnection(), QuotaUtil.QUOTA_TABLE_NAME);
+  }
+
+  /**
    * Checks if the given <code>snapshot</code> is in violation, allowing the snapshot to be null.
    * If the snapshot is null, this is interpreted as no snapshot which implies not in violation.
    *
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java
index 58270c3..aa871f1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java
@@ -82,6 +82,7 @@ public class TestSpaceQuotaViolationPolicyRefresherChore {
     chore = mock(SpaceQuotaRefresherChore.class);
     when(chore.getConnection()).thenReturn(conn);
     when(chore.getManager()).thenReturn(manager);
+    when(chore.checkQuotaTableExists()).thenReturn(true);
     doCallRealMethod().when(chore).chore();
     when(chore.isInViolation(any())).thenCallRealMethod();
     doCallRealMethod().when(chore).extractQuotaSnapshot(any(), any());


[hbase] 02/02: HBASE-22142 Drop table RegionSizes with namespace quota

Posted by el...@apache.org.
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

commit f1d3b5442292a02faacf955feec00936d6731fc8
Author: shardul-cr7 <sh...@gmail.com>
AuthorDate: Fri Sep 20 10:57:15 2019 -0400

    HBASE-22142 Drop table RegionSizes with namespace quota
    
    There was a bug in which we would not drop the RegionSizes
    for a table in a namespace, where the namespace had a quota
    on it. This allowed a scenario in which recreation of a table
    inside of a namespace would unintentionally move into violation
    despite the table being empty. Need to make sure the RegionSizes
    are dropped on table deletion if there is _any_ quota applying
    to that table.
    
    Signed-off-by: Josh Elser <el...@apache.org>
---
 .../hadoop/hbase/quotas/MasterQuotaManager.java    |  9 ++++
 .../hadoop/hbase/quotas/MasterQuotasObserver.java  | 47 +++++++++++++------
 .../hbase/quotas/TestSpaceQuotaDropTable.java      | 52 ++++++++++++++++++++++
 3 files changed, 95 insertions(+), 13 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 d17f1d5..c103c68 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
@@ -709,5 +709,14 @@ public class MasterQuotaManager implements RegionStateListener {
     }
     return numEntriesRemoved;
   }
+
+  /**
+   * Removes each region size entry where the RegionInfo references the provided TableName.
+   *
+   * @param tableName tableName.
+   */
+  public void removeRegionSizesForTable(TableName tableName) {
+    regionSizes.keySet().removeIf(regionInfo -> regionInfo.getTable().equals(tableName));
+  }
 }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotasObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotasObserver.java
index cad3129..1ddc9e3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotasObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotasObserver.java
@@ -24,10 +24,14 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
+import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.HasMasterServices;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 
@@ -35,6 +39,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
  * An observer to automatically delete quotas when a table/namespace
  * is deleted.
  */
+@CoreCoprocessor
 @InterfaceAudience.Private
 public class MasterQuotasObserver implements MasterCoprocessor, MasterObserver {
   public static final String REMOVE_QUOTA_ON_TABLE_DELETE = "hbase.quota.remove.on.table.delete";
@@ -43,6 +48,7 @@ public class MasterQuotasObserver implements MasterCoprocessor, MasterObserver {
   private CoprocessorEnvironment cpEnv;
   private Configuration conf;
   private boolean quotasEnabled = false;
+  private MasterServices masterServices;
 
   @Override
   public Optional<MasterObserver> getMasterObserver() {
@@ -51,9 +57,19 @@ public class MasterQuotasObserver implements MasterCoprocessor, MasterObserver {
 
   @Override
   public void start(CoprocessorEnvironment ctx) throws IOException {
-    this.cpEnv = ctx;
-    this.conf = cpEnv.getConfiguration();
+    this.conf = ctx.getConfiguration();
     this.quotasEnabled = QuotaUtil.isQuotaEnabled(conf);
+
+    if (!(ctx instanceof MasterCoprocessorEnvironment)) {
+      throw new CoprocessorException("Must be loaded on master.");
+    }
+    // if running on master
+    MasterCoprocessorEnvironment mEnv = (MasterCoprocessorEnvironment) ctx;
+    if (mEnv instanceof HasMasterServices) {
+      this.masterServices = ((HasMasterServices) mEnv).getMasterServices();
+    } else {
+      throw new CoprocessorException("Must be loaded on a master having master services.");
+    }
   }
 
   @Override
@@ -64,18 +80,23 @@ public class MasterQuotasObserver implements MasterCoprocessor, MasterObserver {
       return;
     }
     final Connection conn = ctx.getEnvironment().getConnection();
-    Quotas quotas = QuotaUtil.getTableQuota(conn, tableName);
-    if (quotas != null){
-      if (quotas.hasSpace()){
-        QuotaSettings settings = QuotaSettingsFactory.removeTableSpaceLimit(tableName);
-        try (Admin admin = conn.getAdmin()) {
-          admin.setQuota(settings);
+    Quotas tableQuotas = QuotaUtil.getTableQuota(conn, tableName);
+    Quotas namespaceQuotas = QuotaUtil.getNamespaceQuota(conn, tableName.getNamespaceAsString());
+    if (tableQuotas != null || namespaceQuotas != null) {
+      // Remove regions of table from space quota map.
+      this.masterServices.getMasterQuotaManager().removeRegionSizesForTable(tableName);
+      if (tableQuotas != null) {
+        if (tableQuotas.hasSpace()) {
+          QuotaSettings settings = QuotaSettingsFactory.removeTableSpaceLimit(tableName);
+          try (Admin admin = conn.getAdmin()) {
+            admin.setQuota(settings);
+          }
         }
-      }
-      if (quotas.hasThrottle()){
-        QuotaSettings settings = QuotaSettingsFactory.unthrottleTable(tableName);
-        try (Admin admin = conn.getAdmin()) {
-          admin.setQuota(settings);
+        if (tableQuotas.hasThrottle()) {
+          QuotaSettings settings = QuotaSettingsFactory.unthrottleTable(tableName);
+          try (Admin admin = conn.getAdmin()) {
+            admin.setQuota(settings);
+          }
         }
       }
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaDropTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaDropTable.java
index cb60d12..11b9ec6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaDropTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaDropTable.java
@@ -15,16 +15,23 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+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.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.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.HMaster;
 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.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -87,6 +94,51 @@ public class TestSpaceQuotaDropTable {
     setQuotaAndThenDropTable(SpaceViolationPolicy.DISABLE);
   }
 
+  @Test
+  public void testSetQuotaAndThenDropTableWithRegionReport() throws Exception {
+    final TableName tn = helper.createTable();
+    helper.setQuotaLimit(tn, SpaceViolationPolicy.NO_INSERTS, 1L);
+    helper.writeData(tn, 2L);
+
+    final HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
+    final 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(), tn);
+        return regionSizes.containsKey(tableRegions.get(0));
+      }
+    });
+
+    boolean hasRegionSize = false;
+
+    // region report should be present before dropping the table.
+    for (Map.Entry<RegionInfo, Long> entry : quotaManager.snapshotRegionSizes().entrySet()) {
+      if (entry.getKey().getTable().equals(tn)) {
+        hasRegionSize = true;
+        break;
+      }
+    }
+
+    // regionSize report for the given table should be present before dropping the table.
+    Assert.assertTrue(hasRegionSize);
+
+    // drop the table
+    TEST_UTIL.getAdmin().disableTable(tn);
+    TEST_UTIL.getAdmin().deleteTable(tn);
+
+    // check if deleted table region report still present in the map.
+    for (Map.Entry<RegionInfo, Long> entry : quotaManager.snapshotRegionSizes().entrySet()) {
+      if (entry.getKey().getTable().equals(tn)) {
+        Assert.fail("Dropped table regionSizes were not deleted during the drop command");
+      }
+    }
+  }
+
   private void setQuotaAndThenDropTable(SpaceViolationPolicy policy) throws Exception {
     Put put = new Put(Bytes.toBytes("to_reject"));
     put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),