From ef3e8ec76a4cb00cbc9408989b450130448b268d Mon Sep 17 00:00:00 2001 From: Josh Elser Date: Mon, 21 Nov 2016 18:23:03 -0500 Subject: [PATCH] HBASE-17001 Enforce quota violation policies in the RegionServer The nuts-and-bolts of filesystem quotas. The Master must inform RegionServers of the violation of a quota by a table. The RegionServer must apply the violation policy as configured. Need to ensure that the proper interfaces exist to satisfy all necessary policies. --- .../apache/hadoop/hbase/quotas/QuotaTableUtil.java | 100 ++++++- .../org/apache/hadoop/hbase/master/HMaster.java | 29 ++- .../hbase/quotas/ActivePolicyEnforcement.java | 67 +++++ .../hadoop/hbase/quotas/MasterQuotaManager.java | 4 +- .../hadoop/hbase/quotas/QuotaObserverChore.java | 12 +- .../quotas/RegionServerSpaceQuotaManager.java | 158 ++++++++++- .../hbase/quotas/SpaceLimitingException.java | 90 +++++++ .../hbase/quotas/SpaceQuotaViolationNotifier.java | 20 +- .../quotas/SpaceQuotaViolationNotifierFactory.java | 63 +++++ .../SpaceQuotaViolationPolicyRefresherChore.java | 17 +- .../quotas/SpaceViolationPolicyEnforcement.java | 66 +++++ .../SpaceViolationPolicyEnforcementFactory.java | 63 +++++ .../quotas/TableSpaceQuotaViolationNotifier.java | 65 +++++ .../AbstractViolationPolicyEnforcement.java | 60 +++++ .../DisableTableViolationPolicyEnforcement.java | 78 ++++++ .../NoInsertsViolationPolicyEnforcement.java | 53 ++++ ...ritesCompactionsViolationPolicyEnforcement.java | 62 +++++ .../NoWritesViolationPolicyEnforcement.java | 52 ++++ .../policies/NoopViolationPolicyEnforcement.java | 61 +++++ .../hbase/regionserver/CompactSplitThread.java | 9 + .../hadoop/hbase/regionserver/HRegionServer.java | 12 + .../hadoop/hbase/regionserver/RSRpcServices.java | 76 ++++-- .../hadoop/hbase/MockRegionServerServices.java | 6 + .../hadoop/hbase/master/MockRegionServer.java | 6 + .../hbase/quotas/SpaceQuotaHelperForTests.java | 225 ++++++++++++++++ .../quotas/SpaceQuotaViolationNotifierForTest.java | 4 + .../hbase/quotas/TestActivePolicyEnforcement.java | 56 ++++ .../TestQuotaObserverChoreWithMiniCluster.java | 228 +++------------- .../hadoop/hbase/quotas/TestQuotaTableUtil.java | 49 ++++ .../quotas/TestRegionServerSpaceQuotaManager.java | 208 +++++++++++++++ ...estSpaceQuotaViolationPolicyRefresherChore.java | 46 +++- .../hadoop/hbase/quotas/TestSpaceQuotas.java | 289 +++++++++++++++++++++ .../TestTableSpaceQuotaViolationNotifier.java | 141 ++++++++++ .../policies/BaseViolationPolicyEnforcement.java | 31 +++ ...TestDisableTableViolationPolicyEnforcement.java | 56 ++++ .../TestNoInsertsViolationPolicyEnforcement.java | 54 ++++ ...ritesCompactionsViolationPolicyEnforcement.java | 55 ++++ .../TestNoWritesViolationPolicyEnforcement.java | 54 ++++ 38 files changed, 2468 insertions(+), 257 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitingException.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcement.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcementFactory.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/AbstractViolationPolicyEnforcement.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoInsertsViolationPolicyEnforcement.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesCompactionsViolationPolicyEnforcement.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesViolationPolicyEnforcement.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoopViolationPolicyEnforcement.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/BaseViolationPolicyEnforcement.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java index 0528b92..6d4a683 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java @@ -23,16 +23,20 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.regex.Pattern; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; @@ -43,7 +47,12 @@ import org.apache.hadoop.hbase.filter.QualifierFilter; import org.apache.hadoop.hbase.filter.RegexStringComparator; import org.apache.hadoop.hbase.filter.RowFilter; import org.apache.hadoop.hbase.protobuf.ProtobufMagic; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Strings; @@ -52,9 +61,8 @@ import org.apache.hadoop.hbase.util.Strings; *
  *     ROW-KEY      FAM/QUAL        DATA
  *   n.<namespace> q:s         <global-quotas>
- *   n.<namespace> u:du        <size in bytes>
  *   t.<table>     q:s         <global-quotas>
- *   t.<table>     u:du        <size in bytes>
+ *   t.<table>     u:v        <space violation policy>
  *   u.<user>      q:s         <global-quotas>
  *   u.<user>      q:s.<table> <table-quotas>
  *   u.<user>      q:s.<ns>:   <namespace-quotas>
@@ -73,7 +81,7 @@ public class QuotaTableUtil {
   protected static final byte[] QUOTA_FAMILY_USAGE = Bytes.toBytes("u");
   protected static final byte[] QUOTA_QUALIFIER_SETTINGS = Bytes.toBytes("s");
   protected static final byte[] QUOTA_QUALIFIER_SETTINGS_PREFIX = Bytes.toBytes("s.");
-  protected static final byte[] QUOTA_QUALIFIER_DISKUSAGE = Bytes.toBytes("du");
+  protected static final byte[] QUOTA_QUALIFIER_VIOLATION = Bytes.toBytes("v");
   protected static final byte[] QUOTA_USER_ROW_KEY_PREFIX = Bytes.toBytes("u.");
   protected static final byte[] QUOTA_TABLE_ROW_KEY_PREFIX = Bytes.toBytes("t.");
   protected static final byte[] QUOTA_NAMESPACE_ROW_KEY_PREFIX = Bytes.toBytes("n.");
@@ -202,6 +210,49 @@ public class QuotaTableUtil {
     return filterList;
   }
 
+  /**
+   * Creates a {@link Scan} which returns only quota violations from the quota table.
+   */
+  public static Scan makeQuotaViolationScan() {
+    Scan s = new Scan();
+    // Limit to "u:v" column
+    s.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
+    // Limit rowspace to the "t:" prefix
+    s.setRowPrefixFilter(QUOTA_TABLE_ROW_KEY_PREFIX);
+    return s;
+  }
+
+  /**
+   * Extracts the {@link SpaceViolationPolicy} and {@link TableName} from the provided
+   * {@link Result} and adds them to the given {@link Map}. If the result does not contain
+   * the expected information or the serialized policy in the value is invalid, this method
+   * will throw an {@link IllegalArgumentException}.
+   *
+   * @param result A row from the quota table.
+   * @param policies A map of policies to add the result of this method into.
+   */
+  public static void extractViolationPolicy(
+      Result result, Map policies) {
+    byte[] row = Objects.requireNonNull(result).getRow();
+    if (null == row) {
+      throw new IllegalArgumentException("Provided result had a null row");
+    }
+    final TableName targetTableName = getTableFromRowKey(row);
+    Cell c = result.getColumnLatestCell(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
+    if (null == c) {
+      throw new IllegalArgumentException("Result did not contain the expected column.");
+    }
+    ByteString buffer = UnsafeByteOperations.unsafeWrap(
+        c.getValueArray(), c.getValueOffset(), c.getValueLength());
+    try {
+      SpaceQuota quota = SpaceQuota.parseFrom(buffer);
+      policies.put(targetTableName, getViolationPolicy(quota));
+    } catch (InvalidProtocolBufferException e) {
+      throw new IllegalArgumentException(
+          "Result did not contain a valid SpaceQuota protocol buffer message", e);
+    }
+  }
+
   public static interface UserQuotasVisitor {
     void visitUserQuotas(final String userName, final Quotas quotas)
       throws IOException;
@@ -297,6 +348,34 @@ public class QuotaTableUtil {
     }
   }
 
+  /**
+   * Creates a {@link Scan} which only returns violation policy records in the quota table.
+   */
+  public static Scan getScanForViolations() {
+    Scan s = new Scan();
+    s.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
+    return s;
+  }
+
+  /**
+   * Creates a {@link Put} to enable the given policy on the table.
+   */
+  public static Put enableViolationPolicy(TableName tableName, SpaceViolationPolicy policy) {
+    Put p = new Put(getTableRowKey(tableName));
+    SpaceQuota quota = getProtoViolationPolicy(policy);
+    p.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION, quota.toByteArray());
+    return p;
+  }
+
+  /**
+   * Creates a {@link Delete} to remove a policy on the given table.
+   */
+  public static Delete removeViolationPolicy(TableName tableName) {
+    Delete d = new Delete(getTableRowKey(tableName));
+    d.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
+    return d;
+  }
+
   /* =========================================================================
    *  Quotas protobuf helpers
    */
@@ -418,4 +497,17 @@ public class QuotaTableUtil {
   protected static String getUserFromRowKey(final byte[] key) {
     return Bytes.toString(key, QUOTA_USER_ROW_KEY_PREFIX.length);
   }
-}
+
+  protected static SpaceQuota getProtoViolationPolicy(SpaceViolationPolicy policy) {
+    return SpaceQuota.newBuilder()
+          .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(policy))
+          .build();
+  }
+
+  protected static SpaceViolationPolicy getViolationPolicy(SpaceQuota proto) {
+    if (!proto.hasViolationPolicy()) {
+      throw new IllegalArgumentException("Protobuf SpaceQuota does not have violation policy.");
+    }
+    return ProtobufUtil.toViolationPolicy(proto.getViolationPolicy());
+  }
+}
\ No newline at end of file
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 e4ffa6e..027e4ee 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
@@ -125,8 +125,9 @@ 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.QuotaObserverChore;
+import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifier;
-import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifierForTest;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifierFactory;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -137,10 +138,13 @@ import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolic
 import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy;
 import org.apache.hadoop.hbase.replication.master.TableCFsUpdater;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -914,7 +918,10 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   SpaceQuotaViolationNotifier createQuotaViolationNotifier() {
-    return new SpaceQuotaViolationNotifierForTest();
+    SpaceQuotaViolationNotifier notifier =
+        SpaceQuotaViolationNotifierFactory.getInstance().create(getConfiguration());
+    notifier.initialize(getClusterConnection());
+    return notifier;
   }
 
   boolean isCatalogJanitorEnabled() {
@@ -1991,6 +1998,24 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (cpHost != null) {
       cpHost.preEnableTable(tableName);
     }
+    // Normally, it would make sense for this authorization check to exist inside AccessController,
+    // but because the authorization check is done based on internal state (rather than explicit
+    // permissions) we'll do the check here instead of in the coprocessor.
+    MasterQuotaManager quotaManager = getMasterQuotaManager();
+    if (null != quotaManager) {
+      if (quotaManager.isQuotaEnabled()) {
+        Quotas quotaForTable = QuotaUtil.getTableQuota(getConnection(), tableName);
+        if (null != quotaForTable && quotaForTable.hasSpace()) {
+          SpaceViolationPolicy policy = quotaForTable.getSpace().getViolationPolicy();
+          if (SpaceViolationPolicy.DISABLE == policy) {
+            throw new AccessDeniedException("Enabling the table '" + tableName
+                + "' is disallowed due to a violated space quota.");
+          }
+        }
+      } else if (LOG.isTraceEnabled()) {
+        LOG.trace("Unable to check for space quotas as the manager is not enabled");
+      }
+    }
     LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
 
     // Execute the operation asynchronously - client will check the progress of the operation
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
new file mode 100644
index 0000000..59bd1d0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
@@ -0,0 +1,67 @@
+/*
+ * 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.util.Map;
+import java.util.Objects;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.quotas.policies.NoopViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.regionserver.Region;
+
+/**
+ * A class to ease dealing with tables that have and do not have violation policies
+ * being enforced in a uniform manner. Immutable.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ActivePolicyEnforcement {
+  private final Map activePolicies;
+
+  public ActivePolicyEnforcement(Map activePolicies) {
+    this.activePolicies = activePolicies;
+  }
+
+  /**
+   * Returns the proper {@link SpaceViolationPolicyEnforcement} implementation for the given table.
+   * If the given table does not have a violation policy enforced, a "no-op" policy will
+   * be returned which always allows an action.
+   *
+   * @see #getPolicyEnforcement(TableName)
+   */
+  public SpaceViolationPolicyEnforcement getPolicyEnforcement(Region r) {
+    return getPolicyEnforcement(Objects.requireNonNull(r).getTableDesc().getTableName());
+  }
+
+  /**
+   * Returns the proper {@link SpaceViolationPolicyEnforcement} implementation for the given table.
+   * If the given table does not have a violation policy enforced, a "no-op" policy will
+   * be returned which always allows an action.
+   *
+   * @param tableName The table to fetch the policy for.
+   * @return A non-null {@link SpaceViolationPolicyEnforcement} instance.
+   */
+  public SpaceViolationPolicyEnforcement getPolicyEnforcement(TableName tableName) {
+    SpaceViolationPolicyEnforcement policy = activePolicies.get(Objects.requireNonNull(tableName));
+    if (null == policy) {
+      return NoopViolationPolicyEnforcement.getInstance();
+    }
+    return policy;
+  }
+}
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 cd81270..ddac6e9 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
@@ -522,13 +522,13 @@ public class MasterQuotaManager implements RegionStateListener {
   }
 
   public void addRegionSize(HRegionInfo hri, long size) {
-    // TODO Make proper API
+    // TODO Make proper API?
     // TODO Prevent from growing indefinitely
     regionSizes.put(hri, size);
   }
 
   public Map snapshotRegionSizes() {
-    // TODO Make proper API
+    // TODO Make proper API?
     return new HashMap<>(regionSizes);
   }
 }
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 2b65554..3e90c59 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
@@ -103,6 +103,9 @@ public class QuotaObserverChore extends ScheduledChore {
   @Override
   protected void chore() {
     try {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Refreshing space quotas in RegionServer");
+      }
       _chore();
     } catch (IOException e) {
       LOG.warn("Failed to process quota reports and update quota violation state. Will retry.", e);
@@ -309,14 +312,15 @@ public class QuotaObserverChore extends ScheduledChore {
   /**
    * Transitions the given table to violation of its quota, enabling the violation policy.
    */
-  private void transitionTableToViolation(TableName table, SpaceViolationPolicy violationPolicy) {
+  private void transitionTableToViolation(TableName table, SpaceViolationPolicy violationPolicy)
+      throws IOException {
     this.violationNotifier.transitionTableToViolation(table, violationPolicy);
   }
 
   /**
    * Transitions the given table to observance of its quota, disabling the violation policy.
    */
-  private void transitionTableToObservance(TableName table) {
+  private void transitionTableToObservance(TableName table) throws IOException {
     this.violationNotifier.transitionTableToObservance(table);
   }
 
@@ -520,12 +524,12 @@ public class QuotaObserverChore extends ScheduledChore {
         if (ratioReported < percentRegionsReportedThreshold) {
           if (LOG.isTraceEnabled()) {
             LOG.trace("Filtering " + table + " because " + reportedRegionsInQuota  + " of " +
-                numRegionsInTable + " were reported.");
+                numRegionsInTable + " regions were reported.");
           }
           tablesToRemove.add(table);
         } else if (LOG.isTraceEnabled()) {
           LOG.trace("Retaining " + table + " because " + reportedRegionsInQuota  + " of " +
-              numRegionsInTable + " were reported.");
+              numRegionsInTable + " regions were reported.");
         }
       }
       for (TableName tableToRemove : tablesToRemove) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
index 76661e6..2c8124c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
@@ -17,15 +17,25 @@
 package org.apache.hadoop.hbase.quotas;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.Map.Entry;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.annotations.VisibleForTesting;
 
 /**
  * A manager for filesystem space quotas in the RegionServer.
@@ -41,9 +51,19 @@ public class RegionServerSpaceQuotaManager {
   private final RegionServerServices rsServices;
 
   private SpaceQuotaViolationPolicyRefresherChore spaceQuotaRefresher;
+  private ConcurrentHashMap enforcedPolicies;
+  private SpaceViolationPolicyEnforcementFactory factory;
 
   public RegionServerSpaceQuotaManager(RegionServerServices rsServices) {
+    this(rsServices, SpaceViolationPolicyEnforcementFactory.getInstance());
+  }
+
+  @VisibleForTesting
+  RegionServerSpaceQuotaManager(
+      RegionServerServices rsServices, SpaceViolationPolicyEnforcementFactory factory) {
     this.rsServices = Objects.requireNonNull(rsServices);
+    this.factory = factory;
+    this.enforcedPolicies = new ConcurrentHashMap<>();
   }
 
   public synchronized void start() throws IOException {
@@ -51,8 +71,12 @@ public class RegionServerSpaceQuotaManager {
       LOG.info("Quota support disabled");
       return;
     }
-
-    spaceQuotaRefresher = new SpaceQuotaViolationPolicyRefresherChore(this);
+    if (null != spaceQuotaRefresher) {
+      LOG.warn("SpaceQuotaViolationPolicyRefresherChore has already been started!");
+      return;
+    }
+    this.spaceQuotaRefresher = new SpaceQuotaViolationPolicyRefresherChore(this);
+    rsServices.getChoreService().scheduleChore(spaceQuotaRefresher);
   }
 
   public synchronized void stop() {
@@ -63,36 +87,148 @@ public class RegionServerSpaceQuotaManager {
   }
 
   /**
-   * Returns the collection of tables which have quota violation policies enforced.
+   * Creates an object well-suited for the RegionServer to use in verifying active policies.
+   */
+  public ActivePolicyEnforcement getActiveEnforcements() {
+    return new ActivePolicyEnforcement(copyActiveEnforcements());
+  }
+
+  /**
+   * Converts a map of table to {@link SpaceViolationPolicyEnforcement}s into
+   * {@link SpaceViolationPolicy}s.
    */
-  public Map getActivePolicyEnforcements() {
-    return null;
+  public Map getActivePoliciesAsMap() {
+    final Map enforcements =
+        copyActiveEnforcements();
+    final Map policies = new HashMap<>();
+    for (Entry entry : enforcements.entrySet()) {
+      final SpaceViolationPolicy policy = entry.getValue().getPolicy();
+      if (null != policy) {
+        policies.put(entry.getKey(), policy);
+      }
+    }
+    return policies;
   }
 
   /**
    * Reads all quota violation policies which are to be enforced from the quota table.
+   *
    * @return The collection of tables which are in violation of their quota and the policy which
    *    should be enforced.
    */
-  Map getEnforcedViolationPolicies() throws IOException {
-    return null;
+  public Map getViolationPoliciesToEnforce() throws IOException {
+    try (Table quotaTable = getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME);
+        ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaViolationScan())) {
+      Map activePolicies = new HashMap<>();
+      for (Result result : scanner) {
+        try {
+          extractViolationPolicy(result, activePolicies);
+        } catch (IllegalArgumentException e) {
+          final String msg = "Failed to parse result for row " + Bytes.toString(result.getRow());
+          LOG.error(msg, e);
+          throw new IOException(msg, e);
+        }
+      }
+      return activePolicies;
+    }
   }
 
   /**
    * Enforces the given violationPolicy on the given table in this RegionServer.
    */
-  void enforceViolationPolicy(TableName tableName, SpaceViolationPolicy violationPolicy) {
-    throw new RuntimeException();
+  public void enforceViolationPolicy(TableName tableName, SpaceViolationPolicy violationPolicy) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(
+          "Enabling violation policy enforcement on " + tableName
+          + " with policy " + violationPolicy);
+    }
+    // Construct this outside of the lock
+    final SpaceViolationPolicyEnforcement enforcement = getFactory().create(
+        getRegionServerServices(), tableName, violationPolicy);
+    // "Enables" the policy
+    // TODO Should this synchronize on the actual table name instead of the map? That would allow
+    // policy enable/disable on different tables to happen concurrently. As written now, only one
+    // table will be allowed to transition at a time.
+    synchronized (enforcedPolicies) {
+      try {
+        enforcement.enable();
+      } catch (IOException e) {
+        LOG.error("Failed to enable space violation policy for " + tableName
+            + ". This table will not enter violation.", e);
+        return;
+      }
+      enforcedPolicies.put(tableName, enforcement);
+    }
   }
 
   /**
    * Disables enforcement on any violation policy on the given tableName.
    */
-  void disableViolationPolicyEnforcement(TableName tableName) {
-    throw new RuntimeException();
+  public void disableViolationPolicyEnforcement(TableName tableName) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Disabling violation policy enforcement on " + tableName);
+    }
+    // "Disables" the policy
+    // TODO Should this synchronize on the actual table name instead of the map?
+    synchronized (enforcedPolicies) {
+      SpaceViolationPolicyEnforcement enforcement = enforcedPolicies.remove(tableName);
+      if (null != enforcement) {
+        try {
+          enforcement.disable();
+        } catch (IOException e) {
+          LOG.error("Failed to disable space violation policy for " + tableName
+              + ". This table will remain in violation.", e);
+          enforcedPolicies.put(tableName, enforcement);
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns whether or not compactions should be disabled for the given tableName per
+   * a space quota violation policy.
+   *
+   * @param tableName The table to check
+   * @return True if compactions should be disabled for the table, false otherwise.
+   */
+  public boolean areCompactionsDisabled(TableName tableName) {
+    SpaceViolationPolicyEnforcement enforcement = getEnforcement(tableName);
+    if (null != enforcement) {
+      return enforcement.areCompactionsDisabled();
+    }
+    return false;
+  }
+
+  /**
+   * Returns the collection of tables which have quota violation policies enforced on
+   * this RegionServer.
+   */
+  Map copyActiveEnforcements() {
+    // Allows reads to happen concurrently (or while the map is being updated)
+    return new HashMap<>(this.enforcedPolicies);
+  }
+
+  private SpaceViolationPolicyEnforcement getEnforcement(TableName tableName) {
+    // Allows reads to happen concurrently (or while the map is being updated)
+    return this.enforcedPolicies.get(Objects.requireNonNull(tableName));
+  }
+
+  /**
+   * Wrapper around {@link QuotaTableUtil#extractViolationPolicy(Result, Map)} for testing.
+   */
+  void extractViolationPolicy(Result result, Map activePolicies) {
+    QuotaTableUtil.extractViolationPolicy(result, activePolicies);
   }
 
   RegionServerServices getRegionServerServices() {
     return rsServices;
   }
+
+  Connection getConnection() {
+    return rsServices.getConnection();
+  }
+
+  SpaceViolationPolicyEnforcementFactory getFactory() {
+    return factory;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitingException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitingException.java
new file mode 100644
index 0000000..8073d4c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitingException.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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * An Exception that is thrown when a space quota is in violation.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class SpaceLimitingException extends QuotaExceededException {
+  private static final long serialVersionUID = 2319438922387583600L;
+  private static final Log LOG = LogFactory.getLog(SpaceLimitingException.class);
+  private static final String MESSAGE_PREFIX = SpaceLimitingException.class.getName() + ": ";
+
+  private final SpaceViolationPolicy policy;
+
+  public SpaceLimitingException(String msg) {
+    super(parseMessage(msg));
+
+    // Hack around ResponseConverter expecting to invoke a single-arg String constructor
+    // on this class
+    if (null != msg) {
+      for (SpaceViolationPolicy definedPolicy : SpaceViolationPolicy.values()) {
+        if (msg.indexOf(definedPolicy.name()) != -1) {
+          policy = definedPolicy;
+          return;
+        }
+      }
+    }
+    policy = null;
+  }
+
+  public SpaceLimitingException(SpaceViolationPolicy policy, String msg) {
+    super(msg);
+    this.policy = policy;
+  }
+
+  /**
+   * Returns the violation policy in effect.
+   *
+   * @return The violation policy in effect.
+   */
+  public SpaceViolationPolicy getViolationPolicy() {
+    return this.policy;
+  }
+
+  private static String parseMessage(String originalMessage) {
+    // Serialization of the exception places a duplicate class name. Try to strip that off if it
+    // exists. Best effort... Looks something like:
+    // "org.apache.hadoop.hbase.quotas.SpaceLimitingException: NO_INSERTS A Put is disallowed due
+    // to a space quota."
+    if (null != originalMessage && originalMessage.startsWith(MESSAGE_PREFIX)) {
+      // If it starts with the class name, rip off the policy too.
+      try {
+        int index = originalMessage.indexOf(' ', MESSAGE_PREFIX.length());
+        return originalMessage.substring(index + 1);
+      } catch (Exception e) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Failed to trim exception message", e);
+        }
+      }
+    }
+    return originalMessage;
+  }
+
+  @Override
+  public String getMessage() {
+    return (null == policy ? "(unknown policy)" : policy.name()) + " " + super.getMessage();
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
index 745bfc0..c7d237c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
@@ -16,27 +16,41 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+import java.io.IOException;
+
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
 
 /**
  * An interface which abstract away the action taken to enable or disable
- * a space quota violation policy across the HBase cluster.
+ * a space quota violation policy across the HBase cluster. Implementations
+ * must have a no-args constructor.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
 public interface SpaceQuotaViolationNotifier {
 
   /**
+   * Initializes the notifier.
+   */
+  void initialize(Connection conn);
+
+  /**
    * Instructs the cluster that the given table is in violation of a space quota. The
    * provided violation policy is the action which should be taken on the table.
    *
    * @param tableName The name of the table in violation of the quota.
    * @param violationPolicy The policy which should be enacted on the table.
    */
-  void transitionTableToViolation(TableName tableName, SpaceViolationPolicy violationPolicy);
+  void transitionTableToViolation(
+      TableName tableName, SpaceViolationPolicy violationPolicy) throws IOException;
 
   /**
    * Instructs the cluster that the given table is in observance of any applicable space quota.
    *
    * @param tableName The name of the table in observance.
    */
-  void transitionTableToObservance(TableName tableName);
+  void transitionTableToObservance(TableName tableName) throws IOException;
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java
new file mode 100644
index 0000000..522fb80
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java
@@ -0,0 +1,63 @@
+/*
+ * 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.util.Objects;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Factory for creating {@link SpaceQuotaViolationNotifier} implementations. Implementations
+ * must have a no-args constructor.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SpaceQuotaViolationNotifierFactory {
+  private static final SpaceQuotaViolationNotifierFactory INSTANCE =
+      new SpaceQuotaViolationNotifierFactory();
+
+  public static final String VIOLATION_NOTIFIER_KEY = "hbase.master.quota.violation.notifier.impl";
+  public static final Class VIOLATION_NOTIFIER_DEFAULT = TableSpaceQuotaViolationNotifier.class;
+
+  // Use getInstance()
+  private SpaceQuotaViolationNotifierFactory() {}
+
+  public static SpaceQuotaViolationNotifierFactory getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Instantiates the {@link SpaceQuotaViolationNotifier} implementation as defined in the
+   * configuration provided.
+   *
+   * @param conf Configuration object
+   * @return The SpaceQuotaViolationNotifier implementation
+   * @throws IllegalArgumentException if the class could not be instantiated
+   */
+  public SpaceQuotaViolationNotifier create(Configuration conf) {
+    Class clz = Objects.requireNonNull(conf)
+        .getClass(VIOLATION_NOTIFIER_KEY, VIOLATION_NOTIFIER_DEFAULT,
+            SpaceQuotaViolationNotifier.class);
+    try {
+      return clz.newInstance();
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new IllegalArgumentException("Failed to instantiate the implementation", e);
+    }
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java
index a95db09..9193f81 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java
@@ -65,17 +65,24 @@ public class SpaceQuotaViolationPolicyRefresherChore extends ScheduledChore {
   @Override
   protected void chore() {
     try {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Reading current quota violations from hbase:quota.");
+      }
       // Tables with a policy currently enforced
-      final Map locallyEnforcedViolationPolicies =
-          manager.getActivePolicyEnforcements();
+      final Map activeViolationPolicies =
+          manager.getActivePoliciesAsMap();
       // Tables with policies that should be enforced
       final Map violationPolicies =
-          manager.getEnforcedViolationPolicies();
+          manager.getViolationPoliciesToEnforce();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(activeViolationPolicies.size() + " policies are currently active, "
+            + "this server should have " + violationPolicies.size() + " active");
+      }
       // Ensure each policy which should be enacted is enacted.
       for (Entry entry : violationPolicies.entrySet()) {
         final TableName tableName = entry.getKey();
         final SpaceViolationPolicy policyToEnforce = entry.getValue();
-        final SpaceViolationPolicy currentPolicy = locallyEnforcedViolationPolicies.get(tableName);
+        final SpaceViolationPolicy currentPolicy = activeViolationPolicies.get(tableName);
         if (currentPolicy != policyToEnforce) {
           if (LOG.isTraceEnabled()) {
             LOG.trace("Enabling " + policyToEnforce + " on " + tableName);
@@ -84,7 +91,7 @@ public class SpaceQuotaViolationPolicyRefresherChore extends ScheduledChore {
         }
       }
       // Remove policies which should no longer be enforced
-      Iterator iter = locallyEnforcedViolationPolicies.keySet().iterator();
+      Iterator iter = activeViolationPolicies.keySet().iterator();
       while (iter.hasNext()) {
         final TableName localTableWithPolicy = iter.next();
         if (!violationPolicies.containsKey(localTableWithPolicy)) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcement.java
new file mode 100644
index 0000000..29d2f86
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcement.java
@@ -0,0 +1,66 @@
+/*
+ * 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 org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+
+/**
+ * RegionServer implementation of {@link SpaceViolationPolicy}.
+ *
+ * Implementations must have a public, no-args constructor.
+ */
+public interface SpaceViolationPolicyEnforcement {
+
+  /**
+   * Initializes this policy instance.
+   */
+  void initialize(RegionServerServices rss, TableName tableName);
+
+  /**
+   * Enables this policy. Not all policies have enable actions.
+   */
+  void enable() throws IOException;
+
+  /**
+   * Disables this policy. Not all policies have disable actions.
+   */
+  void disable() throws IOException;
+
+  /**
+   * Checks the given {@link Mutation} against this policy. If the
+   * {@link Mutation} violates the policy, this policy should throw a
+   * {@link SpaceLimitingException}.
+   *
+   * @throws SpaceLimitingException When the given mutation violates this policy.
+   */
+  void check(Mutation m) throws SpaceLimitingException;
+
+  /**
+   * Returns the {@link SpaceViolationPolicy} this enforcement is for.
+   */
+  SpaceViolationPolicy getPolicy();
+
+  /**
+   * Returns whether or not compactions on this table should be disabled for this policy.
+   */
+  boolean areCompactionsDisabled();
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcementFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcementFactory.java
new file mode 100644
index 0000000..e81ec40
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcementFactory.java
@@ -0,0 +1,63 @@
+/*
+ * 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 org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.DisableTableViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoInsertsViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesCompactionsViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+
+/**
+ * A factory class for instantiating {@link SpaceViolationPolicyEnforcement} instances.
+ */
+public class SpaceViolationPolicyEnforcementFactory {
+
+  private static final SpaceViolationPolicyEnforcementFactory INSTANCE =
+      new SpaceViolationPolicyEnforcementFactory();
+
+  private SpaceViolationPolicyEnforcementFactory() {}
+
+  public static SpaceViolationPolicyEnforcementFactory getInstance() {
+    return INSTANCE;
+  }
+
+  public SpaceViolationPolicyEnforcement create(
+      RegionServerServices rss, TableName tableName, SpaceViolationPolicy policy) {
+    SpaceViolationPolicyEnforcement enforcement;
+    switch (policy) {
+      case DISABLE:
+        enforcement = new DisableTableViolationPolicyEnforcement();
+        break;
+      case NO_WRITES_COMPACTIONS:
+        enforcement = new NoWritesCompactionsViolationPolicyEnforcement();
+        break;
+      case NO_WRITES:
+        enforcement = new NoWritesViolationPolicyEnforcement();
+        break;
+      case NO_INSERTS:
+        enforcement = new NoInsertsViolationPolicyEnforcement();
+        break;
+      default:
+        throw new IllegalArgumentException("Unhandled SpaceViolationPolicy: " + policy);
+    }
+    enforcement.initialize(rss, tableName);
+    return enforcement;
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java
new file mode 100644
index 0000000..501b0a7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java
@@ -0,0 +1,65 @@
+/*
+ * 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+
+/**
+ * A {@link SpaceQuotaViolationNotifier} which uses the hbase:quota table.
+ */
+public class TableSpaceQuotaViolationNotifier implements SpaceQuotaViolationNotifier {
+  private static final Log LOG = LogFactory.getLog(TableSpaceQuotaViolationNotifier.class);
+
+  private Connection conn;
+
+  @Override
+  public void transitionTableToViolation(
+      TableName tableName, SpaceViolationPolicy violationPolicy) throws IOException {
+    final Put p = QuotaTableUtil.enableViolationPolicy(tableName, violationPolicy);
+    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Persisting the enabling of a space quota violation policy " + violationPolicy
+            + " for " + tableName);
+      }
+      quotaTable.put(p);
+    }
+  }
+
+  @Override
+  public void transitionTableToObservance(TableName tableName) throws IOException {
+    final Delete d = QuotaTableUtil.removeViolationPolicy(tableName);
+    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Persisting the disabling of a space quota violation policy for " + tableName);
+      }
+      quotaTable.delete(d);
+    }
+  }
+
+  @Override
+  public void initialize(Connection conn) {
+    this.conn = conn;
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/AbstractViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/AbstractViolationPolicyEnforcement.java
new file mode 100644
index 0000000..5ba8451
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/AbstractViolationPolicyEnforcement.java
@@ -0,0 +1,60 @@
+/*
+ * 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.policies;
+
+import java.util.Objects;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+
+/**
+ * Abstract implementation for {@link SpaceViolationPolicyEnforcement}.
+ */
+public abstract class AbstractViolationPolicyEnforcement
+    implements SpaceViolationPolicyEnforcement {
+
+  RegionServerServices rss;
+  TableName tableName;
+
+  public void setRegionServerServices(RegionServerServices rss) {
+    this.rss = Objects.requireNonNull(rss);
+  }
+
+  public void setTableName(TableName tableName) {
+    this.tableName = tableName;
+  }
+
+  public RegionServerServices getRegionServerServices() {
+    return this.rss;
+  }
+
+  public TableName getTableName() {
+    return this.tableName;
+  }
+
+  @Override
+  public void initialize(RegionServerServices rss, TableName tableName) {
+    setRegionServerServices(rss);
+    setTableName(tableName);
+  }
+
+  @Override
+  public boolean areCompactionsDisabled() {
+    return false;
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java
new file mode 100644
index 0000000..fb8383c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java
@@ -0,0 +1,78 @@
+/*
+ * 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.policies;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.TableNotEnabledException;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+
+/**
+ * A {@link SpaceViolationPolicyEnforcement} which disables the table. The enforcement
+ * countepart to {@link SpaceViolationPolicy#DISABLE}.
+ */
+public class DisableTableViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+  private static final Log LOG = LogFactory.getLog(DisableTableViolationPolicyEnforcement.class);
+
+  @Override
+  public void enable() throws IOException {
+    try {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Starting disable of " + getTableName());
+      }
+      getRegionServerServices().getClusterConnection().getAdmin().disableTable(getTableName());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Disable is complete for " + getTableName());
+      }
+    } catch (TableNotEnabledException tnee) {
+      // The state we wanted it to be in.
+    }
+  }
+
+  @Override
+  public void disable() throws IOException {
+    try {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Starting enable of " + getTableName());
+      }
+      getRegionServerServices().getClusterConnection().getAdmin().enableTable(getTableName());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Enable is complete for " + getTableName());
+      }
+    } catch (TableNotDisabledException tnde) {
+      // The state we wanted it to be in
+    }
+  }
+
+  @Override
+  public void check(Mutation m) throws SpaceLimitingException {
+    // If this policy is enacted, then the table is (or should be) disabled.
+    throw new SpaceLimitingException(
+        getPolicy(), "This table is disabled due to violating a space quota.");
+  }
+
+  @Override
+  public SpaceViolationPolicy getPolicy() {
+    return SpaceViolationPolicy.DISABLE;
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoInsertsViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoInsertsViolationPolicyEnforcement.java
new file mode 100644
index 0000000..b3e3375
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoInsertsViolationPolicyEnforcement.java
@@ -0,0 +1,53 @@
+/*
+ * 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.policies;
+
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+
+/**
+ * A {@link SpaceViolationPolicyEnforcement} which disallows any inserts to the table. The
+ * enforcement counterpart to {@link SpaceViolationPolicy#NO_INSERTS}.
+ */
+public class NoInsertsViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+
+  @Override
+  public void enable() {}
+
+  @Override
+  public void disable() {}
+
+  @Override
+  public void check(Mutation m) throws SpaceLimitingException {
+    // Disallow all "new" data flowing into HBase, but allow Deletes (even though we know they will
+    // temporarily increase utilization).
+    if (m instanceof Append  || m instanceof Increment || m instanceof Put) {
+      throw new SpaceLimitingException(getPolicy(),
+          m.getClass().getSimpleName() + "s are disallowed due to a space quota.");
+    }
+  }
+
+  @Override
+  public SpaceViolationPolicy getPolicy() {
+    return SpaceViolationPolicy.NO_INSERTS;
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesCompactionsViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesCompactionsViolationPolicyEnforcement.java
new file mode 100644
index 0000000..eacc77a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesCompactionsViolationPolicyEnforcement.java
@@ -0,0 +1,62 @@
+/*
+ * 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.policies;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+
+/**
+ * A {@link SpaceViolationPolicyEnforcement} implementation which disables all updates and
+ * compactions. The enforcement counterpart to {@link SpaceViolationPolicy#NO_WRITES_COMPACTIONS}.
+ */
+public class NoWritesCompactionsViolationPolicyEnforcement
+    extends NoWritesViolationPolicyEnforcement {
+  private static final Log LOG = LogFactory.getLog(
+      NoWritesCompactionsViolationPolicyEnforcement.class);
+
+  private AtomicBoolean disableCompactions = new AtomicBoolean(false);
+
+  @Override
+  public synchronized void enable() {
+    boolean ret = disableCompactions.compareAndSet(false, true);
+    if (!ret && LOG.isTraceEnabled()) {
+      LOG.trace("Compactions were already disabled upon enabling the policy");
+    }
+  }
+
+  @Override
+  public synchronized void disable() {
+    boolean ret = disableCompactions.compareAndSet(true, false);
+    if (!ret && LOG.isTraceEnabled()) {
+      LOG.trace("Compactions were already enabled upon disabling the policy");
+    }
+  }
+
+  @Override
+  public SpaceViolationPolicy getPolicy() {
+    return SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
+  }
+
+  @Override
+  public boolean areCompactionsDisabled() {
+    return disableCompactions.get();
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesViolationPolicyEnforcement.java
new file mode 100644
index 0000000..e9650c2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesViolationPolicyEnforcement.java
@@ -0,0 +1,52 @@
+/*
+ * 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.policies;
+
+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.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+
+/**
+ * A {@link SpaceViolationPolicyEnforcement} implementation which disables all writes flowing
+ * into HBase. The enforcement counterpart to {@link SpaceViolationPolicy#NO_WRITES}.
+ */
+public class NoWritesViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+
+  @Override
+  public void enable() {}
+
+  @Override
+  public void disable() {}
+
+  @Override
+  public void check(Mutation m) throws SpaceLimitingException {
+    if (m instanceof Append || m instanceof Delete || m instanceof Increment || m instanceof Put) {
+      throw new SpaceLimitingException(getPolicy(),
+          m.getClass().getSimpleName() + "s are disallowed due to a space quota.");
+    }
+  }
+
+  @Override
+  public SpaceViolationPolicy getPolicy() {
+    return SpaceViolationPolicy.NO_WRITES;
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoopViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoopViolationPolicyEnforcement.java
new file mode 100644
index 0000000..0843fb2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoopViolationPolicyEnforcement.java
@@ -0,0 +1,61 @@
+/*
+ * 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.policies;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+
+/**
+ * A {@link SpaceViolationPolicyEnforcement} instance which does nothing. Useful for tables
+ * which have no violation policy.
+ */
+public class NoopViolationPolicyEnforcement implements SpaceViolationPolicyEnforcement {
+  private static final NoopViolationPolicyEnforcement INSTANCE =
+      new NoopViolationPolicyEnforcement();
+
+  private NoopViolationPolicyEnforcement() {}
+
+  public static NoopViolationPolicyEnforcement getInstance() {
+    return INSTANCE;
+  }
+
+  @Override
+  public void initialize(RegionServerServices rss, TableName tableName) {}
+
+  @Override
+  public void enable() {}
+
+  @Override
+  public void disable() {}
+
+  @Override
+  public void check(Mutation m) throws SpaceLimitingException {}
+
+  @Override
+  public SpaceViolationPolicy getPolicy() {
+    return null;
+  }
+
+  @Override
+  public boolean areCompactionsDisabled() {
+    return false;
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
index 1331b86..88ce293 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
@@ -351,6 +351,15 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
       if (compaction == null) return null; // message logged inside
     }
 
+    if (this.server.getRegionServerSpaceQuotaManager().areCompactionsDisabled(
+        r.getTableDesc().getTableName())) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Ignoring compaction request for " + r + " as an active space quota violation "
+            + " policy disallows compactions.");
+      }
+      return null;
+    }
+
     // We assume that most compactions are small. So, put system compactions into small
     // pool; we will do selection there, and move to large pool if necessary.
     ThreadPoolExecutor pool = (selectNow && s.throttleCompaction(compaction.getRequest().getSize()))
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 1c00b81..5ea2b74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -117,6 +117,7 @@ import org.apache.hadoop.hbase.mob.MobCacheConfig;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
 import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
@@ -465,6 +466,7 @@ public class HRegionServer extends HasThread implements
   private RegionServerProcedureManagerHost rspmHost;
 
   private RegionServerRpcQuotaManager rsQuotaManager;
+  private RegionServerSpaceQuotaManager rsSpaceQuotaManager;
 
   // Table level lock manager for locking for region operations
   protected TableLockManager tableLockManager;
@@ -916,6 +918,7 @@ public class HRegionServer extends HasThread implements
 
     // Setup the Quota Manager
     rsQuotaManager = new RegionServerRpcQuotaManager(this);
+    rsSpaceQuotaManager = new RegionServerSpaceQuotaManager(this);
 
     this.fsUtilizationChore = new FileSystemUtilizationChore(this);
 
@@ -989,6 +992,7 @@ public class HRegionServer extends HasThread implements
 
         // Start the Quota Manager
         rsQuotaManager.start(getRpcServer().getScheduler());
+        rsSpaceQuotaManager.start();
       }
 
       // We registered with the Master.  Go into run mode.
@@ -1080,6 +1084,9 @@ public class HRegionServer extends HasThread implements
     if (rsQuotaManager != null) {
       rsQuotaManager.stop();
     }
+    if (rsSpaceQuotaManager != null) {
+      rsSpaceQuotaManager.stop();
+    }
 
     // Stop the snapshot and other procedure handlers, forcefully killing all running tasks
     if (rspmHost != null) {
@@ -3655,4 +3662,9 @@ public class HRegionServer extends HasThread implements
   public SecureBulkLoadManager getSecureBulkLoadManager() {
     return this.secureBulkLoadManager;
   }
+
+  @Override
+  public RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager() {
+    return this.rsSpaceQuotaManager;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index d6711fe..4295a59 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -174,8 +174,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescr
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
+import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement;
 import org.apache.hadoop.hbase.quotas.OperationQuota;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.regionserver.Leases.Lease;
 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
@@ -561,8 +564,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    */
   private boolean checkAndRowMutate(final Region region, final List actions,
       final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, ByteArrayComparable comparator,
-                                    RegionActionResult.Builder builder) throws IOException {
+      CompareOp compareOp, ByteArrayComparable comparator, RegionActionResult.Builder builder,
+      ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
     if (!region.getRegionInfo().isMetaTable()) {
       regionServer.cacheFlusher.reclaimMemStoreMemory();
     }
@@ -581,10 +584,14 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       }
       switch (type) {
         case PUT:
-          rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner));
+          Put put = ProtobufUtil.toPut(action.getMutation(), cellScanner);
+          spaceQuotaEnforcement.getPolicyEnforcement(region).check(put);
+          rm.add(put);
           break;
         case DELETE:
-          rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
+          Delete del = ProtobufUtil.toDelete(action.getMutation(), cellScanner);
+          spaceQuotaEnforcement.getPolicyEnforcement(region).check(del);
+          rm.add(del);
           break;
         default:
           throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name());
@@ -611,10 +618,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * @throws IOException
    */
   private Result append(final Region region, final OperationQuota quota,
-      final MutationProto mutation, final CellScanner cellScanner, long nonceGroup)
+      final MutationProto mutation, final CellScanner cellScanner, long nonceGroup,
+      ActivePolicyEnforcement spaceQuota)
       throws IOException {
     long before = EnvironmentEdgeManager.currentTime();
     Append append = ProtobufUtil.toAppend(mutation, cellScanner);
+    spaceQuota.getPolicyEnforcement(region).check(append);
     quota.addMutation(append);
     Result r = null;
     if (region.getCoprocessorHost() != null) {
@@ -659,10 +668,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * @throws IOException
    */
   private Result increment(final Region region, final OperationQuota quota,
-      final MutationProto mutation, final CellScanner cells, long nonceGroup)
+      final MutationProto mutation, final CellScanner cells, long nonceGroup,
+      ActivePolicyEnforcement spaceQuota)
       throws IOException {
     long before = EnvironmentEdgeManager.currentTime();
     Increment increment = ProtobufUtil.toIncrement(mutation, cells);
+    spaceQuota.getPolicyEnforcement(region).check(increment);
     quota.addMutation(increment);
     Result r = null;
     if (region.getCoprocessorHost() != null) {
@@ -714,7 +725,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   private List doNonAtomicRegionMutation(final Region region,
       final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner,
       final RegionActionResult.Builder builder, List cellsToReturn, long nonceGroup,
-      final RegionScannersCloseCallBack closeCallBack, RpcCallContext context) {
+      final RegionScannersCloseCallBack closeCallBack, RpcCallContext context,
+      ActivePolicyEnforcement spaceQuotaEnforcement) {
     // Gather up CONTIGUOUS Puts and Deletes in this mutations List.  Idea is that rather than do
     // one at a time, we instead pass them in batch.  Be aware that the corresponding
     // ResultOrException instance that matches each Put or Delete is then added down in the
@@ -799,15 +811,19 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           if (type != MutationType.PUT && type != MutationType.DELETE && mutations != null &&
               !mutations.isEmpty()) {
             // Flush out any Puts or Deletes already collected.
-            doBatchOp(builder, region, quota, mutations, cellScanner);
+            doBatchOp(builder, region, quota, mutations, cellScanner, spaceQuotaEnforcement);
             mutations.clear();
           }
           switch (type) {
             case APPEND:
-              r = append(region, quota, action.getMutation(), cellScanner, nonceGroup);
+              r = append(
+                  region, quota, action.getMutation(), cellScanner, nonceGroup,
+                  spaceQuotaEnforcement);
               break;
             case INCREMENT:
-              r = increment(region, quota, action.getMutation(), cellScanner, nonceGroup);
+              r = increment(
+                  region, quota, action.getMutation(), cellScanner, nonceGroup,
+                  spaceQuotaEnforcement);
               break;
             case PUT:
             case DELETE:
@@ -856,7 +872,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     }
     // Finish up any outstanding mutations
     if (mutations != null && !mutations.isEmpty()) {
-      doBatchOp(builder, region, quota, mutations, cellScanner);
+      doBatchOp(builder, region, quota, mutations, cellScanner, spaceQuotaEnforcement);
     }
     return cellsToReturn;
   }
@@ -870,7 +886,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    */
   private void doBatchOp(final RegionActionResult.Builder builder, final Region region,
       final OperationQuota quota, final List mutations,
-      final CellScanner cells) {
+      final CellScanner cells, ActivePolicyEnforcement spaceQuotaEnforcement) {
     Mutation[] mArray = new Mutation[mutations.size()];
     long before = EnvironmentEdgeManager.currentTime();
     boolean batchContainsPuts = false, batchContainsDelete = false;
@@ -887,6 +903,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           batchContainsDelete = true;
         }
         mArray[i++] = mutation;
+        // Check if a space quota disallows this mutation
+        spaceQuotaEnforcement.getPolicyEnforcement(region).check(mutation);
         quota.addMutation(mutation);
       }
 
@@ -1250,10 +1268,14 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return regionServer.getConfiguration();
   }
 
-  private RegionServerRpcQuotaManager getQuotaManager() {
+  private RegionServerRpcQuotaManager getRpcQuotaManager() {
     return regionServer.getRegionServerRpcQuotaManager();
   }
 
+  private RegionServerSpaceQuotaManager getSpaceQuotaManager() {
+    return regionServer.getRegionServerSpaceQuotaManager();
+  }
+
   void start() {
     rpcServer.start();
   }
@@ -1421,6 +1443,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       checkOpen();
       requestCount.increment();
       Region region = getRegion(request.getRegion());
+      if (this.regionServer.getRegionServerSpaceQuotaManager().areCompactionsDisabled(
+          region.getTableDesc().getTableName())) {
+        throw new DoNotRetryIOException("Compactions on this region are "
+            + "disabled due to a space quota violation.");
+      }
       region.startRegionOperation(Operation.COMPACT_REGION);
       LOG.info("Compacting " + region.getRegionInfo().getRegionNameAsString());
       boolean major = false;
@@ -2259,7 +2286,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       Boolean existence = null;
       Result r = null;
       RpcCallContext context = RpcServer.getCurrentCall();
-      quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.GET);
+      quota = getRpcQuotaManager().checkQuota(region, OperationQuota.OperationType.GET);
 
       Get clientGet = ProtobufUtil.toGet(get);
       if (get.getExistenceOnly() && region.getCoprocessorHost() != null) {
@@ -2397,6 +2424,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     this.rpcMultiRequestCount.increment();
     Map regionStats = new HashMap<>(request
       .getRegionActionCount());
+    ActivePolicyEnforcement spaceQuotaEnforcement = getSpaceQuotaManager().getActiveEnforcements();
     for (RegionAction regionAction : request.getRegionActionList()) {
       this.requestCount.add(regionAction.getActionCount());
       OperationQuota quota;
@@ -2405,7 +2433,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       RegionSpecifier regionSpecifier = regionAction.getRegion();
       try {
         region = getRegion(regionSpecifier);
-        quota = getQuotaManager().checkQuota(region, regionAction.getActionList());
+        quota = getRpcQuotaManager().checkQuota(region, regionAction.getActionList());
       } catch (IOException e) {
         rpcServer.getMetrics().exception(e);
         regionActionResultBuilder.setException(ResponseConverter.buildException(e));
@@ -2433,7 +2461,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                 ProtobufUtil.toComparator(condition.getComparator());
             processed = checkAndRowMutate(region, regionAction.getActionList(),
                   cellScanner, row, family, qualifier, compareOp,
-                  comparator, regionActionResultBuilder);
+                  comparator, regionActionResultBuilder, spaceQuotaEnforcement);
           } else {
             mutateRows(region, regionAction.getActionList(), cellScanner,
                 regionActionResultBuilder);
@@ -2454,7 +2482,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           context.setCallBack(closeCallBack);
         }
         cellsToReturn = doNonAtomicRegionMutation(region, quota, regionAction, cellScanner,
-            regionActionResultBuilder, cellsToReturn, nonceGroup, closeCallBack, context);
+            regionActionResultBuilder, cellsToReturn, nonceGroup, closeCallBack, context,
+            spaceQuotaEnforcement);
       }
       responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
       quota.close();
@@ -2520,6 +2549,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     HBaseRpcController controller = (HBaseRpcController)rpcc;
     CellScanner cellScanner = controller != null ? controller.cellScanner() : null;
     OperationQuota quota = null;
+    ActivePolicyEnforcement spaceQuotaEnforcement = null;
     // Clear scanner so we are not holding on to reference across call.
     if (controller != null) {
       controller.setCellScanner(null);
@@ -2539,19 +2569,22 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       Boolean processed = null;
       MutationType type = mutation.getMutateType();
 
-      quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.MUTATE);
+      quota = getRpcQuotaManager().checkQuota(region, OperationQuota.OperationType.MUTATE);
+      spaceQuotaEnforcement = getSpaceQuotaManager().getActiveEnforcements();
 
       switch (type) {
       case APPEND:
         // TODO: this doesn't actually check anything.
-        r = append(region, quota, mutation, cellScanner, nonceGroup);
+        r = append(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement);
         break;
       case INCREMENT:
         // TODO: this doesn't actually check anything.
-        r = increment(region, quota, mutation, cellScanner, nonceGroup);
+        r = increment(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement);
         break;
       case PUT:
         Put put = ProtobufUtil.toPut(mutation, cellScanner);
+        // Throws an exception when violated
+        spaceQuotaEnforcement.getPolicyEnforcement(region).check(put);
         quota.addMutation(put);
         if (request.hasCondition()) {
           Condition condition = request.getCondition();
@@ -2581,6 +2614,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         break;
       case DELETE:
         Delete delete = ProtobufUtil.toDelete(mutation, cellScanner);
+        spaceQuotaEnforcement.getPolicyEnforcement(region).check(delete);
         quota.addMutation(delete);
         if (request.hasCondition()) {
           Condition condition = request.getCondition();
@@ -2754,7 +2788,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       RpcCallContext context = RpcServer.getCurrentCall();
       Object lastBlock = null;
 
-      quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.SCAN);
+      quota = getRpcQuotaManager().checkQuota(region, OperationQuota.OperationType.SCAN);
       long maxQuotaResultSize = Math.min(maxScannerResultSize, quota.getReadAvailable());
 
       if (rows > 0) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index 115e660..1807679 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.master.TableLockManager.NullTableLockManager;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
 import org.apache.hadoop.hbase.regionserver.FlushRequester;
 import org.apache.hadoop.hbase.regionserver.HeapMemoryManager;
@@ -356,4 +357,9 @@ public class MockRegionServerServices implements RegionServerServices {
   public SecureBulkLoadManager getSecureBulkLoadManager() {
     return null;
   }
+
+  @Override
+  public RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager() {
+    return null;
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 9b310f0..191e334 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -103,6 +103,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanReques
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
 import org.apache.hadoop.hbase.regionserver.FlushRequester;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -726,4 +727,9 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   public SecureBulkLoadManager getSecureBulkLoadManager() {
     return null;
   }
+
+  @Override
+  public RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager() {
+    return null;
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
new file mode 100644
index 0000000..9432617
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
@@ -0,0 +1,225 @@
+/*
+ * 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.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+
+public class SpaceQuotaHelperForTests {
+  private static final Log LOG = LogFactory.getLog(SpaceQuotaHelperForTests.class);
+
+  public static final int SIZE_PER_VALUE = 256;
+  public static final String F1 = "f1";
+  public static final long ONE_MEGABYTE = 1024L * 1024L;
+
+  private final HBaseTestingUtility testUtil;
+  private final TestName testName;
+  private final AtomicLong counter;
+
+  public SpaceQuotaHelperForTests(
+      HBaseTestingUtility testUtil, TestName testName, AtomicLong counter) {
+    this.testUtil = Objects.requireNonNull(testUtil);
+    this.testName = Objects.requireNonNull(testName);
+    this.counter = Objects.requireNonNull(counter);
+  }
+
+  //
+  // Helpers
+  //
+
+  void writeData(TableName tn, long sizeInBytes) throws IOException {
+    final Connection conn = testUtil.getConnection();
+    final Table table = conn.getTable(tn);
+    try {
+      List updates = new ArrayList<>();
+      long bytesToWrite = sizeInBytes;
+      long rowKeyId = 0L;
+      final StringBuilder sb = new StringBuilder();
+      final Random r = new Random();
+      while (bytesToWrite > 0L) {
+        sb.setLength(0);
+        sb.append(Long.toString(rowKeyId));
+        // Use the reverse counter as the rowKey to get even spread across all regions
+        Put p = new Put(Bytes.toBytes(sb.reverse().toString()));
+        byte[] value = new byte[SIZE_PER_VALUE];
+        r.nextBytes(value);
+        p.addColumn(Bytes.toBytes(F1), Bytes.toBytes("q1"), value);
+        updates.add(p);
+
+        // Batch 50K worth of updates
+        if (updates.size() > 50) {
+          table.put(updates);
+          updates.clear();
+        }
+
+        // Just count the value size, ignore the size of rowkey + column
+        bytesToWrite -= SIZE_PER_VALUE;
+        rowKeyId++;
+      }
+
+      // Write the final batch
+      if (!updates.isEmpty()) {
+        table.put(updates);
+      }
+
+      LOG.debug("Data was written to HBase");
+      // Push the data to disk.
+      testUtil.getAdmin().flush(tn);
+      LOG.debug("Data flushed to disk");
+    } finally {
+      table.close();
+    }
+  }
+
+  Multimap createTablesWithSpaceQuotas() throws Exception {
+    final Admin admin = testUtil.getAdmin();
+    final Multimap tablesWithQuotas = HashMultimap.create();
+
+    final TableName tn1 = createTable();
+    final TableName tn2 = createTable();
+
+    NamespaceDescriptor nd = NamespaceDescriptor.create("ns" + counter.getAndIncrement()).build();
+    admin.createNamespace(nd);
+    final TableName tn3 = createTableInNamespace(nd);
+    final TableName tn4 = createTableInNamespace(nd);
+    final TableName tn5 = createTableInNamespace(nd);
+
+    final long sizeLimit1 = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
+    final SpaceViolationPolicy violationPolicy1 = SpaceViolationPolicy.NO_WRITES;
+    QuotaSettings qs1 = QuotaSettingsFactory.limitTableSpace(tn1, sizeLimit1, violationPolicy1);
+    tablesWithQuotas.put(tn1, qs1);
+    admin.setQuota(qs1);
+
+    final long sizeLimit2 = 1024L * 1024L * 1024L * 200L; // 200GB
+    final SpaceViolationPolicy violationPolicy2 = SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
+    QuotaSettings qs2 = QuotaSettingsFactory.limitTableSpace(tn2, sizeLimit2, violationPolicy2);
+    tablesWithQuotas.put(tn2, qs2);
+    admin.setQuota(qs2);
+
+    final long sizeLimit3 = 1024L * 1024L * 1024L * 1024L * 100L; // 100TB
+    final SpaceViolationPolicy violationPolicy3 = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings qs3 = QuotaSettingsFactory.limitNamespaceSpace(
+        nd.getName(), sizeLimit3, violationPolicy3);
+    tablesWithQuotas.put(tn3, qs3);
+    tablesWithQuotas.put(tn4, qs3);
+    tablesWithQuotas.put(tn5, qs3);
+    admin.setQuota(qs3);
+
+    final long sizeLimit4 = 1024L * 1024L * 1024L * 5L; // 5GB
+    final SpaceViolationPolicy violationPolicy4 = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings qs4 = QuotaSettingsFactory.limitTableSpace(tn5, sizeLimit4, violationPolicy4);
+    // Override the ns quota for tn5, import edge-case to catch table quota taking
+    // precedence over ns quota.
+    tablesWithQuotas.put(tn5, qs4);
+    admin.setQuota(qs4);
+
+    return tablesWithQuotas;
+  }
+
+  TableName createTable() throws Exception {
+    return createTableWithRegions(1);
+  }
+
+  TableName createTableWithRegions(int numRegions) throws Exception {
+    return createTableWithRegions(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR, numRegions);
+  }
+
+  TableName createTableWithRegions(String namespace, int numRegions) throws Exception {
+    final Admin admin = testUtil.getAdmin();
+    final TableName tn = TableName.valueOf(
+        namespace, testName.getMethodName() + counter.getAndIncrement());
+
+    // Delete the old table
+    if (admin.tableExists(tn)) {
+      admin.disableTable(tn);
+      admin.deleteTable(tn);
+    }
+
+    // Create the table
+    HTableDescriptor tableDesc = new HTableDescriptor(tn);
+    tableDesc.addFamily(new HColumnDescriptor(F1));
+    if (numRegions == 1) {
+      admin.createTable(tableDesc);
+    } else {
+      admin.createTable(tableDesc, Bytes.toBytes("0"), Bytes.toBytes("9"), numRegions);
+    }
+    return tn;
+  }
+
+  TableName createTableInNamespace(NamespaceDescriptor nd) throws Exception {
+    final Admin admin = testUtil.getAdmin();
+    final TableName tn = TableName.valueOf(nd.getName(),
+        testName.getMethodName() + counter.getAndIncrement());
+
+    // Delete the old table
+    if (admin.tableExists(tn)) {
+      admin.disableTable(tn);
+      admin.deleteTable(tn);
+    }
+
+    // Create the table
+    HTableDescriptor tableDesc = new HTableDescriptor(tn);
+    tableDesc.addFamily(new HColumnDescriptor(F1));
+
+    admin.createTable(tableDesc);
+    return tn;
+  }
+
+  void partitionTablesByQuotaTarget(Multimap quotas,
+      Set tablesWithTableQuota, Set tablesWithNamespaceQuota) {
+    // Partition the tables with quotas by table and ns quota
+    for (Entry entry : quotas.entries()) {
+      SpaceLimitSettings settings = (SpaceLimitSettings) entry.getValue();
+      TableName tn = entry.getKey();
+      if (null != settings.getTableName()) {
+        tablesWithTableQuota.add(tn);
+      }
+      if (null != settings.getNamespace()) {
+        tablesWithNamespaceQuota.add(tn);
+      }
+
+      if (null == settings.getTableName() && null == settings.getNamespace()) {
+        fail("Unexpected table name with null tableName and namespace: " + tn);
+      }
+    }
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
index 7d1cb56..618fb07 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
@@ -20,6 +20,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 
 /**
  * A SpaceQuotaViolationNotifier implementation for verifying testing.
@@ -29,6 +30,9 @@ public class SpaceQuotaViolationNotifierForTest implements SpaceQuotaViolationNo
   private final Map tablesInViolation = new HashMap<>();
 
   @Override
+  public void initialize(Connection conn) {}
+
+  @Override
   public void transitionTableToViolation(TableName tableName, SpaceViolationPolicy violationPolicy) {
     tablesInViolation.put(tableName, violationPolicy);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java
new file mode 100644
index 0000000..4f37381
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java
@@ -0,0 +1,56 @@
+/*
+ * 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.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoopViolationPolicyEnforcement;
+import org.junit.Test;
+
+/**
+ * Test class for {@link ActivePolicyEnforcement}.
+ */
+public class TestActivePolicyEnforcement {
+
+  @Test
+  public void testGetter() {
+    final TableName tableName = TableName.valueOf("table");
+    Map map = new HashMap<>();
+    map.put(tableName, new NoWritesViolationPolicyEnforcement());
+    ActivePolicyEnforcement ape = new ActivePolicyEnforcement(map);
+    assertEquals(map.get(tableName), ape.getPolicyEnforcement(tableName));
+  }
+
+  @Test
+  public void testNoPolicyReturnsNoopEnforcement() {
+    ActivePolicyEnforcement ape = new ActivePolicyEnforcement(Collections.emptyMap());
+    SpaceViolationPolicyEnforcement enforcement = ape.getPolicyEnforcement(
+        TableName.valueOf("nonexistent"));
+    assertNotNull(enforcement);
+    assertTrue(
+        "Expected an instance of NoopViolationPolicyEnforcement",
+        enforcement instanceof NoopViolationPolicyEnforcement);
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java
index 8638677..e8fdf80 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java
@@ -22,16 +22,12 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -40,20 +36,15 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 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.NamespaceNotFoundException;
 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.client.Put;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.quotas.QuotaObserverChore.TablesWithQuotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -62,7 +53,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
 
@@ -72,11 +62,8 @@ import com.google.common.collect.Multimap;
 @Category(LargeTests.class)
 public class TestQuotaObserverChoreWithMiniCluster {
   private static final Log LOG = LogFactory.getLog(TestQuotaObserverChoreWithMiniCluster.class);
-  private static final int SIZE_PER_VALUE = 256;
-  private static final String F1 = "f1";
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final AtomicLong COUNTER = new AtomicLong(0);
-  private static final long ONE_MEGABYTE = 1024L * 1024L;
 
   @Rule
   public TestName testName = new TestName();
@@ -84,6 +71,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
   private HMaster master;
   private QuotaObserverChore chore;
   private SpaceQuotaViolationNotifierForTest violationNotifier;
+  private SpaceQuotaHelperForTests helper;
 
   @BeforeClass
   public static void setUp() throws Exception {
@@ -93,6 +81,8 @@ public class TestQuotaObserverChoreWithMiniCluster {
     conf.setInt(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_DELAY_KEY, 1000);
     conf.setInt(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_PERIOD_KEY, 1000);
     conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    conf.setClass(SpaceQuotaViolationNotifierFactory.VIOLATION_NOTIFIER_KEY,
+        SpaceQuotaViolationNotifierForTest.class, SpaceQuotaViolationNotifier.class);
     TEST_UTIL.startMiniCluster(1);
   }
 
@@ -132,19 +122,20 @@ public class TestQuotaObserverChoreWithMiniCluster {
         (SpaceQuotaViolationNotifierForTest) master.getSpaceQuotaViolationNotifier();
     violationNotifier.clearTableViolations();
     chore = master.getQuotaObserverChore();
+    helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
   }
 
   @Test
   public void testTableViolatesQuota() throws Exception {
-    TableName tn = createTableWithRegions(10);
+    TableName tn = helper.createTableWithRegions(10);
 
-    final long sizeLimit = 2L * ONE_MEGABYTE;
+    final long sizeLimit = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
     final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_INSERTS;
     QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, violationPolicy);
     TEST_UTIL.getAdmin().setQuota(settings);
 
     // Write more data than should be allowed
-    writeData(tn, 3L * ONE_MEGABYTE);
+    helper.writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
 
     Map violatedQuotas = violationNotifier.snapshotTablesInViolation();
     while (violatedQuotas.isEmpty()) {
@@ -176,16 +167,16 @@ public class TestQuotaObserverChoreWithMiniCluster {
       admin.createNamespace(desc);
     }
 
-    TableName tn1 = createTableWithRegions(namespace, 5);
-    TableName tn2 = createTableWithRegions(namespace, 5);
-    TableName tn3 = createTableWithRegions(namespace, 5);
+    TableName tn1 = helper.createTableWithRegions(namespace, 5);
+    TableName tn2 = helper.createTableWithRegions(namespace, 5);
+    TableName tn3 = helper.createTableWithRegions(namespace, 5);
 
-    final long sizeLimit = 5L * ONE_MEGABYTE;
+    final long sizeLimit = 5L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
     final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.DISABLE;
     QuotaSettings settings = QuotaSettingsFactory.limitNamespaceSpace(namespace, sizeLimit, violationPolicy);
     admin.setQuota(settings);
 
-    writeData(tn1, 2L * ONE_MEGABYTE);
+    helper.writeData(tn1, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn1);
     Map violatedQuotas = violationNotifier.snapshotTablesInViolation();
     for (int i = 0; i < 5; i++) {
@@ -199,7 +190,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
       violatedQuotas = violationNotifier.snapshotTablesInViolation();
     }
 
-    writeData(tn2, 2L * ONE_MEGABYTE);
+    helper.writeData(tn2, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn2);
     violatedQuotas = violationNotifier.snapshotTablesInViolation();
     for (int i = 0; i < 5; i++) {
@@ -216,7 +207,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
     // Writing the final 2MB of data will push the namespace over the 5MB limit (6MB in total)
     // and should push all three tables in the namespace into violation.
-    writeData(tn3, 2L * ONE_MEGABYTE);
+    helper.writeData(tn3, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn3);
     violatedQuotas = violationNotifier.snapshotTablesInViolation();
     while (violatedQuotas.size() < 3) {
@@ -255,16 +246,16 @@ public class TestQuotaObserverChoreWithMiniCluster {
       admin.createNamespace(desc);
     }
 
-    TableName tn1 = createTableWithRegions(namespace, 5);
-    TableName tn2 = createTableWithRegions(namespace, 5);
+    TableName tn1 = helper.createTableWithRegions(namespace, 5);
+    TableName tn2 = helper.createTableWithRegions(namespace, 5);
 
-    final long namespaceSizeLimit = 3L * ONE_MEGABYTE;
+    final long namespaceSizeLimit = 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
     final SpaceViolationPolicy namespaceViolationPolicy = SpaceViolationPolicy.DISABLE;
     QuotaSettings namespaceSettings = QuotaSettingsFactory.limitNamespaceSpace(namespace,
         namespaceSizeLimit, namespaceViolationPolicy);
     admin.setQuota(namespaceSettings);
 
-    writeData(tn1, 2L * ONE_MEGABYTE);
+    helper.writeData(tn1, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn1);
     Map violatedQuotas = violationNotifier.snapshotTablesInViolation();
     for (int i = 0; i < 5; i++) {
@@ -279,7 +270,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
       violatedQuotas = violationNotifier.snapshotTablesInViolation();
     }
 
-    writeData(tn2, 2L * ONE_MEGABYTE);
+    helper.writeData(tn2, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn2);
     violatedQuotas = violationNotifier.snapshotTablesInViolation();
     while (violatedQuotas.size() < 2) {
@@ -302,7 +293,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
     assertEquals(namespaceViolationPolicy, actualPolicyTN2);
 
     // Override the namespace quota with a table quota
-    final long tableSizeLimit = ONE_MEGABYTE;
+    final long tableSizeLimit = SpaceQuotaHelperForTests.ONE_MEGABYTE;
     final SpaceViolationPolicy tableViolationPolicy = SpaceViolationPolicy.NO_INSERTS;
     QuotaSettings tableSettings = QuotaSettingsFactory.limitTableSpace(tn1, tableSizeLimit,
         tableViolationPolicy);
@@ -335,11 +326,11 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
   @Test
   public void testGetAllTablesWithQuotas() throws Exception {
-    final Multimap quotas = createTablesWithSpaceQuotas();
+    final Multimap quotas = helper.createTablesWithSpaceQuotas();
     Set tablesWithQuotas = new HashSet<>();
     Set namespaceTablesWithQuotas = new HashSet<>();
     // Partition the tables with quotas by table and ns quota
-    partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
+    helper.partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
 
     TablesWithQuotas tables = chore.fetchAllTablesWithQuotasDefined();
     assertEquals("Found tables: " + tables, tablesWithQuotas, tables.getTableQuotaTables());
@@ -348,13 +339,13 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
   @Test
   public void testRpcQuotaTablesAreFiltered() throws Exception {
-    final Multimap quotas = createTablesWithSpaceQuotas();
+    final Multimap quotas = helper.createTablesWithSpaceQuotas();
     Set tablesWithQuotas = new HashSet<>();
     Set namespaceTablesWithQuotas = new HashSet<>();
     // Partition the tables with quotas by table and ns quota
-    partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
+    helper.partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
 
-    TableName rpcQuotaTable = createTable();
+    TableName rpcQuotaTable = helper.createTable();
     TEST_UTIL.getAdmin().setQuota(QuotaSettingsFactory
       .throttleTable(rpcQuotaTable, ThrottleType.READ_NUMBER, 6, TimeUnit.MINUTES));
 
@@ -382,9 +373,9 @@ public class TestQuotaObserverChoreWithMiniCluster {
     };
 
     // Create the tables
-    TableName tn1 = createTableWithRegions(20);
-    TableName tn2 = createTableWithRegions(20);
-    TableName tn3 = createTableWithRegions(20);
+    TableName tn1 = helper.createTableWithRegions(20);
+    TableName tn2 = helper.createTableWithRegions(20);
+    TableName tn3 = helper.createTableWithRegions(20);
 
     // Add them to the Tables with Quotas object
     tables.addTableQuotaTable(tn1);
@@ -404,7 +395,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
   @Test
   public void testFetchSpaceQuota() throws Exception {
-    Multimap tables = createTablesWithSpaceQuotas();
+    Multimap tables = helper.createTablesWithSpaceQuotas();
     // Can pass in an empty map, we're not consulting it.
     chore.initializeViolationStores(Collections.emptyMap());
     // All tables that were created should have a quota defined.
@@ -430,166 +421,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
       assertEquals(sls.getProto().getQuota(), spaceQuota);
     }
 
-    TableName tableWithoutQuota = createTable();
+    TableName tableWithoutQuota = helper.createTable();
     assertNull(chore.getTableViolationStore().getSpaceQuota(tableWithoutQuota));
   }
-
-  //
-  // Helpers
-  //
-
-  void writeData(TableName tn, long sizeInBytes) throws IOException {
-    final Connection conn = TEST_UTIL.getConnection();
-    final Table table = conn.getTable(tn);
-    try {
-      List updates = new ArrayList<>();
-      long bytesToWrite = sizeInBytes;
-      long rowKeyId = 0L;
-      final StringBuilder sb = new StringBuilder();
-      final Random r = new Random();
-      while (bytesToWrite > 0L) {
-        sb.setLength(0);
-        sb.append(Long.toString(rowKeyId));
-        // Use the reverse counter as the rowKey to get even spread across all regions
-        Put p = new Put(Bytes.toBytes(sb.reverse().toString()));
-        byte[] value = new byte[SIZE_PER_VALUE];
-        r.nextBytes(value);
-        p.addColumn(Bytes.toBytes(F1), Bytes.toBytes("q1"), value);
-        updates.add(p);
-
-        // Batch 50K worth of updates
-        if (updates.size() > 50) {
-          table.put(updates);
-          updates.clear();
-        }
-
-        // Just count the value size, ignore the size of rowkey + column
-        bytesToWrite -= SIZE_PER_VALUE;
-        rowKeyId++;
-      }
-
-      // Write the final batch
-      if (!updates.isEmpty()) {
-        table.put(updates);
-      }
-
-      LOG.debug("Data was written to HBase");
-      // Push the data to disk.
-      TEST_UTIL.getAdmin().flush(tn);
-      LOG.debug("Data flushed to disk");
-    } finally {
-      table.close();
-    }
-  }
-
-  Multimap createTablesWithSpaceQuotas() throws Exception {
-    final Admin admin = TEST_UTIL.getAdmin();
-    final Multimap tablesWithQuotas = HashMultimap.create();
-
-    final TableName tn1 = createTable();
-    final TableName tn2 = createTable();
-
-    NamespaceDescriptor nd = NamespaceDescriptor.create("ns" + COUNTER.getAndIncrement()).build();
-    admin.createNamespace(nd);
-    final TableName tn3 = createTableInNamespace(nd);
-    final TableName tn4 = createTableInNamespace(nd);
-    final TableName tn5 = createTableInNamespace(nd);
-
-    final long sizeLimit1 = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
-    final SpaceViolationPolicy violationPolicy1 = SpaceViolationPolicy.NO_WRITES;
-    QuotaSettings qs1 = QuotaSettingsFactory.limitTableSpace(tn1, sizeLimit1, violationPolicy1);
-    tablesWithQuotas.put(tn1, qs1);
-    admin.setQuota(qs1);
-
-    final long sizeLimit2 = 1024L * 1024L * 1024L * 200L; // 200GB
-    final SpaceViolationPolicy violationPolicy2 = SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
-    QuotaSettings qs2 = QuotaSettingsFactory.limitTableSpace(tn2, sizeLimit2, violationPolicy2);
-    tablesWithQuotas.put(tn2, qs2);
-    admin.setQuota(qs2);
-
-    final long sizeLimit3 = 1024L * 1024L * 1024L * 1024L * 100L; // 100TB
-    final SpaceViolationPolicy violationPolicy3 = SpaceViolationPolicy.NO_INSERTS;
-    QuotaSettings qs3 = QuotaSettingsFactory.limitNamespaceSpace(nd.getName(), sizeLimit3, violationPolicy3);
-    tablesWithQuotas.put(tn3, qs3);
-    tablesWithQuotas.put(tn4, qs3);
-    tablesWithQuotas.put(tn5, qs3);
-    admin.setQuota(qs3);
-
-    final long sizeLimit4 = 1024L * 1024L * 1024L * 5L; // 5GB
-    final SpaceViolationPolicy violationPolicy4 = SpaceViolationPolicy.NO_INSERTS;
-    QuotaSettings qs4 = QuotaSettingsFactory.limitTableSpace(tn5, sizeLimit4, violationPolicy4);
-    // Override the ns quota for tn5, import edge-case to catch table quota taking
-    // precedence over ns quota.
-    tablesWithQuotas.put(tn5, qs4);
-    admin.setQuota(qs4);
-
-    return tablesWithQuotas;
-  }
-
-  TableName createTable() throws Exception {
-    return createTableWithRegions(1);
-  }
-
-  TableName createTableWithRegions(int numRegions) throws Exception {
-    return createTableWithRegions(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR, numRegions);
-  }
-
-  TableName createTableWithRegions(String namespace, int numRegions) throws Exception {
-    final Admin admin = TEST_UTIL.getAdmin();
-    final TableName tn = TableName.valueOf(namespace, testName.getMethodName() + COUNTER.getAndIncrement());
-
-    // Delete the old table
-    if (admin.tableExists(tn)) {
-      admin.disableTable(tn);
-      admin.deleteTable(tn);
-    }
-
-    // Create the table
-    HTableDescriptor tableDesc = new HTableDescriptor(tn);
-    tableDesc.addFamily(new HColumnDescriptor(F1));
-    if (numRegions == 1) {
-      admin.createTable(tableDesc);
-    } else {
-      admin.createTable(tableDesc, Bytes.toBytes("0"), Bytes.toBytes("9"), numRegions);
-    }
-    return tn;
-  }
-
-  TableName createTableInNamespace(NamespaceDescriptor nd) throws Exception {
-    final Admin admin = TEST_UTIL.getAdmin();
-    final TableName tn = TableName.valueOf(nd.getName(),
-        testName.getMethodName() + COUNTER.getAndIncrement());
-
-    // Delete the old table
-    if (admin.tableExists(tn)) {
-      admin.disableTable(tn);
-      admin.deleteTable(tn);
-    }
-
-    // Create the table
-    HTableDescriptor tableDesc = new HTableDescriptor(tn);
-    tableDesc.addFamily(new HColumnDescriptor(F1));
-
-    admin.createTable(tableDesc);
-    return tn;
-  }
-
-  void partitionTablesByQuotaTarget(Multimap quotas,
-      Set tablesWithTableQuota, Set tablesWithNamespaceQuota) {
-    // Partition the tables with quotas by table and ns quota
-    for (Entry entry : quotas.entries()) {
-      SpaceLimitSettings settings = (SpaceLimitSettings) entry.getValue();
-      TableName tn = entry.getKey();
-      if (null != settings.getTableName()) {
-        tablesWithTableQuota.add(tn);
-      }
-      if (null != settings.getNamespace()) {
-        tablesWithNamespaceQuota.add(tn);
-      }
-
-      if (null == settings.getTableName() && null == settings.getNamespace()) {
-        fail("Unexpected table name with null tableName and namespace: " + tn);
-      }
-    }
-  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
index 5306be9..f897ad4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
@@ -21,6 +21,10 @@ package org.apache.hadoop.hbase.quotas;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -28,6 +32,10 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
@@ -37,8 +45,10 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 
 /**
  * Test the quota table helpers (e.g. CRUD operations)
@@ -48,6 +58,10 @@ public class TestQuotaTableUtil {
 
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private Connection connection;
+  private int tableNameCounter;
+
+  @Rule
+  public TestName testName = new TestName();
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -70,6 +84,7 @@ public class TestQuotaTableUtil {
   @Before
   public void before() throws IOException {
     this.connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+    this.tableNameCounter = 0;
   }
 
   @After
@@ -179,4 +194,38 @@ public class TestQuotaTableUtil {
     resQuotaNS = QuotaUtil.getUserQuota(this.connection, user, namespace);
     assertEquals(null, resQuotaNS);
   }
+
+  @Test
+  public void testSerDeViolationPolicies() throws Exception {
+    final TableName tn1 = getUniqueTableName();
+    final SpaceViolationPolicy policy1 = SpaceViolationPolicy.DISABLE;
+    final TableName tn2 = getUniqueTableName();
+    final SpaceViolationPolicy policy2 = SpaceViolationPolicy.NO_INSERTS;
+    final TableName tn3 = getUniqueTableName();
+    final SpaceViolationPolicy policy3 = SpaceViolationPolicy.NO_WRITES;
+    List puts = new ArrayList<>();
+    puts.add(QuotaTableUtil.enableViolationPolicy(tn1, policy1));
+    puts.add(QuotaTableUtil.enableViolationPolicy(tn2, policy2));
+    puts.add(QuotaTableUtil.enableViolationPolicy(tn3, policy3));
+    final Map expectedPolicies = new HashMap<>();
+    expectedPolicies.put(tn1, policy1);
+    expectedPolicies.put(tn2, policy2);
+    expectedPolicies.put(tn3, policy3);
+
+    final Map actualPolicies = new HashMap<>();
+    try (Table quotaTable = connection.getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
+      quotaTable.put(puts);
+      ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaViolationScan());
+      for (Result r : scanner) {
+        QuotaTableUtil.extractViolationPolicy(r, actualPolicies);
+      }
+      scanner.close();
+    }
+
+    assertEquals(expectedPolicies, actualPolicies);
+  }
+
+  private TableName getUniqueTableName() {
+    return TableName.valueOf(testName.getMethodName() + "_" + tableNameCounter++);
+  }
 }
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java
new file mode 100644
index 0000000..6fa5270
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java
@@ -0,0 +1,208 @@
+/*
+ * 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.util.Bytes.toBytes;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.quotas.policies.DisableTableViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoInsertsViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesCompactionsViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoopViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Test class for {@link RegionServerSpaceQuotaManager}.
+ */
+public class TestRegionServerSpaceQuotaManager {
+
+  private RegionServerSpaceQuotaManager quotaManager;
+  private Connection conn;
+  private Table quotaTable;
+  private ResultScanner scanner;
+
+  @Before
+  @SuppressWarnings("unchecked")
+  public void setup() throws Exception {
+    quotaManager = mock(RegionServerSpaceQuotaManager.class);
+    conn = mock(Connection.class);
+    quotaTable = mock(Table.class);
+    scanner = mock(ResultScanner.class);
+    // Call the real getActivePolicyEnforcements()
+    when(quotaManager.getViolationPoliciesToEnforce()).thenCallRealMethod();
+    // Mock out creating a scanner
+    when(quotaManager.getConnection()).thenReturn(conn);
+    when(conn.getTable(QuotaUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
+    when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
+    // Mock out the static method call with some indirection
+    doAnswer(new Answer(){
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        Result result = invocation.getArgumentAt(0, Result.class);
+        Map policies = invocation.getArgumentAt(1, Map.class);
+        QuotaTableUtil.extractViolationPolicy(result, policies);
+        return null;
+      }
+    }).when(quotaManager).extractViolationPolicy(any(Result.class), any(Map.class));
+  }
+
+  @Test
+  public void testMissingAllColumns() {
+    List results = new ArrayList<>();
+    results.add(Result.create(Collections.emptyList()));
+    when(scanner.iterator()).thenReturn(results.iterator());
+    try {
+      quotaManager.getViolationPoliciesToEnforce();
+      fail("Expected an IOException, but did not receive one.");
+    } catch (IOException e) {
+      // Expected an error because we had no cells in the row.
+      // This should only happen due to programmer error.
+    }
+  }
+
+  @Test
+  public void testMissingDesiredColumn() {
+    List results = new ArrayList<>();
+    // Give a column that isn't the one we want
+    Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("q"), toBytes("s"), new byte[0]);
+    results.add(Result.create(Collections.singletonList(c)));
+    when(scanner.iterator()).thenReturn(results.iterator());
+    try {
+      quotaManager.getViolationPoliciesToEnforce();
+      fail("Expected an IOException, but did not receive one.");
+    } catch (IOException e) {
+      // Expected an error because we were missing the column we expected in this row.
+      // This should only happen due to programmer error.
+    }
+  }
+
+  @Test
+  public void testParsingError() {
+    List results = new ArrayList<>();
+    Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("u"), toBytes("v"), new byte[0]);
+    results.add(Result.create(Collections.singletonList(c)));
+    when(scanner.iterator()).thenReturn(results.iterator());
+    try {
+      quotaManager.getViolationPoliciesToEnforce();
+      fail("Expected an IOException, but did not receive one.");
+    } catch (IOException e) {
+      // We provided a garbage serialized protobuf message (empty byte array), this should
+      // in turn throw an IOException
+    }
+  }
+
+  @Test
+  public void testSpacePoliciesFromEnforcements() {
+    final Map enforcements = new HashMap<>();
+    final Map expectedPolicies = new HashMap<>();
+    when(quotaManager.copyActiveEnforcements()).thenReturn(enforcements);
+    when(quotaManager.getActivePoliciesAsMap()).thenCallRealMethod();
+
+    enforcements.put(TableName.valueOf("no_inserts"), new NoInsertsViolationPolicyEnforcement());
+    expectedPolicies.put(TableName.valueOf("no_inserts"), SpaceViolationPolicy.NO_INSERTS);
+
+    enforcements.put(TableName.valueOf("no_writes"), new NoWritesViolationPolicyEnforcement());
+    expectedPolicies.put(TableName.valueOf("no_writes"), SpaceViolationPolicy.NO_WRITES);
+
+    enforcements.put(TableName.valueOf("no_writes_compactions"),
+        new NoWritesCompactionsViolationPolicyEnforcement());
+    expectedPolicies.put(TableName.valueOf("no_writes_compactions"),
+        SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
+
+    enforcements.put(TableName.valueOf("disable"), new DisableTableViolationPolicyEnforcement());
+    expectedPolicies.put(TableName.valueOf("disable"), SpaceViolationPolicy.DISABLE);
+
+    enforcements.put(TableName.valueOf("no_policy"), NoopViolationPolicyEnforcement.getInstance());
+
+    Map actualPolicies = quotaManager.getActivePoliciesAsMap();
+    assertEquals(expectedPolicies, actualPolicies);
+  }
+
+  @Test
+  public void testExceptionOnPolicyEnforcementEnable() throws Exception {
+    final TableName tableName = TableName.valueOf("foo");
+    final SpaceViolationPolicy policy = SpaceViolationPolicy.DISABLE;
+    RegionServerServices rss = mock(RegionServerServices.class);
+    SpaceViolationPolicyEnforcementFactory factory = mock(
+        SpaceViolationPolicyEnforcementFactory.class);
+    SpaceViolationPolicyEnforcement enforcement = mock(SpaceViolationPolicyEnforcement.class);
+    RegionServerSpaceQuotaManager realManager = new RegionServerSpaceQuotaManager(rss, factory);
+
+    when(factory.create(rss, tableName, policy)).thenReturn(enforcement);
+    doThrow(new IOException("Failed for test!")).when(enforcement).enable();
+
+    realManager.enforceViolationPolicy(tableName, policy);
+    Map enforcements =
+        realManager.copyActiveEnforcements();
+    assertTrue("Expected active enforcements to be empty, but were " + enforcements,
+        enforcements.isEmpty());
+  }
+
+  @Test
+  public void testExceptionOnPolicyEnforcementDisable() throws Exception {
+    final TableName tableName = TableName.valueOf("foo");
+    final SpaceViolationPolicy policy = SpaceViolationPolicy.DISABLE;
+    RegionServerServices rss = mock(RegionServerServices.class);
+    SpaceViolationPolicyEnforcementFactory factory = mock(
+        SpaceViolationPolicyEnforcementFactory.class);
+    SpaceViolationPolicyEnforcement enforcement = mock(SpaceViolationPolicyEnforcement.class);
+    RegionServerSpaceQuotaManager realManager = new RegionServerSpaceQuotaManager(rss, factory);
+
+    when(factory.create(rss, tableName, policy)).thenReturn(enforcement);
+    doNothing().when(enforcement).enable();
+    doThrow(new IOException("Failed for test!")).when(enforcement).disable();
+
+    // Enabling should work
+    realManager.enforceViolationPolicy(tableName, policy);
+    Map enforcements =
+        realManager.copyActiveEnforcements();
+    assertEquals(1, enforcements.size());
+
+    // If the disable fails, we should still treat it as "active"
+    realManager.disableViolationPolicyEnforcement(tableName);
+    enforcements = realManager.copyActiveEnforcements();
+    assertEquals(1, enforcements.size());
+  }
+}
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 038e55c..43db522 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
@@ -16,6 +16,7 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.verify;
@@ -30,6 +31,11 @@ import java.util.Map.Entry;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.quotas.policies.DisableTableViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoInsertsViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesCompactionsViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoopViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.junit.Before;
 import org.junit.Test;
@@ -63,9 +69,9 @@ public class TestSpaceQuotaViolationPolicyRefresherChore {
     policiesToEnforce.put(TableName.valueOf("table4"), SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
 
     // No active enforcements
-    when(manager.getActivePolicyEnforcements()).thenReturn(Collections.emptyMap());
+    when(manager.copyActiveEnforcements()).thenReturn(Collections.emptyMap());
     // Policies to enforce
-    when(manager.getEnforcedViolationPolicies()).thenReturn(policiesToEnforce);
+    when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
 
     chore.chore();
 
@@ -88,9 +94,9 @@ public class TestSpaceQuotaViolationPolicyRefresherChore {
     previousPolicies.put(TableName.valueOf("table4"), SpaceViolationPolicy.NO_WRITES);
 
     // No active enforcements
-    when(manager.getActivePolicyEnforcements()).thenReturn(previousPolicies);
+    when(manager.getActivePoliciesAsMap()).thenReturn(previousPolicies);
     // Policies to enforce
-    when(manager.getEnforcedViolationPolicies()).thenReturn(policiesToEnforce);
+    when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
 
     chore.chore();
 
@@ -114,9 +120,9 @@ public class TestSpaceQuotaViolationPolicyRefresherChore {
     previousPolicies.put(TableName.valueOf("table1"), SpaceViolationPolicy.NO_WRITES);
 
     // No active enforcements
-    when(manager.getActivePolicyEnforcements()).thenReturn(previousPolicies);
+    when(manager.getActivePoliciesAsMap()).thenReturn(previousPolicies);
     // Policies to enforce
-    when(manager.getEnforcedViolationPolicies()).thenReturn(policiesToEnforce);
+    when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
 
     chore.chore();
 
@@ -125,5 +131,31 @@ public class TestSpaceQuotaViolationPolicyRefresherChore {
     }
     verify(manager, never()).disableViolationPolicyEnforcement(TableName.valueOf("table1"));
   }
-  
+
+  @Test
+  public void testFilterNullPoliciesFromEnforcements() {
+    final Map enforcements = new HashMap<>();
+    final Map expectedPolicies = new HashMap<>();
+    when(manager.copyActiveEnforcements()).thenReturn(enforcements);
+    when(manager.getActivePoliciesAsMap()).thenCallRealMethod();
+
+    enforcements.put(TableName.valueOf("no_inserts"), new NoInsertsViolationPolicyEnforcement());
+    expectedPolicies.put(TableName.valueOf("no_inserts"), SpaceViolationPolicy.NO_INSERTS);
+
+    enforcements.put(TableName.valueOf("no_writes"), new NoWritesViolationPolicyEnforcement());
+    expectedPolicies.put(TableName.valueOf("no_writes"), SpaceViolationPolicy.NO_WRITES);
+
+    enforcements.put(TableName.valueOf("no_writes_compactions"),
+        new NoWritesCompactionsViolationPolicyEnforcement());
+    expectedPolicies.put(TableName.valueOf("no_writes_compactions"),
+        SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
+
+    enforcements.put(TableName.valueOf("disable"), new DisableTableViolationPolicyEnforcement());
+    expectedPolicies.put(TableName.valueOf("disable"), SpaceViolationPolicy.DISABLE);
+
+    enforcements.put(TableName.valueOf("no_policy"), NoopViolationPolicyEnforcement.getInstance());
+
+    Map actualPolicies = manager.getActivePoliciesAsMap();
+    assertEquals(expectedPolicies, actualPolicies);
+  }
 }
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
new file mode 100644
index 0000000..db09ba6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
@@ -0,0 +1,289 @@
+/*
+ * 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.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+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.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * End-to-end test class for filesystem space quotas.
+ */
+@Category(LargeTests.class)
+public class TestSpaceQuotas {
+  private static final Log LOG = LogFactory.getLog(TestSpaceQuotas.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final AtomicLong COUNTER = new AtomicLong(0);
+  private static final int NUM_RETRIES = 10;
+
+  @Rule
+  public TestName testName = new TestName();
+  private SpaceQuotaHelperForTests helper;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    // Increase the frequency of some of the chores for responsiveness of the test
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_KEY, 1000);
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 1000);
+    conf.setInt(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_DELAY_KEY, 1000);
+    conf.setInt(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_PERIOD_KEY, 1000);
+    conf.setInt(SpaceQuotaViolationPolicyRefresherChore.POLICY_REFRESHER_CHORE_DELAY_KEY, 1000);
+    conf.setInt(SpaceQuotaViolationPolicyRefresherChore.POLICY_REFRESHER_CHORE_PERIOD_KEY, 1000);
+    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 {
+    final Connection conn = TEST_UTIL.getConnection();
+    // Wait for the quota table to be created
+    if (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME)) {
+      do {
+        LOG.debug("Quota table does not yet exist");
+        Thread.sleep(1000);
+      } while (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME));
+    } else {
+      // Or, clean up any quotas from previous test runs.
+      QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration());
+      for (QuotaSettings quotaSettings : scanner) {
+        final String namespace = quotaSettings.getNamespace();
+        final TableName tableName = quotaSettings.getTableName();
+        if (null != namespace) {
+          LOG.debug("Deleting quota for namespace: " + namespace);
+          QuotaUtil.deleteNamespaceQuota(conn, namespace);
+        } else {
+          assert null != tableName;
+          LOG.debug("Deleting quota for table: "+ tableName);
+          QuotaUtil.deleteTableQuota(conn, tableName);
+        }
+      }
+    }
+    helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
+  }
+
+  @Test
+  public void testNoInsertsWithPut() throws Exception {
+    Put p = new Put(Bytes.toBytes("to_reject"));
+    p.addColumn(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, p);
+  }
+
+  @Test
+  public void testNoInsertsWithAppend() throws Exception {
+    Append a = new Append(Bytes.toBytes("to_reject"));
+    a.add(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, a);
+  }
+
+  @Test
+  public void testNoInsertsWithIncrement() throws Exception {
+    Increment i = new Increment(Bytes.toBytes("to_reject"));
+    i.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("count"), 0);
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, i);
+  }
+
+  @Test
+  public void testDeletesAfterNoInserts() throws Exception {
+    final TableName tn = writeUntilViolation(SpaceViolationPolicy.NO_INSERTS);
+    // Try a couple of times to verify that the quota never gets enforced, same as we
+    // do when we're trying to catch the failure.
+    Delete d = new Delete(Bytes.toBytes("should_not_be_rejected"));
+    for (int i = 0; i < NUM_RETRIES; i++) {
+      try (Table t = TEST_UTIL.getConnection().getTable(tn)) {
+        t.delete(d);
+      }
+    }
+  }
+
+  @Test
+  public void testNoWritesWithPut() throws Exception {
+    Put p = new Put(Bytes.toBytes("to_reject"));
+    p.addColumn(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, p);
+  }
+
+  @Test
+  public void testNoWritesWithAppend() throws Exception {
+    Append a = new Append(Bytes.toBytes("to_reject"));
+    a.add(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, a);
+  }
+
+  @Test
+  public void testNoWritesWithIncrement() throws Exception {
+    Increment i = new Increment(Bytes.toBytes("to_reject"));
+    i.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("count"), 0);
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, i);
+  }
+
+  @Test
+  public void testNoWritesWithDelete() throws Exception {
+    Delete d = new Delete(Bytes.toBytes("to_reject"));
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, d);
+  }
+
+  @Test
+  public void testNoCompactions() throws Exception {
+    Put p = new Put(Bytes.toBytes("to_reject"));
+    p.addColumn(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    final TableName tn = writeUntilViolationAndVerifyViolation(
+        SpaceViolationPolicy.NO_WRITES_COMPACTIONS, p);
+    // We know the policy is active at this point
+
+    // Major compactions should be rejected
+    try {
+      TEST_UTIL.getAdmin().majorCompact(tn);
+      fail("Expected that invoking the compaction should throw an Exception");
+    } catch (DoNotRetryIOException e) {
+      // Expected!
+    }
+    // Minor compactions should also be rejected.
+    try {
+      TEST_UTIL.getAdmin().compact(tn);
+      fail("Expected that invoking the compaction should throw an Exception");
+    } catch (DoNotRetryIOException e) {
+      // Expected!
+    }
+  }
+
+  @Test
+  public void testNoEnableAfterDisablePolicy() throws Exception {
+    Put p = new Put(Bytes.toBytes("to_reject"));
+    p.addColumn(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    final TableName tn = writeUntilViolation(SpaceViolationPolicy.DISABLE);
+    final Admin admin = TEST_UTIL.getAdmin();
+    // Disabling a table relies on some external action (over the other policies), so wait a bit
+    // more than the other tests.
+    for (int i = 0; i < NUM_RETRIES * 2; i++) {
+      if (admin.isTableEnabled(tn)) {
+        LOG.info(tn + " is still enabled, expecting it to be disabled. Will wait and re-check.");
+        Thread.sleep(2000);
+      }
+    }
+    assertFalse(tn + " is still enabled but it should be disabled", admin.isTableEnabled(tn));
+    try {
+      admin.enableTable(tn);
+    } catch (AccessDeniedException e) {
+      String exceptionContents = StringUtils.stringifyException(e);
+      final String expectedText = "violated space quota";
+      assertTrue("Expected the exception to contain " + expectedText + ", but was: "
+          + exceptionContents, exceptionContents.contains(expectedText));
+    }
+  }
+
+  private TableName writeUntilViolation(SpaceViolationPolicy policyToViolate) throws Exception {
+    TableName tn = helper.createTableWithRegions(10);
+
+    final long sizeLimit = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, policyToViolate);
+    TEST_UTIL.getAdmin().setQuota(settings);
+
+    // Write more data than should be allowed and flush it to disk
+    helper.writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
+
+    // This should be sufficient time for the chores to run and see the change.
+    Thread.sleep(5000);
+
+    return tn;
+  }
+
+  private TableName writeUntilViolationAndVerifyViolation(SpaceViolationPolicy policyToViolate, Mutation m) throws Exception {
+    final TableName tn = writeUntilViolation(policyToViolate);
+
+    // But let's try a few times to get the exception before failing
+    boolean sawError = false;
+    for (int i = 0; i < NUM_RETRIES && !sawError; 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");
+        }
+        LOG.info("Did not reject the " + m.getClass().getSimpleName() + ", will sleep and retry");
+        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()));
+        sawError = true;
+      }
+    }
+    if (!sawError) {
+      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 see an exception writing data to a table exceeding its quota", sawError);
+
+    return tn;
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java
new file mode 100644
index 0000000..46aca80
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java
@@ -0,0 +1,141 @@
+/*
+ * 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.mockito.Matchers.argThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.NavigableMap;
+import java.util.Objects;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+
+/**
+ * Test case for {@link TableSpaceQuotaViolationNotifier}.
+ */
+public class TestTableSpaceQuotaViolationNotifier {
+
+  private TableSpaceQuotaViolationNotifier notifier;
+  private Connection conn;
+
+  @Before
+  public void setup() throws Exception {
+    notifier = new TableSpaceQuotaViolationNotifier();
+    conn = mock(Connection.class);
+    notifier.initialize(conn);
+  }
+
+  @Test
+  public void testToViolation() throws Exception {
+    final TableName tn = TableName.valueOf("inviolation");
+    final SpaceViolationPolicy policy = SpaceViolationPolicy.NO_INSERTS;
+    final Table quotaTable = mock(Table.class);
+    when(conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
+
+    final Put expectedPut = new Put(Bytes.toBytes("t." + tn.getNameAsString()));
+    final SpaceQuota protoQuota = SpaceQuota.newBuilder()
+        .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(policy))
+        .build();
+    expectedPut.addColumn(Bytes.toBytes("u"), Bytes.toBytes("v"), protoQuota.toByteArray());
+
+    notifier.transitionTableToViolation(tn, policy);
+
+    verify(quotaTable).put(argThat(new SingleCellPutMatcher(expectedPut)));
+  }
+
+  @Test
+  public void testToObservance() throws Exception {
+    final TableName tn = TableName.valueOf("notinviolation");
+    final Table quotaTable = mock(Table.class);
+    when(conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
+
+    final Delete expectedDelete = new Delete(Bytes.toBytes("t." + tn.getNameAsString()));
+    expectedDelete.addColumn(Bytes.toBytes("u"), Bytes.toBytes("v"));
+
+    notifier.transitionTableToObservance(tn);
+
+    verify(quotaTable).delete(argThat(new SingleCellDeleteMatcher(expectedDelete)));
+  }
+
+  /**
+   * Parameterized for Puts.
+   */
+  private static class SingleCellPutMatcher extends SingleCellMutationMatcher {
+    private SingleCellPutMatcher(Put expected) {
+      super(expected);
+    }
+  }
+
+  /**
+   * Parameterized for Deletes.
+   */
+  private static class SingleCellDeleteMatcher extends SingleCellMutationMatcher {
+    private SingleCellDeleteMatcher(Delete expected) {
+      super(expected);
+    }
+  }
+
+  /**
+   * Quick hack to verify a Mutation with one column.
+   */
+  private static class SingleCellMutationMatcher extends ArgumentMatcher {
+    private final Mutation expected;
+
+    private SingleCellMutationMatcher(Mutation expected) {
+      this.expected = expected;
+    }
+
+    @Override
+    public boolean matches(Object argument) {
+      if (!expected.getClass().isAssignableFrom(argument.getClass())) {
+        return false;
+      }
+      Mutation actual = (Mutation) argument;
+      if (!Arrays.equals(expected.getRow(), actual.getRow())) {
+        return false;
+      }
+      if (expected.size() != actual.size()) {
+        return false;
+      }
+      NavigableMap> expectedCells = expected.getFamilyCellMap();
+      NavigableMap> actualCells = actual.getFamilyCellMap();
+      Entry> expectedEntry = expectedCells.entrySet().iterator().next();
+      Entry> actualEntry = actualCells.entrySet().iterator().next();
+      if (!Arrays.equals(expectedEntry.getKey(), actualEntry.getKey())) {
+        return false;
+      }
+      return Objects.equals(expectedEntry.getValue(), actualEntry.getValue());
+    }
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/BaseViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/BaseViolationPolicyEnforcement.java
new file mode 100644
index 0000000..ec8f1bf
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/BaseViolationPolicyEnforcement.java
@@ -0,0 +1,31 @@
+/*
+ * 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.policies;
+
+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.util.Bytes;
+
+public class BaseViolationPolicyEnforcement {
+
+  static final Append APPEND = new Append(Bytes.toBytes("foo"));
+  static final Delete DELETE = new Delete(Bytes.toBytes("foo"));
+  static final Increment INCREMENT = new Increment(Bytes.toBytes("foo"));
+  static final Put PUT = new Put(Bytes.toBytes("foo"));
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java
new file mode 100644
index 0000000..3248842
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java
@@ -0,0 +1,56 @@
+/*
+ * 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.policies;
+
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test class for {@link DisableTableViolationPolicyEnforcement}.
+ */
+public class TestDisableTableViolationPolicyEnforcement extends BaseViolationPolicyEnforcement {
+
+  private DisableTableViolationPolicyEnforcement enforcement;
+
+  @Before
+  public void setup() {
+    enforcement = new DisableTableViolationPolicyEnforcement();
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckPut() throws SpaceLimitingException {
+    // If the policy is enacted, it will always throw an exception
+    // to avoid constantly re-checking the table state.
+    enforcement.check(PUT);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckAppend() throws SpaceLimitingException {
+    enforcement.check(APPEND);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckDelete() throws SpaceLimitingException {
+    enforcement.check(DELETE);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckIncrement() throws SpaceLimitingException {
+    enforcement.check(INCREMENT);
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java
new file mode 100644
index 0000000..fed7c0d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java
@@ -0,0 +1,54 @@
+/*
+ * 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.policies;
+
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test class for {@link NoInsertsViolationPolicyEnforcement}.
+ */
+public class TestNoInsertsViolationPolicyEnforcement extends BaseViolationPolicyEnforcement {
+
+  private NoInsertsViolationPolicyEnforcement enforcement;
+
+  @Before
+  public void setup() {
+    enforcement = new NoInsertsViolationPolicyEnforcement();
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckAppend() throws Exception {
+    enforcement.check(APPEND);
+  }
+
+  @Test
+  public void testCheckDelete() throws Exception {
+    enforcement.check(DELETE);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckIncrement() throws Exception {
+    enforcement.check(INCREMENT);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckPut() throws Exception {
+    enforcement.check(PUT);
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java
new file mode 100644
index 0000000..e017f59
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java
@@ -0,0 +1,55 @@
+/*
+ * 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.policies;
+
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test class for {@link NoWritesCompactionsViolationPolicyEnforcement};
+ */
+public class TestNoWritesCompactionsViolationPolicyEnforcement
+    extends BaseViolationPolicyEnforcement {
+
+  private NoWritesCompactionsViolationPolicyEnforcement enforcement;
+
+  @Before
+  public void setup() {
+    enforcement = new NoWritesCompactionsViolationPolicyEnforcement();
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckAppend() throws Exception {
+    enforcement.check(APPEND);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckDelete() throws Exception {
+    enforcement.check(DELETE);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckIncrement() throws Exception {
+    enforcement.check(INCREMENT);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckPut() throws Exception {
+    enforcement.check(PUT);
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java
new file mode 100644
index 0000000..3e23bdb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java
@@ -0,0 +1,54 @@
+/*
+ * 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.policies;
+
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test class for {@link NoWritesViolationPolicyEnforcement}.
+ */
+public class TestNoWritesViolationPolicyEnforcement extends BaseViolationPolicyEnforcement {
+
+  private NoWritesViolationPolicyEnforcement enforcement;
+
+  @Before
+  public void setup() {
+    enforcement = new NoWritesViolationPolicyEnforcement();
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckAppend() throws Exception {
+    enforcement.check(APPEND);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckDelete() throws Exception {
+    enforcement.check(DELETE);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckIncrement() throws Exception {
+    enforcement.check(INCREMENT);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckPut() throws Exception {
+    enforcement.check(PUT);
+  }
+}
-- 
2.10.2