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/08/07 18:25:51 UTC

hbase git commit: HBASE-20813 Removed RPC quotas when the associated table/Namespace is dropped off

Repository: hbase
Updated Branches:
  refs/heads/branch-2.0 f3c010da2 -> 2ab3ca014


HBASE-20813 Removed RPC quotas when the associated table/Namespace is dropped off

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/2ab3ca01
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2ab3ca01
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2ab3ca01

Branch: refs/heads/branch-2.0
Commit: 2ab3ca014f3ea7676bb24f666c7a375ffa70d052
Parents: f3c010d
Author: Sakthi <ja...@cloudera.com>
Authored: Mon Aug 6 14:52:44 2018 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Tue Aug 7 14:18:34 2018 -0400

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java |  18 +-
 .../hbase/quotas/MasterQuotasObserver.java      | 108 ++++++++
 .../hbase/quotas/MasterSpaceQuotaObserver.java  |  92 -------
 .../hbase/quotas/TestMasterQuotasObserver.java  | 245 +++++++++++++++++++
 .../TestMasterQuotasObserverWithMocks.java      |  90 +++++++
 .../quotas/TestMasterSpaceQuotaObserver.java    | 177 --------------
 .../TestMasterSpaceQuotaObserverWithMocks.java  |  90 -------
 7 files changed, 452 insertions(+), 368 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2ab3ca01/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 15d44f2..3f296c1 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
@@ -145,7 +145,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
-import org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver;
+import org.apache.hadoop.hbase.quotas.MasterQuotasObserver;
 import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.SnapshotQuotaObserverChore;
@@ -861,10 +861,10 @@ public class HMaster extends HRegionServer implements MasterServices {
             conf, this.clusterConnection);
     tableCFsUpdater.copyTableCFs();
 
-    // Add the Observer to delete space quotas on table deletion before starting all CPs by
+    // Add the Observer to delete quotas on table deletion before starting all CPs by
     // default with quota support, avoiding if user specifically asks to not load this Observer.
     if (QuotaUtil.isQuotaEnabled(conf)) {
-      updateConfigurationForSpaceQuotaObserver(conf);
+      updateConfigurationForQuotasObserver(conf);
     }
     // initialize master side coprocessors before we start handling requests
     status.setStatus("Initializing master coprocessors");
@@ -1019,15 +1019,15 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   /**
-   * Adds the {@code MasterSpaceQuotaObserver} to the list of configured Master observers to
-   * automatically remove space quotas for a table when that table is deleted.
+   * Adds the {@code MasterQuotasObserver} to the list of configured Master observers to
+   * automatically remove quotas for a table when that table is deleted.
    */
   @VisibleForTesting
-  public void updateConfigurationForSpaceQuotaObserver(Configuration conf) {
+  public void updateConfigurationForQuotasObserver(Configuration conf) {
     // We're configured to not delete quotas on table deletion, so we don't need to add the obs.
     if (!conf.getBoolean(
-          MasterSpaceQuotaObserver.REMOVE_QUOTA_ON_TABLE_DELETE,
-          MasterSpaceQuotaObserver.REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT)) {
+          MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE,
+          MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT)) {
       return;
     }
     String[] masterCoprocs = conf.getStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);
@@ -1036,7 +1036,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (length > 0) {
       System.arraycopy(masterCoprocs, 0, updatedCoprocs, 0, masterCoprocs.length);
     }
-    updatedCoprocs[length] = MasterSpaceQuotaObserver.class.getName();
+    updatedCoprocs[length] = MasterQuotasObserver.class.getName();
     conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, updatedCoprocs);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ab3ca01/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotasObserver.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..cad3129
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotasObserver.java
@@ -0,0 +1,108 @@
+/*
+ * 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.quotas;
+
+import java.io.IOException;
+import java.util.Optional;
+
+import org.apache.hadoop.conf.Configuration;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+
+/**
+ * An observer to automatically delete quotas when a table/namespace
+ * is deleted.
+ */
+@InterfaceAudience.Private
+public class MasterQuotasObserver implements MasterCoprocessor, MasterObserver {
+  public static final String REMOVE_QUOTA_ON_TABLE_DELETE = "hbase.quota.remove.on.table.delete";
+  public static final boolean REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT = true;
+
+  private CoprocessorEnvironment cpEnv;
+  private Configuration conf;
+  private boolean quotasEnabled = false;
+
+  @Override
+  public Optional<MasterObserver> getMasterObserver() {
+    return Optional.of(this);
+  }
+
+  @Override
+  public void start(CoprocessorEnvironment ctx) throws IOException {
+    this.cpEnv = ctx;
+    this.conf = cpEnv.getConfiguration();
+    this.quotasEnabled = QuotaUtil.isQuotaEnabled(conf);
+  }
+
+  @Override
+  public void postDeleteTable(
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException {
+    // Do nothing if quotas aren't enabled
+    if (!quotasEnabled) {
+      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);
+        }
+      }
+      if (quotas.hasThrottle()){
+        QuotaSettings settings = QuotaSettingsFactory.unthrottleTable(tableName);
+        try (Admin admin = conn.getAdmin()) {
+          admin.setQuota(settings);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void postDeleteNamespace(
+      ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException {
+    // Do nothing if quotas aren't enabled
+    if (!quotasEnabled) {
+      return;
+    }
+    final Connection conn = ctx.getEnvironment().getConnection();
+    Quotas quotas = QuotaUtil.getNamespaceQuota(conn, namespace);
+    if (quotas != null) {
+      if (quotas.hasSpace()) {
+        QuotaSettings settings = QuotaSettingsFactory.removeNamespaceSpaceLimit(namespace);
+        try (Admin admin = conn.getAdmin()) {
+          admin.setQuota(settings);
+        }
+      }
+      if (quotas.hasThrottle()) {
+        QuotaSettings settings = QuotaSettingsFactory.unthrottleNamespace(namespace);
+        try (Admin admin = conn.getAdmin()) {
+          admin.setQuota(settings);
+        }
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ab3ca01/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.java
deleted file mode 100644
index 9983875..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.quotas;
-
-import java.io.IOException;
-import java.util.Optional;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-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.shaded.protobuf.generated.QuotaProtos.Quotas;
-
-/**
- * An observer to automatically delete space quotas when a table/namespace
- * are deleted.
- */
-@InterfaceAudience.Private
-public class MasterSpaceQuotaObserver implements MasterCoprocessor, MasterObserver {
-  public static final String REMOVE_QUOTA_ON_TABLE_DELETE = "hbase.quota.remove.on.table.delete";
-  public static final boolean REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT = true;
-
-  private CoprocessorEnvironment cpEnv;
-  private Configuration conf;
-  private boolean quotasEnabled = false;
-
-  @Override
-  public Optional<MasterObserver> getMasterObserver() {
-    return Optional.of(this);
-  }
-
-  @Override
-  public void start(CoprocessorEnvironment ctx) throws IOException {
-    this.cpEnv = ctx;
-    this.conf = cpEnv.getConfiguration();
-    this.quotasEnabled = QuotaUtil.isQuotaEnabled(conf);
-  }
-
-  @Override
-  public void postDeleteTable(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException {
-    // Do nothing if quotas aren't enabled
-    if (!quotasEnabled) {
-      return;
-    }
-    final Connection conn = ctx.getEnvironment().getConnection();
-    Quotas quotas = QuotaUtil.getTableQuota(conn, tableName);
-    if (quotas != null && quotas.hasSpace()) {
-      QuotaSettings settings = QuotaSettingsFactory.removeTableSpaceLimit(tableName);
-      try (Admin admin = conn.getAdmin()) {
-        admin.setQuota(settings);
-      }
-    }
-  }
-
-  @Override
-  public void postDeleteNamespace(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException {
-    // Do nothing if quotas aren't enabled
-    if (!quotasEnabled) {
-      return;
-    }
-    final Connection conn = ctx.getEnvironment().getConnection();
-    Quotas quotas = QuotaUtil.getNamespaceQuota(conn, namespace);
-    if (quotas != null && quotas.hasSpace()) {
-      QuotaSettings settings = QuotaSettingsFactory.removeNamespaceSpaceLimit(namespace);
-      try (Admin admin = conn.getAdmin()) {
-        admin.setQuota(settings);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ab3ca01/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotasObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotasObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotasObserver.java
new file mode 100644
index 0000000..b6b7924
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotasObserver.java
@@ -0,0 +1,245 @@
+/**
+ * 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.quotas;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+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.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+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.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test class for {@link MasterQuotasObserver}.
+ */
+@Category(MediumTests.class)
+public class TestMasterQuotasObserver {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterQuotasObserver.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static SpaceQuotaHelperForTests helper;
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void removeAllQuotas() throws Exception {
+    if (helper == null) {
+      helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, new AtomicLong());
+    }
+    final Connection conn = TEST_UTIL.getConnection();
+    // Wait for the quota table to be created
+    if (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME)) {
+      helper.waitForQuotaTable(conn);
+    } else {
+      // Or, clean up any quotas from previous test runs.
+      helper.removeAllQuotas(conn);
+      assertEquals(0, helper.listNumDefinedQuotas(conn));
+    }
+  }
+
+  @Test
+  public void testTableSpaceQuotaRemoved() throws Exception {
+    final Connection conn = TEST_UTIL.getConnection();
+    final Admin admin = conn.getAdmin();
+    final TableName tn = TableName.valueOf(testName.getMethodName());
+    // Drop the table if it somehow exists
+    if (admin.tableExists(tn)) {
+      dropTable(admin, tn);
+    }
+    createTable(admin, tn);
+    assertEquals(0, getNumSpaceQuotas());
+
+    // Set space quota
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, 1024L, SpaceViolationPolicy.NO_INSERTS);
+    admin.setQuota(settings);
+    assertEquals(1, getNumSpaceQuotas());
+
+    // Drop the table and observe the Space quota being automatically deleted as well
+    dropTable(admin, tn);
+    assertEquals(0, getNumSpaceQuotas());
+  }
+
+  @Test
+  public void testTableRPCQuotaRemoved() throws Exception {
+    final Connection conn = TEST_UTIL.getConnection();
+    final Admin admin = conn.getAdmin();
+    final TableName tn = TableName.valueOf(testName.getMethodName());
+    // Drop the table if it somehow exists
+    if (admin.tableExists(tn)) {
+      dropTable(admin, tn);
+    }
+
+    createTable(admin, tn);
+    assertEquals(0, getThrottleQuotas());
+
+    // Set RPC quota
+    QuotaSettings settings =
+        QuotaSettingsFactory.throttleTable(tn, ThrottleType.REQUEST_SIZE, 2L, TimeUnit.HOURS);
+    admin.setQuota(settings);
+
+    assertEquals(1, getThrottleQuotas());
+
+    // Delete the table and observe the RPC quota being automatically deleted as well
+    dropTable(admin, tn);
+    assertEquals(0, getThrottleQuotas());
+  }
+
+  @Test
+  public void testNamespaceSpaceQuotaRemoved() throws Exception {
+    final Connection conn = TEST_UTIL.getConnection();
+    final Admin admin = conn.getAdmin();
+    final String ns = testName.getMethodName();
+    // Drop the ns if it somehow exists
+    if (namespaceExists(ns)) {
+      admin.deleteNamespace(ns);
+    }
+
+    // Create the ns
+    NamespaceDescriptor desc = NamespaceDescriptor.create(ns).build();
+    admin.createNamespace(desc);
+    assertEquals(0, getNumSpaceQuotas());
+
+    // Set a quota
+    QuotaSettings settings = QuotaSettingsFactory.limitNamespaceSpace(
+        ns, 1024L, SpaceViolationPolicy.NO_INSERTS);
+    admin.setQuota(settings);
+    assertEquals(1, getNumSpaceQuotas());
+
+    // Delete the namespace and observe the quota being automatically deleted as well
+    admin.deleteNamespace(ns);
+    assertEquals(0, getNumSpaceQuotas());
+  }
+
+  @Test
+  public void testNamespaceRPCQuotaRemoved() throws Exception {
+    final Connection conn = TEST_UTIL.getConnection();
+    final Admin admin = conn.getAdmin();
+    final String ns = testName.getMethodName();
+    // Drop the ns if it somehow exists
+    if (namespaceExists(ns)) {
+      admin.deleteNamespace(ns);
+    }
+
+    // Create the ns
+    NamespaceDescriptor desc = NamespaceDescriptor.create(ns).build();
+    admin.createNamespace(desc);
+    assertEquals(0, getThrottleQuotas());
+
+    // Set a quota
+    QuotaSettings settings =
+        QuotaSettingsFactory.throttleNamespace(ns, ThrottleType.REQUEST_SIZE, 2L, TimeUnit.HOURS);
+    admin.setQuota(settings);
+    assertEquals(1, getThrottleQuotas());
+
+    // Delete the namespace and observe the quota being automatically deleted as well
+    admin.deleteNamespace(ns);
+    assertEquals(0, getThrottleQuotas());
+  }
+
+  @Test
+  public void testObserverAddedByDefault() throws Exception {
+    final HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
+    final MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
+    Set<String> coprocessorNames = cpHost.getCoprocessors();
+    assertTrue(
+        "Did not find MasterQuotasObserver in list of CPs: " + coprocessorNames,
+        coprocessorNames.contains(MasterQuotasObserver.class.getSimpleName()));
+  }
+
+  public boolean namespaceExists(String ns) throws IOException {
+    NamespaceDescriptor[] descs = TEST_UTIL.getAdmin().listNamespaceDescriptors();
+    for (NamespaceDescriptor desc : descs) {
+      if (ns.equals(desc.getName())) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public int getNumSpaceQuotas() throws Exception {
+    QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration());
+    int numSpaceQuotas = 0;
+    for (QuotaSettings quotaSettings : scanner) {
+      if (quotaSettings.getQuotaType() == QuotaType.SPACE) {
+        numSpaceQuotas++;
+      }
+    }
+    return numSpaceQuotas;
+  }
+
+  public int getThrottleQuotas() throws Exception {
+    QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration());
+    int throttleQuotas = 0;
+    for (QuotaSettings quotaSettings : scanner) {
+      if (quotaSettings.getQuotaType() == QuotaType.THROTTLE) {
+        throttleQuotas++;
+      }
+    }
+    return throttleQuotas;
+  }
+
+  private void createTable(Admin admin, TableName tn) throws Exception {
+    // Create a table
+    HTableDescriptor tableDesc = new HTableDescriptor(tn);
+    tableDesc.addFamily(new HColumnDescriptor("F1"));
+    admin.createTable(tableDesc);
+  }
+
+  private void dropTable(Admin admin, TableName tn) throws  Exception {
+    admin.disableTable(tn);
+    admin.deleteTable(tn);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ab3ca01/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotasObserverWithMocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotasObserverWithMocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotasObserverWithMocks.java
new file mode 100644
index 0000000..abb0d8b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotasObserverWithMocks.java
@@ -0,0 +1,90 @@
+/**
+ * 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.quotas;
+
+import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY;
+import static org.apache.hadoop.hbase.quotas.MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.mock;
+
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.security.access.AccessController;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for MasterQuotasObserver that does not require a cluster.
+ */
+@Category(SmallTests.class)
+public class TestMasterQuotasObserverWithMocks {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterQuotasObserverWithMocks.class);
+
+  private HMaster master;
+  private Configuration conf;
+
+  @Before
+  public void setup() {
+    conf = HBaseConfiguration.create();
+    master = mock(HMaster.class);
+    doCallRealMethod().when(master).updateConfigurationForQuotasObserver(
+        any());
+  }
+
+  @Test
+  public void testAddDefaultObserver() {
+    master.updateConfigurationForQuotasObserver(conf);
+    assertEquals(MasterQuotasObserver.class.getName(), conf.get(MASTER_COPROCESSOR_CONF_KEY));
+  }
+
+  @Test
+  public void testDoNotAddDefaultObserver() {
+    conf.setBoolean(REMOVE_QUOTA_ON_TABLE_DELETE, false);
+    master.updateConfigurationForQuotasObserver(conf);
+    // Configuration#getStrings returns null when unset
+    assertNull(conf.getStrings(MASTER_COPROCESSOR_CONF_KEY));
+  }
+
+  @Test
+  public void testAppendsObserver() {
+    conf.set(MASTER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
+    master.updateConfigurationForQuotasObserver(conf);
+    Set<String> coprocs = new HashSet<>(conf.getStringCollection(MASTER_COPROCESSOR_CONF_KEY));
+    assertEquals(2, coprocs.size());
+    assertTrue(
+        "Observed coprocessors were: " + coprocs,
+        coprocs.contains(AccessController.class.getName()));
+    assertTrue(
+        "Observed coprocessors were: " + coprocs,
+        coprocs.contains(MasterQuotasObserver.class.getName()));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ab3ca01/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java
deleted file mode 100644
index 391b238..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/**
- * 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.quotas;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.Set;
-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.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-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.CoprocessorHost;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-
-/**
- * Test class for {@link MasterSpaceQuotaObserver}.
- */
-@Category(MediumTests.class)
-public class TestMasterSpaceQuotaObserver {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestMasterSpaceQuotaObserver.class);
-
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static SpaceQuotaHelperForTests helper;
-
-  @Rule
-  public TestName testName = new TestName();
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
-    TEST_UTIL.startMiniCluster(1);
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Before
-  public void removeAllQuotas() throws Exception {
-    if (helper == null) {
-      helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, new AtomicLong());
-    }
-    final Connection conn = TEST_UTIL.getConnection();
-    // Wait for the quota table to be created
-    if (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME)) {
-      helper.waitForQuotaTable(conn);
-    } else {
-      // Or, clean up any quotas from previous test runs.
-      helper.removeAllQuotas(conn);
-      assertEquals(0, helper.listNumDefinedQuotas(conn));
-    }
-  }
-
-  @Test
-  public void testTableQuotaRemoved() throws Exception {
-    final Connection conn = TEST_UTIL.getConnection();
-    final Admin admin = conn.getAdmin();
-    final TableName tn = TableName.valueOf(testName.getMethodName());
-    // Drop the table if it somehow exists
-    if (admin.tableExists(tn)) {
-      admin.disableTable(tn);
-      admin.deleteTable(tn);
-    }
-
-    // Create a table
-    HTableDescriptor tableDesc = new HTableDescriptor(tn);
-    tableDesc.addFamily(new HColumnDescriptor("F1"));
-    admin.createTable(tableDesc);
-    assertEquals(0, getNumSpaceQuotas());
-
-    // Set a quota
-    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
-        tn, 1024L, SpaceViolationPolicy.NO_INSERTS);
-    admin.setQuota(settings);
-    assertEquals(1, getNumSpaceQuotas());
-
-    // Delete the table and observe the quota being automatically deleted as well
-    admin.disableTable(tn);
-    admin.deleteTable(tn);
-    assertEquals(0, getNumSpaceQuotas());
-  }
-
-  @Test
-  public void testNamespaceQuotaRemoved() throws Exception {
-    final Connection conn = TEST_UTIL.getConnection();
-    final Admin admin = conn.getAdmin();
-    final String ns = testName.getMethodName();
-    // Drop the ns if it somehow exists
-    if (namespaceExists(ns)) {
-      admin.deleteNamespace(ns);
-    }
-
-    // Create the ns
-    NamespaceDescriptor desc = NamespaceDescriptor.create(ns).build();
-    admin.createNamespace(desc);
-    assertEquals(0, getNumSpaceQuotas());
-
-    // Set a quota
-    QuotaSettings settings = QuotaSettingsFactory.limitNamespaceSpace(
-        ns, 1024L, SpaceViolationPolicy.NO_INSERTS);
-    admin.setQuota(settings);
-    assertEquals(1, getNumSpaceQuotas());
-
-    // Delete the table and observe the quota being automatically deleted as well
-    admin.deleteNamespace(ns);
-    assertEquals(0, getNumSpaceQuotas());
-  }
-
-  @Test
-  public void testObserverAddedByDefault() throws Exception {
-    final HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-    final MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
-    Set<String> coprocessorNames = cpHost.getCoprocessors();
-    assertTrue(
-        "Did not find MasterSpaceQuotaObserver in list of CPs: " + coprocessorNames,
-        coprocessorNames.contains(MasterSpaceQuotaObserver.class.getSimpleName()));
-  }
-
-  public boolean namespaceExists(String ns) throws IOException {
-    NamespaceDescriptor[] descs = TEST_UTIL.getAdmin().listNamespaceDescriptors();
-    for (NamespaceDescriptor desc : descs) {
-      if (ns.equals(desc.getName())) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public int getNumSpaceQuotas() throws Exception {
-    QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration());
-    int numSpaceQuotas = 0;
-    for (QuotaSettings quotaSettings : scanner) {
-      if (quotaSettings.getQuotaType() == QuotaType.SPACE) {
-        numSpaceQuotas++;
-      }
-    }
-    return numSpaceQuotas;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ab3ca01/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserverWithMocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserverWithMocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserverWithMocks.java
deleted file mode 100644
index 9dd9854..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserverWithMocks.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/**
- * 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.quotas;
-
-import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY;
-import static org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver.REMOVE_QUOTA_ON_TABLE_DELETE;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.mock;
-
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.security.access.AccessController;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test class for MasterSpaceQuotaObserver that does not require a cluster.
- */
-@Category(SmallTests.class)
-public class TestMasterSpaceQuotaObserverWithMocks {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestMasterSpaceQuotaObserverWithMocks.class);
-
-  private HMaster master;
-  private Configuration conf;
-
-  @Before
-  public void setup() {
-    conf = HBaseConfiguration.create();
-    master = mock(HMaster.class);
-    doCallRealMethod().when(master).updateConfigurationForSpaceQuotaObserver(
-        any());
-  }
-
-  @Test
-  public void testAddDefaultObserver() {
-    master.updateConfigurationForSpaceQuotaObserver(conf);
-    assertEquals(MasterSpaceQuotaObserver.class.getName(), conf.get(MASTER_COPROCESSOR_CONF_KEY));
-  }
-
-  @Test
-  public void testDoNotAddDefaultObserver() {
-    conf.setBoolean(REMOVE_QUOTA_ON_TABLE_DELETE, false);
-    master.updateConfigurationForSpaceQuotaObserver(conf);
-    // Configuration#getStrings returns null when unset
-    assertNull(conf.getStrings(MASTER_COPROCESSOR_CONF_KEY));
-  }
-
-  @Test
-  public void testAppendsObserver() {
-    conf.set(MASTER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
-    master.updateConfigurationForSpaceQuotaObserver(conf);
-    Set<String> coprocs = new HashSet<>(conf.getStringCollection(MASTER_COPROCESSOR_CONF_KEY));
-    assertEquals(2, coprocs.size());
-    assertTrue(
-        "Observed coprocessors were: " + coprocs,
-        coprocs.contains(AccessController.class.getName()));
-    assertTrue(
-        "Observed coprocessors were: " + coprocs,
-        coprocs.contains(MasterSpaceQuotaObserver.class.getName()));
-  }
-}