Repository: hbase
Updated Branches:
  refs/heads/master a6216db16 -> e5ea45705


http://git-wip-us.apache.org/repos/asf/hbase/blob/e5ea4570/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java
index ed8a2f3..8584d55 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java
@@ -67,9 +67,8 @@ public class TestRegionSizeUse {
   @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.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    // Increase the frequency of some of the chores for responsiveness of the 
test
+    SpaceQuotaHelperForTests.updateConfigForQuotas(conf);
     cluster = TEST_UTIL.startMiniCluster(2);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5ea4570/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSnapshotQuotaObserverChore.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSnapshotQuotaObserverChore.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSnapshotQuotaObserverChore.java
new file mode 100644
index 0000000..4022e3f
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSnapshotQuotaObserverChore.java
@@ -0,0 +1,368 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScanner;
+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.Get;
+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.SnapshotDescription;
+import org.apache.hadoop.hbase.client.SnapshotType;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.HMaster;
+import 
org.apache.hadoop.hbase.quotas.SnapshotQuotaObserverChore.SnapshotWithSize;
+import 
org.apache.hadoop.hbase.quotas.SpaceQuotaHelperForTests.NoFilesToDischarge;
+import 
org.apache.hadoop.hbase.quotas.SpaceQuotaHelperForTests.SpaceQuotaSnapshotPredicate;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+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;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Multimap;
+
+/**
+ * Test class for the {@link SnapshotQuotaObserverChore}.
+ */
+@Category(MediumTests.class)
+public class TestSnapshotQuotaObserverChore {
+  private static final Log LOG = 
LogFactory.getLog(TestSnapshotQuotaObserverChore.class);
+  private static final HBaseTestingUtility TEST_UTIL = new 
HBaseTestingUtility();
+  private static final AtomicLong COUNTER = new AtomicLong();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  private Connection conn;
+  private Admin admin;
+  private SpaceQuotaHelperForTests helper;
+  private HMaster master;
+  private SnapshotQuotaObserverChore testChore;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    SpaceQuotaHelperForTests.updateConfigForQuotas(conf);
+    // Clean up the compacted files faster than normal (15s instead of 2mins)
+    conf.setInt("hbase.hfile.compaction.discharger.interval", 15 * 1000);
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    conn = TEST_UTIL.getConnection();
+    admin = TEST_UTIL.getAdmin();
+    helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
+    master = TEST_UTIL.getHBaseCluster().getMaster();
+    helper.removeAllQuotas(conn);
+    testChore = new SnapshotQuotaObserverChore(
+        TEST_UTIL.getConnection(), TEST_UTIL.getConfiguration(), 
master.getFileSystem(), master,
+        null);
+  }
+
+  @Test
+  public void testSnapshotSizePersistence() throws IOException {
+    final Admin admin = TEST_UTIL.getAdmin();
+    final TableName tn = TableName.valueOf("quota_snapshotSizePersistence");
+    if (admin.tableExists(tn)) {
+      admin.disableTable(tn);
+      admin.deleteTable(tn);
+    }
+    HTableDescriptor desc = new HTableDescriptor(tn);
+    desc.addFamily(new HColumnDescriptor(QuotaTableUtil.QUOTA_FAMILY_USAGE));
+    admin.createTable(desc);
+
+    Multimap<TableName,SnapshotWithSize> snapshotsWithSizes = 
HashMultimap.create();
+    try (Table table = conn.getTable(tn)) {
+      // Writing no values will result in no records written.
+      verify(table, () -> {
+        testChore.persistSnapshotSizes(table, snapshotsWithSizes);
+        assertEquals(0, count(table));
+      });
+
+      verify(table, () -> {
+        TableName originatingTable = TableName.valueOf("t1");
+        snapshotsWithSizes.put(originatingTable, new SnapshotWithSize("ss1", 
1024L));
+        snapshotsWithSizes.put(originatingTable, new SnapshotWithSize("ss2", 
4096L));
+        testChore.persistSnapshotSizes(table, snapshotsWithSizes);
+        assertEquals(2, count(table));
+        assertEquals(1024L, extractSnapshotSize(table, originatingTable, 
"ss1"));
+        assertEquals(4096L, extractSnapshotSize(table, originatingTable, 
"ss2"));
+      });
+
+      snapshotsWithSizes.clear();
+      verify(table, () -> {
+        snapshotsWithSizes.put(TableName.valueOf("t1"), new 
SnapshotWithSize("ss1", 1024L));
+        snapshotsWithSizes.put(TableName.valueOf("t2"), new 
SnapshotWithSize("ss2", 4096L));
+        snapshotsWithSizes.put(TableName.valueOf("t3"), new 
SnapshotWithSize("ss3", 8192L));
+        testChore.persistSnapshotSizes(table, snapshotsWithSizes);
+        assertEquals(3, count(table));
+        assertEquals(1024L, extractSnapshotSize(table, 
TableName.valueOf("t1"), "ss1"));
+        assertEquals(4096L, extractSnapshotSize(table, 
TableName.valueOf("t2"), "ss2"));
+        assertEquals(8192L, extractSnapshotSize(table, 
TableName.valueOf("t3"), "ss3"));
+      });
+    }
+  }
+
+  @Test
+  public void testSnapshotsFromTables() throws Exception {
+    TableName tn1 = helper.createTableWithRegions(1);
+    TableName tn2 = helper.createTableWithRegions(1);
+    TableName tn3 = helper.createTableWithRegions(1);
+
+    // Set a space quota on table 1 and 2 (but not 3)
+    admin.setQuota(QuotaSettingsFactory.limitTableSpace(
+        tn1, SpaceQuotaHelperForTests.ONE_GIGABYTE, 
SpaceViolationPolicy.NO_INSERTS));
+    admin.setQuota(QuotaSettingsFactory.limitTableSpace(
+        tn2, SpaceQuotaHelperForTests.ONE_GIGABYTE, 
SpaceViolationPolicy.NO_INSERTS));
+
+    // Create snapshots on each table (we didn't write any data, so just 
skipflush)
+    admin.snapshot(new SnapshotDescription(tn1 + "snapshot", tn1, 
SnapshotType.SKIPFLUSH));
+    admin.snapshot(new SnapshotDescription(tn2 + "snapshot", tn2, 
SnapshotType.SKIPFLUSH));
+    admin.snapshot(new SnapshotDescription(tn3 + "snapshot", tn3, 
SnapshotType.SKIPFLUSH));
+
+    Multimap<TableName,String> mapping = testChore.getSnapshotsToComputeSize();
+    assertEquals(2, mapping.size());
+    assertEquals(1, mapping.get(tn1).size());
+    assertEquals(tn1 + "snapshot", mapping.get(tn1).iterator().next());
+    assertEquals(1, mapping.get(tn2).size());
+    assertEquals(tn2 + "snapshot", mapping.get(tn2).iterator().next());
+
+    admin.snapshot(new SnapshotDescription(tn2 + "snapshot1", tn2, 
SnapshotType.SKIPFLUSH));
+    admin.snapshot(new SnapshotDescription(tn3 + "snapshot1", tn3, 
SnapshotType.SKIPFLUSH));
+
+    mapping = testChore.getSnapshotsToComputeSize();
+    assertEquals(3, mapping.size());
+    assertEquals(1, mapping.get(tn1).size());
+    assertEquals(tn1 + "snapshot", mapping.get(tn1).iterator().next());
+    assertEquals(2, mapping.get(tn2).size());
+    assertEquals(
+        new HashSet<String>(Arrays.asList(tn2 + "snapshot", tn2 + 
"snapshot1")), mapping.get(tn2));
+  }
+
+  @Test
+  public void testSnapshotsFromNamespaces() throws Exception {
+    NamespaceDescriptor ns = 
NamespaceDescriptor.create("snapshots_from_namespaces").build();
+    admin.createNamespace(ns);
+
+    TableName tn1 = helper.createTableWithRegions(ns.getName(), 1);
+    TableName tn2 = helper.createTableWithRegions(ns.getName(), 1);
+    TableName tn3 = helper.createTableWithRegions(1);
+
+    // Set a space quota on the namespace
+    admin.setQuota(QuotaSettingsFactory.limitNamespaceSpace(
+        ns.getName(), SpaceQuotaHelperForTests.ONE_GIGABYTE, 
SpaceViolationPolicy.NO_INSERTS));
+
+    // Create snapshots on each table (we didn't write any data, so just 
skipflush)
+    admin.snapshot(new SnapshotDescription(
+        tn1.getQualifierAsString() + "snapshot", tn1, SnapshotType.SKIPFLUSH));
+    admin.snapshot(new SnapshotDescription(
+        tn2.getQualifierAsString() + "snapshot", tn2, SnapshotType.SKIPFLUSH));
+    admin.snapshot(new SnapshotDescription(
+        tn3.getQualifierAsString() + "snapshot", tn3, SnapshotType.SKIPFLUSH));
+
+    Multimap<TableName,String> mapping = testChore.getSnapshotsToComputeSize();
+    assertEquals(2, mapping.size());
+    assertEquals(1, mapping.get(tn1).size());
+    assertEquals(tn1.getQualifierAsString() + "snapshot", 
mapping.get(tn1).iterator().next());
+    assertEquals(1, mapping.get(tn2).size());
+    assertEquals(tn2.getQualifierAsString() + "snapshot", 
mapping.get(tn2).iterator().next());
+
+    admin.snapshot(new SnapshotDescription(
+        tn2.getQualifierAsString() + "snapshot1", tn2, 
SnapshotType.SKIPFLUSH));
+    admin.snapshot(new SnapshotDescription(
+        tn3.getQualifierAsString() + "snapshot2", tn3, 
SnapshotType.SKIPFLUSH));
+
+    mapping = testChore.getSnapshotsToComputeSize();
+    assertEquals(3, mapping.size());
+    assertEquals(1, mapping.get(tn1).size());
+    assertEquals(tn1.getQualifierAsString() + "snapshot", 
mapping.get(tn1).iterator().next());
+    assertEquals(2, mapping.get(tn2).size());
+    assertEquals(
+        new HashSet<String>(Arrays.asList(tn2.getQualifierAsString() + 
"snapshot",
+            tn2.getQualifierAsString() + "snapshot1")), mapping.get(tn2));
+  }
+
+  @Test
+  public void testSnapshotSize() throws Exception {
+    // Create a table and set a quota
+    TableName tn1 = helper.createTableWithRegions(5);
+    admin.setQuota(QuotaSettingsFactory.limitTableSpace(
+        tn1, SpaceQuotaHelperForTests.ONE_GIGABYTE, 
SpaceViolationPolicy.NO_INSERTS));
+
+    // Write some data and flush it
+    helper.writeData(tn1, 256L * SpaceQuotaHelperForTests.ONE_KILOBYTE);
+    admin.flush(tn1);
+
+    final AtomicReference<Long> lastSeenSize = new AtomicReference<>();
+    // Wait for the Master chore to run to see the usage (with a fudge factor)
+    TEST_UTIL.waitFor(30_000, new SpaceQuotaSnapshotPredicate(conn, tn1) {
+      @Override
+      boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
+        lastSeenSize.set(snapshot.getUsage());
+        return snapshot.getUsage() > 230L * 
SpaceQuotaHelperForTests.ONE_KILOBYTE;
+      }
+    });
+
+    // Create a snapshot on the table
+    final String snapshotName = tn1 + "snapshot";
+    admin.snapshot(new SnapshotDescription(snapshotName, tn1, 
SnapshotType.SKIPFLUSH));
+
+    // Get the snapshots
+    Multimap<TableName,String> snapshotsToCompute = 
testChore.getSnapshotsToComputeSize();
+    assertEquals(
+        "Expected to see the single snapshot: " + snapshotsToCompute, 1, 
snapshotsToCompute.size());
+
+    // Get the size of our snapshot
+    Multimap<TableName,SnapshotWithSize> snapshotsWithSize = 
testChore.computeSnapshotSizes(
+        snapshotsToCompute);
+    assertEquals(1, snapshotsWithSize.size());
+    SnapshotWithSize sws = 
Iterables.getOnlyElement(snapshotsWithSize.get(tn1));
+    assertEquals(snapshotName, sws.getName());
+    // The snapshot should take up no space since the table refers to it 
completely
+    assertEquals(0, sws.getSize());
+
+    // Write some more data, flush it, and then major_compact the table
+    helper.writeData(tn1, 256L * SpaceQuotaHelperForTests.ONE_KILOBYTE);
+    admin.flush(tn1);
+    TEST_UTIL.compact(tn1, true);
+
+    // Test table should reflect it's original size since ingest was 
deterministic
+    TEST_UTIL.waitFor(30_000, new SpaceQuotaSnapshotPredicate(conn, tn1) {
+      @Override
+      boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
+        LOG.debug("Current usage=" + snapshot.getUsage() + " lastSeenSize=" + 
lastSeenSize.get());
+        return closeInSize(
+            snapshot.getUsage(), lastSeenSize.get(), 
SpaceQuotaHelperForTests.ONE_KILOBYTE);
+      }
+    });
+
+    // Wait for no compacted files on the regions of our table
+    TEST_UTIL.waitFor(30_000, new 
NoFilesToDischarge(TEST_UTIL.getMiniHBaseCluster(), tn1));
+
+    // Still should see only one snapshot
+    snapshotsToCompute = testChore.getSnapshotsToComputeSize();
+    assertEquals(
+        "Expected to see the single snapshot: " + snapshotsToCompute, 1, 
snapshotsToCompute.size());
+    snapshotsWithSize = testChore.computeSnapshotSizes(
+            snapshotsToCompute);
+    assertEquals(1, snapshotsWithSize.size());
+    sws = Iterables.getOnlyElement(snapshotsWithSize.get(tn1));
+    assertEquals(snapshotName, sws.getName());
+    // The snapshot should take up the size the table originally took up
+    assertEquals(lastSeenSize.get().longValue(), sws.getSize());
+  }
+
+  @Test
+  public void testPersistingSnapshotsForNamespaces() throws Exception {
+    Multimap<TableName,SnapshotWithSize> snapshotsWithSizes = 
HashMultimap.create();
+    TableName tn1 = TableName.valueOf("ns1:tn1");
+    TableName tn2 = TableName.valueOf("ns1:tn2");
+    TableName tn3 = TableName.valueOf("ns2:tn1");
+    TableName tn4 = TableName.valueOf("ns2:tn2");
+    TableName tn5 = TableName.valueOf("tn1");
+
+    snapshotsWithSizes.put(tn1, new SnapshotWithSize("", 1024L));
+    snapshotsWithSizes.put(tn2, new SnapshotWithSize("", 1024L));
+    snapshotsWithSizes.put(tn3, new SnapshotWithSize("", 512L));
+    snapshotsWithSizes.put(tn4, new SnapshotWithSize("", 1024L));
+    snapshotsWithSizes.put(tn5, new SnapshotWithSize("", 3072L));
+
+    Map<String,Long> nsSizes = 
testChore.groupSnapshotSizesByNamespace(snapshotsWithSizes);
+    assertEquals(3, nsSizes.size());
+    assertEquals(2048L, (long) nsSizes.get("ns1"));
+    assertEquals(1536L, (long) nsSizes.get("ns2"));
+    assertEquals(3072L, (long) 
nsSizes.get(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR));
+  }
+
+  private long count(Table t) throws IOException {
+    try (ResultScanner rs = t.getScanner(new Scan())) {
+      long sum = 0;
+      for (Result r : rs) {
+        while (r.advance()) {
+          sum++;
+        }
+      }
+      return sum;
+    }
+  }
+
+  private long extractSnapshotSize(
+      Table quotaTable, TableName tn, String snapshot) throws IOException {
+    Get g = QuotaTableUtil.makeGetForSnapshotSize(tn, snapshot);
+    Result r = quotaTable.get(g);
+    assertNotNull(r);
+    CellScanner cs = r.cellScanner();
+    cs.advance();
+    Cell c = cs.current();
+    assertNotNull(c);
+    return QuotaTableUtil.extractSnapshotSize(
+        c.getValueArray(), c.getValueOffset(), c.getValueLength());
+  }
+
+  private void verify(Table t, IOThrowingRunnable test) throws IOException {
+    admin.disableTable(t.getName());
+    admin.truncateTable(t.getName(), false);
+    test.run();
+  }
+
+  @FunctionalInterface
+  private interface IOThrowingRunnable {
+    void run() throws IOException;
+  }
+
+  /**
+   * Computes if {@code size2} is within {@code delta} of {@code size1}, 
inclusive.
+   */
+  boolean closeInSize(long size1, long size2, long delta) {
+    long lower = size1 - delta;
+    long upper = size1 + delta;
+    return lower <= size2 && size2 <= upper;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5ea4570/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
----------------------------------------------------------------------
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
index e21647f..888ad9e 100644
--- 
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
@@ -79,6 +79,7 @@ import org.junit.rules.TestName;
 public class TestSpaceQuotas {
   private static final Log LOG = LogFactory.getLog(TestSpaceQuotas.class);
   private static final HBaseTestingUtility TEST_UTIL = new 
HBaseTestingUtility();
+  // Global for all tests in the class
   private static final AtomicLong COUNTER = new AtomicLong(0);
   private static final int NUM_RETRIES = 10;
 
@@ -89,14 +90,7 @@ public class TestSpaceQuotas {
   @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.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);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5ea4570/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java
new file mode 100644
index 0000000..ebb1a9e
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java
@@ -0,0 +1,448 @@
+/*
+ * 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.io.IOException;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+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.Cell;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.client.Admin;
+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.SnapshotType;
+import org.apache.hadoop.hbase.client.Table;
+import 
org.apache.hadoop.hbase.quotas.SpaceQuotaHelperForTests.SpaceQuotaSnapshotPredicate;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+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;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * Test class to exercise the inclusion of snapshots in space quotas
+ */
+@Category({LargeTests.class})
+public class TestSpaceQuotasWithSnapshots {
+  private static final Log LOG = 
LogFactory.getLog(TestSpaceQuotasWithSnapshots.class);
+  private static final HBaseTestingUtility TEST_UTIL = new 
HBaseTestingUtility();
+  // Global for all tests in the class
+  private static final AtomicLong COUNTER = new AtomicLong(0);
+  private static final long FUDGE_FOR_TABLE_SIZE = 500L * 
SpaceQuotaHelperForTests.ONE_KILOBYTE;
+
+  @Rule
+  public TestName testName = new TestName();
+  private SpaceQuotaHelperForTests helper;
+  private Connection conn;
+  private Admin admin;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    SpaceQuotaHelperForTests.updateConfigForQuotas(conf);
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void removeAllQuotas() throws Exception {
+    helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
+    conn = TEST_UTIL.getConnection();
+    admin = TEST_UTIL.getAdmin();
+  }
+
+  @Test
+  public void testTablesInheritSnapshotSize() throws Exception {
+    TableName tn = helper.createTableWithRegions(1);
+    LOG.info("Writing data");
+    // Set a quota
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, SpaceQuotaHelperForTests.ONE_GIGABYTE, 
SpaceViolationPolicy.NO_INSERTS);
+    admin.setQuota(settings);
+    // Write some data
+    final long initialSize = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    helper.writeData(tn, initialSize);
+
+    LOG.info("Waiting until table size reflects written data");
+    // Wait until that data is seen by the master
+    TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, 
tn) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        return snapshot.getUsage() >= initialSize;
+      }
+    });
+
+    // Make sure we see the final quota usage size
+    waitForStableQuotaSize(conn, tn, null);
+
+    // The actual size on disk after we wrote our data the first time
+    final long actualInitialSize = QuotaTableUtil.getCurrentSnapshot(conn, 
tn).getUsage();
+    LOG.info("Initial table size was " + actualInitialSize);
+
+    LOG.info("Snapshot the table");
+    final String snapshot1 = tn.toString() + "_snapshot1";
+    admin.snapshot(snapshot1, tn);
+
+    // Write the same data again, then flush+compact. This should make sure 
that
+    // the snapshot is referencing files that the table no longer references.
+    LOG.info("Write more data");
+    helper.writeData(tn, initialSize);
+    LOG.info("Flush the table");
+    admin.flush(tn);
+    LOG.info("Synchronously compacting the table");
+    TEST_UTIL.compact(tn, true);
+
+    final long upperBound = initialSize + FUDGE_FOR_TABLE_SIZE;
+    final long lowerBound = initialSize - FUDGE_FOR_TABLE_SIZE;
+
+    // Store the actual size after writing more data and then compacting it 
down to one file
+    LOG.info("Waiting for the region reports to reflect the correct size, 
between ("
+        + lowerBound + ", " + upperBound + ")");
+    TEST_UTIL.waitFor(30 * 1000, 500, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        long size = getRegionSizeReportForTable(conn, tn);
+        return size < upperBound && size > lowerBound;
+      }
+    });
+
+    // Make sure we see the "final" new size for the table, not some 
intermediate
+    waitForStableRegionSizeReport(conn, tn);
+    final long finalSize = getRegionSizeReportForTable(conn, tn);
+    assertNotNull("Did not expect to see a null size", finalSize);
+    LOG.info("Last seen size: " + finalSize);
+
+    // Make sure the QuotaObserverChore has time to reflect the new region 
size reports
+    // (we saw above). The usage of the table should *not* decrease when we 
check it below,
+    // though, because the snapshot on our table will cause the table to 
"retain" the size.
+    TEST_UTIL.waitFor(20 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, 
tn) {
+      @Override
+      public boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
+        return snapshot.getUsage() >= finalSize;
+      }
+    });
+
+    // The final usage should be the sum of the initial size (referenced by 
the snapshot) and the
+    // new size we just wrote above.
+    long expectedFinalSize = actualInitialSize + finalSize;
+    LOG.info(
+        "Expecting table usage to be " + actualInitialSize + " + " + finalSize
+        + " = " + expectedFinalSize);
+    // The size of the table (WRT quotas) should now be approximately double 
what it was previously
+    TEST_UTIL.waitFor(30 * 1000, 1000, new SpaceQuotaSnapshotPredicate(conn, 
tn) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        LOG.debug("Checking for " + expectedFinalSize + " == " + 
snapshot.getUsage());
+        return expectedFinalSize == snapshot.getUsage();
+      }
+    });
+  }
+
+  @Test
+  public void testNamespacesInheritSnapshotSize() throws Exception {
+    String ns = helper.createNamespace().getName();
+    TableName tn = helper.createTableWithRegions(ns, 1);
+    LOG.info("Writing data");
+    // Set a quota
+    QuotaSettings settings = QuotaSettingsFactory.limitNamespaceSpace(
+        ns, SpaceQuotaHelperForTests.ONE_GIGABYTE, 
SpaceViolationPolicy.NO_INSERTS);
+    admin.setQuota(settings);
+
+    // Write some data and flush it to disk
+    final long initialSize = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    helper.writeData(tn, initialSize);
+    admin.flush(tn);
+
+    LOG.info("Waiting until namespace size reflects written data");
+    // Wait until that data is seen by the master
+    TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, 
ns) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        return snapshot.getUsage() >= initialSize;
+      }
+    });
+
+    // Make sure we see the "final" new size for the table, not some 
intermediate
+    waitForStableQuotaSize(conn, null, ns);
+
+    // The actual size on disk after we wrote our data the first time
+    final long actualInitialSize = QuotaTableUtil.getCurrentSnapshot(conn, 
ns).getUsage();
+    LOG.info("Initial table size was " + actualInitialSize);
+
+    LOG.info("Snapshot the table");
+    final String snapshot1 = tn.getQualifierAsString() + "_snapshot1";
+    admin.snapshot(snapshot1, tn);
+
+    // Write the same data again, then flush+compact. This should make sure 
that
+    // the snapshot is referencing files that the table no longer references.
+    LOG.info("Write more data");
+    helper.writeData(tn, initialSize);
+    LOG.info("Flush the table");
+    admin.flush(tn);
+    LOG.info("Synchronously compacting the table");
+    TEST_UTIL.compact(tn, true);
+
+    final long upperBound = initialSize + FUDGE_FOR_TABLE_SIZE;
+    final long lowerBound = initialSize - FUDGE_FOR_TABLE_SIZE;
+
+    LOG.info("Waiting for the region reports to reflect the correct size, 
between ("
+        + lowerBound + ", " + upperBound + ")");
+    TEST_UTIL.waitFor(30 * 1000, 500, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        Map<TableName,Long> sizes = 
QuotaTableUtil.getMasterReportedTableSizes(conn);
+        LOG.debug("Master observed table sizes from region size reports: " + 
sizes);
+        Long size = sizes.get(tn);
+        if (null == size) {
+          return false;
+        }
+        return size < upperBound && size > lowerBound;
+      }
+    });
+
+    // Make sure we see the "final" new size for the table, not some 
intermediate
+    waitForStableRegionSizeReport(conn, tn);
+    final long finalSize = getRegionSizeReportForTable(conn, tn);
+    assertNotNull("Did not expect to see a null size", finalSize);
+    LOG.info("Final observed size of table: " + finalSize);
+
+    // Make sure the QuotaObserverChore has time to reflect the new region 
size reports
+    // (we saw above). The usage of the table should *not* decrease when we 
check it below,
+    // though, because the snapshot on our table will cause the table to 
"retain" the size.
+    TEST_UTIL.waitFor(20 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, 
ns) {
+      @Override
+      public boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
+        return snapshot.getUsage() >= finalSize;
+      }
+    });
+
+    // The final usage should be the sum of the initial size (referenced by 
the snapshot) and the
+    // new size we just wrote above.
+    long expectedFinalSize = actualInitialSize + finalSize;
+    LOG.info(
+        "Expecting namespace usage to be " + actualInitialSize + " + " + 
finalSize
+        + " = " + expectedFinalSize);
+    // The size of the table (WRT quotas) should now be approximately double 
what it was previously
+    TEST_UTIL.waitFor(30 * 1000, 1000, new SpaceQuotaSnapshotPredicate(conn, 
ns) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        LOG.debug("Checking for " + expectedFinalSize + " == " + 
snapshot.getUsage());
+        return expectedFinalSize == snapshot.getUsage();
+      }
+    });
+  }
+
+  @Test
+  public void testTablesWithSnapshots() throws Exception {
+    final Connection conn = TEST_UTIL.getConnection();
+    final SpaceViolationPolicy policy = SpaceViolationPolicy.NO_INSERTS;
+    final TableName tn = helper.createTableWithRegions(10);
+
+    // 3MB limit on the table
+    final long tableLimit = 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    TEST_UTIL.getAdmin().setQuota(QuotaSettingsFactory.limitTableSpace(tn, 
tableLimit, policy));
+
+    LOG.info("Writing first data set");
+    // Write more data than should be allowed and flush it to disk
+    helper.writeData(tn, 1L * SpaceQuotaHelperForTests.ONE_MEGABYTE, "q1");
+
+    LOG.info("Creating snapshot");
+    TEST_UTIL.getAdmin().snapshot(tn.toString() + "snap1", tn, 
SnapshotType.FLUSH);
+
+    LOG.info("Writing second data set");
+    // Write some more data
+    helper.writeData(tn, 1L * SpaceQuotaHelperForTests.ONE_MEGABYTE, "q2");
+
+    LOG.info("Flushing and major compacting table");
+    // Compact the table to force the snapshot to own all of its files
+    TEST_UTIL.getAdmin().flush(tn);
+    TEST_UTIL.compact(tn, true);
+
+    LOG.info("Checking for quota violation");
+    // Wait to observe the quota moving into violation
+    TEST_UTIL.waitFor(60_000, 1_000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        Scan s = QuotaTableUtil.makeQuotaSnapshotScanForTable(tn);
+        try (Table t = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+          ResultScanner rs = t.getScanner(s);
+          try {
+            Result r = Iterables.getOnlyElement(rs);
+            CellScanner cs = r.cellScanner();
+            assertTrue(cs.advance());
+            Cell c = cs.current();
+            SpaceQuotaSnapshot snapshot = 
SpaceQuotaSnapshot.toSpaceQuotaSnapshot(
+                QuotaProtos.SpaceQuotaSnapshot.parseFrom(
+                  UnsafeByteOperations.unsafeWrap(
+                      c.getValueArray(), c.getValueOffset(), 
c.getValueLength())));
+            LOG.info(
+                snapshot.getUsage() + "/" + snapshot.getLimit() + " " + 
snapshot.getQuotaStatus());
+            // We expect to see the table move to violation
+            return snapshot.getQuotaStatus().isInViolation();
+          } finally {
+            if (null != rs) {
+              rs.close();
+            }
+          }
+        }
+      }
+    });
+  }
+
+  @Test
+  public void testRematerializedTablesDoNoInheritSpace() throws Exception {
+    TableName tn = helper.createTableWithRegions(1);
+    TableName tn2 = helper.getNextTableName();
+    LOG.info("Writing data");
+    // Set a quota on both tables
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, SpaceQuotaHelperForTests.ONE_GIGABYTE, 
SpaceViolationPolicy.NO_INSERTS);
+    admin.setQuota(settings);
+    QuotaSettings settings2 = QuotaSettingsFactory.limitTableSpace(
+        tn2, SpaceQuotaHelperForTests.ONE_GIGABYTE, 
SpaceViolationPolicy.NO_INSERTS);
+    admin.setQuota(settings2);
+    // Write some data
+    final long initialSize = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    helper.writeData(tn, initialSize);
+
+    LOG.info("Waiting until table size reflects written data");
+    // Wait until that data is seen by the master
+    TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, 
tn) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        return snapshot.getUsage() >= initialSize;
+      }
+    });
+
+    // Make sure we see the final quota usage size
+    waitForStableQuotaSize(conn, tn, null);
+
+    // The actual size on disk after we wrote our data the first time
+    final long actualInitialSize = QuotaTableUtil.getCurrentSnapshot(conn, 
tn).getUsage();
+    LOG.info("Initial table size was " + actualInitialSize);
+
+    LOG.info("Snapshot the table");
+    final String snapshot1 = tn.toString() + "_snapshot1";
+    admin.snapshot(snapshot1, tn);
+
+    admin.cloneSnapshot(snapshot1, tn2);
+
+    // Write some more data to the first table
+    helper.writeData(tn, initialSize, "q2");
+    admin.flush(tn);
+
+    // Watch the usage of the first table with some more data to know when the 
new
+    // region size reports were sent to the master
+    TEST_UTIL.waitFor(30_000, 1_000, new SpaceQuotaSnapshotPredicate(conn, tn) 
{
+      @Override
+      boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
+        return snapshot.getUsage() >= actualInitialSize * 2;
+      }
+    });
+
+    // We know that reports were sent by our RS, verify that they take up zero 
size.
+    SpaceQuotaSnapshot snapshot = QuotaTableUtil.getCurrentSnapshot(conn, tn2);
+    assertNotNull(snapshot);
+    assertEquals(0, snapshot.getUsage());
+
+    // Compact the cloned table to force it to own its own files.
+    TEST_UTIL.compact(tn2, true);
+    // After the table is compacted, it should have its own files and be the 
same size as originally
+    TEST_UTIL.waitFor(30_000, 1_000, new SpaceQuotaSnapshotPredicate(conn, 
tn2) {
+      @Override
+      boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
+        return snapshot.getUsage() == actualInitialSize;
+      }
+    });
+  }
+
+  void waitForStableQuotaSize(Connection conn, TableName tn, String ns) throws 
Exception {
+    // For some stability in the value before proceeding
+    // Helps make sure that we got the actual last value, not some inbetween
+    AtomicLong lastValue = new AtomicLong(-1);
+    AtomicInteger counter = new AtomicInteger(0);
+    TEST_UTIL.waitFor(15_000, 500, new SpaceQuotaSnapshotPredicate(conn, tn, 
ns) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        LOG.debug("Last observed size=" + lastValue.get());
+        if (snapshot.getUsage() == lastValue.get()) {
+          int numMatches = counter.incrementAndGet();
+          if (numMatches >= 5) {
+            return true;
+          }
+          // Not yet..
+          return false;
+        }
+        counter.set(0);
+        lastValue.set(snapshot.getUsage());
+        return false;
+      }
+    });
+  }
+
+  long getRegionSizeReportForTable(Connection conn, TableName tn) throws 
IOException {
+    Map<TableName,Long> sizes = 
QuotaTableUtil.getMasterReportedTableSizes(conn);
+    Long value = sizes.get(tn);
+    if (null == value) {
+      return 0L;
+    }
+    return value.longValue();
+  }
+
+  void waitForStableRegionSizeReport(Connection conn, TableName tn) throws 
Exception {
+    // For some stability in the value before proceeding
+    // Helps make sure that we got the actual last value, not some inbetween
+    AtomicLong lastValue = new AtomicLong(-1);
+    AtomicInteger counter = new AtomicInteger(0);
+    TEST_UTIL.waitFor(15_000, 500, new Predicate<Exception>() {
+      @Override public boolean evaluate() throws Exception {
+        LOG.debug("Last observed size=" + lastValue.get());
+        long actual = getRegionSizeReportForTable(conn, tn);
+        if (actual == lastValue.get()) {
+          int numMatches = counter.incrementAndGet();
+          if (numMatches >= 5) {
+            return true;
+          }
+          // Not yet..
+          return false;
+        }
+        counter.set(0);
+        lastValue.set(actual);
+        return false;
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5ea4570/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java
index 812e7e0..3a60cbb 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java
@@ -76,13 +76,7 @@ public class TestSuperUserQuotaPermissions {
   public static void setupMiniCluster() 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);
 
     conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, 
AccessController.class.getName());
     conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, 
AccessController.class.getName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5ea4570/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java
index cefed67..5a4969a 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java
@@ -23,6 +23,8 @@ 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.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicReference;
@@ -30,6 +32,10 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.hadoop.hbase.HRegionInfo;
 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.SpaceQuotaSnapshot.SpaceQuotaStatus;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
@@ -87,7 +93,8 @@ public class TestTableQuotaViolationStore {
   }
 
   @Test
-  public void testTargetViolationState() {
+  public void testTargetViolationState() throws IOException {
+    mockNoSnapshotSizes();
     TableName tn1 = TableName.valueOf("violation1");
     TableName tn2 = TableName.valueOf("observance1");
     TableName tn3 = TableName.valueOf("observance2");
@@ -154,4 +161,12 @@ public class TestTableQuotaViolationStore {
     quotaRef.set(quotaWithoutSpace);
     assertNull(mockStore.getSpaceQuota(TableName.valueOf("foo")));
   }
+
+  void mockNoSnapshotSizes() throws IOException {
+    Table quotaTable = mock(Table.class);
+    ResultScanner scanner = mock(ResultScanner.class);
+    
when(conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
+    when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
+    when(scanner.iterator()).thenReturn(Collections.<Result> 
emptyList().iterator());
+  }
 }

Reply via email to