Repository: hbase
Updated Branches:
  refs/heads/master 393ab302a -> bdedcc563


http://git-wip-us.apache.org/repos/asf/hbase/blob/bdedcc56/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestLowLatencySpaceQuotas.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestLowLatencySpaceQuotas.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestLowLatencySpaceQuotas.java
new file mode 100644
index 0000000..f90ed82
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestLowLatencySpaceQuotas.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+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.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ClientServiceCallable;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RpcRetryingCaller;
+import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
+import 
org.apache.hadoop.hbase.quotas.SpaceQuotaHelperForTests.SpaceQuotaSnapshotPredicate;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+
+@Category({MediumTests.class})
+public class TestLowLatencySpaceQuotas {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestLowLatencySpaceQuotas.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new 
HBaseTestingUtility();
+  // Global for all tests in the class
+  private static final AtomicLong COUNTER = new AtomicLong(0);
+
+  @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();
+    // The default 1s period for QuotaObserverChore is good.
+    SpaceQuotaHelperForTests.updateConfigForQuotas(conf);
+    // Set the period to read region size from HDFS to be very long
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 
1000 * 120);
+    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();
+    helper.waitForQuotaTable(conn);
+  }
+
+  @Test
+  public void testFlushes() throws Exception {
+    TableName tn = helper.createTableWithRegions(1);
+    // 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);
+
+    // Make sure a flush happened
+    admin.flush(tn);
+
+    // We should be able to observe the system recording an increase in size 
(even
+    // though we know the filesystem scanning did not happen).
+    TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, 
tn) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        return snapshot.getUsage() >= initialSize;
+      }
+    });
+  }
+
+  @Test
+  public void testMajorCompaction() throws Exception {
+    TableName tn = helper.createTableWithRegions(1);
+    // Set a quota
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, SpaceQuotaHelperForTests.ONE_GIGABYTE, 
SpaceViolationPolicy.NO_INSERTS);
+    admin.setQuota(settings);
+
+    // Write some data and flush it to disk.
+    final long sizePerBatch = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    helper.writeData(tn, sizePerBatch);
+    admin.flush(tn);
+
+    // Write the same data again, flushing it to a second file
+    helper.writeData(tn, sizePerBatch);
+    admin.flush(tn);
+
+    // After two flushes, both hfiles would contain similar data. We should 
see 2x the data.
+    TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, 
tn) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        return snapshot.getUsage() >= 2L * sizePerBatch;
+      }
+    });
+
+    // Rewrite the two files into one.
+    admin.majorCompact(tn);
+
+    // After we major compact the table, we should notice quickly that the 
amount of data in the
+    // table is much closer to reality (the duplicate entries across the two 
files are removed).
+    TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, 
tn) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        return snapshot.getUsage() >= sizePerBatch && snapshot.getUsage() <= 
2L * sizePerBatch;
+      }
+    });
+  }
+
+  @Test
+  public void testMinorCompaction() throws Exception {
+    TableName tn = helper.createTableWithRegions(1);
+    // Set a quota
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, SpaceQuotaHelperForTests.ONE_GIGABYTE, 
SpaceViolationPolicy.NO_INSERTS);
+    admin.setQuota(settings);
+
+    // Write some data and flush it to disk.
+    final long sizePerBatch = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    final long numBatches = 6;
+    for (long i = 0; i < numBatches; i++) {
+      helper.writeData(tn, sizePerBatch);
+      admin.flush(tn);
+    }
+
+    HRegion region = 
Iterables.getOnlyElement(TEST_UTIL.getHBaseCluster().getRegions(tn));
+    long numFiles = getNumHFilesForRegion(region);
+    assertEquals(numBatches, numFiles);
+
+    // After two flushes, both hfiles would contain similar data. We should 
see 2x the data.
+    TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, 
tn) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        return snapshot.getUsage() >= numFiles * sizePerBatch;
+      }
+    });
+
+    // Rewrite some files into fewer
+    TEST_UTIL.compact(tn, false);
+    long numFilesAfterMinorCompaction = getNumHFilesForRegion(region);
+
+    // After we major compact the table, we should notice quickly that the 
amount of data in the
+    // table is much closer to reality (the duplicate entries across the two 
files are removed).
+    TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, 
tn) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        return snapshot.getUsage() >= numFilesAfterMinorCompaction * 
sizePerBatch &&
+            snapshot.getUsage() <= (numFilesAfterMinorCompaction + 1) * 
sizePerBatch;
+      }
+    });
+  }
+
+  private long getNumHFilesForRegion(HRegion region) {
+    return region.getStores().stream().mapToLong((s) -> 
s.getNumHFiles()).sum();
+  }
+
+  @Test
+  public void testBulkLoading() throws Exception {
+    TableName tn = helper.createTableWithRegions(1);
+    // Set a quota
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, SpaceQuotaHelperForTests.ONE_GIGABYTE, 
SpaceViolationPolicy.NO_INSERTS);
+    admin.setQuota(settings);
+
+    ClientServiceCallable<Boolean> callable = helper.generateFileToLoad(tn, 3, 
550);
+    // Make sure the files are about as long as we expect
+    FileSystem fs = TEST_UTIL.getTestFileSystem();
+    FileStatus[] files = fs.listStatus(
+        new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files"));
+    long totalSize = 0;
+    for (FileStatus file : files) {
+      assertTrue(
+          "Expected the file, " + file.getPath() + ",  length to be larger 
than 25KB, but was "
+              + file.getLen(),
+          file.getLen() > 25 * SpaceQuotaHelperForTests.ONE_KILOBYTE);
+      totalSize += file.getLen();
+    }
+
+    RpcRetryingCallerFactory factory = new 
RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
+    RpcRetryingCaller<Boolean> caller = factory.<Boolean> newCaller();
+    assertTrue("The bulk load failed", caller.callWithRetries(callable, 
Integer.MAX_VALUE));
+
+    final long finalTotalSize = totalSize;
+    TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, 
tn) {
+      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception 
{
+        return snapshot.getUsage() >= finalTotalSize;
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdedcc56/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 debd54c..7391fa1 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
@@ -29,17 +29,18 @@ import java.util.Map.Entry;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -86,8 +87,8 @@ public class TestQuotaObserverChoreRegionReports {
   @Test
   public void testReportExpiration() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
-    // Send reports every 30 seconds
-    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 
25000);
+    // Send reports every 25 seconds
+    
conf.setInt(RegionSizeReportingChore.REGION_SIZE_REPORTING_CHORE_PERIOD_KEY, 
25000);
     // Expire the reports after 5 seconds
     conf.setInt(QuotaObserverChore.REGION_REPORT_RETENTION_DURATION_KEY, 5000);
     TEST_UTIL.startMiniCluster(1);
@@ -103,8 +104,8 @@ public class TestQuotaObserverChoreRegionReports {
 
     // Create a table
     final TableName tn = TableName.valueOf("reportExpiration");
-    HTableDescriptor tableDesc = new HTableDescriptor(tn);
-    tableDesc.addFamily(new HColumnDescriptor(FAM1));
+    TableDescriptor tableDesc = 
TableDescriptorBuilder.newBuilder(tn).addColumnFamily(
+        ColumnFamilyDescriptorBuilder.of(FAM1)).build();
     TEST_UTIL.getAdmin().createTable(tableDesc);
 
     // No reports right after we created this table.
@@ -148,8 +149,8 @@ public class TestQuotaObserverChoreRegionReports {
 
     // Create a table
     final TableName tn = TableName.valueOf("quotaAcceptanceWithoutReports");
-    HTableDescriptor tableDesc = new HTableDescriptor(tn);
-    tableDesc.addFamily(new HColumnDescriptor(FAM1));
+    TableDescriptor tableDesc = 
TableDescriptorBuilder.newBuilder(tn).addColumnFamily(
+        ColumnFamilyDescriptorBuilder.of(FAM1)).build();
     TEST_UTIL.getAdmin().createTable(tableDesc);
 
     // Set a quota

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdedcc56/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeImpl.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeImpl.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeImpl.java
new file mode 100644
index 0000000..9217762
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeImpl.java
@@ -0,0 +1,49 @@
+/*
+ * 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 org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({SmallTests.class})
+public class TestRegionSizeImpl {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestRegionSizeImpl.class);
+
+  @Test
+  public void testReportingWithSizeChanges() {
+    long currentSize = 1024L;
+    RegionSizeImpl size = new RegionSizeImpl(currentSize);
+
+    assertEquals(currentSize, size.getSize());
+
+    currentSize *= 2L;
+    size.setSize(currentSize);
+    assertEquals(currentSize, size.getSize());
+
+    long delta = 512L;
+    currentSize += delta;
+    size.incrementSize(delta);
+    assertEquals(currentSize, size.getSize());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdedcc56/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeReportingChore.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeReportingChore.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeReportingChore.java
new file mode 100644
index 0000000..6541cdc
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeReportingChore.java
@@ -0,0 +1,127 @@
+/*
+ * 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 static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({SmallTests.class})
+public class TestRegionSizeReportingChore {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestRegionSizeReportingChore.class);
+
+  @Test
+  public void testDefaultConfigurationProperties() {
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+    RegionSizeReportingChore chore = new RegionSizeReportingChore(rs);
+    assertEquals(
+        RegionSizeReportingChore.REGION_SIZE_REPORTING_CHORE_DELAY_DEFAULT,
+        chore.getInitialDelay());
+    assertEquals(
+        RegionSizeReportingChore.REGION_SIZE_REPORTING_CHORE_PERIOD_DEFAULT, 
chore.getPeriod());
+    assertEquals(
+        
TimeUnit.valueOf(RegionSizeReportingChore.REGION_SIZE_REPORTING_CHORE_TIMEUNIT_DEFAULT),
+        chore.getTimeUnit());
+  }
+
+  @Test
+  public void testNonDefaultConfigurationProperties() {
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+    final int period = 
RegionSizeReportingChore.REGION_SIZE_REPORTING_CHORE_PERIOD_DEFAULT + 1;
+    final long delay = 
RegionSizeReportingChore.REGION_SIZE_REPORTING_CHORE_DELAY_DEFAULT + 1L;
+    final String timeUnit = TimeUnit.SECONDS.name();
+    
conf.setInt(RegionSizeReportingChore.REGION_SIZE_REPORTING_CHORE_PERIOD_KEY, 
period);
+    
conf.setLong(RegionSizeReportingChore.REGION_SIZE_REPORTING_CHORE_DELAY_KEY, 
delay);
+    
conf.set(RegionSizeReportingChore.REGION_SIZE_REPORTING_CHORE_TIMEUNIT_KEY, 
timeUnit);
+    RegionSizeReportingChore chore = new RegionSizeReportingChore(rs);
+    assertEquals(delay, chore.getInitialDelay());
+    assertEquals(period, chore.getPeriod());
+    assertEquals(TimeUnit.valueOf(timeUnit), chore.getTimeUnit());
+  }
+
+  @Test
+  public void testRemovableOfNonOnlineRegions() {
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+    RegionSizeReportingChore chore = new RegionSizeReportingChore(rs);
+
+    RegionInfo infoA = RegionInfoBuilder.newBuilder(TableName.valueOf("T1"))
+        .setStartKey(Bytes.toBytes("a")).setEndKey(Bytes.toBytes("b")).build();
+    RegionInfo infoB = RegionInfoBuilder.newBuilder(TableName.valueOf("T1"))
+        .setStartKey(Bytes.toBytes("b")).setEndKey(Bytes.toBytes("d")).build();
+    RegionInfo infoC = RegionInfoBuilder.newBuilder(TableName.valueOf("T1"))
+        .setStartKey(Bytes.toBytes("c")).setEndKey(Bytes.toBytes("d")).build();
+
+    RegionSizeStore store = new RegionSizeStoreImpl();
+    store.put(infoA, 1024L);
+    store.put(infoB, 1024L);
+    store.put(infoC, 1024L);
+
+    // If there are no online regions, all entries should be removed.
+    chore.removeNonOnlineRegions(store, Collections.<RegionInfo> emptySet());
+    assertTrue(store.isEmpty());
+
+    store.put(infoA, 1024L);
+    store.put(infoB, 1024L);
+    store.put(infoC, 1024L);
+
+    // Remove a single region
+    chore.removeNonOnlineRegions(store, new HashSet<>(Arrays.asList(infoA, 
infoC)));
+    assertEquals(2, store.size());
+    assertNotNull(store.getRegionSize(infoA));
+    assertNotNull(store.getRegionSize(infoC));
+  }
+
+  /**
+   * Creates an HBase Configuration object for the default values.
+   */
+  private Configuration getDefaultHBaseConfiguration() {
+    final Configuration conf = HBaseConfiguration.create();
+    conf.addResource("hbase-default.xml");
+    return conf;
+  }
+
+  private HRegionServer mockRegionServer(Configuration conf) {
+    HRegionServer rs = mock(HRegionServer.class);
+    when(rs.getConfiguration()).thenReturn(conf);
+    return rs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdedcc56/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeStoreImpl.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeStoreImpl.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeStoreImpl.java
new file mode 100644
index 0000000..688fde0
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeStoreImpl.java
@@ -0,0 +1,101 @@
+/*
+ * 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.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({SmallTests.class})
+public class TestRegionSizeStoreImpl {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestRegionSizeStoreImpl.class);
+
+  private static final RegionInfo INFOA = 
RegionInfoBuilder.newBuilder(TableName.valueOf("TEST"))
+      .setStartKey(Bytes.toBytes("a")).setEndKey(Bytes.toBytes("b")).build();
+  private static final RegionInfo INFOB = 
RegionInfoBuilder.newBuilder(TableName.valueOf("TEST"))
+      .setStartKey(Bytes.toBytes("b")).setEndKey(Bytes.toBytes("c")).build();
+
+  @Test
+  public void testSizeUpdates() {
+    RegionSizeStore store = new RegionSizeStoreImpl();
+    assertTrue(store.isEmpty());
+    assertEquals(0, store.size());
+
+    store.put(INFOA, 1024L);
+
+    assertFalse(store.isEmpty());
+    assertEquals(1, store.size());
+    assertEquals(1024L, store.getRegionSize(INFOA).getSize());
+
+    store.put(INFOA, 2048L);
+    assertEquals(1, store.size());
+    assertEquals(2048L, store.getRegionSize(INFOA).getSize());
+
+    store.incrementRegionSize(INFOA, 512L);
+    assertEquals(1, store.size());
+    assertEquals(2048L + 512L, store.getRegionSize(INFOA).getSize());
+
+    store.remove(INFOA);
+    assertTrue(store.isEmpty());
+    assertEquals(0, store.size());
+
+    store.put(INFOA, 64L);
+    store.put(INFOB, 128L);
+
+    assertEquals(2, store.size());
+    Map<RegionInfo,RegionSize> records = new HashMap<>();
+    for (Entry<RegionInfo,RegionSize> entry : store) {
+      records.put(entry.getKey(), entry.getValue());
+    }
+
+    assertEquals(64L, records.remove(INFOA).getSize());
+    assertEquals(128L, records.remove(INFOB).getSize());
+    assertTrue(records.isEmpty());
+  }
+
+  @Test
+  public void testNegativeDeltaForMissingRegion() {
+    RegionSizeStore store = new RegionSizeStoreImpl();
+
+    assertNull(store.getRegionSize(INFOA));
+
+    // We shouldn't allow a negative size to enter the RegionSizeStore. 
Getting a negative size
+    // like this shouldn't be possible, but we can prevent the bad state from 
propagating and
+    // getting worse.
+    store.incrementRegionSize(INFOA, -5);
+    assertNotNull(store.getRegionSize(INFOA));
+    assertEquals(0, store.getRegionSize(INFOA).getSize());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdedcc56/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 ae71b31..9b6a1b8 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
@@ -23,9 +23,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicLong;
@@ -36,7 +34,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
@@ -52,17 +49,13 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.master.HMaster;
 import 
org.apache.hadoop.hbase.quotas.policies.DefaultViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -247,9 +240,9 @@ public class TestSpaceQuotas {
     TableName tableName = 
writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, p);
 
     // The table is now in violation. Try to do a bulk load
-    ClientServiceCallable<Void> callable = generateFileToLoad(tableName, 1, 
50);
+    ClientServiceCallable<Boolean> callable = 
helper.generateFileToLoad(tableName, 1, 50);
     RpcRetryingCallerFactory factory = new 
RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
-    RpcRetryingCaller<Void> caller = factory.<Void> newCaller();
+    RpcRetryingCaller<Boolean> caller = factory.newCaller();
     try {
       caller.callWithRetries(callable, Integer.MAX_VALUE);
       fail("Expected the bulk load call to fail!");
@@ -298,7 +291,7 @@ public class TestSpaceQuotas {
         enforcement instanceof DefaultViolationPolicyEnforcement);
 
     // Should generate two files, each of which is over 25KB each
-    ClientServiceCallable<Void> callable = generateFileToLoad(tn, 2, 500);
+    ClientServiceCallable<Boolean> callable = helper.generateFileToLoad(tn, 2, 
525);
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     FileStatus[] files = fs.listStatus(
         new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files"));
@@ -311,7 +304,7 @@ public class TestSpaceQuotas {
     }
 
     RpcRetryingCallerFactory factory = new 
RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
-    RpcRetryingCaller<Void> caller = factory.<Void> newCaller();
+    RpcRetryingCaller<Boolean> caller = factory.newCaller();
     try {
       caller.callWithRetries(callable, Integer.MAX_VALUE);
       fail("Expected the bulk load call to fail!");
@@ -432,39 +425,4 @@ public class TestSpaceQuotas {
     assertTrue(
         "Expected to see an exception writing data to a table exceeding its 
quota", sawError);
   }
-
-  private ClientServiceCallable<Void> generateFileToLoad(
-      TableName tn, int numFiles, int numRowsPerFile) throws Exception {
-    Connection conn = TEST_UTIL.getConnection();
-    FileSystem fs = TEST_UTIL.getTestFileSystem();
-    Configuration conf = TEST_UTIL.getConfiguration();
-    Path baseDir = new Path(fs.getHomeDirectory(), testName.getMethodName() + 
"_files");
-    fs.mkdirs(baseDir);
-    final List<Pair<byte[], String>> famPaths = new ArrayList<Pair<byte[], 
String>>();
-    for (int i = 1; i <= numFiles; i++) {
-      Path hfile = new Path(baseDir, "file" + i);
-      TestHRegionServerBulkLoad.createHFile(
-          fs, hfile, Bytes.toBytes(SpaceQuotaHelperForTests.F1), 
Bytes.toBytes("to"),
-          Bytes.toBytes("reject"), numRowsPerFile);
-      famPaths.add(new Pair<>(Bytes.toBytes(SpaceQuotaHelperForTests.F1), 
hfile.toString()));
-    }
-
-    // bulk load HFiles
-    Table table = conn.getTable(tn);
-    final String bulkToken = new SecureBulkLoadClient(conf, 
table).prepareBulkLoad(conn);
-    return new ClientServiceCallable<Void>(conn,
-        tn, Bytes.toBytes("row"), new 
RpcControllerFactory(conf).newController(), HConstants.PRIORITY_UNSET) {
-      @Override
-      public Void rpcCall() throws Exception {
-        SecureBulkLoadClient secureClient = null;
-        byte[] regionName = getLocation().getRegionInfo().getRegionName();
-        try (Table table = conn.getTable(getTableName())) {
-          secureClient = new SecureBulkLoadClient(conf, table);
-          secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
-                true, null, bulkToken);
-        }
-        return null;
-      }
-    };
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdedcc56/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java
index 3628738..4995de7 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java
@@ -78,7 +78,7 @@ public class TestBulkLoadCheckingViolationPolicyEnforcement {
 
     policy.initialize(rss, tableName, snapshot);
 
-    policy.checkBulkLoad(fs, paths);
+    policy.computeBulkLoadSize(fs, paths);
   }
 
   @Test(expected = IllegalArgumentException.class)
@@ -97,7 +97,7 @@ public class TestBulkLoadCheckingViolationPolicyEnforcement {
     policy.initialize(rss, tableName, snapshot);
 
     // If the file to bulk load isn't a file, this should throw an exception
-    policy.checkBulkLoad(fs, paths);
+    policy.computeBulkLoadSize(fs, paths);
   }
 
   @Test(expected = SpaceLimitingException.class)
@@ -120,7 +120,7 @@ public class TestBulkLoadCheckingViolationPolicyEnforcement 
{
 
     policy.initialize(rss, tableName, snapshot);
 
-    policy.checkBulkLoad(fs, paths);
+    policy.computeBulkLoadSize(fs, paths);
   }
 
   @Test(expected = SpaceLimitingException.class)
@@ -143,6 +143,6 @@ public class TestBulkLoadCheckingViolationPolicyEnforcement 
{
 
     policy.initialize(rss, tableName, snapshot);
 
-    policy.checkBulkLoad(fs, paths);
+    policy.computeBulkLoadSize(fs, paths);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdedcc56/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
index 9479890..1b2009c 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
@@ -87,6 +87,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
+import org.apache.hadoop.hbase.quotas.RegionSizeStoreImpl;
 import 
org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
 import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
@@ -1658,6 +1659,56 @@ public class TestHStore {
     assertFalse(heap.equals(heap2));
   }
 
+  @Test
+  public void testSpaceQuotaChangeAfterReplacement() throws IOException {
+    final TableName tn = TableName.valueOf(name.getMethodName());
+    init(name.getMethodName());
+
+    RegionSizeStoreImpl sizeStore = new RegionSizeStoreImpl();
+
+    HStoreFile sf1 = mockStoreFileWithLength(1024L);
+    HStoreFile sf2 = mockStoreFileWithLength(2048L);
+    HStoreFile sf3 = mockStoreFileWithLength(4096L);
+    HStoreFile sf4 = mockStoreFileWithLength(8192L);
+
+    RegionInfo regionInfo = 
RegionInfoBuilder.newBuilder(tn).setStartKey(Bytes.toBytes("a"))
+        .setEndKey(Bytes.toBytes("b")).build();
+
+    // Compacting two files down to one, reducing size
+    sizeStore.put(regionInfo, 1024L + 4096L);
+    store.updateSpaceQuotaAfterFileReplacement(
+        sizeStore, regionInfo, Arrays.asList(sf1, sf3), Arrays.asList(sf2));
+
+    assertEquals(2048L, sizeStore.getRegionSize(regionInfo).getSize());
+
+    // The same file length in and out should have no change
+    store.updateSpaceQuotaAfterFileReplacement(
+        sizeStore, regionInfo, Arrays.asList(sf2), Arrays.asList(sf2));
+
+    assertEquals(2048L, sizeStore.getRegionSize(regionInfo).getSize());
+
+    // Increase the total size used
+    store.updateSpaceQuotaAfterFileReplacement(
+        sizeStore, regionInfo, Arrays.asList(sf2), Arrays.asList(sf3));
+
+    assertEquals(4096L, sizeStore.getRegionSize(regionInfo).getSize());
+
+    RegionInfo regionInfo2 = 
RegionInfoBuilder.newBuilder(tn).setStartKey(Bytes.toBytes("b"))
+        .setEndKey(Bytes.toBytes("c")).build();
+    store.updateSpaceQuotaAfterFileReplacement(sizeStore, regionInfo2, null, 
Arrays.asList(sf4));
+
+    assertEquals(8192L, sizeStore.getRegionSize(regionInfo2).getSize());
+  }
+
+  private HStoreFile mockStoreFileWithLength(long length) {
+    HStoreFile sf = mock(HStoreFile.class);
+    StoreFileReader sfr = mock(StoreFileReader.class);
+    when(sf.isHFile()).thenReturn(true);
+    when(sf.getReader()).thenReturn(sfr);
+    when(sfr.length()).thenReturn(length);
+    return sf;
+  }
+
   private static class MyThread extends Thread {
     private StoreScanner scanner;
     private KeyValueHeap heap;

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdedcc56/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
index e17b87c..3cac439 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
@@ -25,12 +25,13 @@ import static org.mockito.Matchers.anyLong;
 import static org.mockito.Mockito.doCallRealMethod;
 import static org.mockito.Mockito.mock;
 
-import java.util.HashMap;
-import java.util.Map;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.quotas.RegionSize;
+import org.apache.hadoop.hbase.quotas.RegionSizeStore;
+import org.apache.hadoop.hbase.quotas.RegionSizeStoreFactory;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.ClassRule;
@@ -68,52 +69,24 @@ public class TestRegionServerRegionSpaceUseReport {
         .setStartKey(Bytes.toBytes("c"))
         .setEndKey(Bytes.toBytes("d"))
         .build();
-    Map<RegionInfo,Long> sizes = new HashMap<>();
-    sizes.put(hri1, 1024L * 1024L);
-    sizes.put(hri2, 1024L * 1024L * 8L);
-    sizes.put(hri3, 1024L * 1024L * 32L);
+    RegionSizeStore store = RegionSizeStoreFactory.getInstance().createStore();
+    store.put(hri1, 1024L * 1024L);
+    store.put(hri2, 1024L * 1024L * 8L);
+    store.put(hri3, 1024L * 1024L * 32L);
 
     // Call the real method to convert the map into a protobuf
     HRegionServer rs = mock(HRegionServer.class);
-    doCallRealMethod().when(rs).buildRegionSpaceUseReportRequest(any());
+    
doCallRealMethod().when(rs).buildRegionSpaceUseReportRequest(any(RegionSizeStore.class));
     doCallRealMethod().when(rs).convertRegionSize(any(), anyLong());
 
-    RegionSpaceUseReportRequest requests = 
rs.buildRegionSpaceUseReportRequest(sizes);
-    assertEquals(sizes.size(), requests.getSpaceUseCount());
+    RegionSpaceUseReportRequest requests = 
rs.buildRegionSpaceUseReportRequest(store);
+    assertEquals(store.size(), requests.getSpaceUseCount());
     for (RegionSpaceUse spaceUse : requests.getSpaceUseList()) {
       RegionInfo hri = ProtobufUtil.toRegionInfo(spaceUse.getRegionInfo());
-      Long expectedSize = sizes.remove(hri);
+      RegionSize expectedSize = store.remove(hri);
       assertNotNull("Could not find size for HRI: " + hri, expectedSize);
-      assertEquals(expectedSize.longValue(), spaceUse.getRegionSize());
+      assertEquals(expectedSize.getSize(), spaceUse.getRegionSize());
     }
-    assertTrue("Should not have any space use entries left: " + sizes, 
sizes.isEmpty());
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testNullMap() {
-    // Call the real method to convert the map into a protobuf
-    HRegionServer rs = mock(HRegionServer.class);
-    doCallRealMethod().when(rs).buildRegionSpaceUseReportRequest(any());
-    doCallRealMethod().when(rs).convertRegionSize(any(), anyLong());
-
-    rs.buildRegionSpaceUseReportRequest(null);
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testMalformedMap() {
-    TableName tn = TableName.valueOf("table1");
-    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tn)
-        .setStartKey(Bytes.toBytes("a"))
-        .setEndKey(Bytes.toBytes("b"))
-        .build();
-    Map<RegionInfo,Long> sizes = new HashMap<>();
-    sizes.put(hri1, null);
-
-    // Call the real method to convert the map into a protobuf
-    HRegionServer rs = mock(HRegionServer.class);
-    doCallRealMethod().when(rs).buildRegionSpaceUseReportRequest(any());
-    doCallRealMethod().when(rs).convertRegionSize(any(), anyLong());
-
-    rs.buildRegionSpaceUseReportRequest(sizes);
+    assertTrue("Should not have any space use entries left: " + store, 
store.isEmpty());
   }
 }

Reply via email to