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/05/23 13:40:41 UTC

hbase git commit: HBASE-18093 Overloading the meaning of 'enabled' in Quota Manager to indicate either quota disabled or quota manager not ready is not good (Stephen Yuan Jiang)

Repository: hbase
Updated Branches:
  refs/heads/master 3f75ba195 -> 1d0295f4e


HBASE-18093 Overloading the meaning of 'enabled' in Quota Manager to indicate either quota disabled or quota manager not ready is not good (Stephen Yuan Jiang)


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

Branch: refs/heads/master
Commit: 1d0295f4e290ce9f0bcc30df9398cd81d75c4d50
Parents: 3f75ba1
Author: Stephen Yuan Jiang <sy...@gmail.com>
Authored: Tue May 23 06:40:33 2017 -0700
Committer: Stephen Yuan Jiang <sy...@gmail.com>
Committed: Tue May 23 06:40:33 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java |  2 +-
 .../hadoop/hbase/quotas/MasterQuotaManager.java | 45 ++++++++++++++------
 .../TestSimpleRegionNormalizerOnCluster.java    |  2 +-
 .../hbase/namespace/TestNamespaceAuditor.java   | 12 +++---
 4 files changed, 39 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1d0295f4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index cd96552..4dd6353 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -2247,7 +2247,7 @@ public class HMaster extends HRegionServer implements MasterServices {
         // coprocessor.
         MasterQuotaManager quotaManager = getMasterQuotaManager();
         if (quotaManager != null) {
-          if (quotaManager.isQuotaEnabled()) {
+          if (quotaManager.isQuotaInitialized()) {
             Quotas quotaForTable = QuotaUtil.getTableQuota(getConnection(), tableName);
             if (quotaForTable != null && quotaForTable.hasSpace()) {
               SpaceViolationPolicy policy = quotaForTable.getSpace().getViolationPolicy();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1d0295f4/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
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 1fb8cf4..9d24e6c 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
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -71,7 +72,7 @@ public class MasterQuotaManager implements RegionStateListener {
   private NamedLock<String> namespaceLocks;
   private NamedLock<TableName> tableLocks;
   private NamedLock<String> userLocks;
-  private boolean enabled = false;
+  private boolean initialized = false;
   private NamespaceAuditor namespaceQuotaManager;
   private ConcurrentHashMap<HRegionInfo, SizeSnapshotWithTimestamp> regionSizes;
 
@@ -101,14 +102,14 @@ public class MasterQuotaManager implements RegionStateListener {
 
     namespaceQuotaManager = new NamespaceAuditor(masterServices);
     namespaceQuotaManager.start();
-    enabled = true;
+    initialized = true;
   }
 
   public void stop() {
   }
 
-  public boolean isQuotaEnabled() {
-    return enabled && namespaceQuotaManager.isInitialized();
+  public boolean isQuotaInitialized() {
+    return initialized && namespaceQuotaManager.isInitialized();
   }
 
   /* ==========================================================================
@@ -284,13 +285,13 @@ public class MasterQuotaManager implements RegionStateListener {
   }
 
   public void setNamespaceQuota(NamespaceDescriptor desc) throws IOException {
-    if (enabled) {
+    if (initialized) {
       this.namespaceQuotaManager.addNamespace(desc);
     }
   }
 
   public void removeNamespaceQuota(String namespace) throws IOException {
-    if (enabled) {
+    if (initialized) {
       this.namespaceQuotaManager.deleteNamespace(namespace);
     }
   }
@@ -325,13 +326,13 @@ public class MasterQuotaManager implements RegionStateListener {
   }
 
   public void checkNamespaceTableAndRegionQuota(TableName tName, int regions) throws IOException {
-    if (enabled) {
+    if (initialized) {
       namespaceQuotaManager.checkQuotaToCreateTable(tName, regions);
     }
   }
   
   public void checkAndUpdateNamespaceRegionQuota(TableName tName, int regions) throws IOException {
-    if (enabled) {
+    if (initialized) {
       namespaceQuotaManager.checkQuotaToUpdateRegion(tName, regions);
     }
   }
@@ -340,20 +341,20 @@ public class MasterQuotaManager implements RegionStateListener {
    * @return cached region count, or -1 if quota manager is disabled or table status not found
   */
   public int getRegionCountOfTable(TableName tName) throws IOException {
-    if (enabled) {
+    if (initialized) {
       return namespaceQuotaManager.getRegionCountOfTable(tName);
     }
     return -1;
   }
 
   public void onRegionMerged(HRegionInfo hri) throws IOException {
-    if (enabled) {
+    if (initialized) {
       namespaceQuotaManager.updateQuotaForRegionMerge(hri);
     }
   }
 
   public void onRegionSplit(HRegionInfo hri) throws IOException {
-    if (enabled) {
+    if (initialized) {
       namespaceQuotaManager.checkQuotaToSplitRegion(hri);
     }
   }
@@ -365,7 +366,7 @@ public class MasterQuotaManager implements RegionStateListener {
    * @throws IOException Signals that an I/O exception has occurred.
    */
   public void removeTableFromNamespaceQuota(TableName tName) throws IOException {
-    if (enabled) {
+    if (initialized) {
       namespaceQuotaManager.removeFromNamespaceUsage(tName);
     }
   }
@@ -499,10 +500,26 @@ public class MasterQuotaManager implements RegionStateListener {
    */
 
   private void checkQuotaSupport() throws IOException {
-    if (!enabled) {
+    if (!QuotaUtil.isQuotaEnabled(masterServices.getConfiguration())) {
       throw new DoNotRetryIOException(
         new UnsupportedOperationException("quota support disabled"));
     }
+    if (!initialized) {
+      long maxWaitTime = masterServices.getConfiguration().getLong(
+        "hbase.master.wait.for.quota.manager.init", 30000); // default is 30 seconds.
+      long startTime = EnvironmentEdgeManager.currentTime();
+      do {
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException e) {
+          LOG.warn("Interrupted while waiting for Quota Manager to be initialized.");
+          break;
+        }
+      } while (!initialized && (EnvironmentEdgeManager.currentTime() - startTime) < maxWaitTime);
+      if (!initialized) {
+        throw new IOException("Quota manager is uninitialized, please retry later.");
+      }
+    }
   }
 
   private void createQuotaTable() throws IOException {
@@ -531,7 +548,7 @@ public class MasterQuotaManager implements RegionStateListener {
 
   @Override
   public void onRegionSplitReverted(HRegionInfo hri) throws IOException {
-    if (enabled) {
+    if (initialized) {
       this.namespaceQuotaManager.removeRegionFromNamespaceUsage(hri);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1d0295f4/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index 70a78de..586f93e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -76,7 +76,7 @@ public class TestSimpleRegionNormalizerOnCluster {
 
     // Start a cluster of two regionservers.
     TEST_UTIL.startMiniCluster(1);
-    TestNamespaceAuditor.waitForQuotaEnabled(TEST_UTIL);
+    TestNamespaceAuditor.waitForQuotaInitialize(TEST_UTIL);
     admin = TEST_UTIL.getAdmin();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1d0295f4/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index cc79915..2fb4741 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -112,7 +112,7 @@ public class TestNamespaceAuditor {
     conf.setClass("hbase.coprocessor.regionserver.classes", CPRegionServerObserver.class,
       RegionServerObserver.class);
     UTIL.startMiniCluster(1, 1);
-    waitForQuotaEnabled(UTIL);
+    waitForQuotaInitialize(UTIL);
     ADMIN = UTIL.getAdmin();
   }
 
@@ -132,8 +132,8 @@ public class TestNamespaceAuditor {
         ADMIN.deleteNamespace(ns.getName());
       }
     }
-    assertTrue("Quota manager not enabled", UTIL.getHBaseCluster().getMaster()
-        .getMasterQuotaManager().isQuotaEnabled());
+    assertTrue("Quota manager not initialized", UTIL.getHBaseCluster().getMaster()
+        .getMasterQuotaManager().isQuotaInitialized());
   }
 
   @Test
@@ -649,7 +649,7 @@ public class TestNamespaceAuditor {
         .getTables().size(), after.getTables().size());
   }
 
-  public static void waitForQuotaEnabled(final HBaseTestingUtility util) throws Exception {
+  public static void waitForQuotaInitialize(final HBaseTestingUtility util) throws Exception {
     util.waitFor(60000, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
@@ -658,7 +658,7 @@ public class TestNamespaceAuditor {
           return false;
         }
         MasterQuotaManager quotaManager = master.getMasterQuotaManager();
-        return quotaManager != null && quotaManager.isQuotaEnabled();
+        return quotaManager != null && quotaManager.isQuotaInitialized();
       }
     });
   }
@@ -667,7 +667,7 @@ public class TestNamespaceAuditor {
     UTIL.getHBaseCluster().getMaster(0).stop("Stopping to start again");
     UTIL.getHBaseCluster().waitOnMaster(0);
     UTIL.getHBaseCluster().startMaster();
-    waitForQuotaEnabled(UTIL);
+    waitForQuotaInitialize(UTIL);
   }
 
   private NamespaceAuditor getQuotaManager() {