You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2017/08/07 21:13:52 UTC

[2/3] hbase git commit: HBASE-18514 Squash of space quota "phase 2" work

HBASE-18514 Squash of space quota "phase 2" work

Contains the following commits:

  HBASE-17748 Include HBase snapshots in space quotas

  Introduces a new Chore in the Master which computes the size
  of the snapshots included in a cluster. The size of these
  snapshots are included in the table's which the snapshot was created
  from HDFS usage.

  Includes some test stabilization, trying to make the tests more
  deterministic by ensuring we observe stable values as we know
  that those values are mutable. This should help avoid problems
  where size reports are delayed and we see an incomplete value.

  HBASE-17752 Shell command to list snapshot sizes WRT quotas

  HBASE-17840 Update hbase book to space quotas on snapshots


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

Branch: refs/heads/branch-2
Commit: f262548b1f3fcd9438773c3e1c5033b8068684db
Parents: 7029c23
Author: Josh Elser <el...@apache.org>
Authored: Wed Mar 8 20:56:37 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Aug 7 16:32:16 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hbase/quotas/QuotaTableUtil.java     | 186 ++++++-
 .../hadoop/hbase/quotas/SpaceQuotaSnapshot.java |   4 +-
 .../hbase/master/MetricsMasterQuotaSource.java  |  27 +
 .../master/MetricsMasterQuotaSourceImpl.java    |  25 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   9 +
 .../hadoop/hbase/master/MetricsMaster.java      |  21 +
 .../quotas/FileSystemUtilizationChore.java      |   9 +-
 .../quotas/NamespaceQuotaSnapshotStore.java     |   7 +-
 .../hadoop/hbase/quotas/QuotaObserverChore.java |   4 +-
 .../hadoop/hbase/quotas/QuotaSnapshotStore.java |   2 +-
 .../quotas/SnapshotQuotaObserverChore.java      | 543 +++++++++++++++++++
 .../hbase/quotas/TableQuotaSnapshotStore.java   |  54 +-
 .../quotas/TableSpaceQuotaSnapshotNotifier.java |   2 +-
 .../hadoop/hbase/regionserver/HStore.java       |  16 +-
 .../apache/hadoop/hbase/regionserver/Store.java |   5 +
 .../hbase/quotas/SpaceQuotaHelperForTests.java  | 184 ++++++-
 .../quotas/TestFileSystemUtilizationChore.java  |  54 +-
 .../TestNamespaceQuotaViolationStore.java       |  15 +-
 .../TestQuotaObserverChoreRegionReports.java    |   7 +-
 .../TestQuotaObserverChoreWithMiniCluster.java  |   6 +-
 .../hbase/quotas/TestQuotaStatusRPCs.java       |  13 +-
 .../hadoop/hbase/quotas/TestQuotaTableUtil.java |  65 ++-
 .../hadoop/hbase/quotas/TestRegionSizeUse.java  |   5 +-
 .../quotas/TestSnapshotQuotaObserverChore.java  | 368 +++++++++++++
 .../hadoop/hbase/quotas/TestSpaceQuotas.java    |  10 +-
 .../quotas/TestSpaceQuotasWithSnapshots.java    | 462 ++++++++++++++++
 .../quotas/TestSuperUserQuotaPermissions.java   |   8 +-
 .../quotas/TestTableQuotaViolationStore.java    |  17 +-
 hbase-shell/src/main/ruby/hbase/quotas.rb       |   4 +
 hbase-shell/src/main/ruby/shell.rb              |   1 +
 .../ruby/shell/commands/list_snapshot_sizes.rb  |   8 +-
 hbase-shell/src/test/ruby/hbase/quotas_test.rb  |  29 +-
 hbase-shell/src/test/ruby/tests_runner.rb       |   2 +
 src/main/asciidoc/_chapters/ops_mgt.adoc        |  45 ++
 34 files changed, 2116 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.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 1b670e6..d1bbade 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
@@ -31,6 +31,7 @@ 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.CellScanner;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -45,6 +46,7 @@ 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.filter.ColumnPrefixFilter;
 import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
@@ -70,16 +72,18 @@ import org.apache.hadoop.hbase.util.Strings;
 
 /**
  * Helper class to interact with the quota table.
- * <pre>
- *     ROW-KEY      FAM/QUAL        DATA
- *   n.&lt;namespace&gt; q:s         &lt;global-quotas&gt;
- *   t.&lt;namespace&gt; u:p        &lt;namespace-quota policy&gt;
- *   t.&lt;table&gt;     q:s         &lt;global-quotas&gt;
- *   t.&lt;table&gt;     u:p        &lt;table-quota policy&gt;
- *   u.&lt;user&gt;      q:s         &lt;global-quotas&gt;
- *   u.&lt;user&gt;      q:s.&lt;table&gt; &lt;table-quotas&gt;
- *   u.&lt;user&gt;      q:s.&lt;ns&gt;:   &lt;namespace-quotas&gt;
- * </pre>
+ * <table>
+ *   <tr><th>ROW-KEY</th><th>FAM/QUAL</th><th>DATA</th></tr>
+ *   <tr><td>n.&lt;namespace&gt;</td><td>q:s</td><td>&lt;global-quotas&gt;</td></tr>
+ *   <tr><td>n.&lt;namespace&gt;</td><td>u:p</td><td>&lt;namespace-quota policy&gt;</td></tr>
+ *   <tr><td>n.&lt;namespace&gt;</td><td>u:s</td><td>&lt;SpaceQuotaSnapshot&gt;</td></tr>
+ *   <tr><td>t.&lt;table&gt;</td><td>q:s</td><td>&lt;global-quotas&gt;</td></tr>
+ *   <tr><td>t.&lt;table&gt;</td><td>u:p</td><td>&lt;table-quota policy&gt;</td></tr>
+ *   <tr><td>t.&lt;table&gt;</td><td>u:ss.&lt;snapshot name&gt;</td><td>&lt;SpaceQuotaSnapshot&gt;</td></tr>
+ *   <tr><td>u.&lt;user&gt;</td><td>q:s</td><td>&lt;global-quotas&gt;</td></tr>
+ *   <tr><td>u.&lt;user&gt;</td><td>q:s.&lt;table&gt;</td><td>&lt;table-quotas&gt;</td></tr>
+ *   <tr><td>u.&lt;user&gt;</td><td>q:s.&lt;ns&gt;</td><td>&lt;namespace-quotas&gt;</td></tr>
+ * </table
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -95,6 +99,7 @@ public class QuotaTableUtil {
   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_POLICY = Bytes.toBytes("p");
+  protected static final byte[] QUOTA_SNAPSHOT_SIZE_QUALIFIER = Bytes.toBytes("ss");
   protected static final String QUOTA_POLICY_COLUMN =
       Bytes.toString(QUOTA_FAMILY_USAGE) + ":" + Bytes.toString(QUOTA_QUALIFIER_POLICY);
   protected static final byte[] QUOTA_USER_ROW_KEY_PREFIX = Bytes.toBytes("u.");
@@ -229,12 +234,7 @@ public class QuotaTableUtil {
    * Creates a {@link Scan} which returns only quota snapshots from the quota table.
    */
   public static Scan makeQuotaSnapshotScan() {
-    Scan s = new Scan();
-    // Limit to "u:v" column
-    s.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY);
-    // Limit rowspace to the "t:" prefix
-    s.setRowPrefixFilter(QUOTA_TABLE_ROW_KEY_PREFIX);
-    return s;
+    return makeQuotaSnapshotScanForTable(null);
   }
 
   /**
@@ -255,6 +255,25 @@ public class QuotaTableUtil {
   }
 
   /**
+   * Creates a {@link Scan} which returns only {@link SpaceQuotaSnapshot} from the quota table for a
+   * specific table.
+   * @param tn Optionally, a table name to limit the scan's rowkey space. Can be null.
+   */
+  public static Scan makeQuotaSnapshotScanForTable(TableName tn) {
+    Scan s = new Scan();
+    // Limit to "u:v" column
+    s.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY);
+    if (null == tn) {
+      s.setRowPrefixFilter(QUOTA_TABLE_ROW_KEY_PREFIX);
+    } else {
+      byte[] row = getTableRowKey(tn);
+      // Limit rowspace to the "t:" prefix
+      s.withStartRow(row, true).withStopRow(row, true);
+    }
+    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
@@ -416,7 +435,7 @@ public class QuotaTableUtil {
    * Creates a {@link Put} to store the given {@code snapshot} for the given {@code tableName} in
    * the quota table.
    */
-  public static Put createPutSpaceSnapshot(TableName tableName, SpaceQuotaSnapshot snapshot) {
+  static Put createPutForSpaceSnapshot(TableName tableName, SpaceQuotaSnapshot snapshot) {
     Put p = new Put(getTableRowKey(tableName));
     p.addColumn(
         QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY,
@@ -424,6 +443,122 @@ public class QuotaTableUtil {
     return p;
   }
 
+  /**
+   * Creates a {@link Get} for the HBase snapshot's size against the given table.
+   */
+  static Get makeGetForSnapshotSize(TableName tn, String snapshot) {
+    Get g = new Get(Bytes.add(QUOTA_TABLE_ROW_KEY_PREFIX, Bytes.toBytes(tn.toString())));
+    g.addColumn(
+        QUOTA_FAMILY_USAGE,
+        Bytes.add(QUOTA_SNAPSHOT_SIZE_QUALIFIER, Bytes.toBytes(snapshot)));
+    return g;
+  }
+
+  /**
+   * Creates a {@link Put} to persist the current size of the {@code snapshot} with respect to
+   * the given {@code table}.
+   */
+  static Put createPutForSnapshotSize(TableName tableName, String snapshot, long size) {
+    // We just need a pb message with some `long usage`, so we can just reuse the
+    // SpaceQuotaSnapshot message instead of creating a new one.
+    Put p = new Put(getTableRowKey(tableName));
+    p.addColumn(QUOTA_FAMILY_USAGE, getSnapshotSizeQualifier(snapshot),
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot
+            .newBuilder().setQuotaUsage(size).build().toByteArray());
+    return p;
+  }
+
+  /**
+   * Creates a {@code Put} for the namespace's total snapshot size.
+   */
+  static Put createPutForNamespaceSnapshotSize(String namespace, long size) {
+    Put p = new Put(getNamespaceRowKey(namespace));
+    p.addColumn(QUOTA_FAMILY_USAGE, QUOTA_SNAPSHOT_SIZE_QUALIFIER,
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot
+            .newBuilder().setQuotaUsage(size).build().toByteArray());
+    return p;
+  }
+
+  /**
+   * Fetches the computed size of all snapshots against tables in a namespace for space quotas.
+   */
+  static long getNamespaceSnapshotSize(
+      Connection conn, String namespace) throws IOException {
+    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      Result r = quotaTable.get(createGetNamespaceSnapshotSize(namespace));
+      if (r.isEmpty()) {
+        return 0L;
+      }
+      r.advance();
+      return parseSnapshotSize(r.current());
+    } catch (InvalidProtocolBufferException e) {
+      throw new IOException("Could not parse snapshot size value for namespace " + namespace, e);
+    }
+  }
+
+  /**
+   * Creates a {@code Get} to fetch the namespace's total snapshot size.
+   */
+  static Get createGetNamespaceSnapshotSize(String namespace) {
+    Get g = new Get(getNamespaceRowKey(namespace));
+    g.addColumn(QUOTA_FAMILY_USAGE, QUOTA_SNAPSHOT_SIZE_QUALIFIER);
+    return g;
+  }
+
+  /**
+   * Parses the snapshot size from the given Cell's value.
+   */
+  static long parseSnapshotSize(Cell c) throws InvalidProtocolBufferException {
+    ByteString bs = UnsafeByteOperations.unsafeWrap(
+        c.getValueArray(), c.getValueOffset(), c.getValueLength());
+    return QuotaProtos.SpaceQuotaSnapshot.parseFrom(bs).getQuotaUsage();
+  }
+
+  static Scan createScanForSpaceSnapshotSizes() {
+    return createScanForSpaceSnapshotSizes(null);
+  }
+
+  static Scan createScanForSpaceSnapshotSizes(TableName table) {
+    Scan s = new Scan();
+    if (null == table) {
+      // Read all tables, just look at the row prefix
+      s.setRowPrefixFilter(QUOTA_TABLE_ROW_KEY_PREFIX);
+    } else {
+      // Fetch the exact row for the table
+      byte[] rowkey = getTableRowKey(table);
+      // Fetch just this one row
+      s.withStartRow(rowkey).withStopRow(rowkey, true);
+    }
+
+    // Just the usage family and only the snapshot size qualifiers
+    return s.addFamily(QUOTA_FAMILY_USAGE).setFilter(
+        new ColumnPrefixFilter(QUOTA_SNAPSHOT_SIZE_QUALIFIER));
+  }
+
+  /**
+   * Fetches any persisted HBase snapshot sizes stored in the quota table. The sizes here are
+   * computed relative to the table which the snapshot was created from. A snapshot's size will
+   * not include the size of files which the table still refers. These sizes, in bytes, are what
+   * is used internally to compute quota violation for tables and namespaces.
+   *
+   * @return A map of snapshot name to size in bytes per space quota computations
+   */
+  public static Map<String,Long> getObservedSnapshotSizes(Connection conn) throws IOException {
+    try (Table quotaTable = conn.getTable(QUOTA_TABLE_NAME);
+        ResultScanner rs = quotaTable.getScanner(createScanForSpaceSnapshotSizes())) {
+      final Map<String,Long> snapshotSizes = new HashMap<>();
+      for (Result r : rs) {
+        CellScanner cs = r.cellScanner();
+        while (cs.advance()) {
+          Cell c = cs.current();
+          final String snapshot = extractSnapshotNameFromSizeCell(c);
+          final long size = parseSnapshotSize(c);
+          snapshotSizes.put(snapshot, size);
+        }
+      }
+      return snapshotSizes;
+    }
+  }
 
   /* =========================================================================
    *  Space quota status RPC helpers
@@ -644,4 +779,21 @@ public class QuotaTableUtil {
     }
     return ProtobufUtil.toViolationPolicy(proto.getViolationPolicy());
   }
+
+  protected static byte[] getSnapshotSizeQualifier(String snapshotName) {
+    return Bytes.add(QUOTA_SNAPSHOT_SIZE_QUALIFIER, Bytes.toBytes(snapshotName));
+  }
+
+  protected static String extractSnapshotNameFromSizeCell(Cell c) {
+    return Bytes.toString(
+        c.getQualifierArray(), c.getQualifierOffset() + QUOTA_SNAPSHOT_SIZE_QUALIFIER.length,
+        c.getQualifierLength() - QUOTA_SNAPSHOT_SIZE_QUALIFIER.length);
+  }
+
+  protected static long extractSnapshotSize(
+      byte[] data, int offset, int length) throws InvalidProtocolBufferException {
+    ByteString byteStr = UnsafeByteOperations.unsafeWrap(data, offset, length);
+    return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot
+        .parseFrom(byteStr).getQuotaUsage();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
index 5c64880..18fe76a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
@@ -22,6 +22,7 @@ import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * A point-in-time view of a space quota on a table.
@@ -175,7 +176,8 @@ public class SpaceQuotaSnapshot {
   public String toString() {
     StringBuilder sb = new StringBuilder(32);
     sb.append("SpaceQuotaSnapshot[policy=").append(quotaStatus).append(", use=");
-    sb.append(usage).append("bytes/").append(limit).append("bytes]");
+    sb.append(StringUtils.byteDesc(usage)).append("/");
+    sb.append(StringUtils.byteDesc(limit)).append("]");
     return sb.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
index 7625f84..99c5441 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
@@ -39,6 +39,15 @@ public interface MetricsMasterQuotaSource extends BaseSource {
   String QUOTA_OBSERVER_CHORE_TIME_NAME = "quotaObserverChoreTime";
   String QUOTA_OBSERVER_CHORE_TIME_DESC =
       "Histogram for the time in millis for the QuotaObserverChore";
+  String SNAPSHOT_OBSERVER_CHORE_TIME_NAME = "snapshotQuotaObserverChoreTime";
+  String SNAPSHOT_OBSERVER_CHORE_TIME_DESC =
+      "Histogram for the time in millis for the SnapshotQuotaObserverChore";
+  String SNAPSHOT_OBSERVER_SIZE_COMPUTATION_TIME_NAME = "snapshotObserverSizeComputationTime";
+  String SNAPSHOT_OBSERVER_SIZE_COMPUTATION_TIME_DESC =
+      "Histogram for the time in millis to compute the size of each snapshot";
+  String SNAPSHOT_OBSERVER_FETCH_TIME_NAME = "snapshotObserverSnapshotFetchTime";
+  String SNAPSHOT_OBSERVER_FETCH_TIME_DESC =
+      "Histogram for the time in millis to fetch all snapshots from HBase";
   String TABLE_QUOTA_USAGE_NAME = "tableSpaceQuotaOverview";
   String TABLE_QUOTA_USAGE_DESC = "A JSON summary of the usage of all tables with space quotas";
   String NS_QUOTA_USAGE_NAME = "namespaceSpaceQuotaOverview";
@@ -83,4 +92,22 @@ public interface MetricsMasterQuotaSource extends BaseSource {
    * @param time The execution time of the chore in milliseconds
    */
   void incrementSpaceQuotaObserverChoreTime(long time);
+
+  /**
+   * Updates the metric tracking the amount of time taken by the {@code SnapshotQuotaObserverChore}
+   * which runs periodically.
+   */
+  void incrementSnapshotObserverChoreTime(long time);
+
+  /**
+   * Updates the metric tracking the amount of time taken by the {@code SnapshotQuotaObserverChore}
+   * to compute the size of one snapshot, relative to the files referenced by the originating table.
+   */
+  void incrementSnapshotObserverSnapshotComputationTime(long time);
+
+  /**
+   * Updates the metric tracking the amount of time taken by the {@code SnapshotQuotaObserverChore}
+   * to fetch all snapshots.
+   */
+  void incrementSnapshotObserverSnapshotFetchTime(long time);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
index 342aa1e..694faca 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
@@ -39,6 +39,9 @@ public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements Metr
   private final MutableGaugeLong namespacesViolatingQuotasGauge;
   private final MutableGaugeLong regionSpaceReportsGauge;
   private final MetricHistogram quotaObserverTimeHisto;
+  private final MetricHistogram snapshotObserverTimeHisto;
+  private final MetricHistogram snapshotObserverSizeComputationTimeHisto;
+  private final MetricHistogram snapshotObserverSnapshotFetchTimeHisto;
 
   public MetricsMasterQuotaSourceImpl(MetricsMasterWrapper wrapper) {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, wrapper);
@@ -61,6 +64,13 @@ public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements Metr
 
     quotaObserverTimeHisto = getMetricsRegistry().newTimeHistogram(
         QUOTA_OBSERVER_CHORE_TIME_NAME, QUOTA_OBSERVER_CHORE_TIME_DESC);
+    snapshotObserverTimeHisto = getMetricsRegistry().newTimeHistogram(
+        SNAPSHOT_OBSERVER_CHORE_TIME_NAME, SNAPSHOT_OBSERVER_CHORE_TIME_DESC);
+
+    snapshotObserverSizeComputationTimeHisto = getMetricsRegistry().newTimeHistogram(
+        SNAPSHOT_OBSERVER_SIZE_COMPUTATION_TIME_NAME, SNAPSHOT_OBSERVER_SIZE_COMPUTATION_TIME_DESC);
+    snapshotObserverSnapshotFetchTimeHisto = getMetricsRegistry().newTimeHistogram(
+        SNAPSHOT_OBSERVER_FETCH_TIME_NAME, SNAPSHOT_OBSERVER_FETCH_TIME_DESC);
   }
 
   @Override
@@ -89,6 +99,11 @@ public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements Metr
   }
 
   @Override
+  public void incrementSnapshotObserverChoreTime(long time) {
+    snapshotObserverTimeHisto.add(time);
+  }
+
+  @Override
   public void getMetrics(MetricsCollector metricsCollector, boolean all) {
     MetricsRecordBuilder record = metricsCollector.addRecord(metricsRegistry.info());
     if (wrapper != null) {
@@ -130,4 +145,14 @@ public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements Metr
     sb.insert(0, "[").append("]");
     return sb.toString();
   }
+
+  @Override
+  public void incrementSnapshotObserverSnapshotComputationTime(long time) {
+    snapshotObserverSizeComputationTimeHisto.add(time);
+  }
+
+  @Override
+  public void incrementSnapshotObserverSnapshotFetchTime(long time) {
+    snapshotObserverSnapshotFetchTimeHisto.add(time);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 5bb6b67..96bf859 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
@@ -144,6 +144,7 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver;
 import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
+import org.apache.hadoop.hbase.quotas.SnapshotQuotaObserverChore;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifier;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifierFactory;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
@@ -391,6 +392,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   private volatile MasterQuotaManager quotaManager;
   private SpaceQuotaSnapshotNotifier spaceQuotaSnapshotNotifier;
   private QuotaObserverChore quotaObserverChore;
+  private SnapshotQuotaObserverChore snapshotQuotaChore;
 
   private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
   private WALProcedureStore procedureStore;
@@ -898,6 +900,10 @@ public class HMaster extends HRegionServer implements MasterServices {
       this.quotaObserverChore = new QuotaObserverChore(this, getMasterMetrics());
       // Start the chore to read the region FS space reports and act on them
       getChoreService().scheduleChore(quotaObserverChore);
+
+      this.snapshotQuotaChore = new SnapshotQuotaObserverChore(this, getMasterMetrics());
+      // Start the chore to read snapshots and add their usage to table/NS quotas
+      getChoreService().scheduleChore(snapshotQuotaChore);
     }
 
     // clear the dead servers with same host name and port of online server because we are not
@@ -1242,6 +1248,9 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (this.quotaObserverChore != null) {
       quotaObserverChore.cancel();
     }
+    if (this.snapshotQuotaChore != null) {
+      snapshotQuotaChore.cancel();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java
index 2810d14..c896441 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java
@@ -156,4 +156,25 @@ public class MetricsMaster {
       }
     };
   }
+
+  /**
+   * Sets the execution time of a period of the {@code SnapshotQuotaObserverChore}.
+   */
+  public void incrementSnapshotObserverTime(final long executionTime) {
+    masterQuotaSource.incrementSnapshotObserverChoreTime(executionTime);
+  }
+
+  /**
+   * Sets the execution time to compute the size of a single snapshot.
+   */
+  public void incrementSnapshotSizeComputationTime(final long executionTime) {
+    masterQuotaSource.incrementSnapshotObserverSnapshotComputationTime(executionTime);
+  }
+
+  /**
+   * Sets the execution time to fetch the mapping of snapshots to originating table.
+   */
+  public void incrementSnapshotFetchTime(long executionTime) {
+    masterQuotaSource.incrementSnapshotObserverSnapshotFetchTime(executionTime);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
index 418a163..1e44686 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
@@ -33,6 +33,8 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
@@ -168,9 +170,10 @@ public class FileSystemUtilizationChore extends ScheduledChore {
   long computeSize(Region r) {
     long regionSize = 0L;
     for (Store store : r.getStores()) {
-      // StoreFile/StoreFileReaders are already instantiated with the file length cached.
-      // Can avoid extra NN ops.
-      regionSize += store.getStorefilesSize();
+      regionSize += store.getHFilesSize();
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Size of " + r + " is " + regionSize);
     }
     return regionSize;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
index 5b7f823..baa1907 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
@@ -18,8 +18,8 @@ package org.apache.hadoop.hbase.quotas;
 
 import java.io.IOException;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Map.Entry;
+import java.util.Objects;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
@@ -77,7 +77,8 @@ public class NamespaceQuotaSnapshotStore implements QuotaSnapshotStore<String> {
   }
 
   @Override
-  public SpaceQuotaSnapshot getTargetState(String subject, SpaceQuota spaceQuota) {
+  public SpaceQuotaSnapshot getTargetState(
+      String subject, SpaceQuota spaceQuota) throws IOException {
     rlock.lock();
     try {
       final long sizeLimitInBytes = spaceQuota.getSoftLimit();
@@ -85,6 +86,8 @@ public class NamespaceQuotaSnapshotStore implements QuotaSnapshotStore<String> {
       for (Entry<HRegionInfo,Long> entry : filterBySubject(subject)) {
         sum += entry.getValue();
       }
+      // Add in the size for any snapshots against this table
+      sum += QuotaTableUtil.getNamespaceSnapshotSize(conn, subject);
       // Observance is defined as the size of the table being less than the limit
       SpaceQuotaStatus status = sum <= sizeLimitInBytes ? SpaceQuotaStatus.notInViolation()
           : new SpaceQuotaStatus(ProtobufUtil.toViolationPolicy(spaceQuota.getViolationPolicy()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
----------------------------------------------------------------------
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 996e0f1..8c9e780 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
@@ -161,7 +161,9 @@ public class QuotaObserverChore extends ScheduledChore {
     // The current "view" of region space use. Used henceforth.
     final Map<HRegionInfo,Long> reportedRegionSpaceUse = quotaManager.snapshotRegionSizes();
     if (LOG.isTraceEnabled()) {
-      LOG.trace("Using " + reportedRegionSpaceUse.size() + " region space use reports");
+      LOG.trace(
+          "Using " + reportedRegionSpaceUse.size() + " region space use reports: " +
+          reportedRegionSpaceUse);
     }
 
     // Remove the "old" region reports

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java
index 8b0b3a7..2b5ba59 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java
@@ -69,7 +69,7 @@ public interface QuotaSnapshotStore<T> {
    * @param subject The object which to determine the target SpaceQuotaSnapshot of
    * @param spaceQuota The quota "definition" for the {@code subject}
    */
-  SpaceQuotaSnapshot getTargetState(T subject, SpaceQuota spaceQuota);
+  SpaceQuotaSnapshot getTargetState(T subject, SpaceQuota spaceQuota) throws IOException;
 
   /**
    * Filters the provided <code>regions</code>, returning those which match the given

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SnapshotQuotaObserverChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SnapshotQuotaObserverChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SnapshotQuotaObserverChore.java
new file mode 100644
index 0000000..46f5a64
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SnapshotQuotaObserverChore.java
@@ -0,0 +1,543 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MetricsMaster;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+
+/**
+ * A Master-invoked {@code Chore} that computes the size of each snapshot which was created from
+ * a table which has a space quota.
+ */
+@InterfaceAudience.Private
+public class SnapshotQuotaObserverChore extends ScheduledChore {
+  private static final Log LOG = LogFactory.getLog(SnapshotQuotaObserverChore.class);
+  static final String SNAPSHOT_QUOTA_CHORE_PERIOD_KEY =
+      "hbase.master.quotas.snapshot.chore.period";
+  static final int SNAPSHOT_QUOTA_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
+
+  static final String SNAPSHOT_QUOTA_CHORE_DELAY_KEY =
+      "hbase.master.quotas.snapshot.chore.delay";
+  static final long SNAPSHOT_QUOTA_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute in millis
+
+  static final String SNAPSHOT_QUOTA_CHORE_TIMEUNIT_KEY =
+      "hbase.master.quotas.snapshot.chore.timeunit";
+  static final String SNAPSHOT_QUOTA_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
+
+  private final Connection conn;
+  private final Configuration conf;
+  private final MetricsMaster metrics;
+  private final FileSystem fs;
+
+  public SnapshotQuotaObserverChore(HMaster master, MetricsMaster metrics) {
+    this(
+        master.getConnection(), master.getConfiguration(), master.getFileSystem(), master, metrics);
+  }
+
+  SnapshotQuotaObserverChore(
+      Connection conn, Configuration conf, FileSystem fs, Stoppable stopper,
+      MetricsMaster metrics) {
+    super(
+        QuotaObserverChore.class.getSimpleName(), stopper, getPeriod(conf),
+        getInitialDelay(conf), getTimeUnit(conf));
+    this.conn = conn;
+    this.conf = conf;
+    this.metrics = metrics;
+    this.fs = fs;
+  }
+
+  @Override
+  protected void chore() {
+    try {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Computing sizes of snapshots for quota management.");
+      }
+      long start = System.nanoTime();
+      _chore();
+      if (null != metrics) {
+        metrics.incrementSnapshotObserverTime((System.nanoTime() - start) / 1_000_000);
+      }
+    } catch (IOException e) {
+      LOG.warn("Failed to compute the size of snapshots, will retry", e);
+    }
+  }
+
+  void _chore() throws IOException {
+    // Gets all tables with quotas that also have snapshots.
+    // This values are all of the snapshots that we need to compute the size of.
+    long start = System.nanoTime();
+    Multimap<TableName,String> snapshotsToComputeSize = getSnapshotsToComputeSize();
+    if (null != metrics) {
+      metrics.incrementSnapshotFetchTime((System.nanoTime() - start) / 1_000_000);
+    }
+
+    // For each table, compute the size of each snapshot
+    Multimap<TableName,SnapshotWithSize> snapshotsWithSize = computeSnapshotSizes(
+        snapshotsToComputeSize);
+
+    // Write the size data to the quota table.
+    persistSnapshotSizes(snapshotsWithSize);
+  }
+
+  /**
+   * Fetches each table with a quota (table or namespace quota), and then fetch the name of each
+   * snapshot which was created from that table.
+   *
+   * @return A mapping of table to snapshots created from that table
+   */
+  Multimap<TableName,String> getSnapshotsToComputeSize() throws IOException {
+    Set<TableName> tablesToFetchSnapshotsFrom = new HashSet<>();
+    QuotaFilter filter = new QuotaFilter();
+    filter.addTypeFilter(QuotaType.SPACE);
+    try (Admin admin = conn.getAdmin()) {
+      // Pull all of the tables that have quotas (direct, or from namespace)
+      for (QuotaSettings qs : QuotaRetriever.open(conf, filter)) {
+        String ns = qs.getNamespace();
+        TableName tn = qs.getTableName();
+        if ((null == ns && null == tn) || (null != ns && null != tn)) {
+          throw new IllegalStateException(
+              "Expected only one of namespace and tablename to be null");
+        }
+        // Collect either the table name itself, or all of the tables in the namespace
+        if (null != ns) {
+          tablesToFetchSnapshotsFrom.addAll(Arrays.asList(admin.listTableNamesByNamespace(ns)));
+        } else {
+          tablesToFetchSnapshotsFrom.add(tn);
+        }
+      }
+      // Fetch all snapshots that were created from these tables
+      return getSnapshotsFromTables(admin, tablesToFetchSnapshotsFrom);
+    }
+  }
+
+  /**
+   * Computes a mapping of originating {@code TableName} to snapshots, when the {@code TableName}
+   * exists in the provided {@code Set}.
+   */
+  Multimap<TableName,String> getSnapshotsFromTables(
+      Admin admin, Set<TableName> tablesToFetchSnapshotsFrom) throws IOException {
+    Multimap<TableName,String> snapshotsToCompute = HashMultimap.create();
+    for (org.apache.hadoop.hbase.client.SnapshotDescription sd : admin.listSnapshots()) {
+      TableName tn = sd.getTableName();
+      if (tablesToFetchSnapshotsFrom.contains(tn)) {
+        snapshotsToCompute.put(tn, sd.getName());
+      }
+    }
+    return snapshotsToCompute;
+  }
+
+  /**
+   * Computes the size of each snapshot provided given the current files referenced by the table.
+   *
+   * @param snapshotsToComputeSize The snapshots to compute the size of
+   * @return A mapping of table to snapshot created from that table and the snapshot's size.
+   */
+  Multimap<TableName,SnapshotWithSize> computeSnapshotSizes(
+      Multimap<TableName,String> snapshotsToComputeSize) throws IOException {
+    Multimap<TableName,SnapshotWithSize> snapshotSizes = HashMultimap.create();
+    for (Entry<TableName,Collection<String>> entry : snapshotsToComputeSize.asMap().entrySet()) {
+      final TableName tn = entry.getKey();
+      final List<String> snapshotNames = new ArrayList<>(entry.getValue());
+      // Sort the snapshots so we process them in lexicographic order. This ensures that multiple
+      // invocations of this Chore do not more the size ownership of some files between snapshots
+      // that reference the file (prevents size ownership from moving between snapshots).
+      Collections.sort(snapshotNames);
+      final Path rootDir = FSUtils.getRootDir(conf);
+      // Get the map of store file names to store file path for this table
+      // TODO is the store-file name unique enough? Does this need to be region+family+storefile?
+      final Set<String> tableReferencedStoreFiles;
+      try {
+        tableReferencedStoreFiles = FSUtils.getTableStoreFilePathMap(fs, rootDir).keySet();
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        return null;
+      }
+
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Paths for " + tn + ": " + tableReferencedStoreFiles);
+      }
+
+      // For each snapshot on this table, get the files which the snapshot references which
+      // the table does not.
+      Set<String> snapshotReferencedFiles = new HashSet<>();
+      for (String snapshotName : snapshotNames) {
+        final long start = System.nanoTime();
+        Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
+        SnapshotDescription sd = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
+        SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, sd);
+
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Files referenced by other snapshots: " + snapshotReferencedFiles);
+        }
+
+        // Get the set of files from the manifest that this snapshot references which are not also
+        // referenced by the originating table.
+        Set<StoreFileReference> unreferencedStoreFileNames = getStoreFilesFromSnapshot(
+            manifest, (sfn) -> !tableReferencedStoreFiles.contains(sfn)
+                && !snapshotReferencedFiles.contains(sfn));
+
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Snapshot " + snapshotName + " solely references the files: "
+              + unreferencedStoreFileNames);
+        }
+
+        // Compute the size of the store files for this snapshot
+        long size = getSizeOfStoreFiles(tn, unreferencedStoreFileNames);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Computed size of " + snapshotName + " to be " + size);
+        }
+
+        // Persist this snapshot's size into the map
+        snapshotSizes.put(tn, new SnapshotWithSize(snapshotName, size));
+
+        // Make sure that we don't double-count the same file
+        for (StoreFileReference ref : unreferencedStoreFileNames) {
+          for (String fileName : ref.getFamilyToFilesMapping().values()) {
+            snapshotReferencedFiles.add(fileName);
+          }
+        }
+        // Update the amount of time it took to compute the snapshot's size
+        if (null != metrics) {
+          metrics.incrementSnapshotSizeComputationTime((System.nanoTime() - start) / 1_000_000);
+        }
+      }
+    }
+    return snapshotSizes;
+  }
+
+  /**
+   * Extracts the names of the store files referenced by this snapshot which satisfy the given
+   * predicate (the predicate returns {@code true}).
+   */
+  Set<StoreFileReference> getStoreFilesFromSnapshot(
+      SnapshotManifest manifest, Predicate<String> filter) {
+    Set<StoreFileReference> references = new HashSet<>();
+    // For each region referenced by the snapshot
+    for (SnapshotRegionManifest rm : manifest.getRegionManifests()) {
+      StoreFileReference regionReference = new StoreFileReference(
+          HRegionInfo.convert(rm.getRegionInfo()).getEncodedName());
+
+      // For each column family in this region
+      for (FamilyFiles ff : rm.getFamilyFilesList()) {
+        final String familyName = ff.getFamilyName().toStringUtf8();
+        // And each store file in that family
+        for (StoreFile sf : ff.getStoreFilesList()) {
+          String storeFileName = sf.getName();
+          // A snapshot only "inherits" a files size if it uniquely refers to it (no table
+          // and no other snapshot references it).
+          if (filter.test(storeFileName)) {
+            regionReference.addFamilyStoreFile(familyName, storeFileName);
+          }
+        }
+      }
+      // Only add this Region reference if we retained any files.
+      if (!regionReference.getFamilyToFilesMapping().isEmpty()) {
+        references.add(regionReference);
+      }
+    }
+    return references;
+  }
+
+  /**
+   * Calculates the directory in HDFS for a table based on the configuration.
+   */
+  Path getTableDir(TableName tn) throws IOException {
+    Path rootDir = FSUtils.getRootDir(conf);
+    return FSUtils.getTableDir(rootDir, tn);
+  }
+
+  /**
+   * Computes the size of each store file in {@code storeFileNames}
+   */
+  long getSizeOfStoreFiles(TableName tn, Set<StoreFileReference> storeFileNames) {
+    return storeFileNames.stream()
+        .collect(Collectors.summingLong((sfr) -> getSizeOfStoreFile(tn, sfr)));
+  }
+
+  /**
+   * Computes the size of the store files for a single region.
+   */
+  long getSizeOfStoreFile(TableName tn, StoreFileReference storeFileName) {
+    String regionName = storeFileName.getRegionName();
+    return storeFileName.getFamilyToFilesMapping()
+        .entries().stream()
+        .collect(Collectors.summingLong((e) ->
+            getSizeOfStoreFile(tn, regionName, e.getKey(), e.getValue())));
+  }
+
+  /**
+   * Computes the size of the store file given its name, region and family name in
+   * the archive directory.
+   */
+  long getSizeOfStoreFile(
+      TableName tn, String regionName, String family, String storeFile) {
+    Path familyArchivePath;
+    try {
+      familyArchivePath = HFileArchiveUtil.getStoreArchivePath(conf, tn, regionName, family);
+    } catch (IOException e) {
+      LOG.warn("Could not compute path for the archive directory for the region", e);
+      return 0L;
+    }
+    Path fileArchivePath = new Path(familyArchivePath, storeFile);
+    try {
+      if (fs.exists(fileArchivePath)) {
+        FileStatus[] status = fs.listStatus(fileArchivePath);
+        if (1 != status.length) {
+          LOG.warn("Expected " + fileArchivePath +
+              " to be a file but was a directory, ignoring reference");
+          return 0L;
+        }
+        return status[0].getLen();
+      }
+    } catch (IOException e) {
+      LOG.warn("Could not obtain the status of " + fileArchivePath, e);
+      return 0L;
+    }
+    LOG.warn("Expected " + fileArchivePath + " to exist but does not, ignoring reference.");
+    return 0L;
+  }
+
+  /**
+   * Writes the snapshot sizes to the {@code hbase:quota} table.
+   *
+   * @param snapshotsWithSize The snapshot sizes to write.
+   */
+  void persistSnapshotSizes(
+      Multimap<TableName,SnapshotWithSize> snapshotsWithSize) throws IOException {
+    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      // Write each snapshot size for the table
+      persistSnapshotSizes(quotaTable, snapshotsWithSize);
+      // Write a size entry for all snapshots in a namespace
+      persistSnapshotSizesByNS(quotaTable, snapshotsWithSize);
+    }
+  }
+
+  /**
+   * Writes the snapshot sizes to the provided {@code table}.
+   */
+  void persistSnapshotSizes(
+      Table table, Multimap<TableName,SnapshotWithSize> snapshotsWithSize) throws IOException {
+    // Convert each entry in the map to a Put and write them to the quota table
+    table.put(snapshotsWithSize.entries()
+        .stream()
+        .map(e -> QuotaTableUtil.createPutForSnapshotSize(
+            e.getKey(), e.getValue().getName(), e.getValue().getSize()))
+        .collect(Collectors.toList()));
+  }
+
+  /**
+   * Rolls up the snapshot sizes by namespace and writes a single record for each namespace
+   * which is the size of all snapshots in that namespace.
+   */
+  void persistSnapshotSizesByNS(
+      Table quotaTable, Multimap<TableName,SnapshotWithSize> snapshotsWithSize) throws IOException {
+    Map<String,Long> namespaceSnapshotSizes = groupSnapshotSizesByNamespace(snapshotsWithSize);
+    quotaTable.put(namespaceSnapshotSizes.entrySet().stream()
+        .map(e -> QuotaTableUtil.createPutForNamespaceSnapshotSize(
+            e.getKey(), e.getValue()))
+        .collect(Collectors.toList()));
+  }
+
+  /**
+   * Sums the snapshot sizes for each namespace.
+   */
+  Map<String,Long> groupSnapshotSizesByNamespace(
+      Multimap<TableName,SnapshotWithSize> snapshotsWithSize) {
+    return snapshotsWithSize.entries().stream()
+        .collect(Collectors.groupingBy(
+            // Convert TableName into the namespace string
+            (e) -> e.getKey().getNamespaceAsString(),
+            // Sum the values for namespace
+            Collectors.mapping(
+                Map.Entry::getValue, Collectors.summingLong((sws) -> sws.getSize()))));
+  }
+
+  /**
+   * A struct encapsulating the name of a snapshot and its "size" on the filesystem. This size is
+   * defined as the amount of filesystem space taken by the files the snapshot refers to which
+   * the originating table no longer refers to.
+   */
+  static class SnapshotWithSize {
+    private final String name;
+    private final long size;
+
+    SnapshotWithSize(String name, long size) {
+      this.name = Objects.requireNonNull(name);
+      this.size = size;
+    }
+
+    String getName() {
+      return name;
+    }
+
+    long getSize() {
+      return size;
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().append(name).append(size).toHashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+
+      if (!(o instanceof SnapshotWithSize)) {
+        return false;
+      }
+
+      SnapshotWithSize other = (SnapshotWithSize) o;
+      return name.equals(other.name) && size == other.size;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder(32);
+      return sb.append("SnapshotWithSize:[").append(name).append(" ")
+          .append(StringUtils.byteDesc(size)).append("]").toString();
+    }
+  }
+
+  /**
+   * A reference to a collection of files in the archive directory for a single region.
+   */
+  static class StoreFileReference {
+    private final String regionName;
+    private final Multimap<String,String> familyToFiles;
+
+    StoreFileReference(String regionName) {
+      this.regionName = Objects.requireNonNull(regionName);
+      familyToFiles = HashMultimap.create();
+    }
+
+    String getRegionName() {
+      return regionName;
+    }
+
+    Multimap<String,String> getFamilyToFilesMapping() {
+      return familyToFiles;
+    }
+
+    void addFamilyStoreFile(String family, String storeFileName) {
+      familyToFiles.put(family, storeFileName);
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().append(regionName).append(familyToFiles).toHashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof StoreFileReference)) {
+        return false;
+      }
+      StoreFileReference other = (StoreFileReference) o;
+      return regionName.equals(other.regionName) && familyToFiles.equals(other.familyToFiles);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      return sb.append("StoreFileReference[region=").append(regionName).append(", files=")
+          .append(familyToFiles).append("]").toString();
+    }
+  }
+
+  /**
+   * Extracts the period for the chore from the configuration.
+   *
+   * @param conf The configuration object.
+   * @return The configured chore period or the default value.
+   */
+  static int getPeriod(Configuration conf) {
+    return conf.getInt(SNAPSHOT_QUOTA_CHORE_PERIOD_KEY,
+        SNAPSHOT_QUOTA_CHORE_PERIOD_DEFAULT);
+  }
+
+  /**
+   * Extracts the initial delay for the chore from the configuration.
+   *
+   * @param conf The configuration object.
+   * @return The configured chore initial delay or the default value.
+   */
+  static long getInitialDelay(Configuration conf) {
+    return conf.getLong(SNAPSHOT_QUOTA_CHORE_DELAY_KEY,
+        SNAPSHOT_QUOTA_CHORE_DELAY_DEFAULT);
+  }
+
+  /**
+   * Extracts the time unit for the chore period and initial delay from the configuration. The
+   * configuration value for {@link #SNAPSHOT_QUOTA_CHORE_TIMEUNIT_KEY} must correspond to
+   * a {@link TimeUnit} value.
+   *
+   * @param conf The configuration object.
+   * @return The configured time unit for the chore period and initial delay or the default value.
+   */
+  static TimeUnit getTimeUnit(Configuration conf) {
+    return TimeUnit.valueOf(conf.get(SNAPSHOT_QUOTA_CHORE_TIMEUNIT_KEY,
+        SNAPSHOT_QUOTA_CHORE_TIMEUNIT_DEFAULT));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
index 970e6a9..27e4c73 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
@@ -18,17 +18,26 @@ package org.apache.hadoop.hbase.quotas;
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Objects;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
-import java.util.Map.Entry;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 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.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 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;
@@ -41,6 +50,8 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
  */
 @InterfaceAudience.Private
 public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
+  private static final Log LOG = LogFactory.getLog(TableQuotaSnapshotStore.class);
+
   private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
   private final ReadLock rlock = lock.readLock();
   private final WriteLock wlock = lock.writeLock();
@@ -77,7 +88,8 @@ public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
   }
 
   @Override
-  public SpaceQuotaSnapshot getTargetState(TableName table, SpaceQuota spaceQuota) {
+  public SpaceQuotaSnapshot getTargetState(
+      TableName table, SpaceQuota spaceQuota) throws IOException {
     rlock.lock();
     try {
       final long sizeLimitInBytes = spaceQuota.getSoftLimit();
@@ -85,6 +97,8 @@ public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
       for (Entry<HRegionInfo,Long> entry : filterBySubject(table)) {
         sum += entry.getValue();
       }
+      // Add in the size for any snapshots against this table
+      sum += getSnapshotSizesForTable(table);
       // Observance is defined as the size of the table being less than the limit
       SpaceQuotaStatus status = sum <= sizeLimitInBytes ? SpaceQuotaStatus.notInViolation()
           : new SpaceQuotaStatus(ProtobufUtil.toViolationPolicy(spaceQuota.getViolationPolicy()));
@@ -94,6 +108,42 @@ public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
     }
   }
 
+  /**
+   * Fetches any serialized snapshot sizes from the quota table for the {@code tn} provided. Any
+   * malformed records are skipped with a warning printed out.
+   */
+  long getSnapshotSizesForTable(TableName tn) throws IOException {
+    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      Scan s = QuotaTableUtil.createScanForSpaceSnapshotSizes(tn);
+      ResultScanner rs = quotaTable.getScanner(s);
+      try {
+        long size = 0L;
+        // Should just be a single row (for our table)
+        for (Result result : rs) {
+          // May have multiple columns, one for each snapshot
+          CellScanner cs = result.cellScanner();
+          while (cs.advance()) {
+            Cell current = cs.current();
+            try {
+              long snapshotSize = QuotaTableUtil.parseSnapshotSize(current);
+              if (LOG.isTraceEnabled()) {
+                LOG.trace("Saw snapshot size of " + snapshotSize + " for " + current);
+              }
+              size += snapshotSize;
+            } catch (InvalidProtocolBufferException e) {
+              LOG.warn("Failed to parse snapshot size from cell: " + current);
+            }
+          }
+        }
+        return size;
+      } finally {
+        if (null != rs) {
+          rs.close();
+        }
+      }
+    }
+  }
+
   @Override
   public Iterable<Entry<HRegionInfo,Long>> filterBySubject(TableName table) {
     rlock.lock();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaSnapshotNotifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaSnapshotNotifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaSnapshotNotifier.java
index 548faf8..f9813e5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaSnapshotNotifier.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaSnapshotNotifier.java
@@ -36,7 +36,7 @@ public class TableSpaceQuotaSnapshotNotifier implements SpaceQuotaSnapshotNotifi
   @Override
   public void transitionTable(
       TableName tableName, SpaceQuotaSnapshot snapshot) throws IOException {
-    final Put p = QuotaTableUtil.createPutSpaceSnapshot(tableName, snapshot);
+    final Put p = QuotaTableUtil.createPutForSpaceSnapshot(tableName, snapshot);
     try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
       if (LOG.isTraceEnabled()) {
         LOG.trace("Persisting a space quota snapshot " + snapshot + " for " + tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 917d183..2d9e0f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -41,6 +41,7 @@ import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -2073,6 +2074,17 @@ public class HStore implements Store {
 
   @Override
   public long getStorefilesSize() {
+    // Include all StoreFiles
+    return getStorefilesSize(storeFile -> true);
+  }
+
+  @Override
+  public long getHFilesSize() {
+    // Include only StoreFiles which are HFiles
+    return getStorefilesSize(storeFile -> storeFile.isHFile());
+  }
+
+  private long getStorefilesSize(Predicate<StoreFile> predicate) {
     long size = 0;
     for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
       StoreFileReader r = s.getReader();
@@ -2080,7 +2092,9 @@ public class HStore implements Store {
         LOG.warn("StoreFile " + s + " has a null Reader");
         continue;
       }
-      size += r.length();
+      if (predicate.test(s)) {
+        size += r.length();
+      }
     }
     return size;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
index e2fabae..fd9de9b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
@@ -430,6 +430,11 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
   long getStorefilesSize();
 
   /**
+   * @return The size of only the store files which are HFiles, in bytes.
+   */
+  long getHFilesSize();
+
+  /**
    * @return The size of the store file indexes, in bytes.
    */
   long getStorefilesIndexSize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
----------------------------------------------------------------------
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
index 9d56d60..bfd6a4b 100644
--- 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
@@ -20,6 +20,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.Objects;
@@ -29,9 +30,11 @@ 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.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.Predicate;
@@ -40,6 +43,10 @@ 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.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.rules.TestName;
 
@@ -55,6 +62,7 @@ public class SpaceQuotaHelperForTests {
   public static final String F1 = "f1";
   public static final long ONE_KILOBYTE = 1024L;
   public static final long ONE_MEGABYTE = ONE_KILOBYTE * ONE_KILOBYTE;
+  public static final long ONE_GIGABYTE = ONE_MEGABYTE * ONE_KILOBYTE;
 
   private final HBaseTestingUtility testUtil;
   private final TestName testName;
@@ -68,6 +76,25 @@ public class SpaceQuotaHelperForTests {
   }
 
   //
+  // Static helpers
+  //
+
+  static void updateConfigForQuotas(Configuration conf) {
+    // 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.QUOTA_OBSERVER_CHORE_DELAY_KEY, 1000);
+    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_PERIOD_KEY, 1000);
+    conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_DELAY_KEY, 1000);
+    conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_PERIOD_KEY, 1000);
+    conf.setInt(SnapshotQuotaObserverChore.SNAPSHOT_QUOTA_CHORE_DELAY_KEY, 1000);
+    conf.setInt(SnapshotQuotaObserverChore.SNAPSHOT_QUOTA_CHORE_PERIOD_KEY, 1000);
+    // The period at which we check for compacted files that should be deleted from HDFS
+    conf.setInt("hbase.hfile.compaction.discharger.interval", 5 * 1000);
+    conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+  }
+
+  //
   // Helpers
   //
 
@@ -88,24 +115,33 @@ public class SpaceQuotaHelperForTests {
   /**
    * Removes all quotas defined in the HBase quota table.
    */
-  void removeAllQuotas(Connection conn) throws IOException {
-    QuotaRetriever scanner = QuotaRetriever.open(conn.getConfiguration());
-    try {
-      for (QuotaSettings quotaSettings : scanner) {
-        final String namespace = quotaSettings.getNamespace();
-        final TableName tableName = quotaSettings.getTableName();
-        if (namespace != null) {
-          LOG.debug("Deleting quota for namespace: " + namespace);
-          QuotaUtil.deleteNamespaceQuota(conn, namespace);
-        } else {
-          assert tableName != null;
-          LOG.debug("Deleting quota for table: "+ tableName);
-          QuotaUtil.deleteTableQuota(conn, tableName);
+  void removeAllQuotas(Connection conn) throws IOException, InterruptedException {
+    // 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(conn.getConfiguration());
+      try {
+        for (QuotaSettings quotaSettings : scanner) {
+          final String namespace = quotaSettings.getNamespace();
+          final TableName tableName = quotaSettings.getTableName();
+          if (namespace != null) {
+            LOG.debug("Deleting quota for namespace: " + namespace);
+            QuotaUtil.deleteNamespaceQuota(conn, namespace);
+          } else {
+            assert tableName != null;
+            LOG.debug("Deleting quota for table: "+ tableName);
+            QuotaUtil.deleteTableQuota(conn, tableName);
+          }
+        }
+      } finally {
+        if (scanner != null) {
+          scanner.close();
         }
-      }
-    } finally {
-      if (scanner != null) {
-        scanner.close();
       }
     }
   }
@@ -146,6 +182,15 @@ public class SpaceQuotaHelperForTests {
   }
 
   void writeData(Connection conn, TableName tn, long sizeInBytes) throws IOException {
+    writeData(tn, sizeInBytes, Bytes.toBytes("q1"));
+  }
+
+  void writeData(TableName tn, long sizeInBytes, String qual) throws IOException {
+    writeData(tn, sizeInBytes, Bytes.toBytes(qual));
+  }
+
+  void writeData(TableName tn, long sizeInBytes, byte[] qual) throws IOException {
+    final Connection conn = testUtil.getConnection();
     final Table table = conn.getTable(tn);
     try {
       List<Put> updates = new ArrayList<>();
@@ -160,7 +205,7 @@ public class SpaceQuotaHelperForTests {
         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);
+        p.addColumn(Bytes.toBytes(F1), qual, value);
         updates.add(p);
 
         // Batch ~13KB worth of updates
@@ -188,6 +233,12 @@ public class SpaceQuotaHelperForTests {
     }
   }
 
+  NamespaceDescriptor createNamespace() throws Exception {
+    NamespaceDescriptor nd = NamespaceDescriptor.create("ns" + counter.getAndIncrement()).build();
+    testUtil.getAdmin().createNamespace(nd);
+    return nd;
+  }
+
   Multimap<TableName, QuotaSettings> createTablesWithSpaceQuotas() throws Exception {
     final Admin admin = testUtil.getAdmin();
     final Multimap<TableName, QuotaSettings> tablesWithQuotas = HashMultimap.create();
@@ -195,8 +246,7 @@ public class SpaceQuotaHelperForTests {
     final TableName tn1 = createTable();
     final TableName tn2 = createTable();
 
-    NamespaceDescriptor nd = NamespaceDescriptor.create("ns" + counter.getAndIncrement()).build();
-    admin.createNamespace(nd);
+    NamespaceDescriptor nd = createNamespace();
     final TableName tn3 = createTableInNamespace(nd);
     final TableName tn4 = createTableInNamespace(nd);
     final TableName tn5 = createTableInNamespace(nd);
@@ -233,6 +283,14 @@ public class SpaceQuotaHelperForTests {
     return tablesWithQuotas;
   }
 
+  TableName getNextTableName() {
+    return getNextTableName(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR);
+  }
+
+  TableName getNextTableName(String namespace) {
+    return TableName.valueOf(namespace, testName.getMethodName() + counter.getAndIncrement());
+  }
+
   TableName createTable() throws Exception {
     return createTableWithRegions(1);
   }
@@ -251,8 +309,7 @@ public class SpaceQuotaHelperForTests {
   }
 
   TableName createTableWithRegions(Admin admin, String namespace, int numRegions) throws Exception {
-    final TableName tn = TableName.valueOf(
-        namespace, testName.getMethodName() + counter.getAndIncrement());
+    final TableName tn = getNextTableName(namespace);
 
     // Delete the old table
     if (admin.tableExists(tn)) {
@@ -308,4 +365,87 @@ public class SpaceQuotaHelperForTests {
       }
     }
   }
+
+  /**
+   * Abstraction to simplify the case where a test needs to verify a certain state
+   * on a {@code SpaceQuotaSnapshot}. This class fails-fast when there is no such
+   * snapshot obtained from the Master. As such, it is not useful to verify the
+   * lack of a snapshot.
+   */
+  static abstract class SpaceQuotaSnapshotPredicate implements Predicate<Exception> {
+    private final Connection conn;
+    private final TableName tn;
+    private final String ns;
+
+    SpaceQuotaSnapshotPredicate(Connection conn, TableName tn) {
+      this(Objects.requireNonNull(conn), Objects.requireNonNull(tn), null);
+    }
+
+    SpaceQuotaSnapshotPredicate(Connection conn, String ns) {
+      this(Objects.requireNonNull(conn), null, Objects.requireNonNull(ns));
+    }
+
+    SpaceQuotaSnapshotPredicate(Connection conn, TableName tn, String ns) {
+      if ((null != tn && null != ns) || (null == tn && null == ns)) {
+        throw new IllegalArgumentException(
+            "One of TableName and Namespace must be non-null, and the other null");
+      }
+      this.conn = conn;
+      this.tn = tn;
+      this.ns = ns;
+    }
+
+    @Override
+    public boolean evaluate() throws Exception {
+      SpaceQuotaSnapshot snapshot;
+      if (null == ns) {
+        snapshot = QuotaTableUtil.getCurrentSnapshot(conn, tn);
+      } else {
+        snapshot = QuotaTableUtil.getCurrentSnapshot(conn, ns);
+      }
+
+      LOG.debug("Saw quota snapshot for " + (null == tn ? ns : tn) + ": " + snapshot);
+      if (null == snapshot) {
+        return false;
+      }
+      return evaluate(snapshot);
+    }
+
+    /**
+     * Must determine if the given {@code SpaceQuotaSnapshot} meets some criteria.
+     *
+     * @param snapshot a non-null snapshot obtained from the HBase Master
+     * @return true if the criteria is met, false otherwise
+     */
+    abstract boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception;
+  }
+
+  /**
+   * Predicate that waits for all store files in a table to have no compacted files.
+   */
+  static class NoFilesToDischarge implements Predicate<Exception> {
+    private final MiniHBaseCluster cluster;
+    private final TableName tn;
+
+    NoFilesToDischarge(MiniHBaseCluster cluster, TableName tn) {
+      this.cluster = cluster;
+      this.tn = tn;
+    }
+
+    @Override
+    public boolean evaluate() throws Exception {
+      for (HRegion region : cluster.getRegions(tn)) {
+        for (Store store : region.getStores()) {
+          HStore hstore = (HStore) store;
+          Collection<StoreFile> files =
+              hstore.getStoreEngine().getStoreFileManager().getCompactedfiles();
+          if (null != files && !files.isEmpty()) {
+            LOG.debug(region.getRegionInfo().getEncodedName() + " still has compacted files");
+            return false;
+          }
+        }
+      }
+      return true;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
index 18e47af..823b1f7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -238,13 +239,13 @@ public class TestFileSystemUtilizationChore {
     final Configuration conf = getDefaultHBaseConfiguration();
     final HRegionServer rs = mockRegionServer(conf);
 
-    // Three regions with multiple store sizes
+    // Two regions with multiple store sizes
     final List<Long> r1Sizes = Arrays.asList(1024L, 2048L);
     final long r1Sum = sum(r1Sizes);
     final List<Long> r2Sizes = Arrays.asList(1024L * 1024L);
 
     final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
-    doAnswer(new ExpectedRegionSizeSummationAnswer(sum(Arrays.asList(r1Sum))))
+    doAnswer(new ExpectedRegionSizeSummationAnswer(r1Sum))
         .when(rs)
         .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
 
@@ -254,6 +255,33 @@ public class TestFileSystemUtilizationChore {
     chore.chore();
   }
 
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testNonHFilesAreIgnored() {
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+
+    // Region r1 has two store files, one hfile link and one hfile
+    final List<Long> r1StoreFileSizes = Arrays.asList(1024L, 2048L);
+    final List<Long> r1HFileSizes = Arrays.asList(0L, 2048L);
+    final long r1HFileSizeSum = sum(r1HFileSizes);
+    // Region r2 has one store file which is a hfile link
+    final List<Long> r2StoreFileSizes = Arrays.asList(1024L * 1024L);
+    final List<Long> r2HFileSizes = Arrays.asList(0L);
+    final long r2HFileSizeSum = sum(r2HFileSizes);
+
+    // We expect that only the hfiles would be counted (hfile links are ignored)
+    final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
+    doAnswer(new ExpectedRegionSizeSummationAnswer(
+        sum(Arrays.asList(r1HFileSizeSum, r2HFileSizeSum))))
+        .when(rs).reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+
+    final Region r1 = mockRegionWithHFileLinks(r1StoreFileSizes, r1HFileSizes);
+    final Region r2 = mockRegionWithHFileLinks(r2StoreFileSizes, r2HFileSizes);
+    when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2));
+    chore.chore();
+  }
+
   /**
    * Creates an HBase Configuration object for the default values.
    */
@@ -300,7 +328,29 @@ public class TestFileSystemUtilizationChore {
     for (Long storeSize : storeSizes) {
       final Store s = mock(Store.class);
       stores.add(s);
+      when(s.getHFilesSize()).thenReturn(storeSize);
+    }
+    return r;
+  }
+
+  private Region mockRegionWithHFileLinks(Collection<Long> storeSizes, Collection<Long> hfileSizes) {
+    final Region r = mock(Region.class);
+    final HRegionInfo info = mock(HRegionInfo.class);
+    when(r.getRegionInfo()).thenReturn(info);
+    List<Store> stores = new ArrayList<>();
+    when(r.getStores()).thenReturn(stores);
+    assertEquals(
+        "Logic error, storeSizes and linkSizes must be equal in size", storeSizes.size(),
+        hfileSizes.size());
+    Iterator<Long> storeSizeIter = storeSizes.iterator();
+    Iterator<Long> hfileSizeIter = hfileSizes.iterator();
+    while (storeSizeIter.hasNext() && hfileSizeIter.hasNext()) {
+      final long storeSize = storeSizeIter.next();
+      final long hfileSize = hfileSizeIter.next();
+      final Store s = mock(Store.class);
+      stores.add(s);
       when(s.getStorefilesSize()).thenReturn(storeSize);
+      when(s.getHFilesSize()).thenReturn(hfileSize);
     }
     return r;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java
index 16bb7dd..a673bcb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java
@@ -23,6 +23,7 @@ import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicReference;
@@ -31,6 +32,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
@@ -91,7 +95,8 @@ public class TestNamespaceQuotaViolationStore {
   }
 
   @Test
-  public void testTargetViolationState() {
+  public void testTargetViolationState() throws IOException {
+    mockNoSnapshotSizes();
     final String NS = "ns";
     TableName tn1 = TableName.valueOf(NS, "tn1");
     TableName tn2 = TableName.valueOf(NS, "tn2");
@@ -123,7 +128,8 @@ public class TestNamespaceQuotaViolationStore {
 
     // Exceeds the quota, should be in violation
     assertEquals(true, store.getTargetState(NS, quota).getQuotaStatus().isInViolation());
-    assertEquals(SpaceViolationPolicy.DISABLE, store.getTargetState(NS, quota).getQuotaStatus().getPolicy());
+    assertEquals(
+        SpaceViolationPolicy.DISABLE, store.getTargetState(NS, quota).getQuotaStatus().getPolicy());
   }
 
   @Test
@@ -153,4 +159,9 @@ public class TestNamespaceQuotaViolationStore {
     assertEquals(18, size(store.filterBySubject("ns")));
   }
 
+  void mockNoSnapshotSizes() throws IOException {
+    Table quotaTable = mock(Table.class);
+    when(conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
+    when(quotaTable.get(any(Get.class))).thenReturn(new Result());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
index 1707168..62c6b53 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
@@ -65,11 +65,8 @@ public class TestQuotaObserverChoreRegionReports {
   @Before
   public void setUp() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_KEY, 1000);
-    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 1000);
-    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_DELAY_KEY, 1000);
-    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_PERIOD_KEY, 1000);
-    conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    // Increase the frequency of some of the chores for responsiveness of the test
+    SpaceQuotaHelperForTests.updateConfigForQuotas(conf);
     conf.setInt(QuotaObserverChore.REGION_REPORT_RETENTION_DURATION_KEY, 1000);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java
----------------------------------------------------------------------
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 b94333d..736be8d 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
@@ -77,11 +77,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
   @BeforeClass
   public static void setUp() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_KEY, 1000);
-    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 1000);
-    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_DELAY_KEY, 1000);
-    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_PERIOD_KEY, 1000);
-    conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    SpaceQuotaHelperForTests.updateConfigForQuotas(conf);
     conf.setClass(SpaceQuotaSnapshotNotifierFactory.SNAPSHOT_NOTIFIER_KEY,
         SpaceQuotaSnapshotNotifierForTest.class, SpaceQuotaSnapshotNotifier.class);
     TEST_UTIL.startMiniCluster(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f262548b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
index 2020e3c..aeae80a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
 import org.apache.hadoop.hbase.quotas.policies.MissingSnapshotViolationPolicyEnforcement;
@@ -65,13 +66,7 @@ public class TestQuotaStatusRPCs {
   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.QUOTA_OBSERVER_CHORE_DELAY_KEY, 1000);
-    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_PERIOD_KEY, 1000);
-    conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_DELAY_KEY, 1000);
-    conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_PERIOD_KEY, 1000);
-    conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    SpaceQuotaHelperForTests.updateConfigForQuotas(conf);
     TEST_UTIL.startMiniCluster(1);
   }
 
@@ -167,7 +162,7 @@ public class TestQuotaStatusRPCs {
     // Write at least `tableSize` data
     try {
       helper.writeData(tn, tableSize);
-    } catch (SpaceLimitingException e) {
+    } catch (RetriesExhaustedWithDetailsException | SpaceLimitingException e) {
       // Pass
     }
 
@@ -245,7 +240,7 @@ public class TestQuotaStatusRPCs {
 
     try {
       helper.writeData(tn, tableSize * 2L);
-    } catch (SpaceLimitingException e) {
+    } catch (RetriesExhaustedWithDetailsException | SpaceLimitingException e) {
       // Pass
     }