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/04/17 20:27:10 UTC

[01/50] [abbrv] hbase git commit: HBASE-16775 Fix flaky TestExportSnapshot#testExportRetry. [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-16961 f46c16aa2 -> 2524f8cd2 (forced update)


HBASE-16775 Fix flaky TestExportSnapshot#testExportRetry.

Reason for flakyness: Current test is probability based fault injection and triggers failure 3% of the time. Earlier when test used LocalJobRunner which didn't honor "mapreduce.map.maxattempts", it'd pass 97% time (when no fault is injected) and fail 3% time (when fault was injected). Point being, even when the test was complete wrong, we couldn't catch it because it was probability based.

This change will inject fault in a deterministic manner.
On design side, it encapsulates all testing hooks in ExportSnapshot.java into single inner class.

Change-Id: Icba866e1d56a5281748df89f4dd374bc45bad249


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

Branch: refs/heads/HBASE-16961
Commit: da5fb27eabed4a4b4d251be973ee945fb52895bf
Parents: cf3215d
Author: Apekshit Sharma <ap...@apache.org>
Authored: Thu Oct 6 14:20:58 2016 -0700
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Wed Apr 12 11:11:31 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/snapshot/ExportSnapshot.java   | 58 +++++++-------
 .../hbase/snapshot/TestExportSnapshot.java      | 84 +++++++++++---------
 .../snapshot/TestExportSnapshotNoCluster.java   |  2 +-
 .../hbase/snapshot/TestMobExportSnapshot.java   |  7 +-
 .../snapshot/TestMobSecureExportSnapshot.java   |  7 +-
 .../snapshot/TestSecureExportSnapshot.java      |  7 +-
 6 files changed, 93 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
index e2086e9..e3ad951 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
@@ -29,7 +29,6 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Random;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -110,9 +109,12 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
   private static final String CONF_BANDWIDTH_MB = "snapshot.export.map.bandwidth.mb";
   protected static final String CONF_SKIP_TMP = "snapshot.export.skip.tmp";
 
-  static final String CONF_TEST_FAILURE = "test.snapshot.export.failure";
-  static final String CONF_TEST_RETRY = "test.snapshot.export.failure.retry";
-
+  static class Testing {
+    static final String CONF_TEST_FAILURE = "test.snapshot.export.failure";
+    static final String CONF_TEST_FAILURE_COUNT = "test.snapshot.export.failure.count";
+    int failuresCountToInject = 0;
+    int injectedFailureCount = 0;
+  }
 
   // Command line options and defaults.
   static final class Options {
@@ -149,12 +151,10 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
 
   private static class ExportMapper extends Mapper<BytesWritable, NullWritable,
                                                    NullWritable, NullWritable> {
+    private static final Log LOG = LogFactory.getLog(ExportMapper.class);
     final static int REPORT_SIZE = 1 * 1024 * 1024;
     final static int BUFFER_SIZE = 64 * 1024;
 
-    private boolean testFailures;
-    private Random random;
-
     private boolean verifyChecksum;
     private String filesGroup;
     private String filesUser;
@@ -169,9 +169,12 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
     private Path inputArchive;
     private Path inputRoot;
 
+    private static Testing testing = new Testing();
+
     @Override
     public void setup(Context context) throws IOException {
       Configuration conf = context.getConfiguration();
+
       Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX);
       Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX);
 
@@ -186,8 +189,6 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
       inputArchive = new Path(inputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
       outputArchive = new Path(outputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
 
-      testFailures = conf.getBoolean(CONF_TEST_FAILURE, false);
-
       try {
         srcConf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true);
         inputFs = FileSystem.get(inputRoot.toUri(), srcConf);
@@ -210,6 +211,12 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
       for (Counter c : Counter.values()) {
         context.getCounter(c).increment(0);
       }
+      if (context.getConfiguration().getBoolean(Testing.CONF_TEST_FAILURE, false)) {
+        testing.failuresCountToInject = conf.getInt(Testing.CONF_TEST_FAILURE_COUNT, 0);
+        // Get number of times we have already injected failure based on attempt number of this
+        // task.
+        testing.injectedFailureCount = context.getTaskAttemptID().getId();
+      }
     }
 
     @Override
@@ -251,35 +258,23 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
       return new Path(outputArchive, path);
     }
 
-    /*
-     * Used by TestExportSnapshot to simulate a failure
+    /**
+     * Used by TestExportSnapshot to test for retries when failures happen.
+     * Failure is injected in {@link #copyFile(Context, SnapshotFileInfo, Path)}.
      */
     private void injectTestFailure(final Context context, final SnapshotFileInfo inputInfo)
         throws IOException {
-      if (testFailures) {
-        if (context.getConfiguration().getBoolean(CONF_TEST_RETRY, false)) {
-          if (random == null) {
-            random = new Random();
-          }
-
-          // FLAKY-TEST-WARN: lower is better, we can get some runs without the
-          // retry, but at least we reduce the number of test failures due to
-          // this test exception from the same map task.
-          if (random.nextFloat() < 0.03) {
-            throw new IOException("TEST RETRY FAILURE: Unable to copy input=" + inputInfo
-                                  + " time=" + System.currentTimeMillis());
-          }
-        } else {
-          context.getCounter(Counter.COPY_FAILED).increment(1);
-          throw new IOException("TEST FAILURE: Unable to copy input=" + inputInfo);
-        }
-      }
+      if (!context.getConfiguration().getBoolean(Testing.CONF_TEST_FAILURE, false)) return;
+      if (testing.injectedFailureCount >= testing.failuresCountToInject) return;
+      testing.injectedFailureCount++;
+      context.getCounter(Counter.COPY_FAILED).increment(1);
+      LOG.debug("Injecting failure. Count: " + testing.injectedFailureCount);
+      throw new IOException(String.format("TEST FAILURE (%d of max %d): Unable to copy input=%s",
+          testing.injectedFailureCount, testing.failuresCountToInject, inputInfo));
     }
 
     private void copyFile(final Context context, final SnapshotFileInfo inputInfo,
         final Path outputPath) throws IOException {
-      injectTestFailure(context, inputInfo);
-
       // Get the file information
       FileStatus inputStat = getSourceFileStatus(context, inputInfo);
 
@@ -318,6 +313,7 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
         }
       } finally {
         in.close();
+        injectTestFailure(context, inputInfo);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
index 1beb518..52412d9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.snapshot;
 import static org.apache.hadoop.util.ToolRunner.run;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 
 import java.io.IOException;
 import java.net.URI;
@@ -44,12 +45,10 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
 import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -57,6 +56,7 @@ import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
 
 /**
@@ -72,6 +72,9 @@ public class TestExportSnapshot {
 
   protected final static byte[] FAMILY = Bytes.toBytes("cf");
 
+  @Rule
+  public final TestName testName = new TestName();
+
   protected TableName tableName;
   private byte[] emptySnapshotName;
   private byte[] snapshotName;
@@ -85,17 +88,27 @@ public class TestExportSnapshot {
     conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
     conf.setBoolean("hbase.master.enabletable.roundrobin", true);
     conf.setInt("mapreduce.map.maxattempts", 10);
+    // If a single node has enough failures (default 3), resource manager will blacklist it.
+    // With only 2 nodes and tests injecting faults, we don't want that.
+    conf.setInt("mapreduce.job.maxtaskfailures.per.tracker", 100);
   }
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     setUpBaseConf(TEST_UTIL.getConfiguration());
-    TEST_UTIL.startMiniCluster(3);
+    // Setup separate test-data directory for MR cluster and set corresponding configurations.
+    // Otherwise, different test classes running MR cluster can step on each other.
+    TEST_UTIL.getDataTestDir();
+    TEST_UTIL.startMiniZKCluster();
+    TEST_UTIL.startMiniMapReduceCluster();
+    TEST_UTIL.startMiniHBaseCluster(1, 3);
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
+    TEST_UTIL.shutdownMiniHBaseCluster();
+    TEST_UTIL.shutdownMiniMapReduceCluster();
+    TEST_UTIL.shutdownMiniZKCluster();
   }
 
   /**
@@ -105,10 +118,9 @@ public class TestExportSnapshot {
   public void setUp() throws Exception {
     this.admin = TEST_UTIL.getAdmin();
 
-    long tid = System.currentTimeMillis();
-    tableName = TableName.valueOf("testtb-" + tid);
-    snapshotName = Bytes.toBytes("snaptb0-" + tid);
-    emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + tid);
+    tableName = TableName.valueOf("testtb-" + testName.getMethodName());
+    snapshotName = Bytes.toBytes("snaptb0-" + testName.getMethodName());
+    emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + testName.getMethodName());
 
     // create Table
     createTable();
@@ -191,16 +203,16 @@ public class TestExportSnapshot {
       Path copyDir, boolean overwrite) throws Exception {
     testExportFileSystemState(TEST_UTIL.getConfiguration(), tableName, snapshotName, targetName,
       filesExpected, TEST_UTIL.getDefaultRootDirPath(), copyDir,
-      overwrite, getBypassRegionPredicate());
+      overwrite, getBypassRegionPredicate(), true);
   }
 
   /**
-   * Test ExportSnapshot
+   * Creates destination directory, runs ExportSnapshot() tool, and runs some verifications.
    */
   protected static void testExportFileSystemState(final Configuration conf, final TableName tableName,
       final byte[] snapshotName, final byte[] targetName, final int filesExpected,
       final Path sourceDir, Path copyDir, final boolean overwrite,
-      final RegionPredicate bypassregionPredicate) throws Exception {
+      final RegionPredicate bypassregionPredicate, boolean success) throws Exception {
     URI hdfsUri = FileSystem.get(conf).getUri();
     FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration());
     copyDir = copyDir.makeQualified(fs);
@@ -218,7 +230,12 @@ public class TestExportSnapshot {
 
     // Export Snapshot
     int res = run(conf, new ExportSnapshot(), opts.toArray(new String[opts.size()]));
-    assertEquals(0, res);
+    assertEquals(success ? 0 : 1, res);
+    if (!success) {
+      final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(targetName));
+      assertFalse(fs.exists(new Path(copyDir, targetDir)));
+      return;
+    }
 
     // Verify File-System state
     FileStatus[] rootFiles = fs.listStatus(copyDir);
@@ -242,42 +259,35 @@ public class TestExportSnapshot {
   }
 
   /**
-   * Check that ExportSnapshot will return a failure if something fails.
+   * Check that ExportSnapshot will succeed if something fails but the retry succeed.
    */
   @Test
-  public void testExportFailure() throws Exception {
-    assertEquals(1, runExportAndInjectFailures(snapshotName, false));
+  public void testExportRetry() throws Exception {
+    Path copyDir = getLocalDestinationDir();
+    FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration());
+    copyDir = copyDir.makeQualified(fs);
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    conf.setBoolean(ExportSnapshot.Testing.CONF_TEST_FAILURE, true);
+    conf.setInt(ExportSnapshot.Testing.CONF_TEST_FAILURE_COUNT, 2);
+    conf.setInt("mapreduce.map.maxattempts", 3);
+    testExportFileSystemState(conf, tableName, snapshotName, snapshotName, tableNumFiles,
+        TEST_UTIL.getDefaultRootDirPath(), copyDir, true, getBypassRegionPredicate(), true);
   }
 
   /**
-   * Check that ExportSnapshot will succede if something fails but the retry succede.
+   * Check that ExportSnapshot will fail if we inject failure more times than MR will retry.
    */
   @Test
-  public void testExportRetry() throws Exception {
-    assertEquals(0, runExportAndInjectFailures(snapshotName, true));
-  }
-
-  /*
-   * Execute the ExportSnapshot job injecting failures
-   */
-  private int runExportAndInjectFailures(final byte[] snapshotName, boolean retry)
-      throws Exception {
+  public void testExportFailure() throws Exception {
     Path copyDir = getLocalDestinationDir();
-    URI hdfsUri = FileSystem.get(TEST_UTIL.getConfiguration()).getUri();
     FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration());
     copyDir = copyDir.makeQualified(fs);
-
     Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
-    conf.setBoolean(ExportSnapshot.CONF_TEST_FAILURE, true);
-    conf.setBoolean(ExportSnapshot.CONF_TEST_RETRY, retry);
-    if (!retry) {
-      conf.setInt("mapreduce.map.maxattempts", 3);
-    }
-    // Export Snapshot
-    Path sourceDir = TEST_UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
-    String[] args = new String[] { "--snapshot", Bytes.toString(snapshotName),
-        "--copy-from", sourceDir.toString(), "--copy-to", copyDir.toString() };
-    return ToolRunner.run(conf, new ExportSnapshot(), args);
+    conf.setBoolean(ExportSnapshot.Testing.CONF_TEST_FAILURE, true);
+    conf.setInt(ExportSnapshot.Testing.CONF_TEST_FAILURE_COUNT, 4);
+    conf.setInt("mapreduce.map.maxattempts", 3);
+    testExportFileSystemState(conf, tableName, snapshotName, snapshotName, tableNumFiles,
+        TEST_UTIL.getDefaultRootDirPath(), copyDir, true, getBypassRegionPredicate(), false);
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
index e2d7c11..cd5ff6c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
@@ -104,7 +104,7 @@ public class TestExportSnapshotNoCluster {
     TableName tableName = builder.getTableDescriptor().getTableName();
     TestExportSnapshot.testExportFileSystemState(TEST_UTIL.getConfiguration(),
       tableName, snapshotName, snapshotName, snapshotFilesCount,
-      testDir, getDestinationDir(), false, null);
+      testDir, getDestinationDir(), false, null, true);
   }
 
   private Path getDestinationDir() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
index c375b0a..55686b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
@@ -45,7 +45,12 @@ public class TestMobExportSnapshot extends TestExportSnapshot {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     setUpBaseConf(TEST_UTIL.getConfiguration());
-    TEST_UTIL.startMiniCluster(3);
+    // Setup separate test-data directory for MR cluster and set corresponding configurations.
+    // Otherwise, different test classes running MR cluster can step on each other.
+    TEST_UTIL.getDataTestDir();
+    TEST_UTIL.startMiniZKCluster();
+    TEST_UTIL.startMiniMapReduceCluster();
+    TEST_UTIL.startMiniHBaseCluster(1, 3);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java
index 8154995..c0f31d5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java
@@ -37,6 +37,9 @@ public class TestMobSecureExportSnapshot extends TestMobExportSnapshot {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     setUpBaseConf(TEST_UTIL.getConfiguration());
+    // Setup separate test-data directory for MR cluster and set corresponding configurations.
+    // Otherwise, different test classes running MR cluster can step on each other.
+    TEST_UTIL.getDataTestDir();
 
     // set the always on security provider
     UserProvider.setUserProviderForTesting(TEST_UTIL.getConfiguration(),
@@ -45,7 +48,9 @@ public class TestMobSecureExportSnapshot extends TestMobExportSnapshot {
     // setup configuration
     SecureTestUtil.enableSecurity(TEST_UTIL.getConfiguration());
 
-    TEST_UTIL.startMiniCluster(3);
+    TEST_UTIL.startMiniZKCluster();
+    TEST_UTIL.startMiniMapReduceCluster();
+    TEST_UTIL.startMiniHBaseCluster(1, 3);
 
     // Wait for the ACL table to become available
     TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java
index f335e4e..4d35b71 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java
@@ -42,6 +42,9 @@ public class TestSecureExportSnapshot extends TestExportSnapshot {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     setUpBaseConf(TEST_UTIL.getConfiguration());
+    // Setup separate test-data directory for MR cluster and set corresponding configurations.
+    // Otherwise, different test classes running MR cluster can step on each other.
+    TEST_UTIL.getDataTestDir();
 
     // set the always on security provider
     UserProvider.setUserProviderForTesting(TEST_UTIL.getConfiguration(),
@@ -50,7 +53,9 @@ public class TestSecureExportSnapshot extends TestExportSnapshot {
     // setup configuration
     SecureTestUtil.enableSecurity(TEST_UTIL.getConfiguration());
 
-    TEST_UTIL.startMiniCluster(3);
+    TEST_UTIL.startMiniZKCluster();
+    TEST_UTIL.startMiniMapReduceCluster();
+    TEST_UTIL.startMiniHBaseCluster(1, 3);
 
     // Wait for the ACL table to become available
     TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);


[22/50] [abbrv] hbase git commit: HBASE-16995 Build client Java API and client protobuf messages (Josh Elser)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
index 05894b9..1925828 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
@@ -217,12 +217,20 @@ public final class QuotaProtos {
      * <code>THROTTLE = 1;</code>
      */
     THROTTLE(0, 1),
+    /**
+     * <code>SPACE = 2;</code>
+     */
+    SPACE(1, 2),
     ;
 
     /**
      * <code>THROTTLE = 1;</code>
      */
     public static final int THROTTLE_VALUE = 1;
+    /**
+     * <code>SPACE = 2;</code>
+     */
+    public static final int SPACE_VALUE = 2;
 
 
     public final int getNumber() { return value; }
@@ -230,6 +238,7 @@ public final class QuotaProtos {
     public static QuotaType valueOf(int value) {
       switch (value) {
         case 1: return THROTTLE;
+        case 2: return SPACE;
         default: return null;
       }
     }
@@ -281,6 +290,142 @@ public final class QuotaProtos {
     // @@protoc_insertion_point(enum_scope:hbase.pb.QuotaType)
   }
 
+  /**
+   * Protobuf enum {@code hbase.pb.SpaceViolationPolicy}
+   *
+   * <pre>
+   * Defines what action should be taken when the SpaceQuota is violated
+   * </pre>
+   */
+  public enum SpaceViolationPolicy
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>DISABLE = 1;</code>
+     *
+     * <pre>
+     * Disable the table(s)
+     * </pre>
+     */
+    DISABLE(0, 1),
+    /**
+     * <code>NO_WRITES_COMPACTIONS = 2;</code>
+     *
+     * <pre>
+     * No writes, bulk-loads, or compactions
+     * </pre>
+     */
+    NO_WRITES_COMPACTIONS(1, 2),
+    /**
+     * <code>NO_WRITES = 3;</code>
+     *
+     * <pre>
+     * No writes or bulk-loads
+     * </pre>
+     */
+    NO_WRITES(2, 3),
+    /**
+     * <code>NO_INSERTS = 4;</code>
+     *
+     * <pre>
+     * No puts or bulk-loads, but deletes are allowed
+     * </pre>
+     */
+    NO_INSERTS(3, 4),
+    ;
+
+    /**
+     * <code>DISABLE = 1;</code>
+     *
+     * <pre>
+     * Disable the table(s)
+     * </pre>
+     */
+    public static final int DISABLE_VALUE = 1;
+    /**
+     * <code>NO_WRITES_COMPACTIONS = 2;</code>
+     *
+     * <pre>
+     * No writes, bulk-loads, or compactions
+     * </pre>
+     */
+    public static final int NO_WRITES_COMPACTIONS_VALUE = 2;
+    /**
+     * <code>NO_WRITES = 3;</code>
+     *
+     * <pre>
+     * No writes or bulk-loads
+     * </pre>
+     */
+    public static final int NO_WRITES_VALUE = 3;
+    /**
+     * <code>NO_INSERTS = 4;</code>
+     *
+     * <pre>
+     * No puts or bulk-loads, but deletes are allowed
+     * </pre>
+     */
+    public static final int NO_INSERTS_VALUE = 4;
+
+
+    public final int getNumber() { return value; }
+
+    public static SpaceViolationPolicy valueOf(int value) {
+      switch (value) {
+        case 1: return DISABLE;
+        case 2: return NO_WRITES_COMPACTIONS;
+        case 3: return NO_WRITES;
+        case 4: return NO_INSERTS;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<SpaceViolationPolicy>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<SpaceViolationPolicy>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<SpaceViolationPolicy>() {
+            public SpaceViolationPolicy findValueByNumber(int number) {
+              return SpaceViolationPolicy.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.getDescriptor().getEnumTypes().get(3);
+    }
+
+    private static final SpaceViolationPolicy[] VALUES = values();
+
+    public static SpaceViolationPolicy valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private SpaceViolationPolicy(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:hbase.pb.SpaceViolationPolicy)
+  }
+
   public interface TimedQuotaOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -3315,6 +3460,20 @@ public final class QuotaProtos {
      * <code>optional .hbase.pb.Throttle throttle = 2;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.ThrottleOrBuilder getThrottleOrBuilder();
+
+    // optional .hbase.pb.SpaceQuota space = 3;
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    boolean hasSpace();
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota getSpace();
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getSpaceOrBuilder();
   }
   /**
    * Protobuf type {@code hbase.pb.Quotas}
@@ -3385,6 +3544,19 @@ public final class QuotaProtos {
               bitField0_ |= 0x00000002;
               break;
             }
+            case 26: {
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = space_.toBuilder();
+              }
+              space_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(space_);
+                space_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000004;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -3463,9 +3635,32 @@ public final class QuotaProtos {
       return throttle_;
     }
 
+    // optional .hbase.pb.SpaceQuota space = 3;
+    public static final int SPACE_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota space_;
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    public boolean hasSpace() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota getSpace() {
+      return space_;
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getSpaceOrBuilder() {
+      return space_;
+    }
+
     private void initFields() {
       bypassGlobals_ = false;
       throttle_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle.getDefaultInstance();
+      space_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -3491,6 +3686,9 @@ public final class QuotaProtos {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeMessage(2, throttle_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, space_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -3508,6 +3706,10 @@ public final class QuotaProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, throttle_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, space_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -3541,6 +3743,11 @@ public final class QuotaProtos {
         result = result && getThrottle()
             .equals(other.getThrottle());
       }
+      result = result && (hasSpace() == other.hasSpace());
+      if (hasSpace()) {
+        result = result && getSpace()
+            .equals(other.getSpace());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -3562,6 +3769,10 @@ public final class QuotaProtos {
         hash = (37 * hash) + THROTTLE_FIELD_NUMBER;
         hash = (53 * hash) + getThrottle().hashCode();
       }
+      if (hasSpace()) {
+        hash = (37 * hash) + SPACE_FIELD_NUMBER;
+        hash = (53 * hash) + getSpace().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -3664,6 +3875,7 @@ public final class QuotaProtos {
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getThrottleFieldBuilder();
+          getSpaceFieldBuilder();
         }
       }
       private static Builder create() {
@@ -3680,6 +3892,12 @@ public final class QuotaProtos {
           throttleBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000002);
+        if (spaceBuilder_ == null) {
+          space_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance();
+        } else {
+          spaceBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -3720,6 +3938,14 @@ public final class QuotaProtos {
         } else {
           result.throttle_ = throttleBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (spaceBuilder_ == null) {
+          result.space_ = space_;
+        } else {
+          result.space_ = spaceBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -3742,6 +3968,9 @@ public final class QuotaProtos {
         if (other.hasThrottle()) {
           mergeThrottle(other.getThrottle());
         }
+        if (other.hasSpace()) {
+          mergeSpace(other.getSpace());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -3925,6 +4154,123 @@ public final class QuotaProtos {
         return throttleBuilder_;
       }
 
+      // optional .hbase.pb.SpaceQuota space = 3;
+      private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota space_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> spaceBuilder_;
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public boolean hasSpace() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota getSpace() {
+        if (spaceBuilder_ == null) {
+          return space_;
+        } else {
+          return spaceBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public Builder setSpace(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota value) {
+        if (spaceBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          space_ = value;
+          onChanged();
+        } else {
+          spaceBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public Builder setSpace(
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder builderForValue) {
+        if (spaceBuilder_ == null) {
+          space_ = builderForValue.build();
+          onChanged();
+        } else {
+          spaceBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public Builder mergeSpace(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota value) {
+        if (spaceBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              space_ != org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance()) {
+            space_ =
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.newBuilder(space_).mergeFrom(value).buildPartial();
+          } else {
+            space_ = value;
+          }
+          onChanged();
+        } else {
+          spaceBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public Builder clearSpace() {
+        if (spaceBuilder_ == null) {
+          space_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance();
+          onChanged();
+        } else {
+          spaceBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder getSpaceBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getSpaceFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getSpaceOrBuilder() {
+        if (spaceBuilder_ != null) {
+          return spaceBuilder_.getMessageOrBuilder();
+        } else {
+          return space_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
+          getSpaceFieldBuilder() {
+        if (spaceBuilder_ == null) {
+          spaceBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>(
+                  space_,
+                  getParentForChildren(),
+                  isClean());
+          space_ = null;
+        }
+        return spaceBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.Quotas)
     }
 
@@ -4274,81 +4620,1257 @@ public final class QuotaProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.QuotaUsage)
   }
 
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_TimedQuota_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_TimedQuota_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_Throttle_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_Throttle_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_ThrottleRequest_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_Quotas_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_Quotas_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_QuotaUsage_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_QuotaUsage_fieldAccessorTable;
+  public interface SpaceQuotaOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
+    // optional uint64 soft_limit = 1;
+    /**
+     * <code>optional uint64 soft_limit = 1;</code>
+     *
+     * <pre>
+     * The limit of bytes for this quota
+     * </pre>
+     */
+    boolean hasSoftLimit();
+    /**
+     * <code>optional uint64 soft_limit = 1;</code>
+     *
+     * <pre>
+     * The limit of bytes for this quota
+     * </pre>
+     */
+    long getSoftLimit();
+
+    // optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+     *
+     * <pre>
+     * The action to take when the quota is violated
+     * </pre>
+     */
+    boolean hasViolationPolicy();
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+     *
+     * <pre>
+     * The action to take when the quota is violated
+     * </pre>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy();
   }
-  private static com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n\013Quota.proto\022\010hbase.pb\032\013HBase.proto\"\204\001\n" +
-      "\nTimedQuota\022%\n\ttime_unit\030\001 \002(\0162\022.hbase.p" +
-      "b.TimeUnit\022\022\n\nsoft_limit\030\002 \001(\004\022\r\n\005share\030" +
-      "\003 \001(\002\022,\n\005scope\030\004 \001(\0162\024.hbase.pb.QuotaSco" +
-      "pe:\007MACHINE\"\375\001\n\010Throttle\022%\n\007req_num\030\001 \001(" +
-      "\0132\024.hbase.pb.TimedQuota\022&\n\010req_size\030\002 \001(" +
-      "\0132\024.hbase.pb.TimedQuota\022\'\n\twrite_num\030\003 \001" +
-      "(\0132\024.hbase.pb.TimedQuota\022(\n\nwrite_size\030\004" +
-      " \001(\0132\024.hbase.pb.TimedQuota\022&\n\010read_num\030\005" +
-      " \001(\0132\024.hbase.pb.TimedQuota\022\'\n\tread_size\030",
-      "\006 \001(\0132\024.hbase.pb.TimedQuota\"b\n\017ThrottleR" +
-      "equest\022$\n\004type\030\001 \001(\0162\026.hbase.pb.Throttle" +
-      "Type\022)\n\013timed_quota\030\002 \001(\0132\024.hbase.pb.Tim" +
-      "edQuota\"M\n\006Quotas\022\035\n\016bypass_globals\030\001 \001(" +
-      "\010:\005false\022$\n\010throttle\030\002 \001(\0132\022.hbase.pb.Th" +
-      "rottle\"\014\n\nQuotaUsage*&\n\nQuotaScope\022\013\n\007CL" +
-      "USTER\020\001\022\013\n\007MACHINE\020\002*v\n\014ThrottleType\022\022\n\016" +
-      "REQUEST_NUMBER\020\001\022\020\n\014REQUEST_SIZE\020\002\022\020\n\014WR" +
-      "ITE_NUMBER\020\003\022\016\n\nWRITE_SIZE\020\004\022\017\n\013READ_NUM" +
-      "BER\020\005\022\r\n\tREAD_SIZE\020\006*\031\n\tQuotaType\022\014\n\010THR",
-      "OTTLE\020\001BA\n*org.apache.hadoop.hbase.proto" +
-      "buf.generatedB\013QuotaProtosH\001\210\001\001\240\001\001"
-    };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
-        public com.google.protobuf.ExtensionRegistry assignDescriptors(
-            com.google.protobuf.Descriptors.FileDescriptor root) {
-          descriptor = root;
-          internal_static_hbase_pb_TimedQuota_descriptor =
-            getDescriptor().getMessageTypes().get(0);
-          internal_static_hbase_pb_TimedQuota_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_hbase_pb_TimedQuota_descriptor,
-              new java.lang.String[] { "TimeUnit", "SoftLimit", "Share", "Scope", });
-          internal_static_hbase_pb_Throttle_descriptor =
-            getDescriptor().getMessageTypes().get(1);
-          internal_static_hbase_pb_Throttle_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_hbase_pb_Throttle_descriptor,
-              new java.lang.String[] { "ReqNum", "ReqSize", "WriteNum", "WriteSize", "ReadNum", "ReadSize", });
-          internal_static_hbase_pb_ThrottleRequest_descriptor =
+  /**
+   * Protobuf type {@code hbase.pb.SpaceQuota}
+   *
+   * <pre>
+   * Defines a limit on the amount of filesystem space used by a table/namespace
+   * </pre>
+   */
+  public static final class SpaceQuota extends
+      com.google.protobuf.GeneratedMessage
+      implements SpaceQuotaOrBuilder {
+    // Use SpaceQuota.newBuilder() to construct.
+    private SpaceQuota(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private SpaceQuota(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final SpaceQuota defaultInstance;
+    public static SpaceQuota getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public SpaceQuota getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SpaceQuota(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              softLimit_ = input.readUInt64();
+              break;
+            }
+            case 16: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy value = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(2, rawValue);
+              } else {
+                bitField0_ |= 0x00000002;
+                violationPolicy_ = value;
+              }
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuota_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuota_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.class, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<SpaceQuota> PARSER =
+        new com.google.protobuf.AbstractParser<SpaceQuota>() {
+      public SpaceQuota parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new SpaceQuota(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<SpaceQuota> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional uint64 soft_limit = 1;
+    public static final int SOFT_LIMIT_FIELD_NUMBER = 1;
+    private long softLimit_;
+    /**
+     * <code>optional uint64 soft_limit = 1;</code>
+     *
+     * <pre>
+     * The limit of bytes for this quota
+     * </pre>
+     */
+    public boolean hasSoftLimit() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional uint64 soft_limit = 1;</code>
+     *
+     * <pre>
+     * The limit of bytes for this quota
+     * </pre>
+     */
+    public long getSoftLimit() {
+      return softLimit_;
+    }
+
+    // optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;
+    public static final int VIOLATION_POLICY_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy violationPolicy_;
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+     *
+     * <pre>
+     * The action to take when the quota is violated
+     * </pre>
+     */
+    public boolean hasViolationPolicy() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+     *
+     * <pre>
+     * The action to take when the quota is violated
+     * </pre>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy() {
+      return violationPolicy_;
+    }
+
+    private void initFields() {
+      softLimit_ = 0L;
+      violationPolicy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt64(1, softLimit_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeEnum(2, violationPolicy_.getNumber());
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(1, softLimit_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(2, violationPolicy_.getNumber());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota other = (org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota) obj;
+
+      boolean result = true;
+      result = result && (hasSoftLimit() == other.hasSoftLimit());
+      if (hasSoftLimit()) {
+        result = result && (getSoftLimit()
+            == other.getSoftLimit());
+      }
+      result = result && (hasViolationPolicy() == other.hasViolationPolicy());
+      if (hasViolationPolicy()) {
+        result = result &&
+            (getViolationPolicy() == other.getViolationPolicy());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasSoftLimit()) {
+        hash = (37 * hash) + SOFT_LIMIT_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getSoftLimit());
+      }
+      if (hasViolationPolicy()) {
+        hash = (37 * hash) + VIOLATION_POLICY_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getViolationPolicy());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.SpaceQuota}
+     *
+     * <pre>
+     * Defines a limit on the amount of filesystem space used by a table/namespace
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuota_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuota_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.class, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        softLimit_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        violationPolicy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuota_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota build() {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota result = new org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.softLimit_ = softLimit_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.violationPolicy_ = violationPolicy_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance()) return this;
+        if (other.hasSoftLimit()) {
+          setSoftLimit(other.getSoftLimit());
+        }
+        if (other.hasViolationPolicy()) {
+          setViolationPolicy(other.getViolationPolicy());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional uint64 soft_limit = 1;
+      private long softLimit_ ;
+      /**
+       * <code>optional uint64 soft_limit = 1;</code>
+       *
+       * <pre>
+       * The limit of bytes for this quota
+       * </pre>
+       */
+      public boolean hasSoftLimit() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional uint64 soft_limit = 1;</code>
+       *
+       * <pre>
+       * The limit of bytes for this quota
+       * </pre>
+       */
+      public long getSoftLimit() {
+        return softLimit_;
+      }
+      /**
+       * <code>optional uint64 soft_limit = 1;</code>
+       *
+       * <pre>
+       * The limit of bytes for this quota
+       * </pre>
+       */
+      public Builder setSoftLimit(long value) {
+        bitField0_ |= 0x00000001;
+        softLimit_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 soft_limit = 1;</code>
+       *
+       * <pre>
+       * The limit of bytes for this quota
+       * </pre>
+       */
+      public Builder clearSoftLimit() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        softLimit_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;
+      private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy violationPolicy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+       *
+       * <pre>
+       * The action to take when the quota is violated
+       * </pre>
+       */
+      public boolean hasViolationPolicy() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+       *
+       * <pre>
+       * The action to take when the quota is violated
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy() {
+        return violationPolicy_;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+       *
+       * <pre>
+       * The action to take when the quota is violated
+       * </pre>
+       */
+      public Builder setViolationPolicy(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000002;
+        violationPolicy_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+       *
+       * <pre>
+       * The action to take when the quota is violated
+       * </pre>
+       */
+      public Builder clearViolationPolicy() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        violationPolicy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SpaceQuota)
+    }
+
+    static {
+      defaultInstance = new SpaceQuota(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.SpaceQuota)
+  }
+
+  public interface SpaceLimitRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional .hbase.pb.SpaceQuota quota = 1;
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    boolean hasQuota();
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota getQuota();
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getQuotaOrBuilder();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.SpaceLimitRequest}
+   *
+   * <pre>
+   * The Request to limit space usage (to allow for schema evolution not tied to SpaceQuota).
+   * </pre>
+   */
+  public static final class SpaceLimitRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements SpaceLimitRequestOrBuilder {
+    // Use SpaceLimitRequest.newBuilder() to construct.
+    private SpaceLimitRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private SpaceLimitRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final SpaceLimitRequest defaultInstance;
+    public static SpaceLimitRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public SpaceLimitRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SpaceLimitRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = quota_.toBuilder();
+              }
+              quota_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(quota_);
+                quota_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceLimitRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest.class, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<SpaceLimitRequest> PARSER =
+        new com.google.protobuf.AbstractParser<SpaceLimitRequest>() {
+      public SpaceLimitRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new SpaceLimitRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<SpaceLimitRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional .hbase.pb.SpaceQuota quota = 1;
+    public static final int QUOTA_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota quota_;
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    public boolean hasQuota() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota getQuota() {
+      return quota_;
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getQuotaOrBuilder() {
+      return quota_;
+    }
+
+    private void initFields() {
+      quota_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, quota_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, quota_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest other = (org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest) obj;
+
+      boolean result = true;
+      result = result && (hasQuota() == other.hasQuota());
+      if (hasQuota()) {
+        result = result && getQuota()
+            .equals(other.getQuota());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasQuota()) {
+        hash = (37 * hash) + QUOTA_FIELD_NUMBER;
+        hash = (53 * hash) + getQuota().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.SpaceLimitRequest}
+     *
+     * <pre>
+     * The Request to limit space usage (to allow for schema evolution not tied to SpaceQuota).
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceLimitRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest.class, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getQuotaFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (quotaBuilder_ == null) {
+          quota_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance();
+        } else {
+          quotaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceLimitRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest result = new org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (quotaBuilder_ == null) {
+          result.quota_ = quota_;
+        } else {
+          result.quota_ = quotaBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance()) return this;
+        if (other.hasQuota()) {
+          mergeQuota(other.getQuota());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceLimitRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional .hbase.pb.SpaceQuota quota = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota quota_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> quotaBuilder_;
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public boolean hasQuota() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota getQuota() {
+        if (quotaBuilder_ == null) {
+          return quota_;
+        } else {
+          return quotaBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public Builder setQuota(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota value) {
+        if (quotaBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          quota_ = value;
+          onChanged();
+        } else {
+          quotaBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public Builder setQuota(
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder builderForValue) {
+        if (quotaBuilder_ == null) {
+          quota_ = builderForValue.build();
+          onChanged();
+        } else {
+          quotaBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public Builder mergeQuota(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota value) {
+        if (quotaBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              quota_ != org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance()) {
+            quota_ =
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.newBuilder(quota_).mergeFrom(value).buildPartial();
+          } else {
+            quota_ = value;
+          }
+          onChanged();
+        } else {
+          quotaBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public Builder clearQuota() {
+        if (quotaBuilder_ == null) {
+          quota_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance();
+          onChanged();
+        } else {
+          quotaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder getQuotaBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getQuotaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getQuotaOrBuilder() {
+        if (quotaBuilder_ != null) {
+          return quotaBuilder_.getMessageOrBuilder();
+        } else {
+          return quota_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
+          getQuotaFieldBuilder() {
+        if (quotaBuilder_ == null) {
+          quotaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>(
+                  quota_,
+                  getParentForChildren(),
+                  isClean());
+          quota_ = null;
+        }
+        return quotaBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SpaceLimitRequest)
+    }
+
+    static {
+      defaultInstance = new SpaceLimitRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.SpaceLimitRequest)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_TimedQuota_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_TimedQuota_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_Throttle_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_Throttle_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_ThrottleRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_Quotas_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_Quotas_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_QuotaUsage_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_QuotaUsage_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_SpaceQuota_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_SpaceQuota_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_SpaceLimitRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\013Quota.proto\022\010hbase.pb\032\013HBase.proto\"\204\001\n" +
+      "\nTimedQuota\022%\n\ttime_unit\030\001 \002(\0162\022.hbase.p" +
+      "b.TimeUnit\022\022\n\nsoft_limit\030\002 \001(\004\022\r\n\005share\030" +
+      "\003 \001(\002\022,\n\005scope\030\004 \001(\0162\024.hbase.pb.QuotaSco" +
+      "pe:\007MACHINE\"\375\001\n\010Throttle\022%\n\007req_num\030\001 \001(" +
+      "\0132\024.hbase.pb.TimedQuota\022&\n\010req_size\030\002 \001(" +
+      "\0132\024.hbase.pb.TimedQuota\022\'\n\twrite_num\030\003 \001" +
+      "(\0132\024.hbase.pb.TimedQuota\022(\n\nwrite_size\030\004" +
+      " \001(\0132\024.hbase.pb.TimedQuota\022&\n\010read_num\030\005" +
+      " \001(\0132\024.hbase.pb.TimedQuota\022\'\n\tread_size\030",
+      "\006 \001(\0132\024.hbase.pb.TimedQuota\"b\n\017ThrottleR" +
+      "equest\022$\n\004type\030\001 \001(\0162\026.hbase.pb.Throttle" +
+      "Type\022)\n\013timed_quota\030\002 \001(\0132\024.hbase.pb.Tim" +
+      "edQuota\"r\n\006Quotas\022\035\n\016bypass_globals\030\001 \001(" +
+      "\010:\005false\022$\n\010throttle\030\002 \001(\0132\022.hbase.pb.Th" +
+      "rottle\022#\n\005space\030\003 \001(\0132\024.hbase.pb.SpaceQu" +
+      "ota\"\014\n\nQuotaUsage\"Z\n\nSpaceQuota\022\022\n\nsoft_" +
+      "limit\030\001 \001(\004\0228\n\020violation_policy\030\002 \001(\0162\036." +
+      "hbase.pb.SpaceViolationPolicy\"8\n\021SpaceLi" +
+      "mitRequest\022#\n\005quota\030\001 \001(\0132\024.hbase.pb.Spa",
+      "ceQuota*&\n\nQuotaScope\022\013\n\007CLUSTER\020\001\022\013\n\007MA" +
+      "CHINE\020\002*v\n\014ThrottleType\022\022\n\016REQUEST_NUMBE" +
+      "R\020\001\022\020\n\014REQUEST_SIZE\020\002\022\020\n\014WRITE_NUMBER\020\003\022" +
+      "\016\n\nWRITE_SIZE\020\004\022\017\n\013READ_NUMBER\020\005\022\r\n\tREAD" +
+      "_SIZE\020\006*$\n\tQuotaType\022\014\n\010THROTTLE\020\001\022\t\n\005SP" +
+      "ACE\020\002*]\n\024SpaceViolationPolicy\022\013\n\007DISABLE" +
+      "\020\001\022\031\n\025NO_WRITES_COMPACTIONS\020\002\022\r\n\tNO_WRIT" +
+      "ES\020\003\022\016\n\nNO_INSERTS\020\004BA\n*org.apache.hadoo" +
+      "p.hbase.protobuf.generatedB\013QuotaProtosH" +
+      "\001\210\001\001\240\001\001"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_hbase_pb_TimedQuota_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_hbase_pb_TimedQuota_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_TimedQuota_descriptor,
+              new java.lang.String[] { "TimeUnit", "SoftLimit", "Share", "Scope", });
+          internal_static_hbase_pb_Throttle_descriptor =
+            getDescriptor().getMessageTypes().get(1);
+          internal_static_hbase_pb_Throttle_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_Throttle_descriptor,
+              new java.lang.String[] { "ReqNum", "ReqSize", "WriteNum", "WriteSize", "ReadNum", "ReadSize", });
+          internal_static_hbase_pb_ThrottleRequest_descriptor =
             getDescriptor().getMessageTypes().get(2);
           internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
@@ -4359,13 +5881,25 @@ public final class QuotaProtos {
           internal_static_hbase_pb_Quotas_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_Quotas_descriptor,
-              new java.lang.String[] { "BypassGlobals", "Throttle", });
+              new java.lang.String[] { "BypassGlobals", "Throttle", "Space", });
           internal_static_hbase_pb_QuotaUsage_descriptor =
             getDescriptor().getMessageTypes().get(4);
           internal_static_hbase_pb_QuotaUsage_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_QuotaUsage_descriptor,
               new java.lang.String[] { });
+          internal_static_hbase_pb_SpaceQuota_descriptor =
+            getDescriptor().getMessageTypes().get(5);
+          internal_static_hbase_pb_SpaceQuota_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_SpaceQuota_descriptor,
+              new java.lang.String[] { "SoftLimit", "ViolationPolicy", });
+          internal_static_hbase_pb_SpaceLimitRequest_descriptor =
+            getDescriptor().getMessageTypes().get(6);
+          internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_SpaceLimitRequest_descriptor,
+              new java.lang.String[] { "Quota", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-protocol/src/main/protobuf/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Quota.proto b/hbase-protocol/src/main/protobuf/Quota.proto
index a8303b1..c12b997 100644
--- a/hbase-protocol/src/main/protobuf/Quota.proto
+++ b/hbase-protocol/src/main/protobuf/Quota.proto
@@ -65,12 +65,33 @@ message ThrottleRequest {
 
 enum QuotaType {
   THROTTLE = 1;
+  SPACE = 2;
 }
 
 message Quotas {
   optional bool bypass_globals = 1 [default = false];
   optional Throttle throttle = 2;
+  optional SpaceQuota space = 3;
 }
 
 message QuotaUsage {
 }
+
+// Defines what action should be taken when the SpaceQuota is violated
+enum SpaceViolationPolicy {
+  DISABLE = 1; // Disable the table(s)
+  NO_WRITES_COMPACTIONS = 2; // No writes, bulk-loads, or compactions
+  NO_WRITES = 3; // No writes or bulk-loads
+  NO_INSERTS = 4; // No puts or bulk-loads, but deletes are allowed
+}
+
+// Defines a limit on the amount of filesystem space used by a table/namespace
+message SpaceQuota {
+  optional uint64 soft_limit = 1; // The limit of bytes for this quota
+  optional SpaceViolationPolicy violation_policy = 2; // The action to take when the quota is violated
+}
+
+// The Request to limit space usage (to allow for schema evolution not tied to SpaceQuota).
+message SpaceLimitRequest {
+  optional SpaceQuota quota = 1;
+}


[33/50] [abbrv] hbase git commit: HBASE-17001 Enforce quota violation policies in the RegionServer

Posted by el...@apache.org.
HBASE-17001 Enforce quota violation policies in the RegionServer

The nuts-and-bolts of filesystem quotas. The Master must inform
RegionServers of the violation of a quota by a table. The RegionServer
must apply the violation policy as configured. Need to ensure
that the proper interfaces exist to satisfy all necessary policies.

This required a massive rewrite of the internal tracking by
the general space quota feature. Instead of tracking "violations",
we need to start tracking "usage". This allows us to make the decision
at the RegionServer level as to when the files in a bulk load request
should be accept or rejected which ultimately lets us avoid bulk loads
dramatically exceeding a configured space quota.


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

Branch: refs/heads/HBASE-16961
Commit: 0d76d667b47c56bb990cda06c719938720049793
Parents: dccfc84
Author: Josh Elser <el...@apache.org>
Authored: Thu Dec 15 13:27:56 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:32 2017 -0400

----------------------------------------------------------------------
 .../hbase/quotas/QuotaExceededException.java    |    4 +
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |   47 +-
 .../hadoop/hbase/quotas/SpaceQuotaSnapshot.java |  192 +++
 .../shaded/protobuf/generated/QuotaProtos.java  | 1384 +++++++++++++++++-
 .../src/main/protobuf/Quota.proto               |   15 +
 .../hbase/protobuf/generated/QuotaProtos.java   | 1324 ++++++++++++++++-
 hbase-protocol/src/main/protobuf/Quota.proto    |   15 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   29 +-
 .../hbase/quotas/ActivePolicyEnforcement.java   |   86 ++
 .../quotas/NamespaceQuotaSnapshotStore.java     |  127 ++
 .../quotas/NamespaceQuotaViolationStore.java    |  127 --
 .../hadoop/hbase/quotas/QuotaObserverChore.java |  344 +++--
 .../hadoop/hbase/quotas/QuotaSnapshotStore.java |   96 ++
 .../hbase/quotas/QuotaViolationStore.java       |   89 --
 .../quotas/RegionServerSpaceQuotaManager.java   |  179 ++-
 .../hbase/quotas/SpaceLimitingException.java    |   95 ++
 .../hbase/quotas/SpaceQuotaRefresherChore.java  |  225 +++
 .../quotas/SpaceQuotaSnapshotNotifier.java      |   45 +
 .../SpaceQuotaSnapshotNotifierFactory.java      |   62 +
 .../quotas/SpaceQuotaViolationNotifier.java     |   54 -
 .../SpaceQuotaViolationNotifierFactory.java     |   62 -
 .../SpaceQuotaViolationNotifierForTest.java     |   54 -
 ...SpaceQuotaViolationPolicyRefresherChore.java |  154 --
 .../quotas/SpaceViolationPolicyEnforcement.java |   91 ++
 .../SpaceViolationPolicyEnforcementFactory.java |   95 ++
 .../hbase/quotas/TableQuotaSnapshotStore.java   |  127 ++
 .../hbase/quotas/TableQuotaViolationStore.java  |  127 --
 .../quotas/TableSpaceQuotaSnapshotNotifier.java |   52 +
 .../TableSpaceQuotaViolationNotifier.java       |   55 -
 .../AbstractViolationPolicyEnforcement.java     |  118 ++
 ...LoadVerifyingViolationPolicyEnforcement.java |   50 +
 .../DisableTableViolationPolicyEnforcement.java |   80 +
 .../NoInsertsViolationPolicyEnforcement.java    |   55 +
 ...esCompactionsViolationPolicyEnforcement.java |   64 +
 .../NoWritesViolationPolicyEnforcement.java     |   54 +
 .../hbase/regionserver/CompactSplitThread.java  |   12 +
 .../hbase/regionserver/RSRpcServices.java       |   92 +-
 .../hbase/quotas/SpaceQuotaHelperForTests.java  |  228 +++
 .../SpaceQuotaSnapshotNotifierForTest.java      |   55 +
 .../quotas/TestActivePolicyEnforcement.java     |   74 +
 .../quotas/TestFileSystemUtilizationChore.java  |    3 +-
 .../TestNamespaceQuotaViolationStore.java       |   16 +-
 .../hbase/quotas/TestQuotaObserverChore.java    |   30 +-
 .../TestQuotaObserverChoreWithMiniCluster.java  |  351 ++---
 .../hadoop/hbase/quotas/TestQuotaTableUtil.java |   34 +-
 .../TestRegionServerSpaceQuotaManager.java      |  174 ++-
 ...SpaceQuotaViolationPolicyRefresherChore.java |  193 ++-
 .../hadoop/hbase/quotas/TestSpaceQuotas.java    |  452 ++++++
 .../quotas/TestTableQuotaViolationStore.java    |   22 +-
 .../TestTableSpaceQuotaViolationNotifier.java   |   48 +-
 .../hbase/quotas/TestTablesWithQuotas.java      |    8 +-
 .../BaseViolationPolicyEnforcement.java         |   31 +
 ...kLoadCheckingViolationPolicyEnforcement.java |  142 ++
 ...tDisableTableViolationPolicyEnforcement.java |   59 +
 ...TestNoInsertsViolationPolicyEnforcement.java |   57 +
 ...esCompactionsViolationPolicyEnforcement.java |   58 +
 .../TestNoWritesViolationPolicyEnforcement.java |   57 +
 57 files changed, 6492 insertions(+), 1481 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaExceededException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaExceededException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaExceededException.java
index 0ab75da..fc1e41c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaExceededException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaExceededException.java
@@ -29,4 +29,8 @@ public class QuotaExceededException extends DoNotRetryIOException {
   public QuotaExceededException(String msg) {
     super(msg);
   }
+
+  public QuotaExceededException(String msg, Throwable cause) {
+    super(msg, cause);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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 b5eac48..66535b2 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
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
@@ -52,6 +51,7 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -62,8 +62,9 @@ import org.apache.hadoop.hbase.util.Strings;
  * <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:v        &lt;space violation policy&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;
@@ -82,7 +83,9 @@ public class QuotaTableUtil {
   protected static final byte[] QUOTA_FAMILY_USAGE = Bytes.toBytes("u");
   protected static final byte[] QUOTA_QUALIFIER_SETTINGS = Bytes.toBytes("s");
   protected static final byte[] QUOTA_QUALIFIER_SETTINGS_PREFIX = Bytes.toBytes("s.");
-  protected static final byte[] QUOTA_QUALIFIER_VIOLATION = Bytes.toBytes("v");
+  protected static final byte[] QUOTA_QUALIFIER_POLICY = Bytes.toBytes("p");
+  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.");
   protected static final byte[] QUOTA_TABLE_ROW_KEY_PREFIX = Bytes.toBytes("t.");
   protected static final byte[] QUOTA_NAMESPACE_ROW_KEY_PREFIX = Bytes.toBytes("n.");
@@ -214,10 +217,10 @@ public class QuotaTableUtil {
   /**
    * Creates a {@link Scan} which returns only quota violations from the quota table.
    */
-  public static Scan makeQuotaViolationScan() {
+  public static Scan makeQuotaSnapshotScan() {
     Scan s = new Scan();
     // Limit to "u:v" column
-    s.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
+    s.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY);
     // Limit rowspace to the "t:" prefix
     s.setRowPrefixFilter(QUOTA_TABLE_ROW_KEY_PREFIX);
     return s;
@@ -230,26 +233,25 @@ public class QuotaTableUtil {
    * will throw an {@link IllegalArgumentException}.
    *
    * @param result A row from the quota table.
-   * @param policies A map of policies to add the result of this method into.
+   * @param snapshots A map of violations to add the result of this method into.
    */
-  public static void extractViolationPolicy(
-      Result result, Map<TableName,SpaceViolationPolicy> policies) {
+  public static void extractQuotaSnapshot(
+      Result result, Map<TableName,SpaceQuotaSnapshot> snapshots) {
     byte[] row = Objects.requireNonNull(result).getRow();
     if (null == row) {
       throw new IllegalArgumentException("Provided result had a null row");
     }
     final TableName targetTableName = getTableFromRowKey(row);
-    Cell c = result.getColumnLatestCell(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
+    Cell c = result.getColumnLatestCell(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY);
     if (null == c) {
       throw new IllegalArgumentException("Result did not contain the expected column "
-          + Bytes.toString(QUOTA_FAMILY_USAGE) + ":" + Bytes.toString(QUOTA_QUALIFIER_VIOLATION)
-          + ", " + result.toString());
+          + QUOTA_POLICY_COLUMN + ", " + result.toString());
     }
     ByteString buffer = UnsafeByteOperations.unsafeWrap(
         c.getValueArray(), c.getValueOffset(), c.getValueLength());
     try {
-      SpaceQuota quota = SpaceQuota.parseFrom(buffer);
-      policies.put(targetTableName, getViolationPolicy(quota));
+      QuotaProtos.SpaceQuotaSnapshot snapshot = QuotaProtos.SpaceQuotaSnapshot.parseFrom(buffer);
+      snapshots.put(targetTableName, SpaceQuotaSnapshot.toSpaceQuotaSnapshot(snapshot));
     } catch (InvalidProtocolBufferException e) {
       throw new IllegalArgumentException(
           "Result did not contain a valid SpaceQuota protocol buffer message", e);
@@ -385,23 +387,14 @@ public class QuotaTableUtil {
   /**
    * Creates a {@link Put} to enable the given <code>policy</code> on the <code>table</code>.
    */
-  public static Put createEnableViolationPolicyUpdate(
-      TableName tableName, SpaceViolationPolicy policy) {
+  public static Put createPutSpaceSnapshot(TableName tableName, SpaceQuotaSnapshot snapshot) {
     Put p = new Put(getTableRowKey(tableName));
-    SpaceQuota quota = getProtoViolationPolicy(policy);
-    p.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION, quota.toByteArray());
+    p.addColumn(
+        QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY,
+        SpaceQuotaSnapshot.toProtoSnapshot(snapshot).toByteArray());
     return p;
   }
 
-  /**
-   * Creates a {@link Delete} to remove a policy on the given <code>table</code>.
-   */
-  public static Delete createRemoveViolationPolicyUpdate(TableName tableName) {
-    Delete d = new Delete(getTableRowKey(tableName));
-    d.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
-    return d;
-  }
-
   /* =========================================================================
    *  Quotas protobuf helpers
    */
@@ -536,4 +529,4 @@ public class QuotaTableUtil {
     }
     return ProtobufUtil.toViolationPolicy(proto.getViolationPolicy());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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
new file mode 100644
index 0000000..86d64d3
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.util.Objects;
+
+import org.apache.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;
+
+/**
+ * A point-in-time view of a space quota on a table.
+ */
+@InterfaceAudience.Private
+public class SpaceQuotaSnapshot {
+  private static final SpaceQuotaSnapshot NO_SUCH_SNAPSHOT = new SpaceQuotaSnapshot(
+      SpaceQuotaStatus.notInViolation(), 0, Long.MAX_VALUE);
+  private final SpaceQuotaStatus quotaStatus;
+  private final long usage;
+  private final long limit;
+
+  /**
+   * Encapsulates the state of a quota on a table. The quota may or may not be in violation.
+   * If it is in violation, there will be a non-null violation policy.
+   */
+  @InterfaceAudience.Private
+  public static class SpaceQuotaStatus {
+    private static final SpaceQuotaStatus NOT_IN_VIOLATION = new SpaceQuotaStatus(null, false);
+    final SpaceViolationPolicy policy;
+    final boolean inViolation;
+
+    public SpaceQuotaStatus(SpaceViolationPolicy policy) {
+      this.policy = Objects.requireNonNull(policy);
+      this.inViolation = true;
+    }
+
+    private SpaceQuotaStatus(SpaceViolationPolicy policy, boolean inViolation) {
+      this.policy = policy;
+      this.inViolation = inViolation;
+    }
+
+    /**
+     * The violation policy which may be null. Is guaranteed to be non-null if
+     * {@link #isInViolation()} is <code>true</code>, and <code>false</code>
+     * otherwise.
+     */
+    public SpaceViolationPolicy getPolicy() {
+      return policy;
+    }
+
+    /**
+     * <code>true</code> if the quota is being violated, <code>false</code> otherwise.
+     */
+    public boolean isInViolation() {
+      return inViolation;
+    }
+
+    /**
+     * Returns a singleton referring to a quota which is not in violation.
+     */
+    public static SpaceQuotaStatus notInViolation() {
+      return NOT_IN_VIOLATION;
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().append(policy == null ? 0 : policy.hashCode())
+          .append(inViolation).toHashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (o instanceof SpaceQuotaStatus) {
+        SpaceQuotaStatus other = (SpaceQuotaStatus) o;
+        return Objects.equals(policy, other.policy) && inViolation == other.inViolation;
+      }
+      return false;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder(getClass().getSimpleName());
+      sb.append("[policy=").append(policy);
+      sb.append(", inViolation=").append(inViolation).append("]");
+      return sb.toString();
+    }
+
+    public static QuotaProtos.SpaceQuotaStatus toProto(SpaceQuotaStatus status) {
+      QuotaProtos.SpaceQuotaStatus.Builder builder = QuotaProtos.SpaceQuotaStatus.newBuilder();
+      builder.setInViolation(status.inViolation);
+      if (status.isInViolation()) {
+        builder.setPolicy(ProtobufUtil.toProtoViolationPolicy(status.getPolicy()));
+      }
+      return builder.build();
+    }
+
+    public static SpaceQuotaStatus toStatus(QuotaProtos.SpaceQuotaStatus proto) {
+      if (proto.getInViolation()) {
+        return new SpaceQuotaStatus(ProtobufUtil.toViolationPolicy(proto.getPolicy()));
+      } else {
+        return NOT_IN_VIOLATION;
+      }
+    }
+  }
+
+  public SpaceQuotaSnapshot(SpaceQuotaStatus quotaStatus, long usage, long limit) {
+    this.quotaStatus = Objects.requireNonNull(quotaStatus);
+    this.usage = usage;
+    this.limit = limit;
+  }
+
+  /**
+   * Returns the status of the quota.
+   */
+  public SpaceQuotaStatus getQuotaStatus() {
+    return quotaStatus;
+  }
+
+  /**
+   * Returns the current usage, in bytes, of the target (e.g. table, namespace).
+   */
+  public long getUsage() {
+    return usage;
+  }
+
+  /**
+   * Returns the limit, in bytes, of the target (e.g. table, namespace).
+   */
+  public long getLimit() {
+    return limit;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder()
+        .append(quotaStatus.hashCode())
+        .append(usage)
+        .append(limit)
+        .toHashCode();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof SpaceQuotaSnapshot) {
+      SpaceQuotaSnapshot other = (SpaceQuotaSnapshot) o;
+      return quotaStatus.equals(other.quotaStatus) && usage == other.usage && limit == other.limit;
+    }
+    return false;
+  }
+
+  @Override
+  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]");
+    return sb.toString();
+  }
+
+  // ProtobufUtil is in hbase-client, and this doesn't need to be public.
+  public static SpaceQuotaSnapshot toSpaceQuotaSnapshot(QuotaProtos.SpaceQuotaSnapshot proto) {
+    return new SpaceQuotaSnapshot(SpaceQuotaStatus.toStatus(proto.getStatus()),
+        proto.getUsage(), proto.getLimit());
+  }
+
+  public static QuotaProtos.SpaceQuotaSnapshot toProtoSnapshot(SpaceQuotaSnapshot snapshot) {
+    return QuotaProtos.SpaceQuotaSnapshot.newBuilder()
+        .setStatus(SpaceQuotaStatus.toProto(snapshot.getQuotaStatus()))
+        .setUsage(snapshot.getUsage()).setLimit(snapshot.getLimit()).build();
+  }
+
+  /**
+   * Returns a singleton that corresponds to no snapshot information.
+   */
+  public static SpaceQuotaSnapshot getNoSuchSnapshot() {
+    return NO_SUCH_SNAPSHOT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
index 0ab2576..117e839 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
@@ -6018,6 +6018,1344 @@ public final class QuotaProtos {
 
   }
 
+  public interface SpaceQuotaStatusOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.SpaceQuotaStatus)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     */
+    boolean hasPolicy();
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy getPolicy();
+
+    /**
+     * <code>optional bool in_violation = 2;</code>
+     */
+    boolean hasInViolation();
+    /**
+     * <code>optional bool in_violation = 2;</code>
+     */
+    boolean getInViolation();
+  }
+  /**
+   * <pre>
+   * Represents the state of a quota on a table. Either the quota is not in violation
+   * or it is in violatino there is a violation policy which should be in effect.
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.SpaceQuotaStatus}
+   */
+  public  static final class SpaceQuotaStatus extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.SpaceQuotaStatus)
+      SpaceQuotaStatusOrBuilder {
+    // Use SpaceQuotaStatus.newBuilder() to construct.
+    private SpaceQuotaStatus(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private SpaceQuotaStatus() {
+      policy_ = 1;
+      inViolation_ = false;
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SpaceQuotaStatus(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy value = org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                policy_ = rawValue;
+              }
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              inViolation_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaStatus_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int POLICY_FIELD_NUMBER = 1;
+    private int policy_;
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     */
+    public boolean hasPolicy() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy getPolicy() {
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy result = org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.valueOf(policy_);
+      return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE : result;
+    }
+
+    public static final int IN_VIOLATION_FIELD_NUMBER = 2;
+    private boolean inViolation_;
+    /**
+     * <code>optional bool in_violation = 2;</code>
+     */
+    public boolean hasInViolation() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional bool in_violation = 2;</code>
+     */
+    public boolean getInViolation() {
+      return inViolation_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeEnum(1, policy_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(2, inViolation_);
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeEnumSize(1, policy_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeBoolSize(2, inViolation_);
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus other = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus) obj;
+
+      boolean result = true;
+      result = result && (hasPolicy() == other.hasPolicy());
+      if (hasPolicy()) {
+        result = result && policy_ == other.policy_;
+      }
+      result = result && (hasInViolation() == other.hasInViolation());
+      if (hasInViolation()) {
+        result = result && (getInViolation()
+            == other.getInViolation());
+      }
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasPolicy()) {
+        hash = (37 * hash) + POLICY_FIELD_NUMBER;
+        hash = (53 * hash) + policy_;
+      }
+      if (hasInViolation()) {
+        hash = (37 * hash) + IN_VIOLATION_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
+            getInViolation());
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     * Represents the state of a quota on a table. Either the quota is not in violation
+     * or it is in violatino there is a violation policy which should be in effect.
+     * </pre>
+     *
+     * Protobuf type {@code hbase.pb.SpaceQuotaStatus}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.SpaceQuotaStatus)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaStatus_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        policy_ = 1;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        inViolation_ = false;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaStatus_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus result = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.policy_ = policy_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.inViolation_ = inViolation_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance()) return this;
+        if (other.hasPolicy()) {
+          setPolicy(other.getPolicy());
+        }
+        if (other.hasInViolation()) {
+          setInViolation(other.getInViolation());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private int policy_ = 1;
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       */
+      public boolean hasPolicy() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy getPolicy() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy result = org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.valueOf(policy_);
+        return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE : result;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       */
+      public Builder setPolicy(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        policy_ = value.getNumber();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       */
+      public Builder clearPolicy() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        policy_ = 1;
+        onChanged();
+        return this;
+      }
+
+      private boolean inViolation_ ;
+      /**
+       * <code>optional bool in_violation = 2;</code>
+       */
+      public boolean hasInViolation() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional bool in_violation = 2;</code>
+       */
+      public boolean getInViolation() {
+        return inViolation_;
+      }
+      /**
+       * <code>optional bool in_violation = 2;</code>
+       */
+      public Builder setInViolation(boolean value) {
+        bitField0_ |= 0x00000002;
+        inViolation_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool in_violation = 2;</code>
+       */
+      public Builder clearInViolation() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        inViolation_ = false;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SpaceQuotaStatus)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.SpaceQuotaStatus)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceQuotaStatus>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<SpaceQuotaStatus>() {
+      public SpaceQuotaStatus parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new SpaceQuotaStatus(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceQuotaStatus> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceQuotaStatus> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface SpaceQuotaSnapshotOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.SpaceQuotaSnapshot)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    boolean hasStatus();
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus getStatus();
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getStatusOrBuilder();
+
+    /**
+     * <code>optional uint64 usage = 2;</code>
+     */
+    boolean hasUsage();
+    /**
+     * <code>optional uint64 usage = 2;</code>
+     */
+    long getUsage();
+
+    /**
+     * <code>optional uint64 limit = 3;</code>
+     */
+    boolean hasLimit();
+    /**
+     * <code>optional uint64 limit = 3;</code>
+     */
+    long getLimit();
+  }
+  /**
+   * <pre>
+   * Message stored in the value of hbase:quota table to denote the status of a table WRT
+   * the quota applicable to it.
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.SpaceQuotaSnapshot}
+   */
+  public  static final class SpaceQuotaSnapshot extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.SpaceQuotaSnapshot)
+      SpaceQuotaSnapshotOrBuilder {
+    // Use SpaceQuotaSnapshot.newBuilder() to construct.
+    private SpaceQuotaSnapshot(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private SpaceQuotaSnapshot() {
+      usage_ = 0L;
+      limit_ = 0L;
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SpaceQuotaSnapshot(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = status_.toBuilder();
+              }
+              status_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(status_);
+                status_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              usage_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              limit_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int STATUS_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus status_;
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    public boolean hasStatus() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus getStatus() {
+      return status_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance() : status_;
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getStatusOrBuilder() {
+      return status_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance() : status_;
+    }
+
+    public static final int USAGE_FIELD_NUMBER = 2;
+    private long usage_;
+    /**
+     * <code>optional uint64 usage = 2;</code>
+     */
+    public boolean hasUsage() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional uint64 usage = 2;</code>
+     */
+    public long getUsage() {
+      return usage_;
+    }
+
+    public static final int LIMIT_FIELD_NUMBER = 3;
+    private long limit_;
+    /**
+     * <code>optional uint64 limit = 3;</code>
+     */
+    public boolean hasLimit() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional uint64 limit = 3;</code>
+     */
+    public long getLimit() {
+      return limit_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, getStatus());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, usage_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt64(3, limit_);
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getStatus());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, usage_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(3, limit_);
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot other = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot) obj;
+
+      boolean result = true;
+      result = result && (hasStatus() == other.hasStatus());
+      if (hasStatus()) {
+        result = result && getStatus()
+            .equals(other.getStatus());
+      }
+      result = result && (hasUsage() == other.hasUsage());
+      if (hasUsage()) {
+        result = result && (getUsage()
+            == other.getUsage());
+      }
+      result = result && (hasLimit() == other.hasLimit());
+      if (hasLimit()) {
+        result = result && (getLimit()
+            == other.getLimit());
+      }
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasStatus()) {
+        hash = (37 * hash) + STATUS_FIELD_NUMBER;
+        hash = (53 * hash) + getStatus().hashCode();
+      }
+      if (hasUsage()) {
+        hash = (37 * hash) + USAGE_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+            getUsage());
+      }
+      if (hasLimit()) {
+        hash = (37 * hash) + LIMIT_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+            getLimit());
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     * Message stored in the value of hbase:quota table to denote the status of a table WRT
+     * the quota applicable to it.
+     * </pre>
+     *
+     * Protobuf type {@code hbase.pb.SpaceQuotaSnapshot}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.SpaceQuotaSnapshot)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getStatusFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (statusBuilder_ == null) {
+          status_ = null;
+        } else {
+          statusBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        usage_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        limit_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot result = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (statusBuilder_ == null) {
+          result.status_ = status_;
+        } else {
+          result.status_ = statusBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.usage_ = usage_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.limit_ = limit_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.getDefaultInstance()) return this;
+        if (other.hasStatus()) {
+          mergeStatus(other.getStatus());
+        }
+        if (other.hasUsage()) {
+          setUsage(other.getUsage());
+        }
+        if (other.hasLimit()) {
+          setLimit(other.getLimit());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus status_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder> statusBuilder_;
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public boolean hasStatus() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus getStatus() {
+        if (statusBuilder_ == null) {
+          return status_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance() : status_;
+        } else {
+          return statusBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public Builder setStatus(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
+        if (statusBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          status_ = value;
+          onChanged();
+        } else {
+          statusBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public Builder setStatus(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder builderForValue) {
+        if (statusBuilder_ == null) {
+          status_ = builderForValue.build();
+          onChanged();
+        } else {
+          statusBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public Builder mergeStatus(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
+        if (statusBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              status_ != null &&
+              status_ != org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance()) {
+            status_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.newBuilder(status_).mergeFrom(value).buildPartial();
+          } else {
+            status_ = value;
+          }
+          onChanged();
+        } else {
+          statusBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public Builder clearStatus() {
+        if (statusBuilder_ == null) {
+          status_ = null;
+          onChanged();
+        } else {
+          statusBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder getStatusBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getStatusFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getStatusOrBuilder() {
+        if (statusBuilder_ != null) {
+          return statusBuilder_.getMessageOrBuilder();
+        } else {
+          return status_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance() : status_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder>
+          getStatusFieldBuilder() {
+        if (statusBuilder_ == null) {
+          statusBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder>(
+                  getStatus(),
+                  getParentForChildren(),
+                  isClean());
+          status_ = null;
+        }
+        return statusBuilder_;
+      }
+
+      private long usage_ ;
+      /**
+       * <code>optional uint64 usage = 2;</code>
+       */
+      public boolean hasUsage() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional uint64 usage = 2;</code>
+       */
+      public long getUsage() {
+        return usage_;
+      }
+      /**
+       * <code>optional uint64 usage = 2;</code>
+       */
+      public Builder setUsage(long value) {
+        bitField0_ |= 0x00000002;
+        usage_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 usage = 2;</code>
+       */
+      public Builder clearUsage() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        usage_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      private long limit_ ;
+      /**
+       * <code>optional uint64 limit = 3;</code>
+       */
+      public boolean hasLimit() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional uint64 limit = 3;</code>
+       */
+      public long getLimit() {
+        return limit_;
+      }
+      /**
+       * <code>optional uint64 limit = 3;</code>
+       */
+      public Builder setLimit(long value) {
+        bitField0_ |= 0x00000004;
+        limit_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 limit = 3;</code>
+       */
+      public Builder clearLimit() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        limit_ = 0L;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SpaceQuotaSnapshot)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.SpaceQuotaSnapshot)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceQuotaSnapshot>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<SpaceQuotaSnapshot>() {
+      public SpaceQuotaSnapshot parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new SpaceQuotaSnapshot(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceQuotaSnapshot> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceQuotaSnapshot> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TimedQuota_descriptor;
   private static final
@@ -6053,6 +7391,16 @@ public final class QuotaProtos {
   private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable;
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_SpaceQuotaStatus_descriptor;
+  private static final
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable;
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor;
+  private static final
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable;
 
   public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -6082,16 +7430,20 @@ public final class QuotaProtos {
       "limit\030\001 \001(\004\0228\n\020violation_policy\030\002 \001(\0162\036." +
       "hbase.pb.SpaceViolationPolicy\"8\n\021SpaceLi" +
       "mitRequest\022#\n\005quota\030\001 \001(\0132\024.hbase.pb.Spa",
-      "ceQuota*&\n\nQuotaScope\022\013\n\007CLUSTER\020\001\022\013\n\007MA" +
-      "CHINE\020\002*v\n\014ThrottleType\022\022\n\016REQUEST_NUMBE" +
-      "R\020\001\022\020\n\014REQUEST_SIZE\020\002\022\020\n\014WRITE_NUMBER\020\003\022" +
-      "\016\n\nWRITE_SIZE\020\004\022\017\n\013READ_NUMBER\020\005\022\r\n\tREAD" +
-      "_SIZE\020\006*$\n\tQuotaType\022\014\n\010THROTTLE\020\001\022\t\n\005SP" +
-      "ACE\020\002*]\n\024SpaceViolationPolicy\022\013\n\007DISABLE" +
-      "\020\001\022\031\n\025NO_WRITES_COMPACTIONS\020\002\022\r\n\tNO_WRIT" +
-      "ES\020\003\022\016\n\nNO_INSERTS\020\004BH\n1org.apache.hadoo" +
-      "p.hbase.shaded.protobuf.generatedB\013Quota" +
-      "ProtosH\001\210\001\001\240\001\001"
+      "ceQuota\"X\n\020SpaceQuotaStatus\022.\n\006policy\030\001 " +
+      "\001(\0162\036.hbase.pb.SpaceViolationPolicy\022\024\n\014i" +
+      "n_violation\030\002 \001(\010\"^\n\022SpaceQuotaSnapshot\022" +
+      "*\n\006status\030\001 \001(\0132\032.hbase.pb.SpaceQuotaSta" +
+      "tus\022\r\n\005usage\030\002 \001(\004\022\r\n\005limit\030\003 \001(\004*&\n\nQuo" +
+      "taScope\022\013\n\007CLUSTER\020\001\022\013\n\007MACHINE\020\002*v\n\014Thr" +
+      "ottleType\022\022\n\016REQUEST_NUMBER\020\001\022\020\n\014REQUEST" +
+      "_SIZE\020\002\022\020\n\014WRITE_NUMBER\020\003\022\016\n\nWRITE_SIZE\020" +
+      "\004\022\017\n\013READ_NUMBER\020\005\022\r\n\tREAD_SIZE\020\006*$\n\tQuo" +
+      "taType\022\014\n\010THROTTLE\020\001\022\t\n\005SPACE\020\002*]\n\024Space",
+      "ViolationPolicy\022\013\n\007DISABLE\020\001\022\031\n\025NO_WRITE" +
+      "S_COMPACTIONS\020\002\022\r\n\tNO_WRITES\020\003\022\016\n\nNO_INS" +
+      "ERTS\020\004BH\n1org.apache.hadoop.hbase.shaded" +
+      ".protobuf.generatedB\013QuotaProtosH\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -6148,6 +7500,18 @@ public final class QuotaProtos {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SpaceLimitRequest_descriptor,
         new java.lang.String[] { "Quota", });
+    internal_static_hbase_pb_SpaceQuotaStatus_descriptor =
+      getDescriptor().getMessageTypes().get(7);
+    internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable = new
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_hbase_pb_SpaceQuotaStatus_descriptor,
+        new java.lang.String[] { "Policy", "InViolation", });
+    internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor =
+      getDescriptor().getMessageTypes().get(8);
+    internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable = new
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor,
+        new java.lang.String[] { "Status", "Usage", "Limit", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-protocol-shaded/src/main/protobuf/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Quota.proto b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
index b53219a..5ea1ddb 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Quota.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
@@ -95,3 +95,18 @@ message SpaceQuota {
 message SpaceLimitRequest {
   optional SpaceQuota quota = 1;
 }
+
+// Represents the state of a quota on a table. Either the quota is not in violation
+// or it is in violatino there is a violation policy which should be in effect.
+message SpaceQuotaStatus {
+  optional SpaceViolationPolicy policy = 1;
+  optional bool in_violation = 2;
+}
+
+// Message stored in the value of hbase:quota table to denote the status of a table WRT
+// the quota applicable to it.
+message SpaceQuotaSnapshot {
+  optional SpaceQuotaStatus status = 1;
+  optional uint64 usage = 2;
+  optional uint64 limit = 3;
+}


[41/50] [abbrv] hbase git commit: HBASE-17428 Implement informational RPCs for space quotas

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
index e90c934..c70b736 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
@@ -10429,7 +10429,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -10824,7 +10824,7 @@ public final class RegionServerStatusProtos {
        * <code>optional .hbase.pb.RegionInfo region = 1;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
           regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -10940,7 +10940,7 @@ public final class RegionServerStatusProtos {
     /**
      * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse>
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse> 
         getSpaceUseList();
     /**
      * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
@@ -10953,7 +10953,7 @@ public final class RegionServerStatusProtos {
     /**
      * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder> 
         getSpaceUseOrBuilderList();
     /**
      * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
@@ -11056,7 +11056,7 @@ public final class RegionServerStatusProtos {
     /**
      * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder> 
         getSpaceUseOrBuilderList() {
       return spaceUse_;
     }
@@ -11142,7 +11142,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getSpaceUseCount() > 0) {
         hash = (37 * hash) + SPACE_USE_FIELD_NUMBER;
         hash = (53 * hash) + getSpaceUseList().hashCode();
@@ -11368,7 +11368,7 @@ public final class RegionServerStatusProtos {
               spaceUseBuilder_ = null;
               spaceUse_ = other.spaceUse_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              spaceUseBuilder_ =
+              spaceUseBuilder_ = 
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getSpaceUseFieldBuilder() : null;
             } else {
@@ -11604,7 +11604,7 @@ public final class RegionServerStatusProtos {
       /**
        * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder> 
            getSpaceUseOrBuilderList() {
         if (spaceUseBuilder_ != null) {
           return spaceUseBuilder_.getMessageOrBuilderList();
@@ -11630,12 +11630,12 @@ public final class RegionServerStatusProtos {
       /**
        * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder>
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder> 
            getSpaceUseBuilderList() {
         return getSpaceUseFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
+          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder> 
           getSpaceUseFieldBuilder() {
         if (spaceUseBuilder_ == null) {
           spaceUseBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -11813,7 +11813,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -12950,17 +12950,17 @@ public final class RegionServerStatusProtos {
       internal_static_hbase_pb_SplitTableRegionResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionSpaceUse_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionSpaceUse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionSpaceUseReportRequest_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionSpaceUseReportRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionSpaceUseReportResponse_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionSpaceUseReportResponse_fieldAccessorTable;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-protocol-shaded/src/main/protobuf/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index 338c80b..1a085e6 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -28,6 +28,7 @@ option optimize_for = SPEED;
 import "ClusterStatus.proto";
 import "HBase.proto";
 import "WAL.proto";
+import "Quota.proto";
 
 message GetRegionInfoRequest {
   required RegionSpecifier region = 1;
@@ -314,4 +315,12 @@ service AdminService {
 
   rpc GetRegionLoad(GetRegionLoadRequest)
     returns(GetRegionLoadResponse);
+
+  /** Fetches the RegionServer's view of space quotas */
+  rpc GetSpaceQuotaSnapshots(GetSpaceQuotaSnapshotsRequest)
+    returns(GetSpaceQuotaSnapshotsResponse);
+
+  /** Fetches the RegionServer's space quota active enforcements */
+  rpc GetSpaceQuotaEnforcements(GetSpaceQuotaEnforcementsRequest)
+    returns(GetSpaceQuotaEnforcementsResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 7b27663..3318a39 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -929,4 +929,8 @@ service MasterService {
   /** Unmark a list of ServerNames marked as draining. */
   rpc removeDrainFromRegionServers(RemoveDrainFromRegionServersRequest)
     returns(RemoveDrainFromRegionServersResponse);
+
+  /** Fetches the Master's view of space quotas */
+  rpc GetSpaceQuotaRegionSizes(GetSpaceQuotaRegionSizesRequest)
+    returns(GetSpaceQuotaRegionSizesResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-protocol-shaded/src/main/protobuf/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Quota.proto b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
index 597b059..2d7e5f5 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Quota.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
@@ -111,3 +111,38 @@ message SpaceQuotaSnapshot {
   optional uint64 usage = 2;
   optional uint64 limit = 3;
 }
+
+message GetSpaceQuotaRegionSizesRequest {
+}
+
+message GetSpaceQuotaRegionSizesResponse {
+  message RegionSizes {
+    optional TableName table_name = 1;
+    optional uint64 size = 2;
+  }
+  repeated RegionSizes sizes = 1;
+}
+
+message GetSpaceQuotaSnapshotsRequest {
+}
+
+message GetSpaceQuotaSnapshotsResponse {
+  // Cannot use TableName as a map key, do the repeated nested message by hand.
+  message TableQuotaSnapshot {
+    optional TableName table_name = 1;
+    optional SpaceQuotaSnapshot snapshot = 2;
+  }
+  repeated TableQuotaSnapshot snapshots = 1;
+}
+
+message GetSpaceQuotaEnforcementsRequest {
+}
+
+message GetSpaceQuotaEnforcementsResponse {
+  // Cannot use TableName as a map key, do the repeated nested message by hand.
+  message TableViolationPolicy {
+    optional TableName table_name = 1;
+    optional SpaceViolationPolicy violation_policy = 2;
+  }
+  repeated TableViolationPolicy violation_policies = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
index 3f65424..96a8ab2 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
@@ -4258,7 +4258,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
           getSpaceFieldBuilder() {
         if (spaceBuilder_ == null) {
           spaceBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -5876,7 +5876,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
           getQuotaFieldBuilder() {
         if (quotaBuilder_ == null) {
           quotaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -7088,7 +7088,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder>
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder> 
           getStatusFieldBuilder() {
         if (statusBuilder_ == null) {
           statusBuilder_ = new com.google.protobuf.SingleFieldBuilder<

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index de437fd..42f99b2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -21,8 +21,11 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.commons.logging.Log;
@@ -211,6 +214,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@@ -2029,4 +2035,38 @@ public class MasterRpcServices extends RSRpcServices
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(
+      RpcController controller, GetSpaceQuotaRegionSizesRequest request) throws ServiceException {
+    try {
+      master.checkInitialized();
+      MasterQuotaManager quotaManager = this.master.getMasterQuotaManager();
+      GetSpaceQuotaRegionSizesResponse.Builder builder =
+          GetSpaceQuotaRegionSizesResponse.newBuilder();
+      if (null != quotaManager) {
+        Map<HRegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
+        Map<TableName,Long> regionSizesByTable = new HashMap<>();
+        // Translate hregioninfo+long -> tablename+long
+        for (Entry<HRegionInfo,Long> entry : regionSizes.entrySet()) {
+          final TableName tableName = entry.getKey().getTable();
+          Long prevSize = regionSizesByTable.get(tableName);
+          if (null == prevSize) {
+            prevSize = 0L;
+          }
+          regionSizesByTable.put(tableName, prevSize + entry.getValue());
+        }
+        // Serialize them into the protobuf
+        for (Entry<TableName,Long> tableSize : regionSizesByTable.entrySet()) {
+          builder.addSizes(RegionSizes.newBuilder()
+              .setTableName(ProtobufUtil.toProtoTableName(tableSize.getKey()))
+              .setSize(tableSize.getValue()).build());
+        }
+        return builder.build();
+      }
+      return builder.build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
index 9408e6c..a313fa1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
@@ -16,6 +16,7 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+import java.util.Collections;
 import java.util.Map;
 import java.util.Objects;
 
@@ -79,6 +80,13 @@ public class ActivePolicyEnforcement {
     return policy;
   }
 
+  /**
+   * Returns an unmodifiable version of the active {@link SpaceViolationPolicyEnforcement}s.
+   */
+  public Map<TableName,SpaceViolationPolicyEnforcement> getPolicies() {
+    return Collections.unmodifiableMap(activePolicies);
+  }
+
   @Override
   public String toString() {
     return getClass().getSimpleName() + ": " + activePolicies;

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 81b87d5..1e4bab2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -101,6 +101,7 @@ import org.apache.hadoop.hbase.quotas.OperationQuota;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.regionserver.Leases.Lease;
@@ -194,6 +195,13 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
@@ -3280,4 +3288,53 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return UpdateConfigurationResponse.getDefaultInstance();
   }
 
+  @Override
+  public GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(
+      RpcController controller, GetSpaceQuotaSnapshotsRequest request) throws ServiceException {
+    try {
+      final RegionServerSpaceQuotaManager manager =
+          regionServer.getRegionServerSpaceQuotaManager();
+      final GetSpaceQuotaSnapshotsResponse.Builder builder =
+          GetSpaceQuotaSnapshotsResponse.newBuilder();
+      if (null != manager) {
+        final Map<TableName,SpaceQuotaSnapshot> snapshots = manager.copyQuotaSnapshots();
+        for (Entry<TableName,SpaceQuotaSnapshot> snapshot : snapshots.entrySet()) {
+          builder.addSnapshots(TableQuotaSnapshot.newBuilder()
+              .setTableName(ProtobufUtil.toProtoTableName(snapshot.getKey()))
+              .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(snapshot.getValue()))
+              .build());
+        }
+      }
+      return builder.build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetSpaceQuotaEnforcementsResponse getSpaceQuotaEnforcements(
+      RpcController controller, GetSpaceQuotaEnforcementsRequest request)
+      throws ServiceException {
+    try {
+      final RegionServerSpaceQuotaManager manager =
+          regionServer.getRegionServerSpaceQuotaManager();
+      final GetSpaceQuotaEnforcementsResponse.Builder builder =
+          GetSpaceQuotaEnforcementsResponse.newBuilder();
+      if (null != manager) {
+        ActivePolicyEnforcement enforcements = manager.getActiveEnforcements();
+        for (Entry<TableName,SpaceViolationPolicyEnforcement> enforcement
+            : enforcements.getPolicies().entrySet()) {
+          SpaceViolationPolicy pbPolicy = SpaceViolationPolicy.valueOf(
+              enforcement.getValue().getPolicyName());
+          builder.addViolationPolicies(TableViolationPolicy.newBuilder()
+              .setTableName(ProtobufUtil.toProtoTableName(enforcement.getKey()))
+              .setViolationPolicy(pbPolicy).build());
+        }
+      }
+      return builder.build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index f133afc..e52114e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -100,6 +100,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBul
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
@@ -726,4 +730,18 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   public RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager() {
     return null;
   }
+
+  @Override
+  public GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(
+      RpcController controller, GetSpaceQuotaSnapshotsRequest request)
+      throws ServiceException {
+    return null;
+  }
+
+  @Override
+  public GetSpaceQuotaEnforcementsResponse getSpaceQuotaEnforcements(
+      RpcController controller, GetSpaceQuotaEnforcementsRequest request)
+      throws ServiceException {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/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
new file mode 100644
index 0000000..d42f3d2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
@@ -0,0 +1,192 @@
+/*
+ * 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.assertTrue;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+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;
+
+/**
+ * Test class for the quota status RPCs in the master and regionserver.
+ */
+@Category({MediumTests.class})
+public class TestQuotaStatusRPCs {
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final AtomicLong COUNTER = new AtomicLong(0);
+
+  @Rule
+  public TestName testName = new TestName();
+  private SpaceQuotaHelperForTests helper;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    // Increase the frequency of some of the chores for responsiveness of the test
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_KEY, 1000);
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 1000);
+    conf.setInt(QuotaObserverChore.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);
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setupForTest() throws Exception {
+    helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
+  }
+
+  @Test
+  public void testRegionSizesFromMaster() throws Exception {
+    final long tableSize = 1024L * 10L; // 10KB
+    final int numRegions = 10;
+    final TableName tn = helper.createTableWithRegions(numRegions);
+    // Will write at least `tableSize` data
+    helper.writeData(tn, tableSize);
+
+    final HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
+    final MasterQuotaManager quotaManager = master.getMasterQuotaManager();
+    // Make sure the master has all of the reports
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return numRegions == countRegionsForTable(tn, quotaManager.snapshotRegionSizes());
+      }
+    });
+
+    Map<TableName,Long> sizes = QuotaTableUtil.getMasterReportedTableSizes(TEST_UTIL.getConnection());
+    Long size = sizes.get(tn);
+    assertNotNull("No reported size for " + tn, size);
+    assertTrue("Reported table size was " + size, size.longValue() >= tableSize);
+  }
+
+  @Test
+  public void testQuotaSnapshotsFromRS() throws Exception {
+    final long sizeLimit = 1024L * 1024L; // 1MB
+    final long tableSize = 1024L * 10L; // 10KB
+    final int numRegions = 10;
+    final TableName tn = helper.createTableWithRegions(numRegions);
+
+    // Define the quota
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, sizeLimit, SpaceViolationPolicy.NO_INSERTS);
+    TEST_UTIL.getAdmin().setQuota(settings);
+
+    // Write at least `tableSize` data
+    helper.writeData(tn, tableSize);
+
+    final HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
+    final RegionServerSpaceQuotaManager manager = rs.getRegionServerSpaceQuotaManager();
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        SpaceQuotaSnapshot snapshot = manager.copyQuotaSnapshots().get(tn);
+        if (null == snapshot) {
+          return false;
+        }
+        return snapshot.getUsage() >= tableSize;
+      }
+    });
+
+    Map<TableName, SpaceQuotaSnapshot> snapshots = QuotaTableUtil.getRegionServerQuotaSnapshots(
+        TEST_UTIL.getConnection(), rs.getServerName());
+    SpaceQuotaSnapshot snapshot = snapshots.get(tn);
+    assertNotNull("Did not find snapshot for " + tn, snapshot);
+    assertTrue(
+        "Observed table usage was " + snapshot.getUsage(),
+        snapshot.getUsage() >= tableSize);
+    assertEquals(snapshot.getLimit(), sizeLimit);
+    SpaceQuotaStatus pbStatus = snapshot.getQuotaStatus();
+    assertFalse(pbStatus.isInViolation());
+  }
+
+  @Test
+  public void testQuotaEnforcementsFromRS() throws Exception {
+    final long sizeLimit = 1024L * 8L; // 8KB
+    final long tableSize = 1024L * 10L; // 10KB
+    final int numRegions = 10;
+    final TableName tn = helper.createTableWithRegions(numRegions);
+
+    // Define the quota
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, sizeLimit, SpaceViolationPolicy.NO_INSERTS);
+    TEST_UTIL.getAdmin().setQuota(settings);
+
+    // Write at least `tableSize` data
+    try {
+      helper.writeData(tn, tableSize);
+    } catch (SpaceLimitingException e) {
+      // Pass
+    }
+
+    final HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
+    final RegionServerSpaceQuotaManager manager = rs.getRegionServerSpaceQuotaManager();
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        ActivePolicyEnforcement enforcements = manager.getActiveEnforcements();
+        SpaceViolationPolicyEnforcement enforcement = enforcements.getPolicyEnforcement(tn);
+        return enforcement.getQuotaSnapshot().getQuotaStatus().isInViolation();
+      }
+    });
+
+    Map<TableName,SpaceViolationPolicy> violations =
+        QuotaTableUtil.getRegionServerQuotaViolations(
+            TEST_UTIL.getConnection(), rs.getServerName());
+    SpaceViolationPolicy policy = violations.get(tn);
+    assertNotNull("Did not find policy for " + tn, policy);
+    assertEquals(SpaceViolationPolicy.NO_INSERTS, policy);
+  }
+
+  private int countRegionsForTable(TableName tn, Map<HRegionInfo,Long> regionSizes) {
+    int size = 0;
+    for (HRegionInfo regionInfo : regionSizes.keySet()) {
+      if (tn.equals(regionInfo.getTable())) {
+        size++;
+      }
+    }
+    return size;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-shell/src/main/ruby/hbase/quotas.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/quotas.rb b/hbase-shell/src/main/ruby/hbase/quotas.rb
index d99fe72..a2b21fa 100644
--- a/hbase-shell/src/main/ruby/hbase/quotas.rb
+++ b/hbase-shell/src/main/ruby/hbase/quotas.rb
@@ -20,10 +20,12 @@
 include Java
 java_import java.util.concurrent.TimeUnit
 java_import org.apache.hadoop.hbase.TableName
+java_import org.apache.hadoop.hbase.ServerName
 java_import org.apache.hadoop.hbase.quotas.ThrottleType
 java_import org.apache.hadoop.hbase.quotas.QuotaFilter
 java_import org.apache.hadoop.hbase.quotas.QuotaRetriever
 java_import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory
+java_import org.apache.hadoop.hbase.quotas.QuotaTableUtil
 java_import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy
 
 module HBaseQuotasConstants
@@ -163,6 +165,20 @@ module Hbase
       @admin.setQuota(settings)
     end
 
+    def get_master_table_sizes()
+      QuotaTableUtil.getMasterReportedTableSizes(@admin.getConnection())
+    end
+
+    def get_rs_quota_snapshots(rs)
+      QuotaTableUtil.getRegionServerQuotaSnapshots(@admin.getConnection(),
+          ServerName.valueOf(rs))
+    end
+
+    def get_rs_quota_violations(rs)
+      QuotaTableUtil.getRegionServerQuotaViolations(@admin.getConnection(),
+          ServerName.valueOf(rs))
+    end
+
     def set_global_bypass(bypass, args)
       raise(ArgumentError, "Arguments should be a Hash") unless args.kind_of?(Hash)
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 66480f9..b203edc 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -419,6 +419,9 @@ Shell.load_command_group(
   :commands => %w[
     set_quota
     list_quotas
+    list_quota_table_sizes
+    list_quota_violations
+    list_quota_snapshots
   ]
 )
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-shell/src/main/ruby/shell/commands/list_quota_snapshots.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_quota_snapshots.rb b/hbase-shell/src/main/ruby/shell/commands/list_quota_snapshots.rb
new file mode 100644
index 0000000..c907762
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_quota_snapshots.rb
@@ -0,0 +1,59 @@
+#
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class ListQuotaSnapshots < Command
+      def help
+        return <<-EOF
+Lists the current snapshot of quotas on the given RegionServer. This
+information filters to each RegionServer from the Master. For each
+table, a snapshot includes the filesystem use, the filesystem limit,
+and the policy to enact when the limit is exceeded. This command is
+useful for debugging the running state of a cluster using filesystem quotas.
+
+For example:
+
+    hbase> list_quota_snapshots 'regionserver1.domain,16020,1483482894742'
+EOF
+      end
+
+      def command(hostname, args = {})
+        formatter.header(["TABLE", "USAGE", "LIMIT", "IN VIOLATION", "POLICY"])
+        count = 0
+        quotas_admin.get_rs_quota_snapshots(hostname).each do |tableName,snapshot|
+          status = snapshot.getQuotaStatus()
+          policy = get_policy(status)
+          formatter.row([tableName.to_s, snapshot.getUsage().to_s, snapshot.getLimit().to_s, status.isInViolation().to_s, policy])
+          count += 1
+        end
+        formatter.footer(count)
+      end
+
+      def get_policy(status)
+        # Unwrap the violation policy if it exists
+        if status.isInViolation()
+          status.getPolicy().name()
+        else
+          "None"
+        end
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-shell/src/main/ruby/shell/commands/list_quota_table_sizes.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_quota_table_sizes.rb b/hbase-shell/src/main/ruby/shell/commands/list_quota_table_sizes.rb
new file mode 100644
index 0000000..9325477
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_quota_table_sizes.rb
@@ -0,0 +1,47 @@
+#
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class ListQuotaTableSizes < Command
+      def help
+        return <<-EOF
+Lists the sizes of the tables in HBase as collected
+for the purpose of implementing filesystem utilization
+quotas. This information is extracted from the HBase
+Master and drives future quota actions in the cluster.
+
+For example:
+
+    hbase> list_quota_table_sizes
+EOF
+      end
+
+      def command(args = {})
+        formatter.header(["TABLE", "SIZE"])
+        count = 0
+        quotas_admin.get_master_table_sizes().each do |tableName,size|
+          formatter.row([tableName.to_s, size.to_s])
+          count += 1
+        end
+        formatter.footer(count)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-shell/src/main/ruby/shell/commands/list_quota_violations.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_quota_violations.rb b/hbase-shell/src/main/ruby/shell/commands/list_quota_violations.rb
new file mode 100644
index 0000000..f1836a2
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_quota_violations.rb
@@ -0,0 +1,48 @@
+#
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class ListQuotaViolations < Command
+      def help
+        return <<-EOF
+Lists the current quota violations being enforced by a RegionServer.
+Violations are enacted based on the quota snapshot information a RegionServer
+holds (see list_quota_snapshots). Each violation contains the action the
+RegionServer is taking on the table. This command is useful in debugging
+the running state of a cluster using filesystem quotas.
+
+For example:
+
+    hbase> list_quota_violations 'regionserver1.domain,16020,1483482894742'
+EOF
+      end
+
+      def command(hostname, args = {})
+        formatter.header(["TABLE", "POLICY"])
+        count = 0
+        quotas_admin.get_rs_quota_violations(hostname).each do |tableName,policy|
+          formatter.row([tableName.to_s, policy.name])
+          count += 1
+        end
+        formatter.footer(count)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-shell/src/test/ruby/hbase/quotas_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/quotas_test.rb b/hbase-shell/src/test/ruby/hbase/quotas_test.rb
index 78c889c..076eaed 100644
--- a/hbase-shell/src/test/ruby/hbase/quotas_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/quotas_test.rb
@@ -96,30 +96,6 @@ module Hbase
       end
     end
 
-    define_test '_parse_size accepts various forms of byte shorthand' do
-      qa = quotas_admin()
-      KILO = 1024
-      MEGA = KILO * KILO
-      GIGA = MEGA * KILO
-      TERA = GIGA * KILO
-      PETA = TERA * KILO
-      assert_equal(1, qa._parse_size("1"))
-      assert_equal(1, qa._parse_size("1b"))
-      assert_equal(1, qa._parse_size("1B"))
-      assert_equal(KILO * 2, qa._parse_size("2k"))
-      assert_equal(KILO * 2, qa._parse_size("2K"))
-      assert_equal(MEGA * 5, qa._parse_size("5m"))
-      assert_equal(MEGA * 5, qa._parse_size("5M"))
-      assert_equal(GIGA * 3, qa._parse_size("3g"))
-      assert_equal(GIGA * 3, qa._parse_size("3G"))
-      assert_equal(TERA * 4, qa._parse_size("4t"))
-      assert_equal(TERA * 4, qa._parse_size("4T"))
-      assert_equal(PETA * 32, qa._parse_size("32p"))
-      assert_equal(PETA * 32, qa._parse_size("32P"))
-      assert_equal(GIGA * 4, qa._parse_size("4096m"))
-      assert_equal(GIGA * 4, qa._parse_size("4096M"))
-    end
-
     define_test 'can set and remove quota' do
       command(:set_quota, TYPE => SPACE, LIMIT => '1G', POLICY => NO_INSERTS, TABLE => @test_name)
       output = capture_stdout{ command(:list_quotas) }

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-shell/src/test/ruby/hbase/quotas_test_no_cluster.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/quotas_test_no_cluster.rb b/hbase-shell/src/test/ruby/hbase/quotas_test_no_cluster.rb
new file mode 100644
index 0000000..7504488
--- /dev/null
+++ b/hbase-shell/src/test/ruby/hbase/quotas_test_no_cluster.rb
@@ -0,0 +1,69 @@
+#
+#
+# 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.
+#
+
+require 'shell'
+require 'stringio'
+require 'hbase_constants'
+require 'hbase/hbase'
+require 'hbase/table'
+
+include HBaseConstants
+
+module Hbase
+  class NoClusterSpaceQuotasTest < Test::Unit::TestCase
+    include TestHelpers
+
+    define_test '_parse_size accepts various forms of byte shorthand' do
+      qa = ::Hbase::QuotasAdmin.new(nil)
+      KILO = 1024
+      MEGA = KILO * KILO
+      GIGA = MEGA * KILO
+      TERA = GIGA * KILO
+      PETA = TERA * KILO
+      assert_equal(1, qa._parse_size("1"))
+      assert_equal(1, qa._parse_size("1b"))
+      assert_equal(1, qa._parse_size("1B"))
+      assert_equal(KILO * 2, qa._parse_size("2k"))
+      assert_equal(KILO * 2, qa._parse_size("2K"))
+      assert_equal(MEGA * 5, qa._parse_size("5m"))
+      assert_equal(MEGA * 5, qa._parse_size("5M"))
+      assert_equal(GIGA * 3, qa._parse_size("3g"))
+      assert_equal(GIGA * 3, qa._parse_size("3G"))
+      assert_equal(TERA * 4, qa._parse_size("4t"))
+      assert_equal(TERA * 4, qa._parse_size("4T"))
+      assert_equal(PETA * 32, qa._parse_size("32p"))
+      assert_equal(PETA * 32, qa._parse_size("32P"))
+      assert_equal(GIGA * 4, qa._parse_size("4096m"))
+      assert_equal(GIGA * 4, qa._parse_size("4096M"))
+    end
+
+    define_test 'get policy name for status not in violation' do
+      java_import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot
+      java_import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy
+
+      okStatus = SpaceQuotaSnapshot::SpaceQuotaStatus::notInViolation()
+      # By default, statuses are in violation
+      violatedStatus = SpaceQuotaSnapshot::SpaceQuotaStatus.new(SpaceViolationPolicy::NO_INSERTS)
+      # Pass in nil for the Shell instance (that we don't care about)
+      quotaSnapshotCommand = ::Shell::Commands::ListQuotaSnapshots.new(nil)
+      assert_equal('None', quotaSnapshotCommand.get_policy(okStatus))
+      assert_equal('NO_INSERTS', quotaSnapshotCommand.get_policy(violatedStatus))
+    end
+  end
+end


[29/50] [abbrv] hbase git commit: HBASE-17001 Enforce quota violation policies in the RegionServer

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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 c493b25..943c898 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java
@@ -22,16 +22,12 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -40,20 +36,15 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.quotas.QuotaObserverChore.TablesWithQuotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -62,7 +53,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
 
@@ -72,11 +62,8 @@ import com.google.common.collect.Multimap;
 @Category(LargeTests.class)
 public class TestQuotaObserverChoreWithMiniCluster {
   private static final Log LOG = LogFactory.getLog(TestQuotaObserverChoreWithMiniCluster.class);
-  private static final int SIZE_PER_VALUE = 256;
-  private static final String F1 = "f1";
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final AtomicLong COUNTER = new AtomicLong(0);
-  private static final long ONE_MEGABYTE = 1024L * 1024L;
   private static final long DEFAULT_WAIT_MILLIS = 500;
 
   @Rule
@@ -84,18 +71,19 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
   private HMaster master;
   private QuotaObserverChore chore;
-  private SpaceQuotaViolationNotifierForTest violationNotifier;
+  private SpaceQuotaSnapshotNotifierForTest snapshotNotifier;
+  private SpaceQuotaHelperForTests helper;
 
   @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.VIOLATION_OBSERVER_CHORE_DELAY_KEY, 1000);
-    conf.setInt(QuotaObserverChore.VIOLATION_OBSERVER_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);
-    conf.setClass(SpaceQuotaViolationNotifierFactory.VIOLATION_NOTIFIER_KEY,
-        SpaceQuotaViolationNotifierForTest.class, SpaceQuotaViolationNotifier.class);
+    conf.setClass(SpaceQuotaSnapshotNotifierFactory.SNAPSHOT_NOTIFIER_KEY,
+        SpaceQuotaSnapshotNotifierForTest.class, SpaceQuotaSnapshotNotifier.class);
     TEST_UTIL.startMiniCluster(1);
   }
 
@@ -131,40 +119,55 @@ public class TestQuotaObserverChoreWithMiniCluster {
     }
 
     master = TEST_UTIL.getMiniHBaseCluster().getMaster();
-    violationNotifier =
-        (SpaceQuotaViolationNotifierForTest) master.getSpaceQuotaViolationNotifier();
-    violationNotifier.clearTableViolations();
+    snapshotNotifier =
+        (SpaceQuotaSnapshotNotifierForTest) master.getSpaceQuotaSnapshotNotifier();
+    snapshotNotifier.clearSnapshots();
     chore = master.getQuotaObserverChore();
+    helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
   }
 
   @Test
   public void testTableViolatesQuota() throws Exception {
-    TableName tn = createTableWithRegions(10);
+    TableName tn = helper.createTableWithRegions(10);
 
-    final long sizeLimit = 2L * ONE_MEGABYTE;
+    final long sizeLimit = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
     final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_INSERTS;
     QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, violationPolicy);
     TEST_UTIL.getAdmin().setQuota(settings);
 
     // Write more data than should be allowed
-    writeData(tn, 3L * ONE_MEGABYTE);
-
-    Map<TableName,SpaceViolationPolicy> violatedQuotas = violationNotifier.snapshotTablesInViolation();
-    while (violatedQuotas.isEmpty()) {
-      LOG.info("Found no violated quotas, sleeping and retrying. Current reports: "
-          + master.getMasterQuotaManager().snapshotRegionSizes());
-      try {
-        Thread.sleep(DEFAULT_WAIT_MILLIS);
-      } catch (InterruptedException e) {
-        LOG.debug("Interrupted while sleeping.", e);
-        Thread.currentThread().interrupt();
+    helper.writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
+
+    Map<TableName,SpaceQuotaSnapshot> quotaSnapshots = snapshotNotifier.copySnapshots();
+    boolean foundSnapshot = false;
+    while (!foundSnapshot) {
+      if (quotaSnapshots.isEmpty()) {
+        LOG.info("Found no violated quotas, sleeping and retrying. Current reports: "
+            + master.getMasterQuotaManager().snapshotRegionSizes());
+        sleepWithInterrupt(DEFAULT_WAIT_MILLIS);
+        quotaSnapshots = snapshotNotifier.copySnapshots();
+      } else {
+        Entry<TableName,SpaceQuotaSnapshot> entry = Iterables.getOnlyElement(quotaSnapshots.entrySet());
+        assertEquals(tn, entry.getKey());
+        final SpaceQuotaSnapshot snapshot = entry.getValue();
+        if (!snapshot.getQuotaStatus().isInViolation()) {
+          LOG.info("Found a snapshot, but it was not yet in violation. " + snapshot);
+          sleepWithInterrupt(DEFAULT_WAIT_MILLIS);
+          quotaSnapshots = snapshotNotifier.copySnapshots();
+        } else {
+          foundSnapshot = true;
+        }
       }
-      violatedQuotas = violationNotifier.snapshotTablesInViolation();
     }
 
-    Entry<TableName,SpaceViolationPolicy> entry = Iterables.getOnlyElement(violatedQuotas.entrySet());
+    Entry<TableName,SpaceQuotaSnapshot> entry = Iterables.getOnlyElement(quotaSnapshots.entrySet());
     assertEquals(tn, entry.getKey());
-    assertEquals(violationPolicy, entry.getValue());
+    final SpaceQuotaSnapshot snapshot = entry.getValue();
+    assertEquals("Snapshot was " + snapshot, violationPolicy, snapshot.getQuotaStatus().getPolicy());
+    assertEquals(sizeLimit, snapshot.getLimit());
+    assertTrue(
+        "The usage should be greater than the limit, but were " + snapshot.getUsage() + " and "
+        + snapshot.getLimit() + ", respectively", snapshot.getUsage() > snapshot.getLimit());
   }
 
   @Test
@@ -179,18 +182,18 @@ public class TestQuotaObserverChoreWithMiniCluster {
       admin.createNamespace(desc);
     }
 
-    TableName tn1 = createTableWithRegions(namespace, 5);
-    TableName tn2 = createTableWithRegions(namespace, 5);
-    TableName tn3 = createTableWithRegions(namespace, 5);
+    TableName tn1 = helper.createTableWithRegions(namespace, 5);
+    TableName tn2 = helper.createTableWithRegions(namespace, 5);
+    TableName tn3 = helper.createTableWithRegions(namespace, 5);
 
-    final long sizeLimit = 5L * ONE_MEGABYTE;
+    final long sizeLimit = 5L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
     final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.DISABLE;
     QuotaSettings settings = QuotaSettingsFactory.limitNamespaceSpace(namespace, sizeLimit, violationPolicy);
     admin.setQuota(settings);
 
-    writeData(tn1, 2L * ONE_MEGABYTE);
+    helper.writeData(tn1, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn1);
-    Map<TableName,SpaceViolationPolicy> violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    Map<TableName,SpaceQuotaSnapshot> violatedQuotas = snapshotNotifier.copySnapshots();
     for (int i = 0; i < 5; i++) {
       // Check a few times to make sure we don't prematurely move to violation
       assertEquals("Should not see any quota violations after writing 2MB of data", 0, violatedQuotas.size());
@@ -199,12 +202,12 @@ public class TestQuotaObserverChoreWithMiniCluster {
       } catch (InterruptedException e) {
         LOG.debug("Interrupted while sleeping." , e);
       }
-      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+      violatedQuotas = snapshotNotifier.copySnapshots();
     }
 
-    writeData(tn2, 2L * ONE_MEGABYTE);
+    helper.writeData(tn2, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn2);
-    violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    violatedQuotas = snapshotNotifier.copySnapshots();
     for (int i = 0; i < 5; i++) {
       // Check a few times to make sure we don't prematurely move to violation
       assertEquals("Should not see any quota violations after writing 4MB of data", 0,
@@ -214,14 +217,14 @@ public class TestQuotaObserverChoreWithMiniCluster {
       } catch (InterruptedException e) {
         LOG.debug("Interrupted while sleeping." , e);
       }
-      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+      violatedQuotas = snapshotNotifier.copySnapshots();
     }
 
     // Writing the final 2MB of data will push the namespace over the 5MB limit (6MB in total)
     // and should push all three tables in the namespace into violation.
-    writeData(tn3, 2L * ONE_MEGABYTE);
+    helper.writeData(tn3, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn3);
-    violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    violatedQuotas = snapshotNotifier.copySnapshots();
     while (violatedQuotas.size() < 3) {
       LOG.debug("Saw fewer violations than desired (expected 3): " + violatedQuotas
           + ". Current reports: " + master.getMasterQuotaManager().snapshotRegionSizes());
@@ -231,18 +234,18 @@ public class TestQuotaObserverChoreWithMiniCluster {
         LOG.debug("Interrupted while sleeping.", e);
         Thread.currentThread().interrupt();
       }
-      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+      violatedQuotas = snapshotNotifier.copySnapshots();
     }
 
-    SpaceViolationPolicy vp1 = violatedQuotas.remove(tn1);
-    assertNotNull("tn1 should be in violation", vp1);
-    assertEquals(violationPolicy, vp1);
-    SpaceViolationPolicy vp2 = violatedQuotas.remove(tn2);
-    assertNotNull("tn2 should be in violation", vp2);
-    assertEquals(violationPolicy, vp2);
-    SpaceViolationPolicy vp3 = violatedQuotas.remove(tn3);
-    assertNotNull("tn3 should be in violation", vp3);
-    assertEquals(violationPolicy, vp3);
+    SpaceQuotaSnapshot snapshot1 = violatedQuotas.remove(tn1);
+    assertNotNull("tn1 should be in violation", snapshot1);
+    assertEquals(violationPolicy, snapshot1.getQuotaStatus().getPolicy());
+    SpaceQuotaSnapshot snapshot2 = violatedQuotas.remove(tn2);
+    assertNotNull("tn2 should be in violation", snapshot2);
+    assertEquals(violationPolicy, snapshot2.getQuotaStatus().getPolicy());
+    SpaceQuotaSnapshot snapshot3 = violatedQuotas.remove(tn3);
+    assertNotNull("tn3 should be in violation", snapshot3);
+    assertEquals(violationPolicy, snapshot3.getQuotaStatus().getPolicy());
     assertTrue("Unexpected additional quota violations: " + violatedQuotas, violatedQuotas.isEmpty());
   }
 
@@ -258,18 +261,18 @@ public class TestQuotaObserverChoreWithMiniCluster {
       admin.createNamespace(desc);
     }
 
-    TableName tn1 = createTableWithRegions(namespace, 5);
-    TableName tn2 = createTableWithRegions(namespace, 5);
+    TableName tn1 = helper.createTableWithRegions(namespace, 5);
+    TableName tn2 = helper.createTableWithRegions(namespace, 5);
 
-    final long namespaceSizeLimit = 3L * ONE_MEGABYTE;
+    final long namespaceSizeLimit = 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
     final SpaceViolationPolicy namespaceViolationPolicy = SpaceViolationPolicy.DISABLE;
     QuotaSettings namespaceSettings = QuotaSettingsFactory.limitNamespaceSpace(namespace,
         namespaceSizeLimit, namespaceViolationPolicy);
     admin.setQuota(namespaceSettings);
 
-    writeData(tn1, 2L * ONE_MEGABYTE);
+    helper.writeData(tn1, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn1);
-    Map<TableName,SpaceViolationPolicy> violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    Map<TableName,SpaceQuotaSnapshot> violatedQuotas = snapshotNotifier.copySnapshots();
     for (int i = 0; i < 5; i++) {
       // Check a few times to make sure we don't prematurely move to violation
       assertEquals("Should not see any quota violations after writing 2MB of data", 0,
@@ -279,12 +282,12 @@ public class TestQuotaObserverChoreWithMiniCluster {
       } catch (InterruptedException e) {
         LOG.debug("Interrupted while sleeping." , e);
       }
-      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+      violatedQuotas = snapshotNotifier.copySnapshots();
     }
 
-    writeData(tn2, 2L * ONE_MEGABYTE);
+    helper.writeData(tn2, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn2);
-    violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    violatedQuotas = snapshotNotifier.copySnapshots();
     while (violatedQuotas.size() < 2) {
       LOG.debug("Saw fewer violations than desired (expected 2): " + violatedQuotas
           + ". Current reports: " + master.getMasterQuotaManager().snapshotRegionSizes());
@@ -294,18 +297,18 @@ public class TestQuotaObserverChoreWithMiniCluster {
         LOG.debug("Interrupted while sleeping.", e);
         Thread.currentThread().interrupt();
       }
-      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+      violatedQuotas = snapshotNotifier.copySnapshots();
     }
 
-    SpaceViolationPolicy actualPolicyTN1 = violatedQuotas.get(tn1);
+    SpaceQuotaSnapshot actualPolicyTN1 = violatedQuotas.get(tn1);
     assertNotNull("Expected to see violation policy for tn1", actualPolicyTN1);
-    assertEquals(namespaceViolationPolicy, actualPolicyTN1);
-    SpaceViolationPolicy actualPolicyTN2 = violatedQuotas.get(tn2);
+    assertEquals(namespaceViolationPolicy, actualPolicyTN1.getQuotaStatus().getPolicy());
+    SpaceQuotaSnapshot actualPolicyTN2 = violatedQuotas.get(tn2);
     assertNotNull("Expected to see violation policy for tn2", actualPolicyTN2);
-    assertEquals(namespaceViolationPolicy, actualPolicyTN2);
+    assertEquals(namespaceViolationPolicy, actualPolicyTN2.getQuotaStatus().getPolicy());
 
     // Override the namespace quota with a table quota
-    final long tableSizeLimit = ONE_MEGABYTE;
+    final long tableSizeLimit = SpaceQuotaHelperForTests.ONE_MEGABYTE;
     final SpaceViolationPolicy tableViolationPolicy = SpaceViolationPolicy.NO_INSERTS;
     QuotaSettings tableSettings = QuotaSettingsFactory.limitTableSpace(tn1, tableSizeLimit,
         tableViolationPolicy);
@@ -313,10 +316,10 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
     // Keep checking for the table quota policy to override the namespace quota
     while (true) {
-      violatedQuotas = violationNotifier.snapshotTablesInViolation();
-      SpaceViolationPolicy actualTableViolationPolicy = violatedQuotas.get(tn1);
-      assertNotNull("Violation policy should never be null", actualTableViolationPolicy);
-      if (tableViolationPolicy != actualTableViolationPolicy) {
+      violatedQuotas = snapshotNotifier.copySnapshots();
+      SpaceQuotaSnapshot actualTableSnapshot = violatedQuotas.get(tn1);
+      assertNotNull("Violation policy should never be null", actualTableSnapshot);
+      if (tableViolationPolicy != actualTableSnapshot.getQuotaStatus().getPolicy()) {
         LOG.debug("Saw unexpected table violation policy, waiting and re-checking.");
         try {
           Thread.sleep(DEFAULT_WAIT_MILLIS);
@@ -326,23 +329,23 @@ public class TestQuotaObserverChoreWithMiniCluster {
         }
         continue;
       }
-      assertEquals(tableViolationPolicy, actualTableViolationPolicy);
+      assertEquals(tableViolationPolicy, actualTableSnapshot.getQuotaStatus().getPolicy());
       break;
     }
 
     // This should not change with the introduction of the table quota for tn1
     actualPolicyTN2 = violatedQuotas.get(tn2);
     assertNotNull("Expected to see violation policy for tn2", actualPolicyTN2);
-    assertEquals(namespaceViolationPolicy, actualPolicyTN2);
+    assertEquals(namespaceViolationPolicy, actualPolicyTN2.getQuotaStatus().getPolicy());
   }
 
   @Test
   public void testGetAllTablesWithQuotas() throws Exception {
-    final Multimap<TableName, QuotaSettings> quotas = createTablesWithSpaceQuotas();
+    final Multimap<TableName, QuotaSettings> quotas = helper.createTablesWithSpaceQuotas();
     Set<TableName> tablesWithQuotas = new HashSet<>();
     Set<TableName> namespaceTablesWithQuotas = new HashSet<>();
     // Partition the tables with quotas by table and ns quota
-    partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
+    helper.partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
 
     TablesWithQuotas tables = chore.fetchAllTablesWithQuotasDefined();
     assertEquals("Found tables: " + tables, tablesWithQuotas, tables.getTableQuotaTables());
@@ -351,13 +354,13 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
   @Test
   public void testRpcQuotaTablesAreFiltered() throws Exception {
-    final Multimap<TableName, QuotaSettings> quotas = createTablesWithSpaceQuotas();
+    final Multimap<TableName, QuotaSettings> quotas = helper.createTablesWithSpaceQuotas();
     Set<TableName> tablesWithQuotas = new HashSet<>();
     Set<TableName> namespaceTablesWithQuotas = new HashSet<>();
     // Partition the tables with quotas by table and ns quota
-    partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
+    helper.partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
 
-    TableName rpcQuotaTable = createTable();
+    TableName rpcQuotaTable = helper.createTable();
     TEST_UTIL.getAdmin().setQuota(QuotaSettingsFactory
       .throttleTable(rpcQuotaTable, ThrottleType.READ_NUMBER, 6, TimeUnit.MINUTES));
 
@@ -375,7 +378,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
     TablesWithQuotas tables = new TablesWithQuotas(TEST_UTIL.getConnection(),
         TEST_UTIL.getConfiguration()) {
       @Override
-      int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore) {
+      int getNumReportedRegions(TableName table, QuotaSnapshotStore<TableName> tableStore) {
         Integer i = mockReportedRegions.get(table);
         if (null == i) {
           return 0;
@@ -385,9 +388,9 @@ public class TestQuotaObserverChoreWithMiniCluster {
     };
 
     // Create the tables
-    TableName tn1 = createTableWithRegions(20);
-    TableName tn2 = createTableWithRegions(20);
-    TableName tn3 = createTableWithRegions(20);
+    TableName tn1 = helper.createTableWithRegions(20);
+    TableName tn2 = helper.createTableWithRegions(20);
+    TableName tn3 = helper.createTableWithRegions(20);
 
     // Add them to the Tables with Quotas object
     tables.addTableQuotaTable(tn1);
@@ -407,9 +410,9 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
   @Test
   public void testFetchSpaceQuota() throws Exception {
-    Multimap<TableName,QuotaSettings> tables = createTablesWithSpaceQuotas();
+    Multimap<TableName,QuotaSettings> tables = helper.createTablesWithSpaceQuotas();
     // Can pass in an empty map, we're not consulting it.
-    chore.initializeViolationStores(Collections.emptyMap());
+    chore.initializeSnapshotStores(Collections.emptyMap());
     // All tables that were created should have a quota defined.
     for (Entry<TableName,QuotaSettings> entry : tables.entries()) {
       final TableName table = entry.getKey();
@@ -420,10 +423,10 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
       SpaceQuota spaceQuota = null;
       if (null != qs.getTableName()) {
-        spaceQuota = chore.getTableViolationStore().getSpaceQuota(table);
+        spaceQuota = chore.getTableSnapshotStore().getSpaceQuota(table);
         assertNotNull("Could not find table space quota for " + table, spaceQuota);
       } else if (null != qs.getNamespace()) {
-        spaceQuota = chore.getNamespaceViolationStore().getSpaceQuota(table.getNamespaceAsString());
+        spaceQuota = chore.getNamespaceSnapshotStore().getSpaceQuota(table.getNamespaceAsString());
         assertNotNull("Could not find namespace space quota for " + table.getNamespaceAsString(), spaceQuota);
       } else {
         fail("Expected table or namespace space quota");
@@ -433,166 +436,16 @@ public class TestQuotaObserverChoreWithMiniCluster {
       assertEquals(sls.getProto().getQuota(), spaceQuota);
     }
 
-    TableName tableWithoutQuota = createTable();
-    assertNull(chore.getTableViolationStore().getSpaceQuota(tableWithoutQuota));
+    TableName tableWithoutQuota = helper.createTable();
+    assertNull(chore.getTableSnapshotStore().getSpaceQuota(tableWithoutQuota));
   }
 
-  //
-  // Helpers
-  //
-
-  void writeData(TableName tn, long sizeInBytes) throws IOException {
-    final Connection conn = TEST_UTIL.getConnection();
-    final Table table = conn.getTable(tn);
+  private void sleepWithInterrupt(long millis) {
     try {
-      List<Put> updates = new ArrayList<>();
-      long bytesToWrite = sizeInBytes;
-      long rowKeyId = 0L;
-      final StringBuilder sb = new StringBuilder();
-      final Random r = new Random();
-      while (bytesToWrite > 0L) {
-        sb.setLength(0);
-        sb.append(Long.toString(rowKeyId));
-        // Use the reverse counter as the rowKey to get even spread across all regions
-        Put p = new Put(Bytes.toBytes(sb.reverse().toString()));
-        byte[] value = new byte[SIZE_PER_VALUE];
-        r.nextBytes(value);
-        p.addColumn(Bytes.toBytes(F1), Bytes.toBytes("q1"), value);
-        updates.add(p);
-
-        // Batch 50K worth of updates
-        if (updates.size() > 50) {
-          table.put(updates);
-          updates.clear();
-        }
-
-        // Just count the value size, ignore the size of rowkey + column
-        bytesToWrite -= SIZE_PER_VALUE;
-        rowKeyId++;
-      }
-
-      // Write the final batch
-      if (!updates.isEmpty()) {
-        table.put(updates);
-      }
-
-      LOG.debug("Data was written to HBase");
-      // Push the data to disk.
-      TEST_UTIL.getAdmin().flush(tn);
-      LOG.debug("Data flushed to disk");
-    } finally {
-      table.close();
-    }
-  }
-
-  Multimap<TableName, QuotaSettings> createTablesWithSpaceQuotas() throws Exception {
-    final Admin admin = TEST_UTIL.getAdmin();
-    final Multimap<TableName, QuotaSettings> tablesWithQuotas = HashMultimap.create();
-
-    final TableName tn1 = createTable();
-    final TableName tn2 = createTable();
-
-    NamespaceDescriptor nd = NamespaceDescriptor.create("ns" + COUNTER.getAndIncrement()).build();
-    admin.createNamespace(nd);
-    final TableName tn3 = createTableInNamespace(nd);
-    final TableName tn4 = createTableInNamespace(nd);
-    final TableName tn5 = createTableInNamespace(nd);
-
-    final long sizeLimit1 = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
-    final SpaceViolationPolicy violationPolicy1 = SpaceViolationPolicy.NO_WRITES;
-    QuotaSettings qs1 = QuotaSettingsFactory.limitTableSpace(tn1, sizeLimit1, violationPolicy1);
-    tablesWithQuotas.put(tn1, qs1);
-    admin.setQuota(qs1);
-
-    final long sizeLimit2 = 1024L * 1024L * 1024L * 200L; // 200GB
-    final SpaceViolationPolicy violationPolicy2 = SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
-    QuotaSettings qs2 = QuotaSettingsFactory.limitTableSpace(tn2, sizeLimit2, violationPolicy2);
-    tablesWithQuotas.put(tn2, qs2);
-    admin.setQuota(qs2);
-
-    final long sizeLimit3 = 1024L * 1024L * 1024L * 1024L * 100L; // 100TB
-    final SpaceViolationPolicy violationPolicy3 = SpaceViolationPolicy.NO_INSERTS;
-    QuotaSettings qs3 = QuotaSettingsFactory.limitNamespaceSpace(nd.getName(), sizeLimit3, violationPolicy3);
-    tablesWithQuotas.put(tn3, qs3);
-    tablesWithQuotas.put(tn4, qs3);
-    tablesWithQuotas.put(tn5, qs3);
-    admin.setQuota(qs3);
-
-    final long sizeLimit4 = 1024L * 1024L * 1024L * 5L; // 5GB
-    final SpaceViolationPolicy violationPolicy4 = SpaceViolationPolicy.NO_INSERTS;
-    QuotaSettings qs4 = QuotaSettingsFactory.limitTableSpace(tn5, sizeLimit4, violationPolicy4);
-    // Override the ns quota for tn5, import edge-case to catch table quota taking
-    // precedence over ns quota.
-    tablesWithQuotas.put(tn5, qs4);
-    admin.setQuota(qs4);
-
-    return tablesWithQuotas;
-  }
-
-  TableName createTable() throws Exception {
-    return createTableWithRegions(1);
-  }
-
-  TableName createTableWithRegions(int numRegions) throws Exception {
-    return createTableWithRegions(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR, numRegions);
-  }
-
-  TableName createTableWithRegions(String namespace, int numRegions) throws Exception {
-    final Admin admin = TEST_UTIL.getAdmin();
-    final TableName tn = TableName.valueOf(namespace, testName.getMethodName() + COUNTER.getAndIncrement());
-
-    // Delete the old table
-    if (admin.tableExists(tn)) {
-      admin.disableTable(tn);
-      admin.deleteTable(tn);
-    }
-
-    // Create the table
-    HTableDescriptor tableDesc = new HTableDescriptor(tn);
-    tableDesc.addFamily(new HColumnDescriptor(F1));
-    if (numRegions == 1) {
-      admin.createTable(tableDesc);
-    } else {
-      admin.createTable(tableDesc, Bytes.toBytes("0"), Bytes.toBytes("9"), numRegions);
-    }
-    return tn;
-  }
-
-  TableName createTableInNamespace(NamespaceDescriptor nd) throws Exception {
-    final Admin admin = TEST_UTIL.getAdmin();
-    final TableName tn = TableName.valueOf(nd.getName(),
-        testName.getMethodName() + COUNTER.getAndIncrement());
-
-    // Delete the old table
-    if (admin.tableExists(tn)) {
-      admin.disableTable(tn);
-      admin.deleteTable(tn);
-    }
-
-    // Create the table
-    HTableDescriptor tableDesc = new HTableDescriptor(tn);
-    tableDesc.addFamily(new HColumnDescriptor(F1));
-
-    admin.createTable(tableDesc);
-    return tn;
-  }
-
-  void partitionTablesByQuotaTarget(Multimap<TableName,QuotaSettings> quotas,
-      Set<TableName> tablesWithTableQuota, Set<TableName> tablesWithNamespaceQuota) {
-    // Partition the tables with quotas by table and ns quota
-    for (Entry<TableName, QuotaSettings> entry : quotas.entries()) {
-      SpaceLimitSettings settings = (SpaceLimitSettings) entry.getValue();
-      TableName tn = entry.getKey();
-      if (null != settings.getTableName()) {
-        tablesWithTableQuota.add(tn);
-      }
-      if (null != settings.getNamespace()) {
-        tablesWithNamespaceQuota.add(tn);
-      }
-
-      if (null == settings.getTableName() && null == settings.getNamespace()) {
-        fail("Unexpected table name with null tableName and namespace: " + tn);
-      }
+      Thread.sleep(millis);
+    } catch (InterruptedException e) {
+      LOG.debug("Interrupted while sleeping");
+      Thread.currentThread().interrupt();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
index 55f671a..f10cdef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
@@ -201,26 +202,29 @@ public class TestQuotaTableUtil {
   @Test
   public void testSerDeViolationPolicies() throws Exception {
     final TableName tn1 = getUniqueTableName();
-    final SpaceViolationPolicy policy1 = SpaceViolationPolicy.DISABLE;
+    final SpaceQuotaSnapshot snapshot1 = new SpaceQuotaSnapshot(
+        new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 512L, 1024L);
     final TableName tn2 = getUniqueTableName();
-    final SpaceViolationPolicy policy2 = SpaceViolationPolicy.NO_INSERTS;
+    final SpaceQuotaSnapshot snapshot2 = new SpaceQuotaSnapshot(
+        new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 512L, 1024L);
     final TableName tn3 = getUniqueTableName();
-    final SpaceViolationPolicy policy3 = SpaceViolationPolicy.NO_WRITES;
+    final SpaceQuotaSnapshot snapshot3 = new SpaceQuotaSnapshot(
+        new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 512L, 1024L);
     List<Put> puts = new ArrayList<>();
-    puts.add(QuotaTableUtil.createEnableViolationPolicyUpdate(tn1, policy1));
-    puts.add(QuotaTableUtil.createEnableViolationPolicyUpdate(tn2, policy2));
-    puts.add(QuotaTableUtil.createEnableViolationPolicyUpdate(tn3, policy3));
-    final Map<TableName,SpaceViolationPolicy> expectedPolicies = new HashMap<>();
-    expectedPolicies.put(tn1, policy1);
-    expectedPolicies.put(tn2, policy2);
-    expectedPolicies.put(tn3, policy3);
-
-    final Map<TableName,SpaceViolationPolicy> actualPolicies = new HashMap<>();
+    puts.add(QuotaTableUtil.createPutSpaceSnapshot(tn1, snapshot1));
+    puts.add(QuotaTableUtil.createPutSpaceSnapshot(tn2, snapshot2));
+    puts.add(QuotaTableUtil.createPutSpaceSnapshot(tn3, snapshot3));
+    final Map<TableName,SpaceQuotaSnapshot> expectedPolicies = new HashMap<>();
+    expectedPolicies.put(tn1, snapshot1);
+    expectedPolicies.put(tn2, snapshot2);
+    expectedPolicies.put(tn3, snapshot3);
+
+    final Map<TableName,SpaceQuotaSnapshot> actualPolicies = new HashMap<>();
     try (Table quotaTable = connection.getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
       quotaTable.put(puts);
-      ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaViolationScan());
+      ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaSnapshotScan());
       for (Result r : scanner) {
-        QuotaTableUtil.extractViolationPolicy(r, actualPolicies);
+        QuotaTableUtil.extractQuotaSnapshot(r, actualPolicies);
       }
       scanner.close();
     }
@@ -231,4 +235,4 @@ public class TestQuotaTableUtil {
   private TableName getUniqueTableName() {
     return TableName.valueOf(testName.getMethodName() + "_" + tableNameCounter++);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java
index e5ab317..38656e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java
@@ -16,33 +16,29 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
-import static org.apache.hadoop.hbase.util.Bytes.toBytes;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doAnswer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
+import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.hadoop.hbase.quotas.policies.DisableTableViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoInsertsViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesCompactionsViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.BulkLoadVerifyingViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
 
 /**
  * Test class for {@link RegionServerSpaceQuotaManager}.
@@ -51,77 +47,105 @@ import org.mockito.stubbing.Answer;
 public class TestRegionServerSpaceQuotaManager {
 
   private RegionServerSpaceQuotaManager quotaManager;
-  private Connection conn;
-  private Table quotaTable;
-  private ResultScanner scanner;
+  private RegionServerServices rss;
 
   @Before
-  @SuppressWarnings("unchecked")
   public void setup() throws Exception {
     quotaManager = mock(RegionServerSpaceQuotaManager.class);
-    conn = mock(Connection.class);
-    quotaTable = mock(Table.class);
-    scanner = mock(ResultScanner.class);
-    // Call the real getViolationPoliciesToEnforce()
-    when(quotaManager.getViolationPoliciesToEnforce()).thenCallRealMethod();
-    // Mock out creating a scanner
-    when(quotaManager.getConnection()).thenReturn(conn);
-    when(conn.getTable(QuotaUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
-    when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
-    // Mock out the static method call with some indirection
-    doAnswer(new Answer<Void>(){
-      @Override
-      public Void answer(InvocationOnMock invocation) throws Throwable {
-        Result result = invocation.getArgumentAt(0, Result.class);
-        Map<TableName,SpaceViolationPolicy> policies = invocation.getArgumentAt(1, Map.class);
-        QuotaTableUtil.extractViolationPolicy(result, policies);
-        return null;
-      }
-    }).when(quotaManager).extractViolationPolicy(any(Result.class), any(Map.class));
+    rss = mock(RegionServerServices.class);
   }
 
   @Test
-  public void testMissingAllColumns() {
-    List<Result> results = new ArrayList<>();
-    results.add(Result.create(Collections.emptyList()));
-    when(scanner.iterator()).thenReturn(results.iterator());
-    try {
-      quotaManager.getViolationPoliciesToEnforce();
-      fail("Expected an IOException, but did not receive one.");
-    } catch (IOException e) {
-      // Expected an error because we had no cells in the row.
-      // This should only happen due to programmer error.
-    }
+  public void testSpacePoliciesFromEnforcements() {
+    final Map<TableName, SpaceViolationPolicyEnforcement> enforcements = new HashMap<>();
+    final Map<TableName, SpaceQuotaSnapshot> expectedPolicies = new HashMap<>();
+    when(quotaManager.copyActiveEnforcements()).thenReturn(enforcements);
+    when(quotaManager.getActivePoliciesAsMap()).thenCallRealMethod();
+
+    NoInsertsViolationPolicyEnforcement noInsertsPolicy = new NoInsertsViolationPolicyEnforcement();
+    SpaceQuotaSnapshot noInsertsSnapshot = new SpaceQuotaSnapshot(
+        new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 256L, 1024L);
+    noInsertsPolicy.initialize(rss, TableName.valueOf("no_inserts"), noInsertsSnapshot);
+    enforcements.put(noInsertsPolicy.getTableName(), noInsertsPolicy);
+    expectedPolicies.put(noInsertsPolicy.getTableName(), noInsertsSnapshot);
+
+    NoWritesViolationPolicyEnforcement noWritesPolicy = new NoWritesViolationPolicyEnforcement();
+    SpaceQuotaSnapshot noWritesSnapshot = new SpaceQuotaSnapshot(
+        new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 512L, 2048L);
+    noWritesPolicy.initialize(rss, TableName.valueOf("no_writes"), noWritesSnapshot);
+    enforcements.put(noWritesPolicy.getTableName(), noWritesPolicy);
+    expectedPolicies.put(noWritesPolicy.getTableName(), noWritesSnapshot);
+
+    NoWritesCompactionsViolationPolicyEnforcement noWritesCompactionsPolicy =
+        new NoWritesCompactionsViolationPolicyEnforcement();
+    SpaceQuotaSnapshot noWritesCompactionsSnapshot = new SpaceQuotaSnapshot(
+        new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES_COMPACTIONS), 1024L, 4096L);
+    noWritesCompactionsPolicy.initialize(
+        rss, TableName.valueOf("no_writes_compactions"), noWritesCompactionsSnapshot);
+    enforcements.put(noWritesCompactionsPolicy.getTableName(), noWritesCompactionsPolicy);
+    expectedPolicies.put(noWritesCompactionsPolicy.getTableName(),
+        noWritesCompactionsSnapshot);
+
+    DisableTableViolationPolicyEnforcement disablePolicy =
+        new DisableTableViolationPolicyEnforcement();
+    SpaceQuotaSnapshot disableSnapshot = new SpaceQuotaSnapshot(
+        new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 2048L, 8192L);
+    disablePolicy.initialize(rss, TableName.valueOf("disable"), disableSnapshot);
+    enforcements.put(disablePolicy.getTableName(), disablePolicy);
+    expectedPolicies.put(disablePolicy.getTableName(), disableSnapshot);
+
+    enforcements.put(
+        TableName.valueOf("no_policy"), new BulkLoadVerifyingViolationPolicyEnforcement());
+
+    Map<TableName, SpaceQuotaSnapshot> actualPolicies = quotaManager.getActivePoliciesAsMap();
+    assertEquals(expectedPolicies, actualPolicies);
   }
 
   @Test
-  public void testMissingDesiredColumn() {
-    List<Result> results = new ArrayList<>();
-    // Give a column that isn't the one we want
-    Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("q"), toBytes("s"), new byte[0]);
-    results.add(Result.create(Collections.singletonList(c)));
-    when(scanner.iterator()).thenReturn(results.iterator());
-    try {
-      quotaManager.getViolationPoliciesToEnforce();
-      fail("Expected an IOException, but did not receive one.");
-    } catch (IOException e) {
-      // Expected an error because we were missing the column we expected in this row.
-      // This should only happen due to programmer error.
-    }
+  public void testExceptionOnPolicyEnforcementEnable() throws Exception {
+    final TableName tableName = TableName.valueOf("foo");
+    final SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(
+        new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L, 2048L);
+    RegionServerServices rss = mock(RegionServerServices.class);
+    SpaceViolationPolicyEnforcementFactory factory = mock(
+        SpaceViolationPolicyEnforcementFactory.class);
+    SpaceViolationPolicyEnforcement enforcement = mock(SpaceViolationPolicyEnforcement.class);
+    RegionServerSpaceQuotaManager realManager = new RegionServerSpaceQuotaManager(rss, factory);
+
+    when(factory.create(rss, tableName, snapshot)).thenReturn(enforcement);
+    doThrow(new IOException("Failed for test!")).when(enforcement).enable();
+
+    realManager.enforceViolationPolicy(tableName, snapshot);
+    Map<TableName, SpaceViolationPolicyEnforcement> enforcements =
+        realManager.copyActiveEnforcements();
+    assertTrue("Expected active enforcements to be empty, but were " + enforcements,
+        enforcements.isEmpty());
   }
 
   @Test
-  public void testParsingError() {
-    List<Result> results = new ArrayList<>();
-    Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("u"), toBytes("v"), new byte[0]);
-    results.add(Result.create(Collections.singletonList(c)));
-    when(scanner.iterator()).thenReturn(results.iterator());
-    try {
-      quotaManager.getViolationPoliciesToEnforce();
-      fail("Expected an IOException, but did not receive one.");
-    } catch (IOException e) {
-      // We provided a garbage serialized protobuf message (empty byte array), this should
-      // in turn throw an IOException
-    }
+  public void testExceptionOnPolicyEnforcementDisable() throws Exception {
+    final TableName tableName = TableName.valueOf("foo");
+    final SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(
+        new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L, 2048L);
+    RegionServerServices rss = mock(RegionServerServices.class);
+    SpaceViolationPolicyEnforcementFactory factory = mock(
+        SpaceViolationPolicyEnforcementFactory.class);
+    SpaceViolationPolicyEnforcement enforcement = mock(SpaceViolationPolicyEnforcement.class);
+    RegionServerSpaceQuotaManager realManager = new RegionServerSpaceQuotaManager(rss, factory);
+
+    when(factory.create(rss, tableName, snapshot)).thenReturn(enforcement);
+    doNothing().when(enforcement).enable();
+    doThrow(new IOException("Failed for test!")).when(enforcement).disable();
+
+    // Enabling should work
+    realManager.enforceViolationPolicy(tableName, snapshot);
+    Map<TableName, SpaceViolationPolicyEnforcement> enforcements =
+        realManager.copyActiveEnforcements();
+    assertEquals(1, enforcements.size());
+
+    // If the disable fails, we should still treat it as "active"
+    realManager.disableViolationPolicyEnforcement(tableName);
+    enforcements = realManager.copyActiveEnforcements();
+    assertEquals(1, enforcements.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java
index 160de46..7f0f9ad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java
@@ -16,20 +16,34 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+import static org.apache.hadoop.hbase.util.Bytes.toBytes;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doCallRealMethod;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Before;
@@ -37,42 +51,62 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 /**
- * Test class for {@link SpaceQuotaViolationPolicyRefresherChore}.
+ * Test class for {@link SpaceQuotaRefresherChore}.
  */
 @Category(SmallTests.class)
 public class TestSpaceQuotaViolationPolicyRefresherChore {
 
   private RegionServerSpaceQuotaManager manager;
   private RegionServerServices rss;
-  private SpaceQuotaViolationPolicyRefresherChore chore;
+  private SpaceQuotaRefresherChore chore;
   private Configuration conf;
+  private Connection conn;
 
+  @SuppressWarnings("unchecked")
   @Before
-  public void setup() {
+  public void setup() throws IOException {
     conf = HBaseConfiguration.create();
     rss = mock(RegionServerServices.class);
     manager = mock(RegionServerSpaceQuotaManager.class);
+    conn = mock(Connection.class);
     when(manager.getRegionServerServices()).thenReturn(rss);
     when(rss.getConfiguration()).thenReturn(conf);
-    chore = new SpaceQuotaViolationPolicyRefresherChore(manager);
+
+
+    chore = mock(SpaceQuotaRefresherChore.class);
+    when(chore.getConnection()).thenReturn(conn);
+    when(chore.getManager()).thenReturn(manager);
+    doCallRealMethod().when(chore).chore();
+    when(chore.isInViolation(any(SpaceQuotaSnapshot.class))).thenCallRealMethod();
+    doCallRealMethod().when(chore).extractQuotaSnapshot(any(Result.class), any(Map.class));
   }
 
   @Test
   public void testPoliciesAreEnforced() throws IOException {
-    final Map<TableName,SpaceViolationPolicy> policiesToEnforce = new HashMap<>();
-    policiesToEnforce.put(TableName.valueOf("table1"), SpaceViolationPolicy.DISABLE);
-    policiesToEnforce.put(TableName.valueOf("table2"), SpaceViolationPolicy.NO_INSERTS);
-    policiesToEnforce.put(TableName.valueOf("table3"), SpaceViolationPolicy.NO_WRITES);
-    policiesToEnforce.put(TableName.valueOf("table4"), SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
+    // Create a number of policies that should be enforced (usage > limit)
+    final Map<TableName,SpaceQuotaSnapshot> policiesToEnforce = new HashMap<>();
+    policiesToEnforce.put(
+        TableName.valueOf("table1"),
+        new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L, 512L));
+    policiesToEnforce.put(
+        TableName.valueOf("table2"),
+        new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 2048L, 512L));
+    policiesToEnforce.put(
+        TableName.valueOf("table3"),
+        new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 4096L, 512L));
+    policiesToEnforce.put(
+        TableName.valueOf("table4"),
+        new SpaceQuotaSnapshot(
+            new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES_COMPACTIONS), 8192L, 512L));
 
     // No active enforcements
-    when(manager.getActiveViolationPolicyEnforcements()).thenReturn(Collections.emptyMap());
+    when(manager.copyQuotaSnapshots()).thenReturn(Collections.emptyMap());
     // Policies to enforce
-    when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
+    when(chore.fetchSnapshotsFromQuotaTable()).thenReturn(policiesToEnforce);
 
     chore.chore();
 
-    for (Entry<TableName,SpaceViolationPolicy> entry : policiesToEnforce.entrySet()) {
+    for (Entry<TableName,SpaceQuotaSnapshot> entry : policiesToEnforce.entrySet()) {
       // Ensure we enforce the policy
       verify(manager).enforceViolationPolicy(entry.getKey(), entry.getValue());
       // Don't disable any policies
@@ -82,50 +116,135 @@ public class TestSpaceQuotaViolationPolicyRefresherChore {
 
   @Test
   public void testOldPoliciesAreRemoved() throws IOException {
-    final Map<TableName,SpaceViolationPolicy> policiesToEnforce = new HashMap<>();
-    policiesToEnforce.put(TableName.valueOf("table1"), SpaceViolationPolicy.DISABLE);
-    policiesToEnforce.put(TableName.valueOf("table2"), SpaceViolationPolicy.NO_INSERTS);
-
-    final Map<TableName,SpaceViolationPolicy> previousPolicies = new HashMap<>();
-    previousPolicies.put(TableName.valueOf("table3"), SpaceViolationPolicy.NO_WRITES);
-    previousPolicies.put(TableName.valueOf("table4"), SpaceViolationPolicy.NO_WRITES);
+    final Map<TableName,SpaceQuotaSnapshot> previousPolicies = new HashMap<>();
+    previousPolicies.put(
+        TableName.valueOf("table3"),
+        new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 4096L, 512L));
+    previousPolicies.put(
+        TableName.valueOf("table4"),
+        new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 8192L, 512L));
+
+    final Map<TableName,SpaceQuotaSnapshot> policiesToEnforce = new HashMap<>();
+    policiesToEnforce.put(
+        TableName.valueOf("table1"),
+        new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L, 512L));
+    policiesToEnforce.put(
+        TableName.valueOf("table2"),
+        new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 2048L, 512L));
+    policiesToEnforce.put(
+        TableName.valueOf("table3"),
+        new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 256L, 512L));
+    policiesToEnforce.put(
+        TableName.valueOf("table4"),
+        new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 128L, 512L));
 
     // No active enforcements
-    when(manager.getActiveViolationPolicyEnforcements()).thenReturn(previousPolicies);
+    when(manager.copyQuotaSnapshots()).thenReturn(previousPolicies);
     // Policies to enforce
-    when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
+    when(chore.fetchSnapshotsFromQuotaTable()).thenReturn(policiesToEnforce);
 
     chore.chore();
 
-    for (Entry<TableName,SpaceViolationPolicy> entry : policiesToEnforce.entrySet()) {
-      verify(manager).enforceViolationPolicy(entry.getKey(), entry.getValue());
-    }
+    verify(manager).enforceViolationPolicy(
+        TableName.valueOf("table1"), policiesToEnforce.get(TableName.valueOf("table1")));
+    verify(manager).enforceViolationPolicy(
+        TableName.valueOf("table2"), policiesToEnforce.get(TableName.valueOf("table2")));
 
-    for (Entry<TableName,SpaceViolationPolicy> entry : previousPolicies.entrySet()) {
-      verify(manager).disableViolationPolicyEnforcement(entry.getKey());
-    }
+    verify(manager).disableViolationPolicyEnforcement(TableName.valueOf("table3"));
+    verify(manager).disableViolationPolicyEnforcement(TableName.valueOf("table4"));
   }
 
   @Test
   public void testNewPolicyOverridesOld() throws IOException {
-    final Map<TableName,SpaceViolationPolicy> policiesToEnforce = new HashMap<>();
-    policiesToEnforce.put(TableName.valueOf("table1"), SpaceViolationPolicy.DISABLE);
-    policiesToEnforce.put(TableName.valueOf("table2"), SpaceViolationPolicy.NO_WRITES);
-    policiesToEnforce.put(TableName.valueOf("table3"), SpaceViolationPolicy.NO_INSERTS);
-
-    final Map<TableName,SpaceViolationPolicy> previousPolicies = new HashMap<>();
-    previousPolicies.put(TableName.valueOf("table1"), SpaceViolationPolicy.NO_WRITES);
+    final Map<TableName,SpaceQuotaSnapshot> policiesToEnforce = new HashMap<>();
+    policiesToEnforce.put(
+        TableName.valueOf("table1"),
+        new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L, 512L));
+    policiesToEnforce.put(
+        TableName.valueOf("table2"),
+        new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 2048L, 512L));
+    policiesToEnforce.put(
+        TableName.valueOf("table3"),
+        new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 4096L, 512L));
+
+    final Map<TableName,SpaceQuotaSnapshot> previousPolicies = new HashMap<>();
+    previousPolicies.put(
+        TableName.valueOf("table1"),
+        new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 8192L, 512L));
 
     // No active enforcements
-    when(manager.getActiveViolationPolicyEnforcements()).thenReturn(previousPolicies);
+    when(manager.getActivePoliciesAsMap()).thenReturn(previousPolicies);
     // Policies to enforce
-    when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
+    when(chore.fetchSnapshotsFromQuotaTable()).thenReturn(policiesToEnforce);
 
     chore.chore();
 
-    for (Entry<TableName,SpaceViolationPolicy> entry : policiesToEnforce.entrySet()) {
+    for (Entry<TableName,SpaceQuotaSnapshot> entry : policiesToEnforce.entrySet()) {
       verify(manager).enforceViolationPolicy(entry.getKey(), entry.getValue());
     }
     verify(manager, never()).disableViolationPolicyEnforcement(TableName.valueOf("table1"));
   }
+
+  @Test
+  public void testMissingAllColumns() throws IOException {
+    when(chore.fetchSnapshotsFromQuotaTable()).thenCallRealMethod();
+    ResultScanner scanner = mock(ResultScanner.class);
+    Table quotaTable = mock(Table.class);
+    when(conn.getTable(QuotaUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
+    when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
+
+    List<Result> results = new ArrayList<>();
+    results.add(Result.create(Collections.emptyList()));
+    when(scanner.iterator()).thenReturn(results.iterator());
+    try {
+      chore.fetchSnapshotsFromQuotaTable();
+      fail("Expected an IOException, but did not receive one.");
+    } catch (IOException e) {
+      // Expected an error because we had no cells in the row.
+      // This should only happen due to programmer error.
+    }
+  }
+
+  @Test
+  public void testMissingDesiredColumn() throws IOException {
+    when(chore.fetchSnapshotsFromQuotaTable()).thenCallRealMethod();
+    ResultScanner scanner = mock(ResultScanner.class);
+    Table quotaTable = mock(Table.class);
+    when(conn.getTable(QuotaUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
+    when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
+
+    List<Result> results = new ArrayList<>();
+    // Give a column that isn't the one we want
+    Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("q"), toBytes("s"), new byte[0]);
+    results.add(Result.create(Collections.singletonList(c)));
+    when(scanner.iterator()).thenReturn(results.iterator());
+    try {
+      chore.fetchSnapshotsFromQuotaTable();
+      fail("Expected an IOException, but did not receive one.");
+    } catch (IOException e) {
+      // Expected an error because we were missing the column we expected in this row.
+      // This should only happen due to programmer error.
+    }
+  }
+
+  @Test
+  public void testParsingError() throws IOException {
+    when(chore.fetchSnapshotsFromQuotaTable()).thenCallRealMethod();
+    ResultScanner scanner = mock(ResultScanner.class);
+    Table quotaTable = mock(Table.class);
+    when(conn.getTable(QuotaUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
+    when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
+
+    List<Result> results = new ArrayList<>();
+    Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("u"), toBytes("v"), new byte[0]);
+    results.add(Result.create(Collections.singletonList(c)));
+    when(scanner.iterator()).thenReturn(results.iterator());
+    try {
+      chore.fetchSnapshotsFromQuotaTable();
+      fail("Expected an IOException, but did not receive one.");
+    } catch (IOException e) {
+      // We provided a garbage serialized protobuf message (empty byte array), this should
+      // in turn throw an IOException
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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
new file mode 100644
index 0000000..ea7dcf1
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
@@ -0,0 +1,452 @@
+/*
+ * 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.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;
+
+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.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.ClientServiceCallable;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.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.BulkLoadVerifyingViolationPolicyEnforcement;
+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;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * End-to-end test class for filesystem space quotas.
+ */
+@Category(LargeTests.class)
+public class TestSpaceQuotas {
+  private static final Log LOG = LogFactory.getLog(TestSpaceQuotas.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final AtomicLong COUNTER = new AtomicLong(0);
+  private static final int NUM_RETRIES = 10;
+
+  @Rule
+  public TestName testName = new TestName();
+  private SpaceQuotaHelperForTests helper;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    // Increase the frequency of some of the chores for responsiveness of the test
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_KEY, 1000);
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 1000);
+    conf.setInt(QuotaObserverChore.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);
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void removeAllQuotas() throws Exception {
+    final Connection conn = TEST_UTIL.getConnection();
+    // Wait for the quota table to be created
+    if (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME)) {
+      do {
+        LOG.debug("Quota table does not yet exist");
+        Thread.sleep(1000);
+      } while (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME));
+    } else {
+      // Or, clean up any quotas from previous test runs.
+      QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration());
+      for (QuotaSettings quotaSettings : scanner) {
+        final String namespace = quotaSettings.getNamespace();
+        final TableName tableName = quotaSettings.getTableName();
+        if (null != namespace) {
+          LOG.debug("Deleting quota for namespace: " + namespace);
+          QuotaUtil.deleteNamespaceQuota(conn, namespace);
+        } else {
+          assert null != tableName;
+          LOG.debug("Deleting quota for table: "+ tableName);
+          QuotaUtil.deleteTableQuota(conn, tableName);
+        }
+      }
+    }
+    helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
+  }
+
+  @Test
+  public void testNoInsertsWithPut() throws Exception {
+    Put p = new Put(Bytes.toBytes("to_reject"));
+    p.addColumn(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, p);
+  }
+
+  @Test
+  public void testNoInsertsWithAppend() throws Exception {
+    Append a = new Append(Bytes.toBytes("to_reject"));
+    a.add(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, a);
+  }
+
+  @Test
+  public void testNoInsertsWithIncrement() throws Exception {
+    Increment i = new Increment(Bytes.toBytes("to_reject"));
+    i.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("count"), 0);
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, i);
+  }
+
+  @Test
+  public void testDeletesAfterNoInserts() throws Exception {
+    final TableName tn = writeUntilViolation(SpaceViolationPolicy.NO_INSERTS);
+    // Try a couple of times to verify that the quota never gets enforced, same as we
+    // do when we're trying to catch the failure.
+    Delete d = new Delete(Bytes.toBytes("should_not_be_rejected"));
+    for (int i = 0; i < NUM_RETRIES; i++) {
+      try (Table t = TEST_UTIL.getConnection().getTable(tn)) {
+        t.delete(d);
+      }
+    }
+  }
+
+  @Test
+  public void testNoWritesWithPut() throws Exception {
+    Put p = new Put(Bytes.toBytes("to_reject"));
+    p.addColumn(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, p);
+  }
+
+  @Test
+  public void testNoWritesWithAppend() throws Exception {
+    Append a = new Append(Bytes.toBytes("to_reject"));
+    a.add(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, a);
+  }
+
+  @Test
+  public void testNoWritesWithIncrement() throws Exception {
+    Increment i = new Increment(Bytes.toBytes("to_reject"));
+    i.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("count"), 0);
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, i);
+  }
+
+  @Test
+  public void testNoWritesWithDelete() throws Exception {
+    Delete d = new Delete(Bytes.toBytes("to_reject"));
+    writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, d);
+  }
+
+  @Test
+  public void testNoCompactions() throws Exception {
+    Put p = new Put(Bytes.toBytes("to_reject"));
+    p.addColumn(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    final TableName tn = writeUntilViolationAndVerifyViolation(
+        SpaceViolationPolicy.NO_WRITES_COMPACTIONS, p);
+    // We know the policy is active at this point
+
+    // Major compactions should be rejected
+    try {
+      TEST_UTIL.getAdmin().majorCompact(tn);
+      fail("Expected that invoking the compaction should throw an Exception");
+    } catch (DoNotRetryIOException e) {
+      // Expected!
+    }
+    // Minor compactions should also be rejected.
+    try {
+      TEST_UTIL.getAdmin().compact(tn);
+      fail("Expected that invoking the compaction should throw an Exception");
+    } catch (DoNotRetryIOException e) {
+      // Expected!
+    }
+  }
+
+  @Test
+  public void testNoEnableAfterDisablePolicy() throws Exception {
+    Put p = new Put(Bytes.toBytes("to_reject"));
+    p.addColumn(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    final TableName tn = writeUntilViolation(SpaceViolationPolicy.DISABLE);
+    final Admin admin = TEST_UTIL.getAdmin();
+    // Disabling a table relies on some external action (over the other policies), so wait a bit
+    // more than the other tests.
+    for (int i = 0; i < NUM_RETRIES * 2; i++) {
+      if (admin.isTableEnabled(tn)) {
+        LOG.info(tn + " is still enabled, expecting it to be disabled. Will wait and re-check.");
+        Thread.sleep(2000);
+      }
+    }
+    assertFalse(tn + " is still enabled but it should be disabled", admin.isTableEnabled(tn));
+    try {
+      admin.enableTable(tn);
+    } catch (AccessDeniedException e) {
+      String exceptionContents = StringUtils.stringifyException(e);
+      final String expectedText = "violated space quota";
+      assertTrue("Expected the exception to contain " + expectedText + ", but was: "
+          + exceptionContents, exceptionContents.contains(expectedText));
+    }
+  }
+
+  @Test(timeout=120000)
+  public void testNoBulkLoadsWithNoWrites() throws Exception {
+    Put p = new Put(Bytes.toBytes("to_reject"));
+    p.addColumn(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    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);
+    RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
+    RpcRetryingCaller<Void> caller = factory.<Void> newCaller();
+    try {
+      caller.callWithRetries(callable, Integer.MAX_VALUE);
+      fail("Expected the bulk load call to fail!");
+    } catch (SpaceLimitingException e) {
+      // Pass
+      LOG.trace("Caught expected exception", e);
+    }
+  }
+
+  @Test(timeout=120000)
+  public void testAtomicBulkLoadUnderQuota() throws Exception {
+    // Need to verify that if the batch of hfiles cannot be loaded, none are loaded.
+    TableName tn = helper.createTableWithRegions(10);
+
+    final long sizeLimit = 50L * SpaceQuotaHelperForTests.ONE_KILOBYTE;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, sizeLimit, SpaceViolationPolicy.NO_INSERTS);
+    TEST_UTIL.getAdmin().setQuota(settings);
+
+    HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
+    RegionServerSpaceQuotaManager spaceQuotaManager = rs.getRegionServerSpaceQuotaManager();
+    Map<TableName,SpaceQuotaSnapshot> snapshots = spaceQuotaManager.copyQuotaSnapshots();
+    Map<HRegionInfo,Long> regionSizes = getReportedSizesForTable(tn);
+    while (true) {
+      SpaceQuotaSnapshot snapshot = snapshots.get(tn);
+      if (null != snapshot && snapshot.getLimit() > 0) {
+        break;
+      }
+      LOG.debug(
+          "Snapshot does not yet realize quota limit: " + snapshots + ", regionsizes: " +
+          regionSizes);
+      Thread.sleep(3000);
+      snapshots = spaceQuotaManager.copyQuotaSnapshots();
+      regionSizes = getReportedSizesForTable(tn);
+    }
+    // Our quota limit should be reflected in the latest snapshot
+    SpaceQuotaSnapshot snapshot = snapshots.get(tn);
+    assertEquals(0L, snapshot.getUsage());
+    assertEquals(sizeLimit, snapshot.getLimit());
+
+    // We would also not have a "real" policy in violation
+    ActivePolicyEnforcement activePolicies = spaceQuotaManager.getActiveEnforcements();
+    SpaceViolationPolicyEnforcement enforcement = activePolicies.getPolicyEnforcement(tn);
+    assertTrue(
+        "Expected to find Noop policy, but got " + enforcement.getClass().getSimpleName(),
+        enforcement instanceof BulkLoadVerifyingViolationPolicyEnforcement);
+
+    // Should generate two files, each of which is over 25KB each
+    ClientServiceCallable<Void> callable = generateFileToLoad(tn, 2, 500);
+    FileSystem fs = TEST_UTIL.getTestFileSystem();
+    FileStatus[] files = fs.listStatus(
+        new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files"));
+    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);
+      LOG.debug(file.getPath() + " -> " + file.getLen() +"B");
+    }
+
+    RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
+    RpcRetryingCaller<Void> caller = factory.<Void> newCaller();
+    try {
+      caller.callWithRetries(callable, Integer.MAX_VALUE);
+      fail("Expected the bulk load call to fail!");
+    } catch (SpaceLimitingException e) {
+      // Pass
+      LOG.trace("Caught expected exception", e);
+    }
+    // Verify that we have no data in the table because neither file should have been
+    // loaded even though one of the files could have.
+    Table table = TEST_UTIL.getConnection().getTable(tn);
+    ResultScanner scanner = table.getScanner(new Scan());
+    try {
+      assertNull("Expected no results", scanner.next());
+    } finally{
+      scanner.close();
+    }
+  }
+
+  private Map<HRegionInfo,Long> getReportedSizesForTable(TableName tn) {
+    HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
+    MasterQuotaManager quotaManager = master.getMasterQuotaManager();
+    Map<HRegionInfo,Long> filteredRegionSizes = new HashMap<>();
+    for (Entry<HRegionInfo,Long> entry : quotaManager.snapshotRegionSizes().entrySet()) {
+      if (entry.getKey().getTable().equals(tn)) {
+        filteredRegionSizes.put(entry.getKey(), entry.getValue());
+      }
+    }
+    return filteredRegionSizes;
+  }
+
+  private TableName writeUntilViolation(SpaceViolationPolicy policyToViolate) throws Exception {
+    TableName tn = helper.createTableWithRegions(10);
+
+    final long sizeLimit = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, policyToViolate);
+    TEST_UTIL.getAdmin().setQuota(settings);
+
+    // Write more data than should be allowed and flush it to disk
+    helper.writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
+
+    // This should be sufficient time for the chores to run and see the change.
+    Thread.sleep(5000);
+
+    return tn;
+  }
+
+  private TableName writeUntilViolationAndVerifyViolation(
+      SpaceViolationPolicy policyToViolate, Mutation m) throws Exception {
+    final TableName tn = writeUntilViolation(policyToViolate);
+
+    // But let's try a few times to get the exception before failing
+    boolean sawError = false;
+    for (int i = 0; i < NUM_RETRIES && !sawError; i++) {
+      try (Table table = TEST_UTIL.getConnection().getTable(tn)) {
+        if (m instanceof Put) {
+          table.put((Put) m);
+        } else if (m instanceof Delete) {
+          table.delete((Delete) m);
+        } else if (m instanceof Append) {
+          table.append((Append) m);
+        } else if (m instanceof Increment) {
+          table.increment((Increment) m);
+        } else {
+          fail(
+              "Failed to apply " + m.getClass().getSimpleName() +
+              " to the table. Programming error");
+        }
+        LOG.info("Did not reject the " + m.getClass().getSimpleName() + ", will sleep and retry");
+        Thread.sleep(2000);
+      } catch (Exception e) {
+        String msg = StringUtils.stringifyException(e);
+        assertTrue("Expected exception message to contain the word '" + policyToViolate.name() +
+            "', but was " + msg, msg.contains(policyToViolate.name()));
+        sawError = true;
+      }
+    }
+    if (!sawError) {
+      try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
+        ResultScanner scanner = quotaTable.getScanner(new Scan());
+        Result result = null;
+        LOG.info("Dumping contents of hbase:quota table");
+        while ((result = scanner.next()) != null) {
+          LOG.info(Bytes.toString(result.getRow()) + " => " + result.toString());
+        }
+        scanner.close();
+      }
+    }
+    assertTrue(
+        "Expected to see an exception writing data to a table exceeding its quota", sawError);
+
+    return tn;
+  }
+
+  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()) {
+      @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/0d76d667/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 efc046b..cefed67 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
@@ -30,7 +30,7 @@ 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.quotas.QuotaViolationStore.ViolationState;
+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;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
@@ -44,7 +44,7 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 /**
- * Test class for {@link TableQuotaViolationStore}.
+ * Test class for {@link TableQuotaSnapshotStore}.
  */
 @Category(SmallTests.class)
 public class TestTableQuotaViolationStore {
@@ -53,14 +53,14 @@ public class TestTableQuotaViolationStore {
   private Connection conn;
   private QuotaObserverChore chore;
   private Map<HRegionInfo, Long> regionReports;
-  private TableQuotaViolationStore store;
+  private TableQuotaSnapshotStore store;
 
   @Before
   public void setup() {
     conn = mock(Connection.class);
     chore = mock(QuotaObserverChore.class);
     regionReports = new HashMap<>();
-    store = new TableQuotaViolationStore(conn, chore, regionReports);
+    store = new TableQuotaSnapshotStore(conn, chore, regionReports);
   }
 
   @Test
@@ -108,23 +108,29 @@ public class TestTableQuotaViolationStore {
     regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(0), Bytes.toBytes(1)), 1024L * 512L);
     regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(1), Bytes.toBytes(2)), 1024L * 256L);
 
+    SpaceQuotaSnapshot tn1Snapshot = new SpaceQuotaSnapshot(
+        SpaceQuotaStatus.notInViolation(), 1024L * 768L, 1024L * 1024L);
+
     // Below the quota
-    assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(tn1, quota));
+    assertEquals(tn1Snapshot, store.getTargetState(tn1, quota));
 
     regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(2), Bytes.toBytes(3)), 1024L * 256L);
+    tn1Snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 1024L * 1024L, 1024L * 1024L);
 
     // Equal to the quota is still in observance
-    assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(tn1, quota));
+    assertEquals(tn1Snapshot, store.getTargetState(tn1, quota));
 
     regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(3), Bytes.toBytes(4)), 1024L);
+    tn1Snapshot = new SpaceQuotaSnapshot(
+        new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L * 1024L + 1024L, 1024L * 1024L);
 
     // Exceeds the quota, should be in violation
-    assertEquals(ViolationState.IN_VIOLATION, store.getTargetState(tn1, quota));
+    assertEquals(tn1Snapshot, store.getTargetState(tn1, quota));
   }
 
   @Test
   public void testGetSpaceQuota() throws Exception {
-    TableQuotaViolationStore mockStore = mock(TableQuotaViolationStore.class);
+    TableQuotaSnapshotStore mockStore = mock(TableQuotaSnapshotStore.class);
     when(mockStore.getSpaceQuota(any(TableName.class))).thenCallRealMethod();
 
     Quotas quotaWithSpace = Quotas.newBuilder().setSpace(

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java
index 4a7000c..d190c8c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java
@@ -30,12 +30,11 @@ import java.util.Objects;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Before;
@@ -44,17 +43,17 @@ import org.junit.experimental.categories.Category;
 import org.mockito.ArgumentMatcher;
 
 /**
- * Test case for {@link TableSpaceQuotaViolationNotifier}.
+ * Test case for {@link TableSpaceQuotaSnapshotNotifier}.
  */
 @Category(SmallTests.class)
 public class TestTableSpaceQuotaViolationNotifier {
 
-  private TableSpaceQuotaViolationNotifier notifier;
+  private TableSpaceQuotaSnapshotNotifier notifier;
   private Connection conn;
 
   @Before
   public void setup() throws Exception {
-    notifier = new TableSpaceQuotaViolationNotifier();
+    notifier = new TableSpaceQuotaSnapshotNotifier();
     conn = mock(Connection.class);
     notifier.initialize(conn);
   }
@@ -62,35 +61,25 @@ public class TestTableSpaceQuotaViolationNotifier {
   @Test
   public void testToViolation() throws Exception {
     final TableName tn = TableName.valueOf("inviolation");
-    final SpaceViolationPolicy policy = SpaceViolationPolicy.NO_INSERTS;
+    final SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(
+        new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 1024L, 512L);
     final Table quotaTable = mock(Table.class);
     when(conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
 
     final Put expectedPut = new Put(Bytes.toBytes("t." + tn.getNameAsString()));
-    final SpaceQuota protoQuota = SpaceQuota.newBuilder()
-        .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(policy))
+    final QuotaProtos.SpaceQuotaSnapshot protoQuota = QuotaProtos.SpaceQuotaSnapshot.newBuilder()
+        .setStatus(QuotaProtos.SpaceQuotaStatus.newBuilder().setInViolation(true).setPolicy(
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.NO_INSERTS))
+        .setLimit(512L)
+        .setUsage(1024L)
         .build();
-    expectedPut.addColumn(Bytes.toBytes("u"), Bytes.toBytes("v"), protoQuota.toByteArray());
+    expectedPut.addColumn(Bytes.toBytes("u"), Bytes.toBytes("p"), protoQuota.toByteArray());
 
-    notifier.transitionTableToViolation(tn, policy);
+    notifier.transitionTable(tn, snapshot);
 
     verify(quotaTable).put(argThat(new SingleCellPutMatcher(expectedPut)));
   }
 
-  @Test
-  public void testToObservance() throws Exception {
-    final TableName tn = TableName.valueOf("notinviolation");
-    final Table quotaTable = mock(Table.class);
-    when(conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
-
-    final Delete expectedDelete = new Delete(Bytes.toBytes("t." + tn.getNameAsString()));
-    expectedDelete.addColumn(Bytes.toBytes("u"), Bytes.toBytes("v"));
-
-    notifier.transitionTableToObservance(tn);
-
-    verify(quotaTable).delete(argThat(new SingleCellDeleteMatcher(expectedDelete)));
-  }
-
   /**
    * Parameterized for Puts.
    */
@@ -101,15 +90,6 @@ public class TestTableSpaceQuotaViolationNotifier {
   }
 
   /**
-   * Parameterized for Deletes.
-   */
-  private static class SingleCellDeleteMatcher extends SingleCellMutationMatcher<Delete> {
-    private SingleCellDeleteMatcher(Delete expected) {
-      super(expected);
-    }
-  }
-
-  /**
    * Quick hack to verify a Mutation with one column.
    */
   private static class SingleCellMutationMatcher<T> extends ArgumentMatcher<T> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
index bb8d5cd..3d0aec6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
@@ -96,7 +96,7 @@ public class TestTablesWithQuotas {
     final Map<TableName,Integer> reportedRegions = new HashMap<>();
     final Map<TableName,Integer> actualRegions = new HashMap<>();
     final Configuration conf = HBaseConfiguration.create();
-    conf.setDouble(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_KEY, 0.95);
+    conf.setDouble(QuotaObserverChore.QUOTA_OBSERVER_CHORE_REPORT_PERCENT_KEY, 0.95);
 
     TableName tooFewRegionsTable = TableName.valueOf("tn1");
     TableName sufficientRegionsTable = TableName.valueOf("tn2");
@@ -114,7 +114,7 @@ public class TestTablesWithQuotas {
       }
 
       @Override
-      int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore) {
+      int getNumReportedRegions(TableName table, QuotaSnapshotStore<TableName> tableStore) {
         return reportedRegions.get(table);
       }
     };
@@ -177,13 +177,13 @@ public class TestTablesWithQuotas {
 
     QuotaObserverChore chore = mock(QuotaObserverChore.class);
     Map<HRegionInfo,Long> regionUsage = new HashMap<>();
-    TableQuotaViolationStore store = new TableQuotaViolationStore(conn, chore, regionUsage);
+    TableQuotaSnapshotStore store = new TableQuotaSnapshotStore(conn, chore, regionUsage);
 
     // A super dirty hack to verify that, after getting no regions for our table,
     // we bail out and start processing the next element (which there is none).
     final TablesWithQuotas tables = new TablesWithQuotas(conn, conf) {
       @Override
-      int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore) {
+      int getNumReportedRegions(TableName table, QuotaSnapshotStore<TableName> tableStore) {
         throw new RuntimeException("Should should not reach here");
       }
     };


[23/50] [abbrv] hbase git commit: HBASE-16995 Build client Java API and client protobuf messages (Josh Elser)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
index 01ba8f6..e3c6bfd 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
@@ -239,12 +239,20 @@ public final class QuotaProtos {
      * <code>THROTTLE = 1;</code>
      */
     THROTTLE(1),
+    /**
+     * <code>SPACE = 2;</code>
+     */
+    SPACE(2),
     ;
 
     /**
      * <code>THROTTLE = 1;</code>
      */
     public static final int THROTTLE_VALUE = 1;
+    /**
+     * <code>SPACE = 2;</code>
+     */
+    public static final int SPACE_VALUE = 2;
 
 
     public final int getNumber() {
@@ -262,6 +270,7 @@ public final class QuotaProtos {
     public static QuotaType forNumber(int value) {
       switch (value) {
         case 1: return THROTTLE;
+        case 2: return SPACE;
         default: return null;
       }
     }
@@ -311,6 +320,150 @@ public final class QuotaProtos {
     // @@protoc_insertion_point(enum_scope:hbase.pb.QuotaType)
   }
 
+  /**
+   * <pre>
+   * Defines what action should be taken when the SpaceQuota is violated
+   * </pre>
+   *
+   * Protobuf enum {@code hbase.pb.SpaceViolationPolicy}
+   */
+  public enum SpaceViolationPolicy
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <pre>
+     * Disable the table(s)
+     * </pre>
+     *
+     * <code>DISABLE = 1;</code>
+     */
+    DISABLE(1),
+    /**
+     * <pre>
+     * No writes, bulk-loads, or compactions
+     * </pre>
+     *
+     * <code>NO_WRITES_COMPACTIONS = 2;</code>
+     */
+    NO_WRITES_COMPACTIONS(2),
+    /**
+     * <pre>
+     * No writes or bulk-loads
+     * </pre>
+     *
+     * <code>NO_WRITES = 3;</code>
+     */
+    NO_WRITES(3),
+    /**
+     * <pre>
+     * No puts or bulk-loads, but deletes are allowed
+     * </pre>
+     *
+     * <code>NO_INSERTS = 4;</code>
+     */
+    NO_INSERTS(4),
+    ;
+
+    /**
+     * <pre>
+     * Disable the table(s)
+     * </pre>
+     *
+     * <code>DISABLE = 1;</code>
+     */
+    public static final int DISABLE_VALUE = 1;
+    /**
+     * <pre>
+     * No writes, bulk-loads, or compactions
+     * </pre>
+     *
+     * <code>NO_WRITES_COMPACTIONS = 2;</code>
+     */
+    public static final int NO_WRITES_COMPACTIONS_VALUE = 2;
+    /**
+     * <pre>
+     * No writes or bulk-loads
+     * </pre>
+     *
+     * <code>NO_WRITES = 3;</code>
+     */
+    public static final int NO_WRITES_VALUE = 3;
+    /**
+     * <pre>
+     * No puts or bulk-loads, but deletes are allowed
+     * </pre>
+     *
+     * <code>NO_INSERTS = 4;</code>
+     */
+    public static final int NO_INSERTS_VALUE = 4;
+
+
+    public final int getNumber() {
+      return value;
+    }
+
+    /**
+     * @deprecated Use {@link #forNumber(int)} instead.
+     */
+    @java.lang.Deprecated
+    public static SpaceViolationPolicy valueOf(int value) {
+      return forNumber(value);
+    }
+
+    public static SpaceViolationPolicy forNumber(int value) {
+      switch (value) {
+        case 1: return DISABLE;
+        case 2: return NO_WRITES_COMPACTIONS;
+        case 3: return NO_WRITES;
+        case 4: return NO_INSERTS;
+        default: return null;
+      }
+    }
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<SpaceViolationPolicy>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+        SpaceViolationPolicy> internalValueMap =
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<SpaceViolationPolicy>() {
+            public SpaceViolationPolicy findValueByNumber(int number) {
+              return SpaceViolationPolicy.forNumber(number);
+            }
+          };
+
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(ordinal());
+    }
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.getDescriptor().getEnumTypes().get(3);
+    }
+
+    private static final SpaceViolationPolicy[] VALUES = values();
+
+    public static SpaceViolationPolicy valueOf(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int value;
+
+    private SpaceViolationPolicy(int value) {
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:hbase.pb.SpaceViolationPolicy)
+  }
+
   public interface TimedQuotaOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TimedQuota)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
@@ -3419,6 +3572,19 @@ public final class QuotaProtos {
      * <code>optional .hbase.pb.Throttle throttle = 2;</code>
      */
     org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleOrBuilder getThrottleOrBuilder();
+
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    boolean hasSpace();
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota getSpace();
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getSpaceOrBuilder();
   }
   /**
    * Protobuf type {@code hbase.pb.Quotas}
@@ -3481,6 +3647,19 @@ public final class QuotaProtos {
               bitField0_ |= 0x00000002;
               break;
             }
+            case 26: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = space_.toBuilder();
+              }
+              space_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(space_);
+                space_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000004;
+              break;
+            }
           }
         }
       } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@@ -3542,6 +3721,27 @@ public final class QuotaProtos {
       return throttle_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle.getDefaultInstance() : throttle_;
     }
 
+    public static final int SPACE_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota space_;
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    public boolean hasSpace() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota getSpace() {
+      return space_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance() : space_;
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getSpaceOrBuilder() {
+      return space_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance() : space_;
+    }
+
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
@@ -3566,6 +3766,9 @@ public final class QuotaProtos {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeMessage(2, getThrottle());
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, getSpace());
+      }
       unknownFields.writeTo(output);
     }
 
@@ -3582,6 +3785,10 @@ public final class QuotaProtos {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getThrottle());
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, getSpace());
+      }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
       return size;
@@ -3609,6 +3816,11 @@ public final class QuotaProtos {
         result = result && getThrottle()
             .equals(other.getThrottle());
       }
+      result = result && (hasSpace() == other.hasSpace());
+      if (hasSpace()) {
+        result = result && getSpace()
+            .equals(other.getSpace());
+      }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
     }
@@ -3629,6 +3841,10 @@ public final class QuotaProtos {
         hash = (37 * hash) + THROTTLE_FIELD_NUMBER;
         hash = (53 * hash) + getThrottle().hashCode();
       }
+      if (hasSpace()) {
+        hash = (37 * hash) + SPACE_FIELD_NUMBER;
+        hash = (53 * hash) + getSpace().hashCode();
+      }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -3744,6 +3960,7 @@ public final class QuotaProtos {
         if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getThrottleFieldBuilder();
+          getSpaceFieldBuilder();
         }
       }
       public Builder clear() {
@@ -3756,6 +3973,12 @@ public final class QuotaProtos {
           throttleBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000002);
+        if (spaceBuilder_ == null) {
+          space_ = null;
+        } else {
+          spaceBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -3792,6 +4015,14 @@ public final class QuotaProtos {
         } else {
           result.throttle_ = throttleBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (spaceBuilder_ == null) {
+          result.space_ = space_;
+        } else {
+          result.space_ = spaceBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -3840,6 +4071,9 @@ public final class QuotaProtos {
         if (other.hasThrottle()) {
           mergeThrottle(other.getThrottle());
         }
+        if (other.hasSpace()) {
+          mergeSpace(other.getSpace());
+        }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
         return this;
@@ -4022,6 +4256,124 @@ public final class QuotaProtos {
         }
         return throttleBuilder_;
       }
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota space_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> spaceBuilder_;
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public boolean hasSpace() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota getSpace() {
+        if (spaceBuilder_ == null) {
+          return space_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance() : space_;
+        } else {
+          return spaceBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public Builder setSpace(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota value) {
+        if (spaceBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          space_ = value;
+          onChanged();
+        } else {
+          spaceBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public Builder setSpace(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder builderForValue) {
+        if (spaceBuilder_ == null) {
+          space_ = builderForValue.build();
+          onChanged();
+        } else {
+          spaceBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public Builder mergeSpace(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota value) {
+        if (spaceBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              space_ != null &&
+              space_ != org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance()) {
+            space_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.newBuilder(space_).mergeFrom(value).buildPartial();
+          } else {
+            space_ = value;
+          }
+          onChanged();
+        } else {
+          spaceBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public Builder clearSpace() {
+        if (spaceBuilder_ == null) {
+          space_ = null;
+          onChanged();
+        } else {
+          spaceBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder getSpaceBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getSpaceFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getSpaceOrBuilder() {
+        if (spaceBuilder_ != null) {
+          return spaceBuilder_.getMessageOrBuilder();
+        } else {
+          return space_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance() : space_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
+          getSpaceFieldBuilder() {
+        if (spaceBuilder_ == null) {
+          spaceBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>(
+                  getSpace(),
+                  getParentForChildren(),
+                  isClean());
+          space_ = null;
+        }
+        return spaceBuilder_;
+      }
       public final Builder setUnknownFields(
           final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
@@ -4444,75 +4796,1314 @@ public final class QuotaProtos {
 
   }
 
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_TimedQuota_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_TimedQuota_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_Throttle_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_Throttle_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_ThrottleRequest_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_Quotas_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_Quotas_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_QuotaUsage_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_QuotaUsage_fieldAccessorTable;
+  public interface SpaceQuotaOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.SpaceQuota)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
-  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
+    /**
+     * <pre>
+     * The limit of bytes for this quota
+     * </pre>
+     *
+     * <code>optional uint64 soft_limit = 1;</code>
+     */
+    boolean hasSoftLimit();
+    /**
+     * <pre>
+     * The limit of bytes for this quota
+     * </pre>
+     *
+     * <code>optional uint64 soft_limit = 1;</code>
+     */
+    long getSoftLimit();
+
+    /**
+     * <pre>
+     * The action to take when the quota is violated
+     * </pre>
+     *
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+     */
+    boolean hasViolationPolicy();
+    /**
+     * <pre>
+     * The action to take when the quota is violated
+     * </pre>
+     *
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy();
   }
-  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n\013Quota.proto\022\010hbase.pb\032\013HBase.proto\"\204\001\n" +
-      "\nTimedQuota\022%\n\ttime_unit\030\001 \002(\0162\022.hbase.p" +
-      "b.TimeUnit\022\022\n\nsoft_limit\030\002 \001(\004\022\r\n\005share\030" +
-      "\003 \001(\002\022,\n\005scope\030\004 \001(\0162\024.hbase.pb.QuotaSco" +
-      "pe:\007MACHINE\"\375\001\n\010Throttle\022%\n\007req_num\030\001 \001(" +
-      "\0132\024.hbase.pb.TimedQuota\022&\n\010req_size\030\002 \001(" +
-      "\0132\024.hbase.pb.TimedQuota\022\'\n\twrite_num\030\003 \001" +
-      "(\0132\024.hbase.pb.TimedQuota\022(\n\nwrite_size\030\004" +
-      " \001(\0132\024.hbase.pb.TimedQuota\022&\n\010read_num\030\005" +
-      " \001(\0132\024.hbase.pb.TimedQuota\022\'\n\tread_size\030",
-      "\006 \001(\0132\024.hbase.pb.TimedQuota\"b\n\017ThrottleR" +
-      "equest\022$\n\004type\030\001 \001(\0162\026.hbase.pb.Throttle" +
-      "Type\022)\n\013timed_quota\030\002 \001(\0132\024.hbase.pb.Tim" +
-      "edQuota\"M\n\006Quotas\022\035\n\016bypass_globals\030\001 \001(" +
-      "\010:\005false\022$\n\010throttle\030\002 \001(\0132\022.hbase.pb.Th" +
-      "rottle\"\014\n\nQuotaUsage*&\n\nQuotaScope\022\013\n\007CL" +
-      "USTER\020\001\022\013\n\007MACHINE\020\002*v\n\014ThrottleType\022\022\n\016" +
-      "REQUEST_NUMBER\020\001\022\020\n\014REQUEST_SIZE\020\002\022\020\n\014WR" +
-      "ITE_NUMBER\020\003\022\016\n\nWRITE_SIZE\020\004\022\017\n\013READ_NUM" +
-      "BER\020\005\022\r\n\tREAD_SIZE\020\006*\031\n\tQuotaType\022\014\n\010THR",
-      "OTTLE\020\001BH\n1org.apache.hadoop.hbase.shade" +
-      "d.protobuf.generatedB\013QuotaProtosH\001\210\001\001\240\001" +
-      "\001"
-    };
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
-              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
-            descriptor = root;
-            return null;
-          }
-        };
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
-      .internalBuildGeneratedFileFrom(descriptorData,
-        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
+  /**
+   * <pre>
+   * Defines a limit on the amount of filesystem space used by a table/namespace
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.SpaceQuota}
+   */
+  public  static final class SpaceQuota extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.SpaceQuota)
+      SpaceQuotaOrBuilder {
+    // Use SpaceQuota.newBuilder() to construct.
+    private SpaceQuota(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private SpaceQuota() {
+      softLimit_ = 0L;
+      violationPolicy_ = 1;
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SpaceQuota(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              softLimit_ = input.readUInt64();
+              break;
+            }
+            case 16: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy value = org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(2, rawValue);
+              } else {
+                bitField0_ |= 0x00000002;
+                violationPolicy_ = rawValue;
+              }
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuota_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuota_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int SOFT_LIMIT_FIELD_NUMBER = 1;
+    private long softLimit_;
+    /**
+     * <pre>
+     * The limit of bytes for this quota
+     * </pre>
+     *
+     * <code>optional uint64 soft_limit = 1;</code>
+     */
+    public boolean hasSoftLimit() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <pre>
+     * The limit of bytes for this quota
+     * </pre>
+     *
+     * <code>optional uint64 soft_limit = 1;</code>
+     */
+    public long getSoftLimit() {
+      return softLimit_;
+    }
+
+    public static final int VIOLATION_POLICY_FIELD_NUMBER = 2;
+    private int violationPolicy_;
+    /**
+     * <pre>
+     * The action to take when the quota is violated
+     * </pre>
+     *
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+     */
+    public boolean hasViolationPolicy() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <pre>
+     * The action to take when the quota is violated
+     * </pre>
+     *
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy() {
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy result = org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.valueOf(violationPolicy_);
+      return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE : result;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt64(1, softLimit_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeEnum(2, violationPolicy_);
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(1, softLimit_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeEnumSize(2, violationPolicy_);
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota other = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota) obj;
+
+      boolean result = true;
+      result = result && (hasSoftLimit() == other.hasSoftLimit());
+      if (hasSoftLimit()) {
+        result = result && (getSoftLimit()
+            == other.getSoftLimit());
+      }
+      result = result && (hasViolationPolicy() == other.hasViolationPolicy());
+      if (hasViolationPolicy()) {
+        result = result && violationPolicy_ == other.violationPolicy_;
+      }
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      if (hasSoftLimit()) {
+        hash = (37 * hash) + SOFT_LIMIT_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+            getSoftLimit());
+      }
+      if (hasViolationPolicy()) {
+        hash = (37 * hash) + VIOLATION_POLICY_FIELD_NUMBER;
+        hash = (53 * hash) + violationPolicy_;
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     * Defines a limit on the amount of filesystem space used by a table/namespace
+     * </pre>
+     *
+     * Protobuf type {@code hbase.pb.SpaceQuota}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.SpaceQuota)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuota_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuota_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        softLimit_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        violationPolicy_ = 1;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuota_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota result = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.softLimit_ = softLimit_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.violationPolicy_ = violationPolicy_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance()) return this;
+        if (other.hasSoftLimit()) {
+          setSoftLimit(other.getSoftLimit());
+        }
+        if (other.hasViolationPolicy()) {
+          setViolationPolicy(other.getViolationPolicy());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private long softLimit_ ;
+      /**
+       * <pre>
+       * The limit of bytes for this quota
+       * </pre>
+       *
+       * <code>optional uint64 soft_limit = 1;</code>
+       */
+      public boolean hasSoftLimit() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <pre>
+       * The limit of bytes for this quota
+       * </pre>
+       *
+       * <code>optional uint64 soft_limit = 1;</code>
+       */
+      public long getSoftLimit() {
+        return softLimit_;
+      }
+      /**
+       * <pre>
+       * The limit of bytes for this quota
+       * </pre>
+       *
+       * <code>optional uint64 soft_limit = 1;</code>
+       */
+      public Builder setSoftLimit(long value) {
+        bitField0_ |= 0x00000001;
+        softLimit_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * The limit of bytes for this quota
+       * </pre>
+       *
+       * <code>optional uint64 soft_limit = 1;</code>
+       */
+      public Builder clearSoftLimit() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        softLimit_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      private int violationPolicy_ = 1;
+      /**
+       * <pre>
+       * The action to take when the quota is violated
+       * </pre>
+       *
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+       */
+      public boolean hasViolationPolicy() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <pre>
+       * The action to take when the quota is violated
+       * </pre>
+       *
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy result = org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.valueOf(violationPolicy_);
+        return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE : result;
+      }
+      /**
+       * <pre>
+       * The action to take when the quota is violated
+       * </pre>
+       *
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+       */
+      public Builder setViolationPolicy(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000002;
+        violationPolicy_ = value.getNumber();
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * The action to take when the quota is violated
+       * </pre>
+       *
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
+       */
+      public Builder clearViolationPolicy() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        violationPolicy_ = 1;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SpaceQuota)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.SpaceQuota)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceQuota>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<SpaceQuota>() {
+      public SpaceQuota parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new SpaceQuota(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceQuota> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceQuota> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface SpaceLimitRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.SpaceLimitRequest)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    boolean hasQuota();
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota getQuota();
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getQuotaOrBuilder();
+  }
+  /**
+   * <pre>
+   * The Request to limit space usage (to allow for schema evolution not tied to SpaceQuota).
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.SpaceLimitRequest}
+   */
+  public  static final class SpaceLimitRequest extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.SpaceLimitRequest)
+      SpaceLimitRequestOrBuilder {
+    // Use SpaceLimitRequest.newBuilder() to construct.
+    private SpaceLimitRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private SpaceLimitRequest() {
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SpaceLimitRequest(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = quota_.toBuilder();
+              }
+              quota_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(quota_);
+                quota_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceLimitRequest_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int QUOTA_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota quota_;
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    public boolean hasQuota() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota getQuota() {
+      return quota_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance() : quota_;
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getQuotaOrBuilder() {
+      return quota_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance() : quota_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, getQuota());
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getQuota());
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest) obj;
+
+      boolean result = true;
+      result = result && (hasQuota() == other.hasQuota());
+      if (hasQuota()) {
+        result = result && getQuota()
+            .equals(other.getQuota());
+      }
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      if (hasQuota()) {
+        hash = (37 * hash) + QUOTA_FIELD_NUMBER;
+        hash = (53 * hash) + getQuota().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     * The Request to limit space usage (to allow for schema evolution not tied to SpaceQuota).
+     * </pre>
+     *
+     * Protobuf type {@code hbase.pb.SpaceLimitRequest}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.SpaceLimitRequest)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceLimitRequest_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getQuotaFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (quotaBuilder_ == null) {
+          quota_ = null;
+        } else {
+          quotaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceLimitRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (quotaBuilder_ == null) {
+          result.quota_ = quota_;
+        } else {
+          result.quota_ = quotaBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance()) return this;
+        if (other.hasQuota()) {
+          mergeQuota(other.getQuota());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota quota_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> quotaBuilder_;
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public boolean hasQuota() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota getQuota() {
+        if (quotaBuilder_ == null) {
+          return quota_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance() : quota_;
+        } else {
+          return quotaBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public Builder setQuota(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota value) {
+        if (quotaBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          quota_ = value;
+          onChanged();
+        } else {
+          quotaBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public Builder setQuota(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder builderForValue) {
+        if (quotaBuilder_ == null) {
+          quota_ = builderForValue.build();
+          onChanged();
+        } else {
+          quotaBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public Builder mergeQuota(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota value) {
+        if (quotaBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              quota_ != null &&
+              quota_ != org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance()) {
+            quota_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.newBuilder(quota_).mergeFrom(value).buildPartial();
+          } else {
+            quota_ = value;
+          }
+          onChanged();
+        } else {
+          quotaBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public Builder clearQuota() {
+        if (quotaBuilder_ == null) {
+          quota_ = null;
+          onChanged();
+        } else {
+          quotaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder getQuotaBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getQuotaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder getQuotaOrBuilder() {
+        if (quotaBuilder_ != null) {
+          return quotaBuilder_.getMessageOrBuilder();
+        } else {
+          return quota_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.getDefaultInstance() : quota_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
+          getQuotaFieldBuilder() {
+        if (quotaBuilder_ == null) {
+          quotaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>(
+                  getQuota(),
+                  getParentForChildren(),
+                  isClean());
+          quota_ = null;
+        }
+        return quotaBuilder_;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SpaceLimitRequest)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.SpaceLimitRequest)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceLimitRequest>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<SpaceLimitRequest>() {
+      public SpaceLimitRequest parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new SpaceLimitRequest(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceLimitRequest> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SpaceLimitRequest> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_TimedQuota_descriptor;
+  private static final 
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_hbase_pb_TimedQuota_fieldAccessorTable;
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_Throttle_descriptor;
+  private static final 
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_hbase_pb_Throttle_fieldAccessorTable;
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_ThrottleRequest_descriptor;
+  private static final 
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable;
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_Quotas_descriptor;
+  private static final 
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_hbase_pb_Quotas_fieldAccessorTable;
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_QuotaUsage_descriptor;
+  private static final 
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_hbase_pb_QuotaUsage_fieldAccessorTable;
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_SpaceQuota_descriptor;
+  private static final 
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_hbase_pb_SpaceQuota_fieldAccessorTable;
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_SpaceLimitRequest_descriptor;
+  private static final 
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable;
+
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\013Quota.proto\022\010hbase.pb\032\013HBase.proto\"\204\001\n" +
+      "\nTimedQuota\022%\n\ttime_unit\030\001 \002(\0162\022.hbase.p" +
+      "b.TimeUnit\022\022\n\nsoft_limit\030\002 \001(\004\022\r\n\005share\030" +
+      "\003 \001(\002\022,\n\005scope\030\004 \001(\0162\024.hbase.pb.QuotaSco" +
+      "pe:\007MACHINE\"\375\001\n\010Throttle\022%\n\007req_num\030\001 \001(" +
+      "\0132\024.hbase.pb.TimedQuota\022&\n\010req_size\030\002 \001(" +
+      "\0132\024.hbase.pb.TimedQuota\022\'\n\twrite_num\030\003 \001" +
+      "(\0132\024.hbase.pb.TimedQuota\022(\n\nwrite_size\030\004" +
+      " \001(\0132\024.hbase.pb.TimedQuota\022&\n\010read_num\030\005" +
+      " \001(\0132\024.hbase.pb.TimedQuota\022\'\n\tread_size\030",
+      "\006 \001(\0132\024.hbase.pb.TimedQuota\"b\n\017ThrottleR" +
+      "equest\022$\n\004type\030\001 \001(\0162\026.hbase.pb.Throttle" +
+      "Type\022)\n\013timed_quota\030\002 \001(\0132\024.hbase.pb.Tim" +
+      "edQuota\"r\n\006Quotas\022\035\n\016bypass_globals\030\001 \001(" +
+      "\010:\005false\022$\n\010throttle\030\002 \001(\0132\022.hbase.pb.Th" +
+      "rottle\022#\n\005space\030\003 \001(\0132\024.hbase.pb.SpaceQu" +
+      "ota\"\014\n\nQuotaUsage\"Z\n\nSpaceQuota\022\022\n\nsoft_" +
+      "limit\030\001 \001(\004\0228\n\020violation_policy\030\002 \001(\0162\036." +
+      "hbase.pb.SpaceViolationPolicy\"8\n\021SpaceLi" +
+      "mitRequest\022#\n\005quota\030\001 \001(\0132\024.hbase.pb.Spa",
+      "ceQuota*&\n\nQuotaScope\022\013\n\007CLUSTER\020\001\022\013\n\007MA" +
+      "CHINE\020\002*v\n\014ThrottleType\022\022\n\016REQUEST_NUMBE" +
+      "R\020\001\022\020\n\014REQUEST_SIZE\020\002\022\020\n\014WRITE_NUMBER\020\003\022" +
+      "\016\n\nWRITE_SIZE\020\004\022\017\n\013READ_NUMBER\020\005\022\r\n\tREAD" +
+      "_SIZE\020\006*$\n\tQuotaType\022\014\n\010THROTTLE\020\001\022\t\n\005SP" +
+      "ACE\020\002*]\n\024SpaceViolationPolicy\022\013\n\007DISABLE" +
+      "\020\001\022\031\n\025NO_WRITES_COMPACTIONS\020\002\022\r\n\tNO_WRIT" +
+      "ES\020\003\022\016\n\nNO_INSERTS\020\004BH\n1org.apache.hadoo" +
+      "p.hbase.shaded.protobuf.generatedB\013Quota" +
+      "ProtosH\001\210\001\001\240\001\001"
+    };
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
+            descriptor = root;
+            return null;
+          }
+        };
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_TimedQuota_descriptor =
@@ -4538,13 +6129,25 @@ public final class QuotaProtos {
     internal_static_hbase_pb_Quotas_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Quotas_descriptor,
-        new java.lang.String[] { "BypassGlobals", "Throttle", });
+        new java.lang.String[] { "BypassGlobals", "Throttle", "Space", });
     internal_static_hbase_pb_QuotaUsage_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_QuotaUsage_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_QuotaUsage_descriptor,
         new java.lang.String[] { });
+    internal_static_hbase_pb_SpaceQuota_descriptor =
+      getDescriptor().getMessageTypes().get(5);
+    internal_static_hbase_pb_SpaceQuota_fieldAccessorTable = new
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_hbase_pb_SpaceQuota_descriptor,
+        new java.lang.String[] { "SoftLimit", "ViolationPolicy", });
+    internal_static_hbase_pb_SpaceLimitRequest_descriptor =
+      getDescriptor().getMessageTypes().get(6);
+    internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable = new
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_hbase_pb_SpaceLimitRequest_descriptor,
+        new java.lang.String[] { "Quota", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index d7d51e2..7b27663 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -543,6 +543,8 @@ message SetQuotaRequest {
   optional bool remove_all = 5;
   optional bool bypass_globals = 6;
   optional ThrottleRequest throttle = 7;
+
+  optional SpaceLimitRequest space_limit = 8;
 }
 
 message SetQuotaResponse {

http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-protocol-shaded/src/main/protobuf/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Quota.proto b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
index 240c535..b53219a 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Quota.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
@@ -65,12 +65,33 @@ message ThrottleRequest {
 
 enum QuotaType {
   THROTTLE = 1;
+  SPACE = 2;
 }
 
 message Quotas {
   optional bool bypass_globals = 1 [default = false];
   optional Throttle throttle = 2;
+  optional SpaceQuota space = 3;
 }
 
 message QuotaUsage {
 }
+
+// Defines what action should be taken when the SpaceQuota is violated
+enum SpaceViolationPolicy {
+  DISABLE = 1; // Disable the table(s)
+  NO_WRITES_COMPACTIONS = 2; // No writes, bulk-loads, or compactions
+  NO_WRITES = 3; // No writes or bulk-loads
+  NO_INSERTS = 4; // No puts or bulk-loads, but deletes are allowed
+}
+
+// Defines a limit on the amount of filesystem space used by a table/namespace
+message SpaceQuota {
+  optional uint64 soft_limit = 1; // The limit of bytes for this quota
+  optional SpaceViolationPolicy violation_policy = 2; // The action to take when the quota is violated
+}
+
+// The Request to limit space usage (to allow for schema evolution not tied to SpaceQuota).
+message SpaceLimitRequest {
+  optional SpaceQuota quota = 1;
+}


[37/50] [abbrv] hbase git commit: HBASE-16999 Implement master and regionserver synchronization of quota state

Posted by el...@apache.org.
HBASE-16999 Implement master and regionserver synchronization of quota state

* Implement the RegionServer reading violation from the quota table
* Implement the Master reporting violations to the quota table
* RegionServers need to track its enforced policies


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

Branch: refs/heads/HBASE-16961
Commit: dccfc8464e22c0a4c0efde50c7087db49a2ec1b0
Parents: bcf6da4
Author: Josh Elser <el...@apache.org>
Authored: Fri Nov 18 15:38:19 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:32 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |  92 ++++++++-
 .../org/apache/hadoop/hbase/master/HMaster.java |  35 +++-
 .../hadoop/hbase/quotas/QuotaObserverChore.java |   5 +-
 .../hbase/quotas/RegionServerQuotaManager.java  | 200 -------------------
 .../quotas/RegionServerRpcQuotaManager.java     | 200 +++++++++++++++++++
 .../quotas/RegionServerSpaceQuotaManager.java   | 169 ++++++++++++++++
 .../quotas/SpaceQuotaViolationNotifier.java     |  16 +-
 .../SpaceQuotaViolationNotifierFactory.java     |  62 ++++++
 .../SpaceQuotaViolationNotifierForTest.java     |   4 +
 ...SpaceQuotaViolationPolicyRefresherChore.java | 154 ++++++++++++++
 .../TableSpaceQuotaViolationNotifier.java       |  55 +++++
 .../hbase/regionserver/HRegionServer.java       |  21 +-
 .../hbase/regionserver/RSRpcServices.java       |   7 +-
 .../regionserver/RegionServerServices.java      |  12 +-
 .../hadoop/hbase/MockRegionServerServices.java  |  10 +-
 .../hadoop/hbase/master/MockRegionServer.java   |  10 +-
 .../TestQuotaObserverChoreWithMiniCluster.java  |   2 +
 .../hadoop/hbase/quotas/TestQuotaTableUtil.java |  47 +++++
 .../hadoop/hbase/quotas/TestQuotaThrottle.java  |   4 +-
 .../TestRegionServerSpaceQuotaManager.java      | 127 ++++++++++++
 ...SpaceQuotaViolationPolicyRefresherChore.java | 131 ++++++++++++
 .../TestTableSpaceQuotaViolationNotifier.java   | 144 +++++++++++++
 22 files changed, 1281 insertions(+), 226 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/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 8ef4f08..b5eac48 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
@@ -24,16 +24,20 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
@@ -44,7 +48,12 @@ import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.filter.RegexStringComparator;
 import org.apache.hadoop.hbase.filter.RowFilter;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Strings;
 
@@ -53,9 +62,8 @@ import org.apache.hadoop.hbase.util.Strings;
  * <pre>
  *     ROW-KEY      FAM/QUAL        DATA
  *   n.&lt;namespace&gt; q:s         &lt;global-quotas&gt;
- *   n.&lt;namespace&gt; u:du        &lt;size in bytes&gt;
  *   t.&lt;table&gt;     q:s         &lt;global-quotas&gt;
- *   t.&lt;table&gt;     u:du        &lt;size in bytes&gt;
+ *   t.&lt;table&gt;     u:v        &lt;space violation 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;
@@ -74,7 +82,7 @@ public class QuotaTableUtil {
   protected static final byte[] QUOTA_FAMILY_USAGE = Bytes.toBytes("u");
   protected static final byte[] QUOTA_QUALIFIER_SETTINGS = Bytes.toBytes("s");
   protected static final byte[] QUOTA_QUALIFIER_SETTINGS_PREFIX = Bytes.toBytes("s.");
-  protected static final byte[] QUOTA_QUALIFIER_DISKUSAGE = Bytes.toBytes("du");
+  protected static final byte[] QUOTA_QUALIFIER_VIOLATION = Bytes.toBytes("v");
   protected static final byte[] QUOTA_USER_ROW_KEY_PREFIX = Bytes.toBytes("u.");
   protected static final byte[] QUOTA_TABLE_ROW_KEY_PREFIX = Bytes.toBytes("t.");
   protected static final byte[] QUOTA_NAMESPACE_ROW_KEY_PREFIX = Bytes.toBytes("n.");
@@ -203,6 +211,51 @@ public class QuotaTableUtil {
     return filterList;
   }
 
+  /**
+   * Creates a {@link Scan} which returns only quota violations from the quota table.
+   */
+  public static Scan makeQuotaViolationScan() {
+    Scan s = new Scan();
+    // Limit to "u:v" column
+    s.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
+    // Limit rowspace to the "t:" prefix
+    s.setRowPrefixFilter(QUOTA_TABLE_ROW_KEY_PREFIX);
+    return s;
+  }
+
+  /**
+   * Extracts the {@link SpaceViolationPolicy} and {@link TableName} from the provided
+   * {@link Result} and adds them to the given {@link Map}. If the result does not contain
+   * the expected information or the serialized policy in the value is invalid, this method
+   * will throw an {@link IllegalArgumentException}.
+   *
+   * @param result A row from the quota table.
+   * @param policies A map of policies to add the result of this method into.
+   */
+  public static void extractViolationPolicy(
+      Result result, Map<TableName,SpaceViolationPolicy> policies) {
+    byte[] row = Objects.requireNonNull(result).getRow();
+    if (null == row) {
+      throw new IllegalArgumentException("Provided result had a null row");
+    }
+    final TableName targetTableName = getTableFromRowKey(row);
+    Cell c = result.getColumnLatestCell(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
+    if (null == c) {
+      throw new IllegalArgumentException("Result did not contain the expected column "
+          + Bytes.toString(QUOTA_FAMILY_USAGE) + ":" + Bytes.toString(QUOTA_QUALIFIER_VIOLATION)
+          + ", " + result.toString());
+    }
+    ByteString buffer = UnsafeByteOperations.unsafeWrap(
+        c.getValueArray(), c.getValueOffset(), c.getValueLength());
+    try {
+      SpaceQuota quota = SpaceQuota.parseFrom(buffer);
+      policies.put(targetTableName, getViolationPolicy(quota));
+    } catch (InvalidProtocolBufferException e) {
+      throw new IllegalArgumentException(
+          "Result did not contain a valid SpaceQuota protocol buffer message", e);
+    }
+  }
+
   public static interface UserQuotasVisitor {
     void visitUserQuotas(final String userName, final Quotas quotas)
       throws IOException;
@@ -329,6 +382,26 @@ public class QuotaTableUtil {
     }
   }
 
+  /**
+   * Creates a {@link Put} to enable the given <code>policy</code> on the <code>table</code>.
+   */
+  public static Put createEnableViolationPolicyUpdate(
+      TableName tableName, SpaceViolationPolicy policy) {
+    Put p = new Put(getTableRowKey(tableName));
+    SpaceQuota quota = getProtoViolationPolicy(policy);
+    p.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION, quota.toByteArray());
+    return p;
+  }
+
+  /**
+   * Creates a {@link Delete} to remove a policy on the given <code>table</code>.
+   */
+  public static Delete createRemoveViolationPolicyUpdate(TableName tableName) {
+    Delete d = new Delete(getTableRowKey(tableName));
+    d.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
+    return d;
+  }
+
   /* =========================================================================
    *  Quotas protobuf helpers
    */
@@ -450,4 +523,17 @@ public class QuotaTableUtil {
   protected static String getUserFromRowKey(final byte[] key) {
     return Bytes.toString(key, QUOTA_USER_ROW_KEY_PREFIX.length);
   }
+
+  protected static SpaceQuota getProtoViolationPolicy(SpaceViolationPolicy policy) {
+    return SpaceQuota.newBuilder()
+          .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(policy))
+          .build();
+  }
+
+  protected static SpaceViolationPolicy getViolationPolicy(SpaceQuota proto) {
+    if (!proto.hasViolationPolicy()) {
+      throw new IllegalArgumentException("Protobuf SpaceQuota does not have violation policy.");
+    }
+    return ProtobufUtil.toViolationPolicy(proto.getViolationPolicy());
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/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 5b2d89c..6f9dc8b 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
@@ -133,8 +133,9 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
+import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifier;
-import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifierForTest;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifierFactory;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -150,11 +151,14 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
 import org.apache.hadoop.hbase.replication.master.TableCFsUpdater;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -900,7 +904,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     status.setStatus("Starting quota manager");
     initQuotaManager();
-    this.spaceQuotaViolationNotifier = new SpaceQuotaViolationNotifierForTest();
+    this.spaceQuotaViolationNotifier = createQuotaViolationNotifier();
     this.quotaObserverChore = new QuotaObserverChore(this);
     // Start the chore to read the region FS space reports and act on them
     getChoreService().scheduleChore(quotaObserverChore);
@@ -991,6 +995,13 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.quotaManager = quotaManager;
   }
 
+  SpaceQuotaViolationNotifier createQuotaViolationNotifier() {
+    SpaceQuotaViolationNotifier notifier =
+        SpaceQuotaViolationNotifierFactory.getInstance().create(getConfiguration());
+    notifier.initialize(getClusterConnection());
+    return notifier;
+  }
+
   boolean isCatalogJanitorEnabled() {
     return catalogJanitorChore != null ?
       catalogJanitorChore.getEnabled() : false;
@@ -2183,6 +2194,26 @@ public class HMaster extends HRegionServer implements MasterServices {
       protected void run() throws IOException {
         getMaster().getMasterCoprocessorHost().preEnableTable(tableName);
 
+        // Normally, it would make sense for this authorization check to exist inside
+        // AccessController, but because the authorization check is done based on internal state
+        // (rather than explicit permissions) we'll do the check here instead of in the
+        // coprocessor.
+        MasterQuotaManager quotaManager = getMasterQuotaManager();
+        if (null != quotaManager) {
+          if (quotaManager.isQuotaEnabled()) {
+            Quotas quotaForTable = QuotaUtil.getTableQuota(getConnection(), tableName);
+            if (null != quotaForTable && quotaForTable.hasSpace()) {
+              SpaceViolationPolicy policy = quotaForTable.getSpace().getViolationPolicy();
+              if (SpaceViolationPolicy.DISABLE == policy) {
+                throw new AccessDeniedException("Enabling the table '" + tableName
+                    + "' is disallowed due to a violated space quota.");
+              }
+            }
+          } else if (LOG.isTraceEnabled()) {
+            LOG.trace("Unable to check for space quotas as the MasterQuotaManager is not enabled");
+          }
+        }
+
         LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
 
         // Execute the operation asynchronously - client will check the progress of the operation

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/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 88a6149..8b127d9 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
@@ -352,14 +352,15 @@ public class QuotaObserverChore extends ScheduledChore {
   /**
    * Transitions the given table to violation of its quota, enabling the violation policy.
    */
-  private void transitionTableToViolation(TableName table, SpaceViolationPolicy violationPolicy) {
+  private void transitionTableToViolation(TableName table, SpaceViolationPolicy violationPolicy)
+      throws IOException {
     this.violationNotifier.transitionTableToViolation(table, violationPolicy);
   }
 
   /**
    * Transitions the given table to observance of its quota, disabling the violation policy.
    */
-  private void transitionTableToObservance(TableName table) {
+  private void transitionTableToObservance(TableName table) throws IOException {
     this.violationNotifier.transitionTableToObservance(table);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerQuotaManager.java
deleted file mode 100644
index 4961e06..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerQuotaManager.java
+++ /dev/null
@@ -1,200 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.quotas;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ipc.RpcScheduler;
-import org.apache.hadoop.hbase.ipc.RpcServer;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.security.UserGroupInformation;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * Region Server Quota Manager.
- * It is responsible to provide access to the quota information of each user/table.
- *
- * The direct user of this class is the RegionServer that will get and check the
- * user/table quota for each operation (put, get, scan).
- * For system tables and user/table with a quota specified, the quota check will be a noop.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class RegionServerQuotaManager {
-  private static final Log LOG = LogFactory.getLog(RegionServerQuotaManager.class);
-
-  private final RegionServerServices rsServices;
-
-  private QuotaCache quotaCache = null;
-
-  public RegionServerQuotaManager(final RegionServerServices rsServices) {
-    this.rsServices = rsServices;
-  }
-
-  public void start(final RpcScheduler rpcScheduler) throws IOException {
-    if (!QuotaUtil.isQuotaEnabled(rsServices.getConfiguration())) {
-      LOG.info("Quota support disabled");
-      return;
-    }
-
-    LOG.info("Initializing quota support");
-
-    // Initialize quota cache
-    quotaCache = new QuotaCache(rsServices);
-    quotaCache.start();
-  }
-
-  public void stop() {
-    if (isQuotaEnabled()) {
-      quotaCache.stop("shutdown");
-    }
-  }
-
-  public boolean isQuotaEnabled() {
-    return quotaCache != null;
-  }
-
-  @VisibleForTesting
-  QuotaCache getQuotaCache() {
-    return quotaCache;
-  }
-
-  /**
-   * Returns the quota for an operation.
-   *
-   * @param ugi the user that is executing the operation
-   * @param table the table where the operation will be executed
-   * @return the OperationQuota
-   */
-  public OperationQuota getQuota(final UserGroupInformation ugi, final TableName table) {
-    if (isQuotaEnabled() && !table.isSystemTable()) {
-      UserQuotaState userQuotaState = quotaCache.getUserQuotaState(ugi);
-      QuotaLimiter userLimiter = userQuotaState.getTableLimiter(table);
-      boolean useNoop = userLimiter.isBypass();
-      if (userQuotaState.hasBypassGlobals()) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("get quota for ugi=" + ugi + " table=" + table + " userLimiter=" + userLimiter);
-        }
-        if (!useNoop) {
-          return new DefaultOperationQuota(userLimiter);
-        }
-      } else {
-        QuotaLimiter nsLimiter = quotaCache.getNamespaceLimiter(table.getNamespaceAsString());
-        QuotaLimiter tableLimiter = quotaCache.getTableLimiter(table);
-        useNoop &= tableLimiter.isBypass() && nsLimiter.isBypass();
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("get quota for ugi=" + ugi + " table=" + table + " userLimiter=" +
-                    userLimiter + " tableLimiter=" + tableLimiter + " nsLimiter=" + nsLimiter);
-        }
-        if (!useNoop) {
-          return new DefaultOperationQuota(userLimiter, tableLimiter, nsLimiter);
-        }
-      }
-    }
-    return NoopOperationQuota.get();
-  }
-
-  /**
-   * Check the quota for the current (rpc-context) user.
-   * Returns the OperationQuota used to get the available quota and
-   * to report the data/usage of the operation.
-   * @param region the region where the operation will be performed
-   * @param type the operation type
-   * @return the OperationQuota
-   * @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
-   */
-  public OperationQuota checkQuota(final Region region,
-      final OperationQuota.OperationType type) throws IOException, ThrottlingException {
-    switch (type) {
-      case SCAN:   return checkQuota(region, 0, 0, 1);
-      case GET:    return checkQuota(region, 0, 1, 0);
-      case MUTATE: return checkQuota(region, 1, 0, 0);
-    }
-    throw new RuntimeException("Invalid operation type: " + type);
-  }
-
-  /**
-   * Check the quota for the current (rpc-context) user.
-   * Returns the OperationQuota used to get the available quota and
-   * to report the data/usage of the operation.
-   * @param region the region where the operation will be performed
-   * @param actions the "multi" actions to perform
-   * @return the OperationQuota
-   * @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
-   */
-  public OperationQuota checkQuota(final Region region,
-      final List<ClientProtos.Action> actions) throws IOException, ThrottlingException {
-    int numWrites = 0;
-    int numReads = 0;
-    for (final ClientProtos.Action action: actions) {
-      if (action.hasMutation()) {
-        numWrites++;
-      } else if (action.hasGet()) {
-        numReads++;
-      }
-    }
-    return checkQuota(region, numWrites, numReads, 0);
-  }
-
-  /**
-   * Check the quota for the current (rpc-context) user.
-   * Returns the OperationQuota used to get the available quota and
-   * to report the data/usage of the operation.
-   * @param region the region where the operation will be performed
-   * @param numWrites number of writes to perform
-   * @param numReads number of short-reads to perform
-   * @param numScans number of scan to perform
-   * @return the OperationQuota
-   * @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
-   */
-  private OperationQuota checkQuota(final Region region,
-      final int numWrites, final int numReads, final int numScans)
-      throws IOException, ThrottlingException {
-    User user = RpcServer.getRequestUser();
-    UserGroupInformation ugi;
-    if (user != null) {
-      ugi = user.getUGI();
-    } else {
-      ugi = User.getCurrent().getUGI();
-    }
-    TableName table = region.getTableDesc().getTableName();
-
-    OperationQuota quota = getQuota(ugi, table);
-    try {
-      quota.checkQuota(numWrites, numReads, numScans);
-    } catch (ThrottlingException e) {
-      LOG.debug("Throttling exception for user=" + ugi.getUserName() +
-                " table=" + table + " numWrites=" + numWrites +
-                " numReads=" + numReads + " numScans=" + numScans +
-                ": " + e.getMessage());
-      throw e;
-    }
-    return quota;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java
new file mode 100644
index 0000000..756251a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java
@@ -0,0 +1,200 @@
+/**
+ * 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.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ipc.RpcScheduler;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Region Server Quota Manager.
+ * It is responsible to provide access to the quota information of each user/table.
+ *
+ * The direct user of this class is the RegionServer that will get and check the
+ * user/table quota for each operation (put, get, scan).
+ * For system tables and user/table with a quota specified, the quota check will be a noop.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RegionServerRpcQuotaManager {
+  private static final Log LOG = LogFactory.getLog(RegionServerRpcQuotaManager.class);
+
+  private final RegionServerServices rsServices;
+
+  private QuotaCache quotaCache = null;
+
+  public RegionServerRpcQuotaManager(final RegionServerServices rsServices) {
+    this.rsServices = rsServices;
+  }
+
+  public void start(final RpcScheduler rpcScheduler) throws IOException {
+    if (!QuotaUtil.isQuotaEnabled(rsServices.getConfiguration())) {
+      LOG.info("Quota support disabled");
+      return;
+    }
+
+    LOG.info("Initializing RPC quota support");
+
+    // Initialize quota cache
+    quotaCache = new QuotaCache(rsServices);
+    quotaCache.start();
+  }
+
+  public void stop() {
+    if (isQuotaEnabled()) {
+      quotaCache.stop("shutdown");
+    }
+  }
+
+  public boolean isQuotaEnabled() {
+    return quotaCache != null;
+  }
+
+  @VisibleForTesting
+  QuotaCache getQuotaCache() {
+    return quotaCache;
+  }
+
+  /**
+   * Returns the quota for an operation.
+   *
+   * @param ugi the user that is executing the operation
+   * @param table the table where the operation will be executed
+   * @return the OperationQuota
+   */
+  public OperationQuota getQuota(final UserGroupInformation ugi, final TableName table) {
+    if (isQuotaEnabled() && !table.isSystemTable()) {
+      UserQuotaState userQuotaState = quotaCache.getUserQuotaState(ugi);
+      QuotaLimiter userLimiter = userQuotaState.getTableLimiter(table);
+      boolean useNoop = userLimiter.isBypass();
+      if (userQuotaState.hasBypassGlobals()) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("get quota for ugi=" + ugi + " table=" + table + " userLimiter=" + userLimiter);
+        }
+        if (!useNoop) {
+          return new DefaultOperationQuota(userLimiter);
+        }
+      } else {
+        QuotaLimiter nsLimiter = quotaCache.getNamespaceLimiter(table.getNamespaceAsString());
+        QuotaLimiter tableLimiter = quotaCache.getTableLimiter(table);
+        useNoop &= tableLimiter.isBypass() && nsLimiter.isBypass();
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("get quota for ugi=" + ugi + " table=" + table + " userLimiter=" +
+                    userLimiter + " tableLimiter=" + tableLimiter + " nsLimiter=" + nsLimiter);
+        }
+        if (!useNoop) {
+          return new DefaultOperationQuota(userLimiter, tableLimiter, nsLimiter);
+        }
+      }
+    }
+    return NoopOperationQuota.get();
+  }
+
+  /**
+   * Check the quota for the current (rpc-context) user.
+   * Returns the OperationQuota used to get the available quota and
+   * to report the data/usage of the operation.
+   * @param region the region where the operation will be performed
+   * @param type the operation type
+   * @return the OperationQuota
+   * @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
+   */
+  public OperationQuota checkQuota(final Region region,
+      final OperationQuota.OperationType type) throws IOException, ThrottlingException {
+    switch (type) {
+      case SCAN:   return checkQuota(region, 0, 0, 1);
+      case GET:    return checkQuota(region, 0, 1, 0);
+      case MUTATE: return checkQuota(region, 1, 0, 0);
+    }
+    throw new RuntimeException("Invalid operation type: " + type);
+  }
+
+  /**
+   * Check the quota for the current (rpc-context) user.
+   * Returns the OperationQuota used to get the available quota and
+   * to report the data/usage of the operation.
+   * @param region the region where the operation will be performed
+   * @param actions the "multi" actions to perform
+   * @return the OperationQuota
+   * @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
+   */
+  public OperationQuota checkQuota(final Region region,
+      final List<ClientProtos.Action> actions) throws IOException, ThrottlingException {
+    int numWrites = 0;
+    int numReads = 0;
+    for (final ClientProtos.Action action: actions) {
+      if (action.hasMutation()) {
+        numWrites++;
+      } else if (action.hasGet()) {
+        numReads++;
+      }
+    }
+    return checkQuota(region, numWrites, numReads, 0);
+  }
+
+  /**
+   * Check the quota for the current (rpc-context) user.
+   * Returns the OperationQuota used to get the available quota and
+   * to report the data/usage of the operation.
+   * @param region the region where the operation will be performed
+   * @param numWrites number of writes to perform
+   * @param numReads number of short-reads to perform
+   * @param numScans number of scan to perform
+   * @return the OperationQuota
+   * @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
+   */
+  private OperationQuota checkQuota(final Region region,
+      final int numWrites, final int numReads, final int numScans)
+      throws IOException, ThrottlingException {
+    User user = RpcServer.getRequestUser();
+    UserGroupInformation ugi;
+    if (user != null) {
+      ugi = user.getUGI();
+    } else {
+      ugi = User.getCurrent().getUGI();
+    }
+    TableName table = region.getTableDesc().getTableName();
+
+    OperationQuota quota = getQuota(ugi, table);
+    try {
+      quota.checkQuota(numWrites, numReads, numScans);
+    } catch (ThrottlingException e) {
+      LOG.debug("Throttling exception for user=" + ugi.getUserName() +
+                " table=" + table + " numWrites=" + numWrites +
+                " numReads=" + numReads + " numScans=" + numScans +
+                ": " + e.getMessage());
+      throw e;
+    }
+    return quota;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
new file mode 100644
index 0000000..9a8edb9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
@@ -0,0 +1,169 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * A manager for filesystem space quotas in the RegionServer.
+ *
+ * This class is responsible for reading quota violation policies from the quota
+ * table and then enacting them on the given table.
+ */
+@InterfaceAudience.Private
+public class RegionServerSpaceQuotaManager {
+  private static final Log LOG = LogFactory.getLog(RegionServerSpaceQuotaManager.class);
+
+  private final RegionServerServices rsServices;
+
+  private SpaceQuotaViolationPolicyRefresherChore spaceQuotaRefresher;
+  private Map<TableName,SpaceViolationPolicy> enforcedPolicies;
+  private boolean started = false;
+
+  public RegionServerSpaceQuotaManager(RegionServerServices rsServices) {
+    this.rsServices = Objects.requireNonNull(rsServices);
+  }
+
+  public synchronized void start() throws IOException {
+    if (!QuotaUtil.isQuotaEnabled(rsServices.getConfiguration())) {
+      LOG.info("Quota support disabled, not starting space quota manager.");
+      return;
+    }
+
+    spaceQuotaRefresher = new SpaceQuotaViolationPolicyRefresherChore(this);
+    enforcedPolicies = new HashMap<>();
+    started = true;
+  }
+
+  public synchronized void stop() {
+    if (null != spaceQuotaRefresher) {
+      spaceQuotaRefresher.cancel();
+      spaceQuotaRefresher = null;
+    }
+    started = false;
+  }
+
+  /**
+   * @return if the {@code Chore} has been started.
+   */
+  public boolean isStarted() {
+    return started;
+  }
+
+  Connection getConnection() {
+    return rsServices.getConnection();
+  }
+
+  /**
+   * Returns the collection of tables which have quota violation policies enforced on
+   * this RegionServer.
+   */
+  public synchronized Map<TableName,SpaceViolationPolicy> getActiveViolationPolicyEnforcements()
+      throws IOException {
+    return new HashMap<>(this.enforcedPolicies);
+  }
+
+  /**
+   * Wrapper around {@link QuotaTableUtil#extractViolationPolicy(Result, Map)} for testing.
+   */
+  void extractViolationPolicy(Result result, Map<TableName,SpaceViolationPolicy> activePolicies) {
+    QuotaTableUtil.extractViolationPolicy(result, activePolicies);
+  }
+
+  /**
+   * Reads all quota violation policies which are to be enforced from the quota table.
+   *
+   * @return The collection of tables which are in violation of their quota and the policy which
+   *    should be enforced.
+   */
+  public Map<TableName, SpaceViolationPolicy> getViolationPoliciesToEnforce() throws IOException {
+    try (Table quotaTable = getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME);
+        ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaViolationScan())) {
+      Map<TableName,SpaceViolationPolicy> activePolicies = new HashMap<>();
+      for (Result result : scanner) {
+        try {
+          extractViolationPolicy(result, activePolicies);
+        } catch (IllegalArgumentException e) {
+          final String msg = "Failed to parse result for row " + Bytes.toString(result.getRow());
+          LOG.error(msg, e);
+          throw new IOException(msg, e);
+        }
+      }
+      return activePolicies;
+    }
+  }
+
+  /**
+   * Enforces the given violationPolicy on the given table in this RegionServer.
+   */
+  synchronized void enforceViolationPolicy(
+      TableName tableName, SpaceViolationPolicy violationPolicy) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(
+          "Enabling violation policy enforcement on " + tableName
+          + " with policy " + violationPolicy);
+    }
+    // Enact the policy
+    enforceOnRegionServer(tableName, violationPolicy);
+    // Publicize our enacting of the policy
+    enforcedPolicies.put(tableName, violationPolicy);
+  }
+
+  /**
+   * Enacts the given violation policy on this table in the RegionServer.
+   */
+  void enforceOnRegionServer(TableName tableName, SpaceViolationPolicy violationPolicy) {
+    throw new UnsupportedOperationException("TODO");
+  }
+
+  /**
+   * Disables enforcement on any violation policy on the given <code>tableName</code>.
+   */
+  synchronized void disableViolationPolicyEnforcement(TableName tableName) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Disabling violation policy enforcement on " + tableName);
+    }
+    disableOnRegionServer(tableName);
+    enforcedPolicies.remove(tableName);
+  }
+
+  /**
+   * Disables any violation policy on this table in the RegionServer.
+   */
+  void disableOnRegionServer(TableName tableName) {
+    throw new UnsupportedOperationException("TODO");
+  }
+
+  RegionServerServices getRegionServerServices() {
+    return rsServices;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
index bccf519..261dea7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
@@ -16,29 +16,39 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+import java.io.IOException;
+
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
 
 /**
  * An interface which abstract away the action taken to enable or disable
- * a space quota violation policy across the HBase cluster.
+ * a space quota violation policy across the HBase cluster. Implementations
+ * must have a no-args constructor.
  */
 @InterfaceAudience.Private
 public interface SpaceQuotaViolationNotifier {
 
   /**
+   * Initializes the notifier.
+   */
+  void initialize(Connection conn);
+
+  /**
    * Instructs the cluster that the given table is in violation of a space quota. The
    * provided violation policy is the action which should be taken on the table.
    *
    * @param tableName The name of the table in violation of the quota.
    * @param violationPolicy The policy which should be enacted on the table.
    */
-  void transitionTableToViolation(TableName tableName, SpaceViolationPolicy violationPolicy);
+  void transitionTableToViolation(
+      TableName tableName, SpaceViolationPolicy violationPolicy) throws IOException;
 
   /**
    * Instructs the cluster that the given table is in observance of any applicable space quota.
    *
    * @param tableName The name of the table in observance.
    */
-  void transitionTableToObservance(TableName tableName);
+  void transitionTableToObservance(TableName tableName) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java
new file mode 100644
index 0000000..43f5513
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.util.Objects;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Factory for creating {@link SpaceQuotaViolationNotifier} implementations. Implementations
+ * must have a no-args constructor.
+ */
+@InterfaceAudience.Private
+public class SpaceQuotaViolationNotifierFactory {
+  private static final SpaceQuotaViolationNotifierFactory INSTANCE =
+      new SpaceQuotaViolationNotifierFactory();
+
+  public static final String VIOLATION_NOTIFIER_KEY = "hbase.master.quota.violation.notifier.impl";
+  public static final Class<? extends SpaceQuotaViolationNotifier> VIOLATION_NOTIFIER_DEFAULT =
+      SpaceQuotaViolationNotifierForTest.class;
+
+  // Private
+  private SpaceQuotaViolationNotifierFactory() {}
+
+  public static SpaceQuotaViolationNotifierFactory getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Instantiates the {@link SpaceQuotaViolationNotifier} implementation as defined in the
+   * configuration provided.
+   *
+   * @param conf Configuration object
+   * @return The SpaceQuotaViolationNotifier implementation
+   * @throws IllegalArgumentException if the class could not be instantiated
+   */
+  public SpaceQuotaViolationNotifier create(Configuration conf) {
+    Class<? extends SpaceQuotaViolationNotifier> clz = Objects.requireNonNull(conf)
+        .getClass(VIOLATION_NOTIFIER_KEY, VIOLATION_NOTIFIER_DEFAULT,
+            SpaceQuotaViolationNotifier.class);
+    try {
+      return clz.newInstance();
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new IllegalArgumentException("Failed to instantiate the implementation", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
index 4ab9834..65dc979 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
@@ -21,6 +21,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
 
 /**
  * A SpaceQuotaViolationNotifier implementation for verifying testing.
@@ -31,6 +32,9 @@ public class SpaceQuotaViolationNotifierForTest implements SpaceQuotaViolationNo
   private final Map<TableName,SpaceViolationPolicy> tablesInViolation = new HashMap<>();
 
   @Override
+  public void initialize(Connection conn) {}
+
+  @Override
   public void transitionTableToViolation(TableName tableName, SpaceViolationPolicy violationPolicy) {
     tablesInViolation.put(tableName, violationPolicy);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java
new file mode 100644
index 0000000..778ea0b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java
@@ -0,0 +1,154 @@
+/*
+ * 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.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A {@link ScheduledChore} which periodically updates a local copy of tables which have
+ * space quota violation policies enacted on them.
+ */
+@InterfaceAudience.Private
+public class SpaceQuotaViolationPolicyRefresherChore extends ScheduledChore {
+  private static final Log LOG = LogFactory.getLog(SpaceQuotaViolationPolicyRefresherChore.class);
+
+  static final String POLICY_REFRESHER_CHORE_PERIOD_KEY =
+      "hbase.regionserver.quotas.policy.refresher.chore.period";
+  static final int POLICY_REFRESHER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
+
+  static final String POLICY_REFRESHER_CHORE_DELAY_KEY =
+      "hbase.regionserver.quotas.policy.refresher.chore.delay";
+  static final long POLICY_REFRESHER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
+
+  static final String POLICY_REFRESHER_CHORE_TIMEUNIT_KEY =
+      "hbase.regionserver.quotas.policy.refresher.chore.timeunit";
+  static final String POLICY_REFRESHER_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
+
+  static final String POLICY_REFRESHER_CHORE_REPORT_PERCENT_KEY =
+      "hbase.regionserver.quotas.policy.refresher.report.percent";
+  static final double POLICY_REFRESHER_CHORE_REPORT_PERCENT_DEFAULT= 0.95;
+
+  private final RegionServerSpaceQuotaManager manager;
+
+  public SpaceQuotaViolationPolicyRefresherChore(RegionServerSpaceQuotaManager manager) {
+    super(SpaceQuotaViolationPolicyRefresherChore.class.getSimpleName(),
+        manager.getRegionServerServices(),
+        getPeriod(manager.getRegionServerServices().getConfiguration()),
+        getInitialDelay(manager.getRegionServerServices().getConfiguration()),
+        getTimeUnit(manager.getRegionServerServices().getConfiguration()));
+    this.manager = manager;
+  }
+
+  @Override
+  protected void chore() {
+    // Tables with a policy currently enforced
+    final Map<TableName, SpaceViolationPolicy> activeViolationPolicies;
+    // Tables with policies that should be enforced
+    final Map<TableName, SpaceViolationPolicy> violationPolicies;
+    try {
+      // Tables with a policy currently enforced
+      activeViolationPolicies = manager.getActiveViolationPolicyEnforcements();
+      // Tables with policies that should be enforced
+      violationPolicies = manager.getViolationPoliciesToEnforce();
+    } catch (IOException e) {
+      LOG.warn("Failed to fetch enforced quota violation policies, will retry.", e);
+      return;
+    }
+    // Ensure each policy which should be enacted is enacted.
+    for (Entry<TableName, SpaceViolationPolicy> entry : violationPolicies.entrySet()) {
+      final TableName tableName = entry.getKey();
+      final SpaceViolationPolicy policyToEnforce = entry.getValue();
+      final SpaceViolationPolicy currentPolicy = activeViolationPolicies.get(tableName);
+      if (currentPolicy != policyToEnforce) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Enabling " + policyToEnforce + " on " + tableName);
+        }
+        manager.enforceViolationPolicy(tableName, policyToEnforce);
+      }
+    }
+    // Remove policies which should no longer be enforced
+    Iterator<TableName> iter = activeViolationPolicies.keySet().iterator();
+    while (iter.hasNext()) {
+      final TableName localTableWithPolicy = iter.next();
+      if (!violationPolicies.containsKey(localTableWithPolicy)) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Removing quota violation policy on " + localTableWithPolicy);
+        }
+        manager.disableViolationPolicyEnforcement(localTableWithPolicy);
+        iter.remove();
+      }
+    }
+  }
+
+  /**
+   * 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(POLICY_REFRESHER_CHORE_PERIOD_KEY,
+        POLICY_REFRESHER_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(POLICY_REFRESHER_CHORE_DELAY_KEY,
+        POLICY_REFRESHER_CHORE_DELAY_DEFAULT);
+  }
+
+  /**
+   * Extracts the time unit for the chore period and initial delay from the configuration. The
+   * configuration value for {@link #POLICY_REFRESHER_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(POLICY_REFRESHER_CHORE_TIMEUNIT_KEY,
+        POLICY_REFRESHER_CHORE_TIMEUNIT_DEFAULT));
+  }
+
+  /**
+   * Extracts the percent of Regions for a table to have been reported to enable quota violation
+   * state change.
+   *
+   * @param conf The configuration object.
+   * @return The percent of regions reported to use.
+   */
+  static Double getRegionReportPercent(Configuration conf) {
+    return conf.getDouble(POLICY_REFRESHER_CHORE_REPORT_PERCENT_KEY,
+        POLICY_REFRESHER_CHORE_REPORT_PERCENT_DEFAULT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java
new file mode 100644
index 0000000..a8b1c55
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+
+/**
+ * A {@link SpaceQuotaViolationNotifier} which uses the hbase:quota table.
+ */
+public class TableSpaceQuotaViolationNotifier implements SpaceQuotaViolationNotifier {
+
+  private Connection conn;
+
+  @Override
+  public void transitionTableToViolation(
+      TableName tableName, SpaceViolationPolicy violationPolicy) throws IOException {
+    final Put p = QuotaTableUtil.createEnableViolationPolicyUpdate(tableName, violationPolicy);
+    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      quotaTable.put(p);
+    }
+  }
+
+  @Override
+  public void transitionTableToObservance(TableName tableName) throws IOException {
+    final Delete d = QuotaTableUtil.createRemoveViolationPolicyUpdate(tableName);
+    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      quotaTable.delete(d);
+    }
+  }
+
+  @Override
+  public void initialize(Connection conn) {
+    this.conn = conn;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 053e4ac..79e2c46 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -119,7 +119,8 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.mob.MobCacheConfig;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
 import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore;
-import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
@@ -477,7 +478,8 @@ public class HRegionServer extends HasThread implements
 
   private RegionServerProcedureManagerHost rspmHost;
 
-  private RegionServerQuotaManager rsQuotaManager;
+  private RegionServerRpcQuotaManager rsQuotaManager;
+  private RegionServerSpaceQuotaManager rsSpaceQuotaManager;
 
   /**
    * Nonce manager. Nonces are used to make operations like increment and append idempotent
@@ -928,7 +930,8 @@ public class HRegionServer extends HasThread implements
     }
 
     // Setup the Quota Manager
-    rsQuotaManager = new RegionServerQuotaManager(this);
+    rsQuotaManager = new RegionServerRpcQuotaManager(this);
+    rsSpaceQuotaManager = new RegionServerSpaceQuotaManager(this);
 
     this.fsUtilizationChore = new FileSystemUtilizationChore(this);
 
@@ -1000,6 +1003,7 @@ public class HRegionServer extends HasThread implements
 
         // Start the Quota Manager
         rsQuotaManager.start(getRpcServer().getScheduler());
+        rsSpaceQuotaManager.start();
       }
 
       // We registered with the Master.  Go into run mode.
@@ -1091,6 +1095,10 @@ public class HRegionServer extends HasThread implements
     if (rsQuotaManager != null) {
       rsQuotaManager.stop();
     }
+    if (rsSpaceQuotaManager != null) {
+      rsSpaceQuotaManager.stop();
+      rsSpaceQuotaManager = null;
+    }
 
     // Stop the snapshot and other procedure handlers, forcefully killing all running tasks
     if (rspmHost != null) {
@@ -2882,7 +2890,7 @@ public class HRegionServer extends HasThread implements
   }
 
   @Override
-  public RegionServerQuotaManager getRegionServerQuotaManager() {
+  public RegionServerRpcQuotaManager getRegionServerRpcQuotaManager() {
     return rsQuotaManager;
   }
 
@@ -3745,4 +3753,9 @@ public class HRegionServer extends HasThread implements
   public void unassign(byte[] regionName) throws IOException {
     clusterConnection.getAdmin().unassign(regionName, false);
   }
+
+  @Override
+  public RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager() {
+    return this.rsSpaceQuotaManager;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 8d4ea4d..fd9ca92 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -97,7 +97,7 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
 import org.apache.hadoop.hbase.quotas.OperationQuota;
-import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.regionserver.Leases.Lease;
 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
@@ -195,6 +195,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescr
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
+import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.DNS;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -1266,8 +1267,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return regionServer.getConfiguration();
   }
 
-  private RegionServerQuotaManager getQuotaManager() {
-    return regionServer.getRegionServerQuotaManager();
+  private RegionServerRpcQuotaManager getQuotaManager() {
+    return regionServer.getRegionServerRpcQuotaManager();
   }
 
   void start() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
index 3382263..54aeaa6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
@@ -35,7 +35,8 @@ import org.apache.hadoop.hbase.client.locking.EntityLock;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.zookeeper.KeeperException;
@@ -78,9 +79,9 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
   RegionServerAccounting getRegionServerAccounting();
 
   /**
-   * @return RegionServer's instance of {@link RegionServerQuotaManager}
+   * @return RegionServer's instance of {@link RegionServerRpcQuotaManager}
    */
-  RegionServerQuotaManager getRegionServerQuotaManager();
+  RegionServerRpcQuotaManager getRegionServerRpcQuotaManager();
 
   /**
    * @return RegionServer's instance of {@link SecureBulkLoadManager}
@@ -88,6 +89,11 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
   SecureBulkLoadManager getSecureBulkLoadManager();
 
   /**
+   * @return RegionServer's instance of {@link RegionServerSpaceQuotaManager}
+   */
+  RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager();
+
+  /**
    * Context for postOpenDeployTasks().
    */
   class PostOpenDeployContext {

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index 81b3489..eefde94 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -38,7 +38,8 @@ import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
 import org.apache.hadoop.hbase.regionserver.FlushRequester;
 import org.apache.hadoop.hbase.regionserver.HeapMemoryManager;
@@ -189,7 +190,7 @@ public class MockRegionServerServices implements RegionServerServices {
   }
 
   @Override
-  public RegionServerQuotaManager getRegionServerQuotaManager() {
+  public RegionServerRpcQuotaManager getRegionServerRpcQuotaManager() {
     return null;
   }
 
@@ -360,4 +361,9 @@ public class MockRegionServerServices implements RegionServerServices {
   @Override
   public void unassign(byte[] regionName) throws IOException {
   }
+
+  @Override
+  public RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 78c8214..f133afc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -101,7 +101,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBul
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
 import org.apache.hadoop.hbase.regionserver.FlushRequester;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -331,7 +332,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   }
 
   @Override
-  public RegionServerQuotaManager getRegionServerQuotaManager() {
+  public RegionServerRpcQuotaManager getRegionServerRpcQuotaManager() {
     return null;
   }
 
@@ -720,4 +721,9 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   @Override
   public void unassign(byte[] regionName) throws IOException {
   }
+
+  @Override
+  public RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/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 98236c2..c493b25 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
@@ -94,6 +94,8 @@ public class TestQuotaObserverChoreWithMiniCluster {
     conf.setInt(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_DELAY_KEY, 1000);
     conf.setInt(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_PERIOD_KEY, 1000);
     conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    conf.setClass(SpaceQuotaViolationNotifierFactory.VIOLATION_NOTIFIER_KEY,
+        SpaceQuotaViolationNotifierForTest.class, SpaceQuotaViolationNotifier.class);
     TEST_UTIL.startMiniCluster(1);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
index 238c4c0..55f671a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
@@ -21,6 +21,10 @@ package org.apache.hadoop.hbase.quotas;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -28,6 +32,10 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
@@ -50,6 +58,10 @@ public class TestQuotaTableUtil {
 
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private Connection connection;
+  private int tableNameCounter;
+
+  @Rule
+  public TestName testName = new TestName();
 
   @Rule
   public TestName name = new TestName();
@@ -75,6 +87,7 @@ public class TestQuotaTableUtil {
   @Before
   public void before() throws IOException {
     this.connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+    this.tableNameCounter = 0;
   }
 
   @After
@@ -184,4 +197,38 @@ public class TestQuotaTableUtil {
     resQuotaNS = QuotaUtil.getUserQuota(this.connection, user, namespace);
     assertEquals(null, resQuotaNS);
   }
+
+  @Test
+  public void testSerDeViolationPolicies() throws Exception {
+    final TableName tn1 = getUniqueTableName();
+    final SpaceViolationPolicy policy1 = SpaceViolationPolicy.DISABLE;
+    final TableName tn2 = getUniqueTableName();
+    final SpaceViolationPolicy policy2 = SpaceViolationPolicy.NO_INSERTS;
+    final TableName tn3 = getUniqueTableName();
+    final SpaceViolationPolicy policy3 = SpaceViolationPolicy.NO_WRITES;
+    List<Put> puts = new ArrayList<>();
+    puts.add(QuotaTableUtil.createEnableViolationPolicyUpdate(tn1, policy1));
+    puts.add(QuotaTableUtil.createEnableViolationPolicyUpdate(tn2, policy2));
+    puts.add(QuotaTableUtil.createEnableViolationPolicyUpdate(tn3, policy3));
+    final Map<TableName,SpaceViolationPolicy> expectedPolicies = new HashMap<>();
+    expectedPolicies.put(tn1, policy1);
+    expectedPolicies.put(tn2, policy2);
+    expectedPolicies.put(tn3, policy3);
+
+    final Map<TableName,SpaceViolationPolicy> actualPolicies = new HashMap<>();
+    try (Table quotaTable = connection.getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
+      quotaTable.put(puts);
+      ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaViolationScan());
+      for (Result r : scanner) {
+        QuotaTableUtil.extractViolationPolicy(r, actualPolicies);
+      }
+      scanner.close();
+    }
+
+    assertEquals(expectedPolicies, actualPolicies);
+  }
+
+  private TableName getUniqueTableName() {
+    return TableName.valueOf(testName.getMethodName() + "_" + tableNameCounter++);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
index 91c2d80..7a330fb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
@@ -105,7 +105,7 @@ public class TestQuotaThrottle {
   @After
   public void tearDown() throws Exception {
     for (RegionServerThread rst: TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
-      RegionServerQuotaManager quotaManager = rst.getRegionServer().getRegionServerQuotaManager();
+      RegionServerRpcQuotaManager quotaManager = rst.getRegionServer().getRegionServerRpcQuotaManager();
       QuotaCache quotaCache = quotaManager.getQuotaCache();
       quotaCache.getNamespaceQuotaCache().clear();
       quotaCache.getTableQuotaCache().clear();
@@ -557,7 +557,7 @@ public class TestQuotaThrottle {
       boolean nsLimiter, final TableName... tables) throws Exception {
     envEdge.incValue(2 * REFRESH_TIME);
     for (RegionServerThread rst: TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
-      RegionServerQuotaManager quotaManager = rst.getRegionServer().getRegionServerQuotaManager();
+      RegionServerRpcQuotaManager quotaManager = rst.getRegionServer().getRegionServerRpcQuotaManager();
       QuotaCache quotaCache = quotaManager.getQuotaCache();
 
       quotaCache.triggerCacheRefresh();

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java
new file mode 100644
index 0000000..e5ab317
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.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.apache.hadoop.hbase.util.Bytes.toBytes;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Test class for {@link RegionServerSpaceQuotaManager}.
+ */
+@Category(SmallTests.class)
+public class TestRegionServerSpaceQuotaManager {
+
+  private RegionServerSpaceQuotaManager quotaManager;
+  private Connection conn;
+  private Table quotaTable;
+  private ResultScanner scanner;
+
+  @Before
+  @SuppressWarnings("unchecked")
+  public void setup() throws Exception {
+    quotaManager = mock(RegionServerSpaceQuotaManager.class);
+    conn = mock(Connection.class);
+    quotaTable = mock(Table.class);
+    scanner = mock(ResultScanner.class);
+    // Call the real getViolationPoliciesToEnforce()
+    when(quotaManager.getViolationPoliciesToEnforce()).thenCallRealMethod();
+    // Mock out creating a scanner
+    when(quotaManager.getConnection()).thenReturn(conn);
+    when(conn.getTable(QuotaUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
+    when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
+    // Mock out the static method call with some indirection
+    doAnswer(new Answer<Void>(){
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        Result result = invocation.getArgumentAt(0, Result.class);
+        Map<TableName,SpaceViolationPolicy> policies = invocation.getArgumentAt(1, Map.class);
+        QuotaTableUtil.extractViolationPolicy(result, policies);
+        return null;
+      }
+    }).when(quotaManager).extractViolationPolicy(any(Result.class), any(Map.class));
+  }
+
+  @Test
+  public void testMissingAllColumns() {
+    List<Result> results = new ArrayList<>();
+    results.add(Result.create(Collections.emptyList()));
+    when(scanner.iterator()).thenReturn(results.iterator());
+    try {
+      quotaManager.getViolationPoliciesToEnforce();
+      fail("Expected an IOException, but did not receive one.");
+    } catch (IOException e) {
+      // Expected an error because we had no cells in the row.
+      // This should only happen due to programmer error.
+    }
+  }
+
+  @Test
+  public void testMissingDesiredColumn() {
+    List<Result> results = new ArrayList<>();
+    // Give a column that isn't the one we want
+    Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("q"), toBytes("s"), new byte[0]);
+    results.add(Result.create(Collections.singletonList(c)));
+    when(scanner.iterator()).thenReturn(results.iterator());
+    try {
+      quotaManager.getViolationPoliciesToEnforce();
+      fail("Expected an IOException, but did not receive one.");
+    } catch (IOException e) {
+      // Expected an error because we were missing the column we expected in this row.
+      // This should only happen due to programmer error.
+    }
+  }
+
+  @Test
+  public void testParsingError() {
+    List<Result> results = new ArrayList<>();
+    Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("u"), toBytes("v"), new byte[0]);
+    results.add(Result.create(Collections.singletonList(c)));
+    when(scanner.iterator()).thenReturn(results.iterator());
+    try {
+      quotaManager.getViolationPoliciesToEnforce();
+      fail("Expected an IOException, but did not receive one.");
+    } catch (IOException e) {
+      // We provided a garbage serialized protobuf message (empty byte array), this should
+      // in turn throw an IOException
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java
new file mode 100644
index 0000000..160de46
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+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.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link SpaceQuotaViolationPolicyRefresherChore}.
+ */
+@Category(SmallTests.class)
+public class TestSpaceQuotaViolationPolicyRefresherChore {
+
+  private RegionServerSpaceQuotaManager manager;
+  private RegionServerServices rss;
+  private SpaceQuotaViolationPolicyRefresherChore chore;
+  private Configuration conf;
+
+  @Before
+  public void setup() {
+    conf = HBaseConfiguration.create();
+    rss = mock(RegionServerServices.class);
+    manager = mock(RegionServerSpaceQuotaManager.class);
+    when(manager.getRegionServerServices()).thenReturn(rss);
+    when(rss.getConfiguration()).thenReturn(conf);
+    chore = new SpaceQuotaViolationPolicyRefresherChore(manager);
+  }
+
+  @Test
+  public void testPoliciesAreEnforced() throws IOException {
+    final Map<TableName,SpaceViolationPolicy> policiesToEnforce = new HashMap<>();
+    policiesToEnforce.put(TableName.valueOf("table1"), SpaceViolationPolicy.DISABLE);
+    policiesToEnforce.put(TableName.valueOf("table2"), SpaceViolationPolicy.NO_INSERTS);
+    policiesToEnforce.put(TableName.valueOf("table3"), SpaceViolationPolicy.NO_WRITES);
+    policiesToEnforce.put(TableName.valueOf("table4"), SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
+
+    // No active enforcements
+    when(manager.getActiveViolationPolicyEnforcements()).thenReturn(Collections.emptyMap());
+    // Policies to enforce
+    when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
+
+    chore.chore();
+
+    for (Entry<TableName,SpaceViolationPolicy> entry : policiesToEnforce.entrySet()) {
+      // Ensure we enforce the policy
+      verify(manager).enforceViolationPolicy(entry.getKey(), entry.getValue());
+      // Don't disable any policies
+      verify(manager, never()).disableViolationPolicyEnforcement(entry.getKey());
+    }
+  }
+
+  @Test
+  public void testOldPoliciesAreRemoved() throws IOException {
+    final Map<TableName,SpaceViolationPolicy> policiesToEnforce = new HashMap<>();
+    policiesToEnforce.put(TableName.valueOf("table1"), SpaceViolationPolicy.DISABLE);
+    policiesToEnforce.put(TableName.valueOf("table2"), SpaceViolationPolicy.NO_INSERTS);
+
+    final Map<TableName,SpaceViolationPolicy> previousPolicies = new HashMap<>();
+    previousPolicies.put(TableName.valueOf("table3"), SpaceViolationPolicy.NO_WRITES);
+    previousPolicies.put(TableName.valueOf("table4"), SpaceViolationPolicy.NO_WRITES);
+
+    // No active enforcements
+    when(manager.getActiveViolationPolicyEnforcements()).thenReturn(previousPolicies);
+    // Policies to enforce
+    when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
+
+    chore.chore();
+
+    for (Entry<TableName,SpaceViolationPolicy> entry : policiesToEnforce.entrySet()) {
+      verify(manager).enforceViolationPolicy(entry.getKey(), entry.getValue());
+    }
+
+    for (Entry<TableName,SpaceViolationPolicy> entry : previousPolicies.entrySet()) {
+      verify(manager).disableViolationPolicyEnforcement(entry.getKey());
+    }
+  }
+
+  @Test
+  public void testNewPolicyOverridesOld() throws IOException {
+    final Map<TableName,SpaceViolationPolicy> policiesToEnforce = new HashMap<>();
+    policiesToEnforce.put(TableName.valueOf("table1"), SpaceViolationPolicy.DISABLE);
+    policiesToEnforce.put(TableName.valueOf("table2"), SpaceViolationPolicy.NO_WRITES);
+    policiesToEnforce.put(TableName.valueOf("table3"), SpaceViolationPolicy.NO_INSERTS);
+
+    final Map<TableName,SpaceViolationPolicy> previousPolicies = new HashMap<>();
+    previousPolicies.put(TableName.valueOf("table1"), SpaceViolationPolicy.NO_WRITES);
+
+    // No active enforcements
+    when(manager.getActiveViolationPolicyEnforcements()).thenReturn(previousPolicies);
+    // Policies to enforce
+    when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
+
+    chore.chore();
+
+    for (Entry<TableName,SpaceViolationPolicy> entry : policiesToEnforce.entrySet()) {
+      verify(manager).enforceViolationPolicy(entry.getKey(), entry.getValue());
+    }
+    verify(manager, never()).disableViolationPolicyEnforcement(TableName.valueOf("table1"));
+  }
+}


[27/50] [abbrv] hbase git commit: HBASE-16996 Implement storage/retrieval of filesystem-use quotas into quota table (Josh Elser)

Posted by el...@apache.org.
HBASE-16996 Implement storage/retrieval of filesystem-use quotas into quota table (Josh Elser)


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

Branch: refs/heads/HBASE-16961
Commit: a29abe646e2b1cd75ee1d0f186cc7486fe78b79e
Parents: 988a23e
Author: tedyu <yu...@gmail.com>
Authored: Sat Dec 3 14:30:48 2016 -0800
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:31 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |  13 +-
 .../hadoop/hbase/quotas/MasterQuotaManager.java |  30 +++++
 .../hadoop/hbase/quotas/TestQuotaAdmin.java     | 125 ++++++++++++++++++-
 3 files changed, 165 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a29abe64/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 c44090f..8ef4f08 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
@@ -53,7 +53,9 @@ import org.apache.hadoop.hbase.util.Strings;
  * <pre>
  *     ROW-KEY      FAM/QUAL        DATA
  *   n.&lt;namespace&gt; q:s         &lt;global-quotas&gt;
+ *   n.&lt;namespace&gt; u:du        &lt;size in bytes&gt;
  *   t.&lt;table&gt;     q:s         &lt;global-quotas&gt;
+ *   t.&lt;table&gt;     u:du        &lt;size in bytes&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;
@@ -72,6 +74,7 @@ public class QuotaTableUtil {
   protected static final byte[] QUOTA_FAMILY_USAGE = Bytes.toBytes("u");
   protected static final byte[] QUOTA_QUALIFIER_SETTINGS = Bytes.toBytes("s");
   protected static final byte[] QUOTA_QUALIFIER_SETTINGS_PREFIX = Bytes.toBytes("s.");
+  protected static final byte[] QUOTA_QUALIFIER_DISKUSAGE = Bytes.toBytes("du");
   protected static final byte[] QUOTA_USER_ROW_KEY_PREFIX = Bytes.toBytes("u.");
   protected static final byte[] QUOTA_TABLE_ROW_KEY_PREFIX = Bytes.toBytes("t.");
   protected static final byte[] QUOTA_NAMESPACE_ROW_KEY_PREFIX = Bytes.toBytes("n.");
@@ -330,11 +333,16 @@ public class QuotaTableUtil {
    *  Quotas protobuf helpers
    */
   protected static Quotas quotasFromData(final byte[] data) throws IOException {
+    return quotasFromData(data, 0, data.length);
+  }
+
+  protected static Quotas quotasFromData(
+      final byte[] data, int offset, int length) throws IOException {
     int magicLen = ProtobufMagic.lengthOfPBMagic();
-    if (!ProtobufMagic.isPBMagicPrefix(data, 0, magicLen)) {
+    if (!ProtobufMagic.isPBMagicPrefix(data, offset, magicLen)) {
       throw new IOException("Missing pb magic prefix");
     }
-    return Quotas.parseFrom(new ByteArrayInputStream(data, magicLen, data.length - magicLen));
+    return Quotas.parseFrom(new ByteArrayInputStream(data, offset + magicLen, length - magicLen));
   }
 
   protected static byte[] quotasToData(final Quotas data) throws IOException {
@@ -348,6 +356,7 @@ public class QuotaTableUtil {
     boolean hasSettings = false;
     hasSettings |= quotas.hasThrottle();
     hasSettings |= quotas.hasBypassGlobals();
+    hasSettings |= quotas.hasSpace();
     return !hasSettings;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a29abe64/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index 5dab2e3..1469268 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -37,6 +37,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
@@ -293,9 +295,11 @@ public class MasterQuotaManager implements RegionStateListener {
     Quotas quotas = quotaOps.fetch();
     quotaOps.preApply(quotas);
 
+    // Copy the user request into the Quotas object
     Quotas.Builder builder = (quotas != null) ? quotas.toBuilder() : Quotas.newBuilder();
     if (req.hasThrottle()) applyThrottle(builder, req.getThrottle());
     if (req.hasBypassGlobals()) applyBypassGlobals(builder, req.getBypassGlobals());
+    if (req.hasSpaceLimit()) applySpaceLimit(builder, req.getSpaceLimit());
 
     // Submit new changes
     quotas = builder.build();
@@ -437,6 +441,32 @@ public class MasterQuotaManager implements RegionStateListener {
     }
   }
 
+  /**
+   * Adds the information from the provided {@link SpaceLimitRequest} to the {@link Quotas} builder.
+   *
+   * @param quotas The builder to update.
+   * @param req The request to extract space quota information from.
+   */
+  void applySpaceLimit(final Quotas.Builder quotas, final SpaceLimitRequest req) {
+    if (req.hasQuota()) {
+      applySpaceQuota(quotas, req.getQuota());
+    }
+  }
+
+  /**
+   * Merges the provided {@link SpaceQuota} into the given {@link Quotas} builder.
+   *
+   * @param quotas The Quotas builder instance to update
+   * @param quota The SpaceQuota instance to update from
+   */
+  void applySpaceQuota(final Quotas.Builder quotas, final SpaceQuota quota) {
+    // Create a builder for Quotas
+    SpaceQuota.Builder builder = quotas.hasSpace() ? quotas.getSpace().toBuilder() :
+        SpaceQuota.newBuilder();
+    // Update the values from the provided quota into the new one and set it on Quotas.
+    quotas.setSpace(builder.mergeFrom(quota).build());
+  }
+
   private void validateTimedQuota(final TimedQuota timedQuota) throws IOException {
     if (timedQuota.getSoftLimit() < 1) {
       throw new DoNotRetryIOException(new UnsupportedOperationException(

http://git-wip-us.apache.org/repos/asf/hbase/blob/a29abe64/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
index ef26c25..be93c2f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
@@ -22,20 +22,32 @@ import java.util.concurrent.TimeUnit;
 
 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.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.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.security.User;
+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;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.google.common.collect.Iterables;
+
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 /**
@@ -233,10 +245,121 @@ public class TestQuotaAdmin {
     assertNumResults(0, null);
   }
 
+  @Test
+  public void testSetAndGetSpaceQuota() throws Exception {
+    Admin admin = TEST_UTIL.getAdmin();
+    final TableName tn = TableName.valueOf("table1");
+    final long sizeLimit = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
+    final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_WRITES;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, violationPolicy);
+    admin.setQuota(settings);
+
+    // Verify the Quotas in the table
+    try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      ResultScanner scanner = quotaTable.getScanner(new Scan());
+      try {
+        Result r = Iterables.getOnlyElement(scanner);
+        CellScanner cells = r.cellScanner();
+        assertTrue("Expected to find a cell", cells.advance());
+        assertSpaceQuota(sizeLimit, violationPolicy, cells.current());
+      } finally {
+        scanner.close();
+      }
+    }
+
+    // Verify we can retrieve it via the QuotaRetriever API
+    QuotaRetriever scanner = QuotaRetriever.open(admin.getConfiguration());
+    try {
+      assertSpaceQuota(sizeLimit, violationPolicy, Iterables.getOnlyElement(scanner));
+    } finally {
+      scanner.close();
+    }
+  }
+
+  @Test
+  public void testSetAndModifyQuota() throws Exception {
+    Admin admin = TEST_UTIL.getAdmin();
+    final TableName tn = TableName.valueOf("table1");
+    final long originalSizeLimit = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
+    final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_WRITES;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, originalSizeLimit, violationPolicy);
+    admin.setQuota(settings);
+
+    // Verify the Quotas in the table
+    try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      ResultScanner scanner = quotaTable.getScanner(new Scan());
+      try {
+        Result r = Iterables.getOnlyElement(scanner);
+        CellScanner cells = r.cellScanner();
+        assertTrue("Expected to find a cell", cells.advance());
+        assertSpaceQuota(originalSizeLimit, violationPolicy, cells.current());
+      } finally {
+        scanner.close();
+      }
+    }
+
+    // Verify we can retrieve it via the QuotaRetriever API
+    QuotaRetriever quotaScanner = QuotaRetriever.open(admin.getConfiguration());
+    try {
+      assertSpaceQuota(originalSizeLimit, violationPolicy, Iterables.getOnlyElement(quotaScanner));
+    } finally {
+      quotaScanner.close();
+    }
+
+    // Setting a new size and policy should be reflected
+    final long newSizeLimit = 1024L * 1024L * 1024L * 1024L; // 1TB
+    final SpaceViolationPolicy newViolationPolicy = SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
+    QuotaSettings newSettings = QuotaSettingsFactory.limitTableSpace(
+        tn, newSizeLimit, newViolationPolicy);
+    admin.setQuota(newSettings);
+
+    // Verify the new Quotas in the table
+    try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      ResultScanner scanner = quotaTable.getScanner(new Scan());
+      try {
+        Result r = Iterables.getOnlyElement(scanner);
+        CellScanner cells = r.cellScanner();
+        assertTrue("Expected to find a cell", cells.advance());
+        assertSpaceQuota(newSizeLimit, newViolationPolicy, cells.current());
+      } finally {
+        scanner.close();
+      }
+    }
+
+    // Verify we can retrieve the new quota via the QuotaRetriever API
+    quotaScanner = QuotaRetriever.open(admin.getConfiguration());
+    try {
+      assertSpaceQuota(newSizeLimit, newViolationPolicy, Iterables.getOnlyElement(quotaScanner));
+    } finally {
+      quotaScanner.close();
+    }
+  }
+
   private void assertNumResults(int expected, final QuotaFilter filter) throws Exception {
     assertEquals(expected, countResults(filter));
   }
 
+  private void assertSpaceQuota(
+      long sizeLimit, SpaceViolationPolicy violationPolicy, Cell cell) throws Exception {
+    Quotas q = QuotaTableUtil.quotasFromData(
+        cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+    assertTrue("Quota should have space quota defined", q.hasSpace());
+    QuotaProtos.SpaceQuota spaceQuota = q.getSpace();
+    assertEquals(sizeLimit, spaceQuota.getSoftLimit());
+    assertEquals(violationPolicy, ProtobufUtil.toViolationPolicy(spaceQuota.getViolationPolicy()));
+  }
+
+  private void assertSpaceQuota(
+      long sizeLimit, SpaceViolationPolicy violationPolicy, QuotaSettings actualSettings) {
+    assertTrue("The actual QuotaSettings was not an instance of " + SpaceLimitSettings.class
+        + " but of " + actualSettings.getClass(), actualSettings instanceof SpaceLimitSettings);
+    SpaceLimitRequest spaceLimitRequest = ((SpaceLimitSettings) actualSettings).getProto();
+    assertEquals(sizeLimit, spaceLimitRequest.getQuota().getSoftLimit());
+    assertEquals(violationPolicy,
+        ProtobufUtil.toViolationPolicy(spaceLimitRequest.getQuota().getViolationPolicy()));
+  }
+
   private int countResults(final QuotaFilter filter) throws Exception {
     QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration(), filter);
     try {


[17/50] [abbrv] hbase git commit: HBASE-16998 Implement Master-side analysis of region space reports

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/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
new file mode 100644
index 0000000..98236c2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java
@@ -0,0 +1,596 @@
+/*
+ * 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.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.quotas.QuotaObserverChore.TablesWithQuotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+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 {@link QuotaObserverChore} that uses a live HBase cluster.
+ */
+@Category(LargeTests.class)
+public class TestQuotaObserverChoreWithMiniCluster {
+  private static final Log LOG = LogFactory.getLog(TestQuotaObserverChoreWithMiniCluster.class);
+  private static final int SIZE_PER_VALUE = 256;
+  private static final String F1 = "f1";
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final AtomicLong COUNTER = new AtomicLong(0);
+  private static final long ONE_MEGABYTE = 1024L * 1024L;
+  private static final long DEFAULT_WAIT_MILLIS = 500;
+
+  @Rule
+  public TestName testName = new TestName();
+
+  private HMaster master;
+  private QuotaObserverChore chore;
+  private SpaceQuotaViolationNotifierForTest violationNotifier;
+
+  @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.VIOLATION_OBSERVER_CHORE_DELAY_KEY, 1000);
+    conf.setInt(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_PERIOD_KEY, 1000);
+    conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void removeAllQuotas() throws Exception {
+    final Connection conn = TEST_UTIL.getConnection();
+    // Wait for the quota table to be created
+    if (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME)) {
+      do {
+        LOG.debug("Quota table does not yet exist");
+        Thread.sleep(DEFAULT_WAIT_MILLIS);
+      } while (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME));
+    } else {
+      // Or, clean up any quotas from previous test runs.
+      QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration());
+      for (QuotaSettings quotaSettings : scanner) {
+        final String namespace = quotaSettings.getNamespace();
+        final TableName tableName = quotaSettings.getTableName();
+        if (null != namespace) {
+          LOG.debug("Deleting quota for namespace: " + namespace);
+          QuotaUtil.deleteNamespaceQuota(conn, namespace);
+        } else {
+          assert null != tableName;
+          LOG.debug("Deleting quota for table: "+ tableName);
+          QuotaUtil.deleteTableQuota(conn, tableName);
+        }
+      }
+    }
+
+    master = TEST_UTIL.getMiniHBaseCluster().getMaster();
+    violationNotifier =
+        (SpaceQuotaViolationNotifierForTest) master.getSpaceQuotaViolationNotifier();
+    violationNotifier.clearTableViolations();
+    chore = master.getQuotaObserverChore();
+  }
+
+  @Test
+  public void testTableViolatesQuota() throws Exception {
+    TableName tn = createTableWithRegions(10);
+
+    final long sizeLimit = 2L * ONE_MEGABYTE;
+    final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, violationPolicy);
+    TEST_UTIL.getAdmin().setQuota(settings);
+
+    // Write more data than should be allowed
+    writeData(tn, 3L * ONE_MEGABYTE);
+
+    Map<TableName,SpaceViolationPolicy> violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    while (violatedQuotas.isEmpty()) {
+      LOG.info("Found no violated quotas, sleeping and retrying. Current reports: "
+          + master.getMasterQuotaManager().snapshotRegionSizes());
+      try {
+        Thread.sleep(DEFAULT_WAIT_MILLIS);
+      } catch (InterruptedException e) {
+        LOG.debug("Interrupted while sleeping.", e);
+        Thread.currentThread().interrupt();
+      }
+      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    }
+
+    Entry<TableName,SpaceViolationPolicy> entry = Iterables.getOnlyElement(violatedQuotas.entrySet());
+    assertEquals(tn, entry.getKey());
+    assertEquals(violationPolicy, entry.getValue());
+  }
+
+  @Test
+  public void testNamespaceViolatesQuota() throws Exception {
+    final String namespace = testName.getMethodName();
+    final Admin admin = TEST_UTIL.getAdmin();
+    // Ensure the namespace exists
+    try {
+      admin.getNamespaceDescriptor(namespace);
+    } catch (NamespaceNotFoundException e) {
+      NamespaceDescriptor desc = NamespaceDescriptor.create(namespace).build();
+      admin.createNamespace(desc);
+    }
+
+    TableName tn1 = createTableWithRegions(namespace, 5);
+    TableName tn2 = createTableWithRegions(namespace, 5);
+    TableName tn3 = createTableWithRegions(namespace, 5);
+
+    final long sizeLimit = 5L * ONE_MEGABYTE;
+    final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.DISABLE;
+    QuotaSettings settings = QuotaSettingsFactory.limitNamespaceSpace(namespace, sizeLimit, violationPolicy);
+    admin.setQuota(settings);
+
+    writeData(tn1, 2L * ONE_MEGABYTE);
+    admin.flush(tn1);
+    Map<TableName,SpaceViolationPolicy> violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    for (int i = 0; i < 5; i++) {
+      // Check a few times to make sure we don't prematurely move to violation
+      assertEquals("Should not see any quota violations after writing 2MB of data", 0, violatedQuotas.size());
+      try {
+        Thread.sleep(DEFAULT_WAIT_MILLIS);
+      } catch (InterruptedException e) {
+        LOG.debug("Interrupted while sleeping." , e);
+      }
+      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    }
+
+    writeData(tn2, 2L * ONE_MEGABYTE);
+    admin.flush(tn2);
+    violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    for (int i = 0; i < 5; i++) {
+      // Check a few times to make sure we don't prematurely move to violation
+      assertEquals("Should not see any quota violations after writing 4MB of data", 0,
+          violatedQuotas.size());
+      try {
+        Thread.sleep(DEFAULT_WAIT_MILLIS);
+      } catch (InterruptedException e) {
+        LOG.debug("Interrupted while sleeping." , e);
+      }
+      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    }
+
+    // Writing the final 2MB of data will push the namespace over the 5MB limit (6MB in total)
+    // and should push all three tables in the namespace into violation.
+    writeData(tn3, 2L * ONE_MEGABYTE);
+    admin.flush(tn3);
+    violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    while (violatedQuotas.size() < 3) {
+      LOG.debug("Saw fewer violations than desired (expected 3): " + violatedQuotas
+          + ". Current reports: " + master.getMasterQuotaManager().snapshotRegionSizes());
+      try {
+        Thread.sleep(DEFAULT_WAIT_MILLIS);
+      } catch (InterruptedException e) {
+        LOG.debug("Interrupted while sleeping.", e);
+        Thread.currentThread().interrupt();
+      }
+      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    }
+
+    SpaceViolationPolicy vp1 = violatedQuotas.remove(tn1);
+    assertNotNull("tn1 should be in violation", vp1);
+    assertEquals(violationPolicy, vp1);
+    SpaceViolationPolicy vp2 = violatedQuotas.remove(tn2);
+    assertNotNull("tn2 should be in violation", vp2);
+    assertEquals(violationPolicy, vp2);
+    SpaceViolationPolicy vp3 = violatedQuotas.remove(tn3);
+    assertNotNull("tn3 should be in violation", vp3);
+    assertEquals(violationPolicy, vp3);
+    assertTrue("Unexpected additional quota violations: " + violatedQuotas, violatedQuotas.isEmpty());
+  }
+
+  @Test
+  public void testTableQuotaOverridesNamespaceQuota() throws Exception {
+    final String namespace = testName.getMethodName();
+    final Admin admin = TEST_UTIL.getAdmin();
+    // Ensure the namespace exists
+    try {
+      admin.getNamespaceDescriptor(namespace);
+    } catch (NamespaceNotFoundException e) {
+      NamespaceDescriptor desc = NamespaceDescriptor.create(namespace).build();
+      admin.createNamespace(desc);
+    }
+
+    TableName tn1 = createTableWithRegions(namespace, 5);
+    TableName tn2 = createTableWithRegions(namespace, 5);
+
+    final long namespaceSizeLimit = 3L * ONE_MEGABYTE;
+    final SpaceViolationPolicy namespaceViolationPolicy = SpaceViolationPolicy.DISABLE;
+    QuotaSettings namespaceSettings = QuotaSettingsFactory.limitNamespaceSpace(namespace,
+        namespaceSizeLimit, namespaceViolationPolicy);
+    admin.setQuota(namespaceSettings);
+
+    writeData(tn1, 2L * ONE_MEGABYTE);
+    admin.flush(tn1);
+    Map<TableName,SpaceViolationPolicy> violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    for (int i = 0; i < 5; i++) {
+      // Check a few times to make sure we don't prematurely move to violation
+      assertEquals("Should not see any quota violations after writing 2MB of data", 0,
+          violatedQuotas.size());
+      try {
+        Thread.sleep(DEFAULT_WAIT_MILLIS);
+      } catch (InterruptedException e) {
+        LOG.debug("Interrupted while sleeping." , e);
+      }
+      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    }
+
+    writeData(tn2, 2L * ONE_MEGABYTE);
+    admin.flush(tn2);
+    violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    while (violatedQuotas.size() < 2) {
+      LOG.debug("Saw fewer violations than desired (expected 2): " + violatedQuotas
+          + ". Current reports: " + master.getMasterQuotaManager().snapshotRegionSizes());
+      try {
+        Thread.sleep(DEFAULT_WAIT_MILLIS);
+      } catch (InterruptedException e) {
+        LOG.debug("Interrupted while sleeping.", e);
+        Thread.currentThread().interrupt();
+      }
+      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+    }
+
+    SpaceViolationPolicy actualPolicyTN1 = violatedQuotas.get(tn1);
+    assertNotNull("Expected to see violation policy for tn1", actualPolicyTN1);
+    assertEquals(namespaceViolationPolicy, actualPolicyTN1);
+    SpaceViolationPolicy actualPolicyTN2 = violatedQuotas.get(tn2);
+    assertNotNull("Expected to see violation policy for tn2", actualPolicyTN2);
+    assertEquals(namespaceViolationPolicy, actualPolicyTN2);
+
+    // Override the namespace quota with a table quota
+    final long tableSizeLimit = ONE_MEGABYTE;
+    final SpaceViolationPolicy tableViolationPolicy = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings tableSettings = QuotaSettingsFactory.limitTableSpace(tn1, tableSizeLimit,
+        tableViolationPolicy);
+    admin.setQuota(tableSettings);
+
+    // Keep checking for the table quota policy to override the namespace quota
+    while (true) {
+      violatedQuotas = violationNotifier.snapshotTablesInViolation();
+      SpaceViolationPolicy actualTableViolationPolicy = violatedQuotas.get(tn1);
+      assertNotNull("Violation policy should never be null", actualTableViolationPolicy);
+      if (tableViolationPolicy != actualTableViolationPolicy) {
+        LOG.debug("Saw unexpected table violation policy, waiting and re-checking.");
+        try {
+          Thread.sleep(DEFAULT_WAIT_MILLIS);
+        } catch (InterruptedException e) {
+          LOG.debug("Interrupted while sleeping");
+          Thread.currentThread().interrupt();
+        }
+        continue;
+      }
+      assertEquals(tableViolationPolicy, actualTableViolationPolicy);
+      break;
+    }
+
+    // This should not change with the introduction of the table quota for tn1
+    actualPolicyTN2 = violatedQuotas.get(tn2);
+    assertNotNull("Expected to see violation policy for tn2", actualPolicyTN2);
+    assertEquals(namespaceViolationPolicy, actualPolicyTN2);
+  }
+
+  @Test
+  public void testGetAllTablesWithQuotas() throws Exception {
+    final Multimap<TableName, QuotaSettings> quotas = createTablesWithSpaceQuotas();
+    Set<TableName> tablesWithQuotas = new HashSet<>();
+    Set<TableName> namespaceTablesWithQuotas = new HashSet<>();
+    // Partition the tables with quotas by table and ns quota
+    partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
+
+    TablesWithQuotas tables = chore.fetchAllTablesWithQuotasDefined();
+    assertEquals("Found tables: " + tables, tablesWithQuotas, tables.getTableQuotaTables());
+    assertEquals("Found tables: " + tables, namespaceTablesWithQuotas, tables.getNamespaceQuotaTables());
+  }
+
+  @Test
+  public void testRpcQuotaTablesAreFiltered() throws Exception {
+    final Multimap<TableName, QuotaSettings> quotas = createTablesWithSpaceQuotas();
+    Set<TableName> tablesWithQuotas = new HashSet<>();
+    Set<TableName> namespaceTablesWithQuotas = new HashSet<>();
+    // Partition the tables with quotas by table and ns quota
+    partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
+
+    TableName rpcQuotaTable = createTable();
+    TEST_UTIL.getAdmin().setQuota(QuotaSettingsFactory
+      .throttleTable(rpcQuotaTable, ThrottleType.READ_NUMBER, 6, TimeUnit.MINUTES));
+
+    // The `rpcQuotaTable` should not be included in this Set
+    TablesWithQuotas tables = chore.fetchAllTablesWithQuotasDefined();
+    assertEquals("Found tables: " + tables, tablesWithQuotas, tables.getTableQuotaTables());
+    assertEquals("Found tables: " + tables, namespaceTablesWithQuotas, tables.getNamespaceQuotaTables());
+  }
+
+  @Test
+  public void testFilterRegions() throws Exception {
+    Map<TableName,Integer> mockReportedRegions = new HashMap<>();
+    // Can't mock because of primitive int as a return type -- Mockito
+    // can only handle an Integer.
+    TablesWithQuotas tables = new TablesWithQuotas(TEST_UTIL.getConnection(),
+        TEST_UTIL.getConfiguration()) {
+      @Override
+      int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore) {
+        Integer i = mockReportedRegions.get(table);
+        if (null == i) {
+          return 0;
+        }
+        return i;
+      }
+    };
+
+    // Create the tables
+    TableName tn1 = createTableWithRegions(20);
+    TableName tn2 = createTableWithRegions(20);
+    TableName tn3 = createTableWithRegions(20);
+
+    // Add them to the Tables with Quotas object
+    tables.addTableQuotaTable(tn1);
+    tables.addTableQuotaTable(tn2);
+    tables.addTableQuotaTable(tn3);
+
+    // Mock the number of regions reported
+    mockReportedRegions.put(tn1, 10); // 50%
+    mockReportedRegions.put(tn2, 19); // 95%
+    mockReportedRegions.put(tn3, 20); // 100%
+
+    // Argument is un-used
+    tables.filterInsufficientlyReportedTables(null);
+    // The default of 95% reported should prevent tn1 from appearing
+    assertEquals(new HashSet<>(Arrays.asList(tn2, tn3)), tables.getTableQuotaTables());
+  }
+
+  @Test
+  public void testFetchSpaceQuota() throws Exception {
+    Multimap<TableName,QuotaSettings> tables = createTablesWithSpaceQuotas();
+    // Can pass in an empty map, we're not consulting it.
+    chore.initializeViolationStores(Collections.emptyMap());
+    // All tables that were created should have a quota defined.
+    for (Entry<TableName,QuotaSettings> entry : tables.entries()) {
+      final TableName table = entry.getKey();
+      final QuotaSettings qs = entry.getValue();
+
+      assertTrue("QuotaSettings was an instance of " + qs.getClass(),
+          qs instanceof SpaceLimitSettings);
+
+      SpaceQuota spaceQuota = null;
+      if (null != qs.getTableName()) {
+        spaceQuota = chore.getTableViolationStore().getSpaceQuota(table);
+        assertNotNull("Could not find table space quota for " + table, spaceQuota);
+      } else if (null != qs.getNamespace()) {
+        spaceQuota = chore.getNamespaceViolationStore().getSpaceQuota(table.getNamespaceAsString());
+        assertNotNull("Could not find namespace space quota for " + table.getNamespaceAsString(), spaceQuota);
+      } else {
+        fail("Expected table or namespace space quota");
+      }
+
+      final SpaceLimitSettings sls = (SpaceLimitSettings) qs;
+      assertEquals(sls.getProto().getQuota(), spaceQuota);
+    }
+
+    TableName tableWithoutQuota = createTable();
+    assertNull(chore.getTableViolationStore().getSpaceQuota(tableWithoutQuota));
+  }
+
+  //
+  // Helpers
+  //
+
+  void writeData(TableName tn, long sizeInBytes) throws IOException {
+    final Connection conn = TEST_UTIL.getConnection();
+    final Table table = conn.getTable(tn);
+    try {
+      List<Put> updates = new ArrayList<>();
+      long bytesToWrite = sizeInBytes;
+      long rowKeyId = 0L;
+      final StringBuilder sb = new StringBuilder();
+      final Random r = new Random();
+      while (bytesToWrite > 0L) {
+        sb.setLength(0);
+        sb.append(Long.toString(rowKeyId));
+        // Use the reverse counter as the rowKey to get even spread across all regions
+        Put p = new Put(Bytes.toBytes(sb.reverse().toString()));
+        byte[] value = new byte[SIZE_PER_VALUE];
+        r.nextBytes(value);
+        p.addColumn(Bytes.toBytes(F1), Bytes.toBytes("q1"), value);
+        updates.add(p);
+
+        // Batch 50K worth of updates
+        if (updates.size() > 50) {
+          table.put(updates);
+          updates.clear();
+        }
+
+        // Just count the value size, ignore the size of rowkey + column
+        bytesToWrite -= SIZE_PER_VALUE;
+        rowKeyId++;
+      }
+
+      // Write the final batch
+      if (!updates.isEmpty()) {
+        table.put(updates);
+      }
+
+      LOG.debug("Data was written to HBase");
+      // Push the data to disk.
+      TEST_UTIL.getAdmin().flush(tn);
+      LOG.debug("Data flushed to disk");
+    } finally {
+      table.close();
+    }
+  }
+
+  Multimap<TableName, QuotaSettings> createTablesWithSpaceQuotas() throws Exception {
+    final Admin admin = TEST_UTIL.getAdmin();
+    final Multimap<TableName, QuotaSettings> tablesWithQuotas = HashMultimap.create();
+
+    final TableName tn1 = createTable();
+    final TableName tn2 = createTable();
+
+    NamespaceDescriptor nd = NamespaceDescriptor.create("ns" + COUNTER.getAndIncrement()).build();
+    admin.createNamespace(nd);
+    final TableName tn3 = createTableInNamespace(nd);
+    final TableName tn4 = createTableInNamespace(nd);
+    final TableName tn5 = createTableInNamespace(nd);
+
+    final long sizeLimit1 = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
+    final SpaceViolationPolicy violationPolicy1 = SpaceViolationPolicy.NO_WRITES;
+    QuotaSettings qs1 = QuotaSettingsFactory.limitTableSpace(tn1, sizeLimit1, violationPolicy1);
+    tablesWithQuotas.put(tn1, qs1);
+    admin.setQuota(qs1);
+
+    final long sizeLimit2 = 1024L * 1024L * 1024L * 200L; // 200GB
+    final SpaceViolationPolicy violationPolicy2 = SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
+    QuotaSettings qs2 = QuotaSettingsFactory.limitTableSpace(tn2, sizeLimit2, violationPolicy2);
+    tablesWithQuotas.put(tn2, qs2);
+    admin.setQuota(qs2);
+
+    final long sizeLimit3 = 1024L * 1024L * 1024L * 1024L * 100L; // 100TB
+    final SpaceViolationPolicy violationPolicy3 = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings qs3 = QuotaSettingsFactory.limitNamespaceSpace(nd.getName(), sizeLimit3, violationPolicy3);
+    tablesWithQuotas.put(tn3, qs3);
+    tablesWithQuotas.put(tn4, qs3);
+    tablesWithQuotas.put(tn5, qs3);
+    admin.setQuota(qs3);
+
+    final long sizeLimit4 = 1024L * 1024L * 1024L * 5L; // 5GB
+    final SpaceViolationPolicy violationPolicy4 = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings qs4 = QuotaSettingsFactory.limitTableSpace(tn5, sizeLimit4, violationPolicy4);
+    // Override the ns quota for tn5, import edge-case to catch table quota taking
+    // precedence over ns quota.
+    tablesWithQuotas.put(tn5, qs4);
+    admin.setQuota(qs4);
+
+    return tablesWithQuotas;
+  }
+
+  TableName createTable() throws Exception {
+    return createTableWithRegions(1);
+  }
+
+  TableName createTableWithRegions(int numRegions) throws Exception {
+    return createTableWithRegions(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR, numRegions);
+  }
+
+  TableName createTableWithRegions(String namespace, int numRegions) throws Exception {
+    final Admin admin = TEST_UTIL.getAdmin();
+    final TableName tn = TableName.valueOf(namespace, testName.getMethodName() + COUNTER.getAndIncrement());
+
+    // Delete the old table
+    if (admin.tableExists(tn)) {
+      admin.disableTable(tn);
+      admin.deleteTable(tn);
+    }
+
+    // Create the table
+    HTableDescriptor tableDesc = new HTableDescriptor(tn);
+    tableDesc.addFamily(new HColumnDescriptor(F1));
+    if (numRegions == 1) {
+      admin.createTable(tableDesc);
+    } else {
+      admin.createTable(tableDesc, Bytes.toBytes("0"), Bytes.toBytes("9"), numRegions);
+    }
+    return tn;
+  }
+
+  TableName createTableInNamespace(NamespaceDescriptor nd) throws Exception {
+    final Admin admin = TEST_UTIL.getAdmin();
+    final TableName tn = TableName.valueOf(nd.getName(),
+        testName.getMethodName() + COUNTER.getAndIncrement());
+
+    // Delete the old table
+    if (admin.tableExists(tn)) {
+      admin.disableTable(tn);
+      admin.deleteTable(tn);
+    }
+
+    // Create the table
+    HTableDescriptor tableDesc = new HTableDescriptor(tn);
+    tableDesc.addFamily(new HColumnDescriptor(F1));
+
+    admin.createTable(tableDesc);
+    return tn;
+  }
+
+  void partitionTablesByQuotaTarget(Multimap<TableName,QuotaSettings> quotas,
+      Set<TableName> tablesWithTableQuota, Set<TableName> tablesWithNamespaceQuota) {
+    // Partition the tables with quotas by table and ns quota
+    for (Entry<TableName, QuotaSettings> entry : quotas.entries()) {
+      SpaceLimitSettings settings = (SpaceLimitSettings) entry.getValue();
+      TableName tn = entry.getKey();
+      if (null != settings.getTableName()) {
+        tablesWithTableQuota.add(tn);
+      }
+      if (null != settings.getNamespace()) {
+        tablesWithNamespaceQuota.add(tn);
+      }
+
+      if (null == settings.getTableName() && null == settings.getNamespace()) {
+        fail("Unexpected table name with null tableName and namespace: " + tn);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
index 3b276ad..238c4c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java
@@ -23,14 +23,11 @@ import static org.junit.Assert.assertEquals;
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
@@ -50,7 +47,6 @@ import org.junit.rules.TestName;
  */
 @Category({MasterTests.class, MediumTests.class})
 public class TestQuotaTableUtil {
-  private static final Log LOG = LogFactory.getLog(TestQuotaTableUtil.class);
 
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private Connection connection;

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/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
new file mode 100644
index 0000000..efc046b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java
@@ -0,0 +1,151 @@
+/*
+ * 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 com.google.common.collect.Iterables.size;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashMap;
+import java.util.Map;
+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.quotas.QuotaViolationStore.ViolationState;
+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;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Test class for {@link TableQuotaViolationStore}.
+ */
+@Category(SmallTests.class)
+public class TestTableQuotaViolationStore {
+  private static final long ONE_MEGABYTE = 1024L * 1024L;
+
+  private Connection conn;
+  private QuotaObserverChore chore;
+  private Map<HRegionInfo, Long> regionReports;
+  private TableQuotaViolationStore store;
+
+  @Before
+  public void setup() {
+    conn = mock(Connection.class);
+    chore = mock(QuotaObserverChore.class);
+    regionReports = new HashMap<>();
+    store = new TableQuotaViolationStore(conn, chore, regionReports);
+  }
+
+  @Test
+  public void testFilterRegionsByTable() throws Exception {
+    TableName tn1 = TableName.valueOf("foo");
+    TableName tn2 = TableName.valueOf("bar");
+    TableName tn3 = TableName.valueOf("ns", "foo");
+
+    assertEquals(0, size(store.filterBySubject(tn1)));
+
+    for (int i = 0; i < 5; i++) {
+      regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+    }
+    for (int i = 0; i < 3; i++) {
+      regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+    }
+    for (int i = 0; i < 10; i++) {
+      regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+    }
+    assertEquals(18, regionReports.size());
+    assertEquals(5, size(store.filterBySubject(tn1)));
+    assertEquals(3, size(store.filterBySubject(tn2)));
+    assertEquals(10, size(store.filterBySubject(tn3)));
+  }
+
+  @Test
+  public void testTargetViolationState() {
+    TableName tn1 = TableName.valueOf("violation1");
+    TableName tn2 = TableName.valueOf("observance1");
+    TableName tn3 = TableName.valueOf("observance2");
+    SpaceQuota quota = SpaceQuota.newBuilder()
+        .setSoftLimit(1024L * 1024L)
+        .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(SpaceViolationPolicy.DISABLE))
+        .build();
+
+    // Create some junk data to filter. Makes sure it's so large that it would
+    // immediately violate the quota.
+    for (int i = 0; i < 3; i++) {
+      regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(i), Bytes.toBytes(i + 1)),
+          5L * ONE_MEGABYTE);
+      regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i + 1)),
+          5L * ONE_MEGABYTE);
+    }
+
+    regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(0), Bytes.toBytes(1)), 1024L * 512L);
+    regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(1), Bytes.toBytes(2)), 1024L * 256L);
+
+    // Below the quota
+    assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(tn1, quota));
+
+    regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(2), Bytes.toBytes(3)), 1024L * 256L);
+
+    // Equal to the quota is still in observance
+    assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(tn1, quota));
+
+    regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(3), Bytes.toBytes(4)), 1024L);
+
+    // Exceeds the quota, should be in violation
+    assertEquals(ViolationState.IN_VIOLATION, store.getTargetState(tn1, quota));
+  }
+
+  @Test
+  public void testGetSpaceQuota() throws Exception {
+    TableQuotaViolationStore mockStore = mock(TableQuotaViolationStore.class);
+    when(mockStore.getSpaceQuota(any(TableName.class))).thenCallRealMethod();
+
+    Quotas quotaWithSpace = Quotas.newBuilder().setSpace(
+        SpaceQuota.newBuilder()
+            .setSoftLimit(1024L)
+            .setViolationPolicy(QuotaProtos.SpaceViolationPolicy.DISABLE)
+            .build())
+        .build();
+    Quotas quotaWithoutSpace = Quotas.newBuilder().build();
+
+    AtomicReference<Quotas> quotaRef = new AtomicReference<>();
+    when(mockStore.getQuotaForTable(any(TableName.class))).then(new Answer<Quotas>() {
+      @Override
+      public Quotas answer(InvocationOnMock invocation) throws Throwable {
+        return quotaRef.get();
+      }
+    });
+
+    quotaRef.set(quotaWithSpace);
+    assertEquals(quotaWithSpace.getSpace(), mockStore.getSpaceQuota(TableName.valueOf("foo")));
+    quotaRef.set(quotaWithoutSpace);
+    assertNull(mockStore.getSpaceQuota(TableName.valueOf("foo")));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
new file mode 100644
index 0000000..bb8d5cd
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
@@ -0,0 +1,198 @@
+/*
+ * 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 static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HRegionInfo;
+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.quotas.QuotaObserverChore.TablesWithQuotas;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Multimap;
+
+/**
+ * Non-HBase cluster unit tests for {@link TablesWithQuotas}.
+ */
+@Category(SmallTests.class)
+public class TestTablesWithQuotas {
+  private Connection conn;
+  private Configuration conf;
+
+  @Before
+  public void setup() throws Exception {
+    conn = mock(Connection.class);
+    conf = HBaseConfiguration.create();
+  }
+
+  @Test
+  public void testImmutableGetters() {
+    Set<TableName> tablesWithTableQuotas = new HashSet<>();
+    Set<TableName> tablesWithNamespaceQuotas = new HashSet<>();
+    final TablesWithQuotas tables = new TablesWithQuotas(conn, conf);
+    for (int i = 0; i < 5; i++) {
+      TableName tn = TableName.valueOf("tn" + i);
+      tablesWithTableQuotas.add(tn);
+      tables.addTableQuotaTable(tn);
+    }
+    for (int i = 0; i < 3; i++) {
+      TableName tn = TableName.valueOf("tn_ns" + i);
+      tablesWithNamespaceQuotas.add(tn);
+      tables.addNamespaceQuotaTable(tn);
+    }
+    Set<TableName> actualTableQuotaTables = tables.getTableQuotaTables();
+    Set<TableName> actualNamespaceQuotaTables = tables.getNamespaceQuotaTables();
+    assertEquals(tablesWithTableQuotas, actualTableQuotaTables);
+    assertEquals(tablesWithNamespaceQuotas, actualNamespaceQuotaTables);
+    try {
+      actualTableQuotaTables.add(null);
+      fail("Should not be able to add an element");
+    } catch (UnsupportedOperationException e) {
+      // pass
+    }
+    try {
+      actualNamespaceQuotaTables.add(null);
+      fail("Should not be able to add an element");
+    } catch (UnsupportedOperationException e) {
+      // pass
+    }
+  }
+
+  @Test
+  public void testInsufficientlyReportedTableFiltering() throws Exception {
+    final Map<TableName,Integer> reportedRegions = new HashMap<>();
+    final Map<TableName,Integer> actualRegions = new HashMap<>();
+    final Configuration conf = HBaseConfiguration.create();
+    conf.setDouble(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_KEY, 0.95);
+
+    TableName tooFewRegionsTable = TableName.valueOf("tn1");
+    TableName sufficientRegionsTable = TableName.valueOf("tn2");
+    TableName tooFewRegionsNamespaceTable = TableName.valueOf("ns1", "tn2");
+    TableName sufficientRegionsNamespaceTable = TableName.valueOf("ns1", "tn2");
+    final TablesWithQuotas tables = new TablesWithQuotas(conn, conf) {
+      @Override
+      Configuration getConfiguration() {
+        return conf;
+      }
+
+      @Override
+      int getNumRegions(TableName tableName) {
+        return actualRegions.get(tableName);
+      }
+
+      @Override
+      int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore) {
+        return reportedRegions.get(table);
+      }
+    };
+    tables.addTableQuotaTable(tooFewRegionsTable);
+    tables.addTableQuotaTable(sufficientRegionsTable);
+    tables.addNamespaceQuotaTable(tooFewRegionsNamespaceTable);
+    tables.addNamespaceQuotaTable(sufficientRegionsNamespaceTable);
+
+    reportedRegions.put(tooFewRegionsTable, 5);
+    actualRegions.put(tooFewRegionsTable, 10);
+    reportedRegions.put(sufficientRegionsTable, 19);
+    actualRegions.put(sufficientRegionsTable, 20);
+    reportedRegions.put(tooFewRegionsNamespaceTable, 9);
+    actualRegions.put(tooFewRegionsNamespaceTable, 10);
+    reportedRegions.put(sufficientRegionsNamespaceTable, 98);
+    actualRegions.put(sufficientRegionsNamespaceTable, 100);
+
+    // Unused argument
+    tables.filterInsufficientlyReportedTables(null);
+    Set<TableName> filteredTablesWithTableQuotas = tables.getTableQuotaTables();
+    assertEquals(Collections.singleton(sufficientRegionsTable), filteredTablesWithTableQuotas);
+    Set<TableName> filteredTablesWithNamespaceQutoas = tables.getNamespaceQuotaTables();
+    assertEquals(Collections.singleton(sufficientRegionsNamespaceTable), filteredTablesWithNamespaceQutoas);
+  }
+
+  @Test
+  public void testGetTablesByNamespace() {
+    final TablesWithQuotas tables = new TablesWithQuotas(conn, conf);
+    tables.addTableQuotaTable(TableName.valueOf("ignored1"));
+    tables.addTableQuotaTable(TableName.valueOf("ignored2"));
+    tables.addNamespaceQuotaTable(TableName.valueOf("ns1", "t1"));
+    tables.addNamespaceQuotaTable(TableName.valueOf("ns1", "t2"));
+    tables.addNamespaceQuotaTable(TableName.valueOf("ns1", "t3"));
+    tables.addNamespaceQuotaTable(TableName.valueOf("ns2", "t1"));
+    tables.addNamespaceQuotaTable(TableName.valueOf("ns2", "t2"));
+
+    Multimap<String,TableName> tablesByNamespace = tables.getTablesByNamespace();
+    Collection<TableName> tablesInNs = tablesByNamespace.get("ns1");
+    assertEquals(3, tablesInNs.size());
+    assertTrue("Unexpected results for ns1: " + tablesInNs,
+        tablesInNs.containsAll(Arrays.asList(
+            TableName.valueOf("ns1", "t1"),
+            TableName.valueOf("ns1", "t2"),
+            TableName.valueOf("ns1", "t3"))));
+    tablesInNs = tablesByNamespace.get("ns2");
+    assertEquals(2, tablesInNs.size());
+    assertTrue("Unexpected results for ns2: " + tablesInNs,
+        tablesInNs.containsAll(Arrays.asList(
+            TableName.valueOf("ns2", "t1"),
+            TableName.valueOf("ns2", "t2"))));
+  }
+
+  @Test
+  public void testFilteringMissingTables() throws Exception {
+    final TableName missingTable = TableName.valueOf("doesNotExist");
+    // Set up Admin to return null (match the implementation)
+    Admin admin = mock(Admin.class);
+    when(conn.getAdmin()).thenReturn(admin);
+    when(admin.getTableRegions(missingTable)).thenReturn(null);
+
+    QuotaObserverChore chore = mock(QuotaObserverChore.class);
+    Map<HRegionInfo,Long> regionUsage = new HashMap<>();
+    TableQuotaViolationStore store = new TableQuotaViolationStore(conn, chore, regionUsage);
+
+    // A super dirty hack to verify that, after getting no regions for our table,
+    // we bail out and start processing the next element (which there is none).
+    final TablesWithQuotas tables = new TablesWithQuotas(conn, conf) {
+      @Override
+      int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore) {
+        throw new RuntimeException("Should should not reach here");
+      }
+    };
+    tables.addTableQuotaTable(missingTable);
+
+    tables.filterInsufficientlyReportedTables(store);
+
+    final Set<TableName> tablesWithQuotas = tables.getTableQuotaTables();
+    assertTrue(
+        "Expected to find no tables, but found " + tablesWithQuotas, tablesWithQuotas.isEmpty());
+  }
+}


[31/50] [abbrv] hbase git commit: HBASE-17001 Enforce quota violation policies in the RegionServer

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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 8b127d9..973ac8c 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
@@ -37,9 +37,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.quotas.QuotaViolationStore.ViolationState;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -54,51 +53,51 @@ import com.google.common.collect.Multimap;
 @InterfaceAudience.Private
 public class QuotaObserverChore extends ScheduledChore {
   private static final Log LOG = LogFactory.getLog(QuotaObserverChore.class);
-  static final String VIOLATION_OBSERVER_CHORE_PERIOD_KEY =
-      "hbase.master.quotas.violation.observer.chore.period";
-  static final int VIOLATION_OBSERVER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
+  static final String QUOTA_OBSERVER_CHORE_PERIOD_KEY =
+      "hbase.master.quotas.observer.chore.period";
+  static final int QUOTA_OBSERVER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
 
-  static final String VIOLATION_OBSERVER_CHORE_DELAY_KEY =
-      "hbase.master.quotas.violation.observer.chore.delay";
-  static final long VIOLATION_OBSERVER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
+  static final String QUOTA_OBSERVER_CHORE_DELAY_KEY =
+      "hbase.master.quotas.observer.chore.delay";
+  static final long QUOTA_OBSERVER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
 
-  static final String VIOLATION_OBSERVER_CHORE_TIMEUNIT_KEY =
-      "hbase.master.quotas.violation.observer.chore.timeunit";
-  static final String VIOLATION_OBSERVER_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
+  static final String QUOTA_OBSERVER_CHORE_TIMEUNIT_KEY =
+      "hbase.master.quotas.observer.chore.timeunit";
+  static final String QUOTA_OBSERVER_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
 
-  static final String VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_KEY =
-      "hbase.master.quotas.violation.observer.report.percent";
-  static final double VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT= 0.95;
+  static final String QUOTA_OBSERVER_CHORE_REPORT_PERCENT_KEY =
+      "hbase.master.quotas.observer.report.percent";
+  static final double QUOTA_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT= 0.95;
 
   private final Connection conn;
   private final Configuration conf;
   private final MasterQuotaManager quotaManager;
   /*
-   * Callback that changes in quota violation are passed to.
+   * Callback that changes in quota snapshots are passed to.
    */
-  private final SpaceQuotaViolationNotifier violationNotifier;
+  private final SpaceQuotaSnapshotNotifier snapshotNotifier;
 
   /*
-   * Preserves the state of quota violations for tables and namespaces
+   * Preserves the state of quota snapshots for tables and namespaces
    */
-  private final Map<TableName,ViolationState> tableQuotaViolationStates;
-  private final Map<String,ViolationState> namespaceQuotaViolationStates;
+  private final Map<TableName,SpaceQuotaSnapshot> tableQuotaSnapshots;
+  private final Map<String,SpaceQuotaSnapshot> namespaceQuotaSnapshots;
 
   /*
-   * Encapsulates logic for moving tables/namespaces into or out of quota violation
+   * Encapsulates logic for tracking the state of a table/namespace WRT space quotas
    */
-  private QuotaViolationStore<TableName> tableViolationStore;
-  private QuotaViolationStore<String> namespaceViolationStore;
+  private QuotaSnapshotStore<TableName> tableSnapshotStore;
+  private QuotaSnapshotStore<String> namespaceSnapshotStore;
 
   public QuotaObserverChore(HMaster master) {
     this(
         master.getConnection(), master.getConfiguration(),
-        master.getSpaceQuotaViolationNotifier(), master.getMasterQuotaManager(),
+        master.getSpaceQuotaSnapshotNotifier(), master.getMasterQuotaManager(),
         master);
   }
 
   QuotaObserverChore(
-      Connection conn, Configuration conf, SpaceQuotaViolationNotifier violationNotifier,
+      Connection conn, Configuration conf, SpaceQuotaSnapshotNotifier snapshotNotifier,
       MasterQuotaManager quotaManager, Stoppable stopper) {
     super(
         QuotaObserverChore.class.getSimpleName(), stopper, getPeriod(conf),
@@ -106,17 +105,20 @@ public class QuotaObserverChore extends ScheduledChore {
     this.conn = conn;
     this.conf = conf;
     this.quotaManager = quotaManager;
-    this.violationNotifier = violationNotifier;
-    this.tableQuotaViolationStates = new HashMap<>();
-    this.namespaceQuotaViolationStates = new HashMap<>();
+    this.snapshotNotifier = Objects.requireNonNull(snapshotNotifier);
+    this.tableQuotaSnapshots = new HashMap<>();
+    this.namespaceQuotaSnapshots = new HashMap<>();
   }
 
   @Override
   protected void chore() {
     try {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Refreshing space quotas in RegionServer");
+      }
       _chore();
     } catch (IOException e) {
-      LOG.warn("Failed to process quota reports and update quota violation state. Will retry.", e);
+      LOG.warn("Failed to process quota reports and update quota state. Will retry.", e);
     }
   }
 
@@ -134,12 +136,12 @@ public class QuotaObserverChore extends ScheduledChore {
       LOG.trace("Using " + reportedRegionSpaceUse.size() + " region space use reports");
     }
 
-    // Create the stores to track table and namespace violations
-    initializeViolationStores(reportedRegionSpaceUse);
+    // Create the stores to track table and namespace snapshots
+    initializeSnapshotStores(reportedRegionSpaceUse);
 
     // Filter out tables for which we don't have adequate regionspace reports yet.
     // Important that we do this after we instantiate the stores above
-    tablesWithQuotas.filterInsufficientlyReportedTables(tableViolationStore);
+    tablesWithQuotas.filterInsufficientlyReportedTables(tableSnapshotStore);
 
     if (LOG.isTraceEnabled()) {
       LOG.trace("Filtered insufficiently reported tables, left with " +
@@ -158,18 +160,18 @@ public class QuotaObserverChore extends ScheduledChore {
     processNamespacesWithQuotas(namespacesWithQuotas, tablesByNamespace);
   }
 
-  void initializeViolationStores(Map<HRegionInfo,Long> regionSizes) {
+  void initializeSnapshotStores(Map<HRegionInfo,Long> regionSizes) {
     Map<HRegionInfo,Long> immutableRegionSpaceUse = Collections.unmodifiableMap(regionSizes);
-    if (null == tableViolationStore) {
-      tableViolationStore = new TableQuotaViolationStore(conn, this, immutableRegionSpaceUse);
+    if (null == tableSnapshotStore) {
+      tableSnapshotStore = new TableQuotaSnapshotStore(conn, this, immutableRegionSpaceUse);
     } else {
-      tableViolationStore.setRegionUsage(immutableRegionSpaceUse);
+      tableSnapshotStore.setRegionUsage(immutableRegionSpaceUse);
     }
-    if (null == namespaceViolationStore) {
-      namespaceViolationStore = new NamespaceQuotaViolationStore(
+    if (null == namespaceSnapshotStore) {
+      namespaceSnapshotStore = new NamespaceQuotaSnapshotStore(
           conn, this, immutableRegionSpaceUse);
     } else {
-      namespaceViolationStore.setRegionUsage(immutableRegionSpaceUse);
+      namespaceSnapshotStore.setRegionUsage(immutableRegionSpaceUse);
     }
   }
 
@@ -181,7 +183,7 @@ public class QuotaObserverChore extends ScheduledChore {
    */
   void processTablesWithQuotas(final Set<TableName> tablesWithTableQuotas) throws IOException {
     for (TableName table : tablesWithTableQuotas) {
-      final SpaceQuota spaceQuota = tableViolationStore.getSpaceQuota(table);
+      final SpaceQuota spaceQuota = tableSnapshotStore.getSpaceQuota(table);
       if (null == spaceQuota) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Unexpectedly did not find a space quota for " + table
@@ -189,32 +191,12 @@ public class QuotaObserverChore extends ScheduledChore {
         }
         continue;
       }
-      final ViolationState currentState = tableViolationStore.getCurrentState(table);
-      final ViolationState targetState = tableViolationStore.getTargetState(table, spaceQuota);
-
-      if (currentState == ViolationState.IN_VIOLATION) {
-        if (targetState == ViolationState.IN_OBSERVANCE) {
-          LOG.info(table + " moving into observance of table space quota.");
-          transitionTableToObservance(table);
-          tableViolationStore.setCurrentState(table, ViolationState.IN_OBSERVANCE);
-        } else if (targetState == ViolationState.IN_VIOLATION) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(table + " remains in violation of quota.");
-          }
-          tableViolationStore.setCurrentState(table, ViolationState.IN_VIOLATION);
-        }
-      } else if (currentState == ViolationState.IN_OBSERVANCE) {
-        if (targetState == ViolationState.IN_VIOLATION) {
-          LOG.info(table + " moving into violation of table space quota.");
-          transitionTableToViolation(table, getViolationPolicy(spaceQuota));
-          tableViolationStore.setCurrentState(table, ViolationState.IN_VIOLATION);
-        } else if (targetState == ViolationState.IN_OBSERVANCE) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(table + " remains in observance of quota.");
-          }
-          tableViolationStore.setCurrentState(table, ViolationState.IN_OBSERVANCE);
-        }
+      final SpaceQuotaSnapshot currentSnapshot = tableSnapshotStore.getCurrentState(table);
+      final SpaceQuotaSnapshot targetSnapshot = tableSnapshotStore.getTargetState(table, spaceQuota);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Processing " + table + " with current=" + currentSnapshot + ", target=" + targetSnapshot);
       }
+      updateTableQuota(table, currentSnapshot, targetSnapshot);
     }
   }
 
@@ -233,7 +215,7 @@ public class QuotaObserverChore extends ScheduledChore {
       final Multimap<String,TableName> tablesByNamespace) throws IOException {
     for (String namespace : namespacesWithQuotas) {
       // Get the quota definition for the namespace
-      final SpaceQuota spaceQuota = namespaceViolationStore.getSpaceQuota(namespace);
+      final SpaceQuota spaceQuota = namespaceSnapshotStore.getSpaceQuota(namespace);
       if (null == spaceQuota) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Could not get Namespace space quota for " + namespace
@@ -241,50 +223,117 @@ public class QuotaObserverChore extends ScheduledChore {
         }
         continue;
       }
-      final ViolationState currentState = namespaceViolationStore.getCurrentState(namespace);
-      final ViolationState targetState = namespaceViolationStore.getTargetState(namespace, spaceQuota);
-      // When in observance, check if we need to move to violation.
-      if (ViolationState.IN_OBSERVANCE == currentState) {
-        if (ViolationState.IN_VIOLATION == targetState) {
-          for (TableName tableInNS : tablesByNamespace.get(namespace)) {
-            if (ViolationState.IN_VIOLATION == tableViolationStore.getCurrentState(tableInNS)) {
-              // Table-level quota violation policy is being applied here.
-              if (LOG.isTraceEnabled()) {
-                LOG.trace("Not activating Namespace violation policy because Table violation"
-                    + " policy is already in effect for " + tableInNS);
-              }
-              continue;
-            } else {
-              LOG.info(tableInNS + " moving into violation of namespace space quota");
-              transitionTableToViolation(tableInNS, getViolationPolicy(spaceQuota));
+      final SpaceQuotaSnapshot currentSnapshot = namespaceSnapshotStore.getCurrentState(namespace);
+      final SpaceQuotaSnapshot targetSnapshot = namespaceSnapshotStore.getTargetState(namespace, spaceQuota);
+      updateNamespaceQuota(namespace, currentSnapshot, targetSnapshot, tablesByNamespace);
+    }
+  }
+
+  /**
+   * Updates the hbase:quota table with the new quota policy for this <code>table</code>
+   * if necessary.
+   *
+   * @param table The table being checked
+   * @param currentSnapshot The state of the quota on this table from the previous invocation.
+   * @param targetSnapshot The state the quota should be in for this table.
+   */
+  void updateTableQuota(
+      TableName table, SpaceQuotaSnapshot currentSnapshot, SpaceQuotaSnapshot targetSnapshot)
+          throws IOException {
+    final SpaceQuotaStatus currentStatus = currentSnapshot.getQuotaStatus();
+    final SpaceQuotaStatus targetStatus = targetSnapshot.getQuotaStatus();
+
+    // If we're changing something, log it.
+    if (!currentSnapshot.equals(targetSnapshot)) {
+      // If the target is none, we're moving out of violation. Update the hbase:quota table
+      if (!targetStatus.isInViolation()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(table + " moving into observance of table space quota.");
+        }
+      } else if (LOG.isDebugEnabled()) {
+        // We're either moving into violation or changing violation policies
+        LOG.debug(table + " moving into violation of table space quota with policy of " + targetStatus.getPolicy());
+      }
+
+      this.snapshotNotifier.transitionTable(table, targetSnapshot);
+      // Update it in memory
+      tableSnapshotStore.setCurrentState(table, targetSnapshot);
+    } else if (LOG.isTraceEnabled()) {
+      // Policies are the same, so we have nothing to do except log this. Don't need to re-update the quota table
+      if (!currentStatus.isInViolation()) {
+        LOG.trace(table + " remains in observance of quota.");
+      } else {
+        LOG.trace(table + " remains in violation of quota.");
+      }
+    }
+  }
+
+  /**
+   * Updates the hbase:quota table with the target quota policy for this <code>namespace</code>
+   * if necessary.
+   *
+   * @param namespace The namespace being checked
+   * @param currentSnapshot The state of the quota on this namespace from the previous invocation
+   * @param targetSnapshot The state the quota should be in for this namespace
+   * @param tablesByNamespace A mapping of tables in namespaces.
+   */
+  void updateNamespaceQuota(
+      String namespace, SpaceQuotaSnapshot currentSnapshot, SpaceQuotaSnapshot targetSnapshot,
+      final Multimap<String,TableName> tablesByNamespace) throws IOException {
+    final SpaceQuotaStatus targetStatus = targetSnapshot.getQuotaStatus();
+
+    // When the policies differ, we need to move into or out of violatino
+    if (!currentSnapshot.equals(targetSnapshot)) {
+      // We want to have a policy of "NONE", moving out of violation
+      if (!targetStatus.isInViolation()) {
+        for (TableName tableInNS : tablesByNamespace.get(namespace)) {
+          if (!tableSnapshotStore.getCurrentState(tableInNS).getQuotaStatus().isInViolation()) {
+            // Table-level quota violation policy is being applied here.
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Not activating Namespace violation policy because a Table violation"
+                  + " policy is already in effect for " + tableInNS);
             }
-          }
-        } else {
-          // still in observance
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(namespace + " remains in observance of quota.");
+          } else {
+            LOG.info(tableInNS + " moving into observance of namespace space quota");
+            this.snapshotNotifier.transitionTable(tableInNS, targetSnapshot);
           }
         }
-      } else if (ViolationState.IN_VIOLATION == currentState) {
-        // When in violation, check if we need to move to observance.
-        if (ViolationState.IN_OBSERVANCE == targetState) {
-          for (TableName tableInNS : tablesByNamespace.get(namespace)) {
-            if (ViolationState.IN_VIOLATION == tableViolationStore.getCurrentState(tableInNS)) {
-              // Table-level quota violation policy is being applied here.
-              if (LOG.isTraceEnabled()) {
-                LOG.trace("Not activating Namespace violation policy because Table violation"
-                    + " policy is already in effect for " + tableInNS);
-              }
-              continue;
-            } else {
-              LOG.info(tableInNS + " moving into observance of namespace space quota");
-              transitionTableToObservance(tableInNS);
+      } else {
+        // Moving tables in the namespace into violation or to a different violation policy
+        for (TableName tableInNS : tablesByNamespace.get(namespace)) {
+          if (tableSnapshotStore.getCurrentState(tableInNS).getQuotaStatus().isInViolation()) {
+            // Table-level quota violation policy is being applied here.
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Not activating Namespace violation policy because a Table violation"
+                  + " policy is already in effect for " + tableInNS);
             }
+          } else {
+            LOG.info(tableInNS + " moving into violation of namespace space quota with policy " + targetStatus.getPolicy());
+            this.snapshotNotifier.transitionTable(tableInNS, targetSnapshot);
           }
-        } else {
-          // Remains in violation
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(namespace + " remains in violation of quota.");
+        }
+      }
+    } else {
+      // Policies are the same
+      if (!targetStatus.isInViolation()) {
+        // Both are NONE, so we remain in observance
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(namespace + " remains in observance of quota.");
+        }
+      } else {
+        // Namespace quota is still in violation, need to enact if the table quota is not taking priority.
+        for (TableName tableInNS : tablesByNamespace.get(namespace)) {
+          // Does a table policy exist
+          if (tableSnapshotStore.getCurrentState(tableInNS).getQuotaStatus().isInViolation()) {
+            // Table-level quota violation policy is being applied here.
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Not activating Namespace violation policy because Table violation"
+                  + " policy is already in effect for " + tableInNS);
+            }
+          } else {
+            // No table policy, so enact namespace policy
+            LOG.info(tableInNS + " moving into violation of namespace space quota");
+            this.snapshotNotifier.transitionTable(tableInNS, targetSnapshot);
           }
         }
       }
@@ -340,39 +389,24 @@ public class QuotaObserverChore extends ScheduledChore {
   }
 
   @VisibleForTesting
-  QuotaViolationStore<TableName> getTableViolationStore() {
-    return tableViolationStore;
+  QuotaSnapshotStore<TableName> getTableSnapshotStore() {
+    return tableSnapshotStore;
   }
 
   @VisibleForTesting
-  QuotaViolationStore<String> getNamespaceViolationStore() {
-    return namespaceViolationStore;
+  QuotaSnapshotStore<String> getNamespaceSnapshotStore() {
+    return namespaceSnapshotStore;
   }
 
   /**
-   * Transitions the given table to violation of its quota, enabling the violation policy.
+   * Fetches the {@link SpaceQuotaSnapshot} for the given table.
    */
-  private void transitionTableToViolation(TableName table, SpaceViolationPolicy violationPolicy)
-      throws IOException {
-    this.violationNotifier.transitionTableToViolation(table, violationPolicy);
-  }
-
-  /**
-   * Transitions the given table to observance of its quota, disabling the violation policy.
-   */
-  private void transitionTableToObservance(TableName table) throws IOException {
-    this.violationNotifier.transitionTableToObservance(table);
-  }
-
-  /**
-   * Fetch the {@link ViolationState} for the given table.
-   */
-  ViolationState getTableQuotaViolation(TableName table) {
+  SpaceQuotaSnapshot getTableQuotaSnapshot(TableName table) {
     // TODO Can one instance of a Chore be executed concurrently?
-    ViolationState state = this.tableQuotaViolationStates.get(table);
+    SpaceQuotaSnapshot state = this.tableQuotaSnapshots.get(table);
     if (null == state) {
       // No tracked state implies observance.
-      return ViolationState.IN_OBSERVANCE;
+      return QuotaSnapshotStore.NO_QUOTA;
     }
     return state;
   }
@@ -380,19 +414,19 @@ public class QuotaObserverChore extends ScheduledChore {
   /**
    * Stores the quota violation state for the given table.
    */
-  void setTableQuotaViolation(TableName table, ViolationState state) {
-    this.tableQuotaViolationStates.put(table, state);
+  void setTableQuotaViolation(TableName table, SpaceQuotaSnapshot snapshot) {
+    this.tableQuotaSnapshots.put(table, snapshot);
   }
 
   /**
-   * Fetches the {@link ViolationState} for the given namespace.
+   * Fetches the {@link SpaceQuotaSnapshot} for the given namespace.
    */
-  ViolationState getNamespaceQuotaViolation(String namespace) {
+  SpaceQuotaSnapshot getNamespaceQuotaSnapshot(String namespace) {
     // TODO Can one instance of a Chore be executed concurrently?
-    ViolationState state = this.namespaceQuotaViolationStates.get(namespace);
+    SpaceQuotaSnapshot state = this.namespaceQuotaSnapshots.get(namespace);
     if (null == state) {
       // No tracked state implies observance.
-      return ViolationState.IN_OBSERVANCE;
+      return QuotaSnapshotStore.NO_QUOTA;
     }
     return state;
   }
@@ -400,20 +434,8 @@ public class QuotaObserverChore extends ScheduledChore {
   /**
    * Stores the quota violation state for the given namespace.
    */
-  void setNamespaceQuotaViolation(String namespace, ViolationState state) {
-    this.namespaceQuotaViolationStates.put(namespace, state);
-  }
-
-  /**
-   * Extracts the {@link SpaceViolationPolicy} from the serialized {@link Quotas} protobuf.
-   * @throws IllegalArgumentException If the SpaceQuota lacks a ViolationPolicy
-   */
-  SpaceViolationPolicy getViolationPolicy(SpaceQuota spaceQuota) {
-    if (!spaceQuota.hasViolationPolicy()) {
-      throw new IllegalArgumentException("SpaceQuota had no associated violation policy: "
-          + spaceQuota);
-    }
-    return ProtobufUtil.toViolationPolicy(spaceQuota.getViolationPolicy());
+  void setNamespaceQuotaSnapshot(String namespace, SpaceQuotaSnapshot snapshot) {
+    this.namespaceQuotaSnapshots.put(namespace, snapshot);
   }
 
   /**
@@ -423,8 +445,8 @@ public class QuotaObserverChore extends ScheduledChore {
    * @return The configured chore period or the default value.
    */
   static int getPeriod(Configuration conf) {
-    return conf.getInt(VIOLATION_OBSERVER_CHORE_PERIOD_KEY,
-        VIOLATION_OBSERVER_CHORE_PERIOD_DEFAULT);
+    return conf.getInt(QUOTA_OBSERVER_CHORE_PERIOD_KEY,
+        QUOTA_OBSERVER_CHORE_PERIOD_DEFAULT);
   }
 
   /**
@@ -434,21 +456,21 @@ public class QuotaObserverChore extends ScheduledChore {
    * @return The configured chore initial delay or the default value.
    */
   static long getInitialDelay(Configuration conf) {
-    return conf.getLong(VIOLATION_OBSERVER_CHORE_DELAY_KEY,
-        VIOLATION_OBSERVER_CHORE_DELAY_DEFAULT);
+    return conf.getLong(QUOTA_OBSERVER_CHORE_DELAY_KEY,
+        QUOTA_OBSERVER_CHORE_DELAY_DEFAULT);
   }
 
   /**
    * Extracts the time unit for the chore period and initial delay from the configuration. The
-   * configuration value for {@link #VIOLATION_OBSERVER_CHORE_TIMEUNIT_KEY} must correspond to
+   * configuration value for {@link #QUOTA_OBSERVER_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(VIOLATION_OBSERVER_CHORE_TIMEUNIT_KEY,
-        VIOLATION_OBSERVER_CHORE_TIMEUNIT_DEFAULT));
+    return TimeUnit.valueOf(conf.get(QUOTA_OBSERVER_CHORE_TIMEUNIT_KEY,
+        QUOTA_OBSERVER_CHORE_TIMEUNIT_DEFAULT));
   }
 
   /**
@@ -459,8 +481,8 @@ public class QuotaObserverChore extends ScheduledChore {
    * @return The percent of regions reported to use.
    */
   static Double getRegionReportPercent(Configuration conf) {
-    return conf.getDouble(VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_KEY,
-        VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT);
+    return conf.getDouble(QUOTA_OBSERVER_CHORE_REPORT_PERCENT_KEY,
+        QUOTA_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT);
   }
 
   /**
@@ -549,7 +571,7 @@ public class QuotaObserverChore extends ScheduledChore {
      * Filters out all tables for which the Master currently doesn't have enough region space
      * reports received from RegionServers yet.
      */
-    public void filterInsufficientlyReportedTables(QuotaViolationStore<TableName> tableStore)
+    public void filterInsufficientlyReportedTables(QuotaSnapshotStore<TableName> tableStore)
         throws IOException {
       final double percentRegionsReportedThreshold = getRegionReportPercent(getConfiguration());
       Set<TableName> tablesToRemove = new HashSet<>();
@@ -572,12 +594,12 @@ public class QuotaObserverChore extends ScheduledChore {
         if (ratioReported < percentRegionsReportedThreshold) {
           if (LOG.isTraceEnabled()) {
             LOG.trace("Filtering " + table + " because " + reportedRegionsInQuota  + " of " +
-                numRegionsInTable + " were reported.");
+                numRegionsInTable + " regions were reported.");
           }
           tablesToRemove.add(table);
         } else if (LOG.isTraceEnabled()) {
           LOG.trace("Retaining " + table + " because " + reportedRegionsInQuota  + " of " +
-              numRegionsInTable + " were reported.");
+              numRegionsInTable + " regions were reported.");
         }
       }
       for (TableName tableToRemove : tablesToRemove) {
@@ -600,7 +622,7 @@ public class QuotaObserverChore extends ScheduledChore {
     /**
      * Computes the number of regions reported for a table.
      */
-    int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore)
+    int getNumReportedRegions(TableName table, QuotaSnapshotStore<TableName> tableStore)
         throws IOException {
       return Iterables.size(tableStore.filterBySubject(table));
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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
new file mode 100644
index 0000000..8b0b3a7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java
@@ -0,0 +1,96 @@
+/*
+ * 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.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+
+/**
+ * A common interface for computing and storing space quota observance/violation for entities.
+ *
+ * An entity is presently a table or a namespace.
+ */
+@InterfaceAudience.Private
+public interface QuotaSnapshotStore<T> {
+
+  /**
+   * The current state of a table with respect to the policy set forth by a quota.
+   */
+  @InterfaceAudience.Private
+  public enum ViolationState {
+    IN_VIOLATION,
+    IN_OBSERVANCE,
+  }
+
+  /**
+   * Singleton to represent a table without a quota defined. It is never in violation.
+   */
+  public static final SpaceQuotaSnapshot NO_QUOTA = new SpaceQuotaSnapshot(
+      SpaceQuotaStatus.notInViolation(), -1, -1);
+
+  /**
+   * Fetch the Quota for the given {@code subject}. May be null.
+   *
+   * @param subject The object for which the quota should be fetched
+   */
+  SpaceQuota getSpaceQuota(T subject) throws IOException;
+
+  /**
+   * Returns the current {@link SpaceQuotaSnapshot} for the given {@code subject}.
+   *
+   * @param subject The object which the quota snapshot should be fetched
+   */
+  SpaceQuotaSnapshot getCurrentState(T subject);
+
+  /**
+   * Computes the target {@link SpaceQuotaSnapshot} for the given {@code subject} and
+   * {@code spaceQuota}.
+   *
+   * @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);
+
+  /**
+   * Filters the provided <code>regions</code>, returning those which match the given
+   * <code>subject</code>.
+   *
+   * @param subject The filter criteria. Only regions belonging to this parameter will be returned
+   */
+  Iterable<Entry<HRegionInfo,Long>> filterBySubject(T subject);
+
+  /**
+   * Persists the current {@link SpaceQuotaSnapshot} for the {@code subject}.
+   *
+   * @param subject The object which the {@link SpaceQuotaSnapshot} is being persisted for
+   * @param state The current state of the {@code subject}
+   */
+  void setCurrentState(T subject, SpaceQuotaSnapshot state);
+
+  /**
+   * Updates {@code this} with the latest snapshot of filesystem use by region.
+   *
+   * @param regionUsage A map of {@code HRegionInfo} objects to their filesystem usage in bytes
+   */
+  void setRegionUsage(Map<HRegionInfo,Long> regionUsage);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaViolationStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaViolationStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaViolationStore.java
deleted file mode 100644
index 381ac8e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaViolationStore.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.quotas;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
-
-/**
- * A common interface for computing and storing space quota observance/violation for entities.
- *
- * An entity is presently a table or a namespace.
- */
-@InterfaceAudience.Private
-public interface QuotaViolationStore<T> {
-
-  /**
-   * The current state of a table with respect to the policy set forth by a quota.
-   */
-  @InterfaceAudience.Private
-  public enum ViolationState {
-    IN_VIOLATION,
-    IN_OBSERVANCE,
-  }
-
-  /**
-   * Fetch the Quota for the given {@code subject}. May be null.
-   *
-   * @param subject The object for which the quota should be fetched
-   */
-  SpaceQuota getSpaceQuota(T subject) throws IOException;
-
-  /**
-   * Returns the current {@link ViolationState} for the given {@code subject}.
-   *
-   * @param subject The object which the quota violation state should be fetched
-   */
-  ViolationState getCurrentState(T subject);
-
-  /**
-   * Computes the target {@link ViolationState} for the given {@code subject} and
-   * {@code spaceQuota}.
-   *
-   * @param subject The object which to determine the target quota violation state of
-   * @param spaceQuota The quota "definition" for the {@code subject}
-   */
-  ViolationState getTargetState(T subject, SpaceQuota spaceQuota);
-
-  /**
-   * Filters the provided <code>regions</code>, returning those which match the given
-   * <code>subject</code>.
-   *
-   * @param subject The filter criteria. Only regions belonging to this parameter will be returned
-   */
-  Iterable<Entry<HRegionInfo,Long>> filterBySubject(T subject);
-
-  /**
-   * Persists the current {@link ViolationState} for the {@code subject}.
-   *
-   * @param subject The object which the {@link ViolationState} is being persisted for
-   * @param state The current {@link ViolationState} of the {@code subject}
-   */
-  void setCurrentState(T subject, ViolationState state);
-
-  /**
-   * Updates {@code this} with the latest snapshot of filesystem use by region.
-   *
-   * @param regionUsage A map of {@code HRegionInfo} objects to their filesystem usage in bytes
-   */
-  void setRegionUsage(Map<HRegionInfo,Long> regionUsage);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
index 9a8edb9..1c82808 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
@@ -20,24 +20,29 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.Map.Entry;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.annotations.VisibleForTesting;
 
 /**
  * A manager for filesystem space quotas in the RegionServer.
  *
- * This class is responsible for reading quota violation policies from the quota
- * table and then enacting them on the given table.
+ * This class is the centralized point for what a RegionServer knows about space quotas
+ * on tables. For each table, it tracks two different things: the {@link SpaceQuotaSnapshot}
+ * and a {@link SpaceViolationPolicyEnforcement} (which may be null when a quota is not
+ * being violated). Both of these are sensitive on when they were last updated. The
+ * {link SpaceQutoaViolationPolicyRefresherChore} periodically runs and updates
+ * the state on <code>this</code>.
  */
 @InterfaceAudience.Private
 public class RegionServerSpaceQuotaManager {
@@ -45,12 +50,23 @@ public class RegionServerSpaceQuotaManager {
 
   private final RegionServerServices rsServices;
 
-  private SpaceQuotaViolationPolicyRefresherChore spaceQuotaRefresher;
-  private Map<TableName,SpaceViolationPolicy> enforcedPolicies;
+  private SpaceQuotaRefresherChore spaceQuotaRefresher;
+  private AtomicReference<Map<TableName, SpaceQuotaSnapshot>> currentQuotaSnapshots;
   private boolean started = false;
+  private ConcurrentHashMap<TableName,SpaceViolationPolicyEnforcement> enforcedPolicies;
+  private SpaceViolationPolicyEnforcementFactory factory;
 
   public RegionServerSpaceQuotaManager(RegionServerServices rsServices) {
+    this(rsServices, SpaceViolationPolicyEnforcementFactory.getInstance());
+  }
+
+  @VisibleForTesting
+  RegionServerSpaceQuotaManager(
+      RegionServerServices rsServices, SpaceViolationPolicyEnforcementFactory factory) {
     this.rsServices = Objects.requireNonNull(rsServices);
+    this.factory = factory;
+    this.enforcedPolicies = new ConcurrentHashMap<>();
+    this.currentQuotaSnapshots = new AtomicReference<>(new HashMap<>());
   }
 
   public synchronized void start() throws IOException {
@@ -59,8 +75,12 @@ public class RegionServerSpaceQuotaManager {
       return;
     }
 
-    spaceQuotaRefresher = new SpaceQuotaViolationPolicyRefresherChore(this);
-    enforcedPolicies = new HashMap<>();
+    if (started) {
+      LOG.warn("RegionServerSpaceQuotaManager has already been started!");
+      return;
+    }
+    this.spaceQuotaRefresher = new SpaceQuotaRefresherChore(this, rsServices.getClusterConnection());
+    rsServices.getChoreService().scheduleChore(spaceQuotaRefresher);
     started = true;
   }
 
@@ -79,91 +99,136 @@ public class RegionServerSpaceQuotaManager {
     return started;
   }
 
-  Connection getConnection() {
-    return rsServices.getConnection();
+  /**
+   * Copies the last {@link SpaceQuotaSnapshot}s that were recorded. The current view
+   * of what the RegionServer thinks the table's utilization is.
+   */
+  public Map<TableName,SpaceQuotaSnapshot> copyQuotaSnapshots() {
+    return new HashMap<>(currentQuotaSnapshots.get());
   }
 
   /**
-   * Returns the collection of tables which have quota violation policies enforced on
-   * this RegionServer.
+   * Updates the current {@link SpaceQuotaSnapshot}s for the RegionServer.
+   *
+   * @param newSnapshots The space quota snapshots.
    */
-  public synchronized Map<TableName,SpaceViolationPolicy> getActiveViolationPolicyEnforcements()
-      throws IOException {
-    return new HashMap<>(this.enforcedPolicies);
+  public void updateQuotaSnapshot(Map<TableName,SpaceQuotaSnapshot> newSnapshots) {
+    currentQuotaSnapshots.set(Objects.requireNonNull(newSnapshots));
   }
 
   /**
-   * Wrapper around {@link QuotaTableUtil#extractViolationPolicy(Result, Map)} for testing.
+   * Creates an object well-suited for the RegionServer to use in verifying active policies.
    */
-  void extractViolationPolicy(Result result, Map<TableName,SpaceViolationPolicy> activePolicies) {
-    QuotaTableUtil.extractViolationPolicy(result, activePolicies);
+  public ActivePolicyEnforcement getActiveEnforcements() {
+    return new ActivePolicyEnforcement(copyActiveEnforcements(), copyQuotaSnapshots(), rsServices);
   }
 
   /**
-   * Reads all quota violation policies which are to be enforced from the quota table.
-   *
-   * @return The collection of tables which are in violation of their quota and the policy which
-   *    should be enforced.
+   * Converts a map of table to {@link SpaceViolationPolicyEnforcement}s into
+   * {@link SpaceViolationPolicy}s.
    */
-  public Map<TableName, SpaceViolationPolicy> getViolationPoliciesToEnforce() throws IOException {
-    try (Table quotaTable = getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME);
-        ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaViolationScan())) {
-      Map<TableName,SpaceViolationPolicy> activePolicies = new HashMap<>();
-      for (Result result : scanner) {
-        try {
-          extractViolationPolicy(result, activePolicies);
-        } catch (IllegalArgumentException e) {
-          final String msg = "Failed to parse result for row " + Bytes.toString(result.getRow());
-          LOG.error(msg, e);
-          throw new IOException(msg, e);
-        }
+  public Map<TableName, SpaceQuotaSnapshot> getActivePoliciesAsMap() {
+    final Map<TableName, SpaceViolationPolicyEnforcement> enforcements =
+        copyActiveEnforcements();
+    final Map<TableName, SpaceQuotaSnapshot> policies = new HashMap<>();
+    for (Entry<TableName, SpaceViolationPolicyEnforcement> entry : enforcements.entrySet()) {
+      final SpaceQuotaSnapshot snapshot = entry.getValue().getQuotaSnapshot();
+      if (null != snapshot) {
+        policies.put(entry.getKey(), snapshot);
       }
-      return activePolicies;
     }
+    return policies;
   }
 
   /**
    * Enforces the given violationPolicy on the given table in this RegionServer.
    */
-  synchronized void enforceViolationPolicy(
-      TableName tableName, SpaceViolationPolicy violationPolicy) {
+  public void enforceViolationPolicy(TableName tableName, SpaceQuotaSnapshot snapshot) {
+    SpaceQuotaStatus status = snapshot.getQuotaStatus();
+    if (!status.isInViolation()) {
+      throw new IllegalStateException(
+          tableName + " is not in violation. Violation policy should not be enabled.");
+    }
     if (LOG.isTraceEnabled()) {
       LOG.trace(
           "Enabling violation policy enforcement on " + tableName
-          + " with policy " + violationPolicy);
+          + " with policy " + status.getPolicy());
+    }
+    // Construct this outside of the lock
+    final SpaceViolationPolicyEnforcement enforcement = getFactory().create(
+        getRegionServerServices(), tableName, snapshot);
+    // "Enables" the policy
+    // TODO Should this synchronize on the actual table name instead of the map? That would allow
+    // policy enable/disable on different tables to happen concurrently. As written now, only one
+    // table will be allowed to transition at a time.
+    synchronized (enforcedPolicies) {
+      try {
+        enforcement.enable();
+      } catch (IOException e) {
+        LOG.error("Failed to enable space violation policy for " + tableName
+            + ". This table will not enter violation.", e);
+        return;
+      }
+      enforcedPolicies.put(tableName, enforcement);
     }
-    // Enact the policy
-    enforceOnRegionServer(tableName, violationPolicy);
-    // Publicize our enacting of the policy
-    enforcedPolicies.put(tableName, violationPolicy);
   }
 
   /**
-   * Enacts the given violation policy on this table in the RegionServer.
+   * Disables enforcement on any violation policy on the given <code>tableName</code>.
    */
-  void enforceOnRegionServer(TableName tableName, SpaceViolationPolicy violationPolicy) {
-    throw new UnsupportedOperationException("TODO");
+  public void disableViolationPolicyEnforcement(TableName tableName) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Disabling violation policy enforcement on " + tableName);
+    }
+    // "Disables" the policy
+    // TODO Should this synchronize on the actual table name instead of the map?
+    synchronized (enforcedPolicies) {
+      SpaceViolationPolicyEnforcement enforcement = enforcedPolicies.remove(tableName);
+      if (null != enforcement) {
+        try {
+          enforcement.disable();
+        } catch (IOException e) {
+          LOG.error("Failed to disable space violation policy for " + tableName
+              + ". This table will remain in violation.", e);
+          enforcedPolicies.put(tableName, enforcement);
+        }
+      }
+    }
   }
 
   /**
-   * Disables enforcement on any violation policy on the given <code>tableName</code>.
+   * Returns whether or not compactions should be disabled for the given <code>tableName</code> per
+   * a space quota violation policy. A convenience method.
+   *
+   * @param tableName The table to check
+   * @return True if compactions should be disabled for the table, false otherwise.
    */
-  synchronized void disableViolationPolicyEnforcement(TableName tableName) {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Disabling violation policy enforcement on " + tableName);
+  public boolean areCompactionsDisabled(TableName tableName) {
+    SpaceViolationPolicyEnforcement enforcement = this.enforcedPolicies.get(Objects.requireNonNull(tableName));
+    if (null != enforcement) {
+      return enforcement.areCompactionsDisabled();
     }
-    disableOnRegionServer(tableName);
-    enforcedPolicies.remove(tableName);
+    return false;
   }
 
   /**
-   * Disables any violation policy on this table in the RegionServer.
+   * Returns the collection of tables which have quota violation policies enforced on
+   * this RegionServer.
    */
-  void disableOnRegionServer(TableName tableName) {
-    throw new UnsupportedOperationException("TODO");
+  Map<TableName,SpaceViolationPolicyEnforcement> copyActiveEnforcements() {
+    // Allows reads to happen concurrently (or while the map is being updated)
+    return new HashMap<>(this.enforcedPolicies);
   }
 
   RegionServerServices getRegionServerServices() {
     return rsServices;
   }
+
+  Connection getConnection() {
+    return rsServices.getConnection();
+  }
+
+  SpaceViolationPolicyEnforcementFactory getFactory() {
+    return factory;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitingException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitingException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitingException.java
new file mode 100644
index 0000000..904903f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitingException.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * An Exception that is thrown when a space quota is in violation.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class SpaceLimitingException extends QuotaExceededException {
+  private static final long serialVersionUID = 2319438922387583600L;
+  private static final Log LOG = LogFactory.getLog(SpaceLimitingException.class);
+  private static final String MESSAGE_PREFIX = SpaceLimitingException.class.getName() + ": ";
+
+  private final String policyName;
+
+  public SpaceLimitingException(String msg) {
+    super(parseMessage(msg));
+
+    // Hack around ResponseConverter expecting to invoke a single-arg String constructor
+    // on this class
+    if (null != msg) {
+      for (SpaceViolationPolicy definedPolicy : SpaceViolationPolicy.values()) {
+        if (msg.indexOf(definedPolicy.name()) != -1) {
+          policyName = definedPolicy.name();
+          return;
+        }
+      }
+    }
+    policyName = null;
+  }
+
+  public SpaceLimitingException(String policyName, String msg) {
+    super(msg);
+    this.policyName = policyName;
+  }
+
+  public SpaceLimitingException(String policyName, String msg, Throwable e) {
+    super(msg, e);
+    this.policyName = policyName;
+  }
+
+  /**
+   * Returns the violation policy in effect.
+   *
+   * @return The violation policy in effect.
+   */
+  public String getViolationPolicy() {
+    return this.policyName;
+  }
+
+  private static String parseMessage(String originalMessage) {
+    // Serialization of the exception places a duplicate class name. Try to strip that off if it
+    // exists. Best effort... Looks something like:
+    // "org.apache.hadoop.hbase.quotas.SpaceLimitingException: NO_INSERTS A Put is disallowed due
+    // to a space quota."
+    if (null != originalMessage && originalMessage.startsWith(MESSAGE_PREFIX)) {
+      // If it starts with the class name, rip off the policy too.
+      try {
+        int index = originalMessage.indexOf(' ', MESSAGE_PREFIX.length());
+        return originalMessage.substring(index + 1);
+      } catch (Exception e) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Failed to trim exception message", e);
+        }
+      }
+    }
+    return originalMessage;
+  }
+
+  @Override
+  public String getMessage() {
+    return (null == policyName ? "(unknown policy)" : policyName) + " " + super.getMessage();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
new file mode 100644
index 0000000..e1a2693
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ScheduledChore;
+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.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * A {@link ScheduledChore} which periodically updates the {@link RegionServerSpaceQuotaManager}
+ * with information from the hbase:quota.
+ */
+@InterfaceAudience.Private
+public class SpaceQuotaRefresherChore extends ScheduledChore {
+  private static final Log LOG = LogFactory.getLog(SpaceQuotaRefresherChore.class);
+
+  static final String POLICY_REFRESHER_CHORE_PERIOD_KEY =
+      "hbase.regionserver.quotas.policy.refresher.chore.period";
+  static final int POLICY_REFRESHER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
+
+  static final String POLICY_REFRESHER_CHORE_DELAY_KEY =
+      "hbase.regionserver.quotas.policy.refresher.chore.delay";
+  static final long POLICY_REFRESHER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
+
+  static final String POLICY_REFRESHER_CHORE_TIMEUNIT_KEY =
+      "hbase.regionserver.quotas.policy.refresher.chore.timeunit";
+  static final String POLICY_REFRESHER_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
+
+  static final String POLICY_REFRESHER_CHORE_REPORT_PERCENT_KEY =
+      "hbase.regionserver.quotas.policy.refresher.report.percent";
+  static final double POLICY_REFRESHER_CHORE_REPORT_PERCENT_DEFAULT= 0.95;
+
+  private final RegionServerSpaceQuotaManager manager;
+  private final Connection conn;
+
+  public SpaceQuotaRefresherChore(RegionServerSpaceQuotaManager manager, Connection conn) {
+    super(SpaceQuotaRefresherChore.class.getSimpleName(),
+        manager.getRegionServerServices(),
+        getPeriod(manager.getRegionServerServices().getConfiguration()),
+        getInitialDelay(manager.getRegionServerServices().getConfiguration()),
+        getTimeUnit(manager.getRegionServerServices().getConfiguration()));
+    this.manager = manager;
+    this.conn = conn;
+  }
+
+  @Override
+  protected void chore() {
+    try {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Reading current quota snapshots from hbase:quota.");
+      }
+      // Get the snapshots that the quota manager is currently aware of
+      final Map<TableName, SpaceQuotaSnapshot> currentSnapshots =
+          getManager().copyQuotaSnapshots();
+      // Read the new snapshots from the quota table
+      final Map<TableName, SpaceQuotaSnapshot> newSnapshots = fetchSnapshotsFromQuotaTable();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(currentSnapshots.size() + " table quota snapshots are collected, "
+            + "read " + newSnapshots.size() + " from the quota table.");
+      }
+      // Iterate over each new quota snapshot
+      for (Entry<TableName, SpaceQuotaSnapshot> entry : newSnapshots.entrySet()) {
+        final TableName tableName = entry.getKey();
+        final SpaceQuotaSnapshot newSnapshot = entry.getValue();
+        // May be null!
+        final SpaceQuotaSnapshot currentSnapshot = currentSnapshots.get(tableName);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(tableName + ": current=" + currentSnapshot + ", new=" + newSnapshot);
+        }
+        if (!newSnapshot.equals(currentSnapshot)) {
+          // We have a new snapshot. We might need to enforce it or disable the enforcement
+          if (!isInViolation(currentSnapshot) && newSnapshot.getQuotaStatus().isInViolation()) {
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Enabling " + newSnapshot + " on " + tableName);
+            }
+            getManager().enforceViolationPolicy(tableName, newSnapshot);
+          }
+          if (isInViolation(currentSnapshot) && !newSnapshot.getQuotaStatus().isInViolation()) {
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Removing quota violation policy on " + tableName);
+            }
+            getManager().disableViolationPolicyEnforcement(tableName);
+          }
+        }
+      }
+
+      // We're intentionally ignoring anything extra with the currentSnapshots. If we were missing
+      // information from the RegionServers to create an accurate SpaceQuotaSnapshot in the Master,
+      // the Master will generate a new SpaceQuotaSnapshot which represents this state. This lets
+      // us avoid having to do anything special with currentSnapshots here.
+
+      // Update the snapshots in the manager
+      getManager().updateQuotaSnapshot(newSnapshots);
+    } catch (IOException e) {
+      LOG.warn(
+          "Caught exception while refreshing enforced quota violation policies, will retry.", e);
+    }
+  }
+
+  /**
+   * Checks if the given <code>snapshot</code> is in violation, allowing the snapshot to be null.
+   * If the snapshot is null, this is interpreted as no snapshot which implies not in violation.
+   *
+   * @param snapshot The snapshot to operate on.
+   * @return true if the snapshot is in violation, false otherwise.
+   */
+  boolean isInViolation(SpaceQuotaSnapshot snapshot) {
+    if (null == snapshot) {
+      return false;
+    }
+    return snapshot.getQuotaStatus().isInViolation();
+  }
+
+  /**
+   * Reads all quota snapshots from the quota table.
+   *
+   * @return The current "view" of space use by each table.
+   */
+  public Map<TableName, SpaceQuotaSnapshot> fetchSnapshotsFromQuotaTable() throws IOException {
+    try (Table quotaTable = getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME);
+        ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaSnapshotScan())) {
+      Map<TableName,SpaceQuotaSnapshot> snapshots = new HashMap<>();
+      for (Result result : scanner) {
+        try {
+          extractQuotaSnapshot(result, snapshots);
+        } catch (IllegalArgumentException e) {
+          final String msg = "Failed to parse result for row " + Bytes.toString(result.getRow());
+          LOG.error(msg, e);
+          throw new IOException(msg, e);
+        }
+      }
+      return snapshots;
+    }
+  }
+
+  /**
+   * Wrapper around {@link QuotaTableUtil#extractQuotaSnapshot(Result, Map)} for testing.
+   */
+  void extractQuotaSnapshot(Result result, Map<TableName,SpaceQuotaSnapshot> snapshots) {
+    QuotaTableUtil.extractQuotaSnapshot(result, snapshots);
+  }
+
+  Connection getConnection() {
+    return conn;
+  }
+
+  RegionServerSpaceQuotaManager getManager() {
+    return manager;
+  }
+
+  /**
+   * 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(POLICY_REFRESHER_CHORE_PERIOD_KEY,
+        POLICY_REFRESHER_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(POLICY_REFRESHER_CHORE_DELAY_KEY,
+        POLICY_REFRESHER_CHORE_DELAY_DEFAULT);
+  }
+
+  /**
+   * Extracts the time unit for the chore period and initial delay from the configuration. The
+   * configuration value for {@link #POLICY_REFRESHER_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(POLICY_REFRESHER_CHORE_TIMEUNIT_KEY,
+        POLICY_REFRESHER_CHORE_TIMEUNIT_DEFAULT));
+  }
+
+  /**
+   * Extracts the percent of Regions for a table to have been reported to enable quota violation
+   * state change.
+   *
+   * @param conf The configuration object.
+   * @return The percent of regions reported to use.
+   */
+  static Double getRegionReportPercent(Configuration conf) {
+    return conf.getDouble(POLICY_REFRESHER_CHORE_REPORT_PERCENT_KEY,
+        POLICY_REFRESHER_CHORE_REPORT_PERCENT_DEFAULT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifier.java
new file mode 100644
index 0000000..46e93c0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifier.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+
+/**
+ * An interface which abstract away the action taken to enable or disable
+ * a space quota violation policy across the HBase cluster. Implementations
+ * must have a no-args constructor.
+ */
+@InterfaceAudience.Private
+public interface SpaceQuotaSnapshotNotifier {
+
+  /**
+   * Initializes the notifier.
+   */
+  void initialize(Connection conn);
+
+  /**
+   * Informs the cluster of the current state of a space quota for a table.
+   *
+   * @param tableName The name of the table.
+   * @param snapshot The details of the space quota utilization.
+   */
+  void transitionTable(TableName tableName, SpaceQuotaSnapshot snapshot) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifierFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifierFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifierFactory.java
new file mode 100644
index 0000000..cb34529
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifierFactory.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.util.Objects;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Factory for creating {@link SpaceQuotaSnapshotNotifier} implementations. Implementations
+ * must have a no-args constructor.
+ */
+@InterfaceAudience.Private
+public class SpaceQuotaSnapshotNotifierFactory {
+  private static final SpaceQuotaSnapshotNotifierFactory INSTANCE =
+      new SpaceQuotaSnapshotNotifierFactory();
+
+  public static final String SNAPSHOT_NOTIFIER_KEY = "hbase.master.quota.snapshot.notifier.impl";
+  public static final Class<? extends SpaceQuotaSnapshotNotifier> SNAPSHOT_NOTIFIER_DEFAULT =
+      TableSpaceQuotaSnapshotNotifier.class;
+
+  // Private
+  private SpaceQuotaSnapshotNotifierFactory() {}
+
+  public static SpaceQuotaSnapshotNotifierFactory getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Instantiates the {@link SpaceQuotaSnapshotNotifier} implementation as defined in the
+   * configuration provided.
+   *
+   * @param conf Configuration object
+   * @return The SpaceQuotaSnapshotNotifier implementation
+   * @throws IllegalArgumentException if the class could not be instantiated
+   */
+  public SpaceQuotaSnapshotNotifier create(Configuration conf) {
+    Class<? extends SpaceQuotaSnapshotNotifier> clz = Objects.requireNonNull(conf)
+        .getClass(SNAPSHOT_NOTIFIER_KEY, SNAPSHOT_NOTIFIER_DEFAULT,
+            SpaceQuotaSnapshotNotifier.class);
+    try {
+      return clz.newInstance();
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new IllegalArgumentException("Failed to instantiate the implementation", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
deleted file mode 100644
index 261dea7..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.quotas;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Connection;
-
-/**
- * An interface which abstract away the action taken to enable or disable
- * a space quota violation policy across the HBase cluster. Implementations
- * must have a no-args constructor.
- */
-@InterfaceAudience.Private
-public interface SpaceQuotaViolationNotifier {
-
-  /**
-   * Initializes the notifier.
-   */
-  void initialize(Connection conn);
-
-  /**
-   * Instructs the cluster that the given table is in violation of a space quota. The
-   * provided violation policy is the action which should be taken on the table.
-   *
-   * @param tableName The name of the table in violation of the quota.
-   * @param violationPolicy The policy which should be enacted on the table.
-   */
-  void transitionTableToViolation(
-      TableName tableName, SpaceViolationPolicy violationPolicy) throws IOException;
-
-  /**
-   * Instructs the cluster that the given table is in observance of any applicable space quota.
-   *
-   * @param tableName The name of the table in observance.
-   */
-  void transitionTableToObservance(TableName tableName) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java
deleted file mode 100644
index 43f5513..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierFactory.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.quotas;
-
-import java.util.Objects;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Factory for creating {@link SpaceQuotaViolationNotifier} implementations. Implementations
- * must have a no-args constructor.
- */
-@InterfaceAudience.Private
-public class SpaceQuotaViolationNotifierFactory {
-  private static final SpaceQuotaViolationNotifierFactory INSTANCE =
-      new SpaceQuotaViolationNotifierFactory();
-
-  public static final String VIOLATION_NOTIFIER_KEY = "hbase.master.quota.violation.notifier.impl";
-  public static final Class<? extends SpaceQuotaViolationNotifier> VIOLATION_NOTIFIER_DEFAULT =
-      SpaceQuotaViolationNotifierForTest.class;
-
-  // Private
-  private SpaceQuotaViolationNotifierFactory() {}
-
-  public static SpaceQuotaViolationNotifierFactory getInstance() {
-    return INSTANCE;
-  }
-
-  /**
-   * Instantiates the {@link SpaceQuotaViolationNotifier} implementation as defined in the
-   * configuration provided.
-   *
-   * @param conf Configuration object
-   * @return The SpaceQuotaViolationNotifier implementation
-   * @throws IllegalArgumentException if the class could not be instantiated
-   */
-  public SpaceQuotaViolationNotifier create(Configuration conf) {
-    Class<? extends SpaceQuotaViolationNotifier> clz = Objects.requireNonNull(conf)
-        .getClass(VIOLATION_NOTIFIER_KEY, VIOLATION_NOTIFIER_DEFAULT,
-            SpaceQuotaViolationNotifier.class);
-    try {
-      return clz.newInstance();
-    } catch (InstantiationException | IllegalAccessException e) {
-      throw new IllegalArgumentException("Failed to instantiate the implementation", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
deleted file mode 100644
index 65dc979..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.quotas;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Connection;
-
-/**
- * A SpaceQuotaViolationNotifier implementation for verifying testing.
- */
-@InterfaceAudience.Private
-public class SpaceQuotaViolationNotifierForTest implements SpaceQuotaViolationNotifier {
-
-  private final Map<TableName,SpaceViolationPolicy> tablesInViolation = new HashMap<>();
-
-  @Override
-  public void initialize(Connection conn) {}
-
-  @Override
-  public void transitionTableToViolation(TableName tableName, SpaceViolationPolicy violationPolicy) {
-    tablesInViolation.put(tableName, violationPolicy);
-  }
-
-  @Override
-  public void transitionTableToObservance(TableName tableName) {
-    tablesInViolation.remove(tableName);
-  }
-
-  public Map<TableName,SpaceViolationPolicy> snapshotTablesInViolation() {
-    return new HashMap<>(this.tablesInViolation);
-  }
-
-  public void clearTableViolations() {
-    this.tablesInViolation.clear();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java
deleted file mode 100644
index 778ea0b..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationPolicyRefresherChore.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.quotas;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ScheduledChore;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * A {@link ScheduledChore} which periodically updates a local copy of tables which have
- * space quota violation policies enacted on them.
- */
-@InterfaceAudience.Private
-public class SpaceQuotaViolationPolicyRefresherChore extends ScheduledChore {
-  private static final Log LOG = LogFactory.getLog(SpaceQuotaViolationPolicyRefresherChore.class);
-
-  static final String POLICY_REFRESHER_CHORE_PERIOD_KEY =
-      "hbase.regionserver.quotas.policy.refresher.chore.period";
-  static final int POLICY_REFRESHER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
-
-  static final String POLICY_REFRESHER_CHORE_DELAY_KEY =
-      "hbase.regionserver.quotas.policy.refresher.chore.delay";
-  static final long POLICY_REFRESHER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
-
-  static final String POLICY_REFRESHER_CHORE_TIMEUNIT_KEY =
-      "hbase.regionserver.quotas.policy.refresher.chore.timeunit";
-  static final String POLICY_REFRESHER_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
-
-  static final String POLICY_REFRESHER_CHORE_REPORT_PERCENT_KEY =
-      "hbase.regionserver.quotas.policy.refresher.report.percent";
-  static final double POLICY_REFRESHER_CHORE_REPORT_PERCENT_DEFAULT= 0.95;
-
-  private final RegionServerSpaceQuotaManager manager;
-
-  public SpaceQuotaViolationPolicyRefresherChore(RegionServerSpaceQuotaManager manager) {
-    super(SpaceQuotaViolationPolicyRefresherChore.class.getSimpleName(),
-        manager.getRegionServerServices(),
-        getPeriod(manager.getRegionServerServices().getConfiguration()),
-        getInitialDelay(manager.getRegionServerServices().getConfiguration()),
-        getTimeUnit(manager.getRegionServerServices().getConfiguration()));
-    this.manager = manager;
-  }
-
-  @Override
-  protected void chore() {
-    // Tables with a policy currently enforced
-    final Map<TableName, SpaceViolationPolicy> activeViolationPolicies;
-    // Tables with policies that should be enforced
-    final Map<TableName, SpaceViolationPolicy> violationPolicies;
-    try {
-      // Tables with a policy currently enforced
-      activeViolationPolicies = manager.getActiveViolationPolicyEnforcements();
-      // Tables with policies that should be enforced
-      violationPolicies = manager.getViolationPoliciesToEnforce();
-    } catch (IOException e) {
-      LOG.warn("Failed to fetch enforced quota violation policies, will retry.", e);
-      return;
-    }
-    // Ensure each policy which should be enacted is enacted.
-    for (Entry<TableName, SpaceViolationPolicy> entry : violationPolicies.entrySet()) {
-      final TableName tableName = entry.getKey();
-      final SpaceViolationPolicy policyToEnforce = entry.getValue();
-      final SpaceViolationPolicy currentPolicy = activeViolationPolicies.get(tableName);
-      if (currentPolicy != policyToEnforce) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Enabling " + policyToEnforce + " on " + tableName);
-        }
-        manager.enforceViolationPolicy(tableName, policyToEnforce);
-      }
-    }
-    // Remove policies which should no longer be enforced
-    Iterator<TableName> iter = activeViolationPolicies.keySet().iterator();
-    while (iter.hasNext()) {
-      final TableName localTableWithPolicy = iter.next();
-      if (!violationPolicies.containsKey(localTableWithPolicy)) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Removing quota violation policy on " + localTableWithPolicy);
-        }
-        manager.disableViolationPolicyEnforcement(localTableWithPolicy);
-        iter.remove();
-      }
-    }
-  }
-
-  /**
-   * 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(POLICY_REFRESHER_CHORE_PERIOD_KEY,
-        POLICY_REFRESHER_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(POLICY_REFRESHER_CHORE_DELAY_KEY,
-        POLICY_REFRESHER_CHORE_DELAY_DEFAULT);
-  }
-
-  /**
-   * Extracts the time unit for the chore period and initial delay from the configuration. The
-   * configuration value for {@link #POLICY_REFRESHER_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(POLICY_REFRESHER_CHORE_TIMEUNIT_KEY,
-        POLICY_REFRESHER_CHORE_TIMEUNIT_DEFAULT));
-  }
-
-  /**
-   * Extracts the percent of Regions for a table to have been reported to enable quota violation
-   * state change.
-   *
-   * @param conf The configuration object.
-   * @return The percent of regions reported to use.
-   */
-  static Double getRegionReportPercent(Configuration conf) {
-    return conf.getDouble(POLICY_REFRESHER_CHORE_REPORT_PERCENT_KEY,
-        POLICY_REFRESHER_CHORE_REPORT_PERCENT_DEFAULT);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcement.java
new file mode 100644
index 0000000..34b88e5
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcement.java
@@ -0,0 +1,91 @@
+/*
+ * 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.List;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+
+/**
+ * RegionServer implementation of {@link SpaceViolationPolicy}.
+ *
+ * Implementations must have a public, no-args constructor.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface SpaceViolationPolicyEnforcement {
+
+  /**
+   * Initializes this policy instance.
+   */
+  void initialize(RegionServerServices rss, TableName tableName, SpaceQuotaSnapshot snapshot);
+
+  /**
+   * Enables this policy. Not all policies have enable actions.
+   */
+  void enable() throws IOException;
+
+  /**
+   * Disables this policy. Not all policies have disable actions.
+   */
+  void disable() throws IOException;
+
+  /**
+   * Checks the given {@link Mutation} against <code>this</code> policy. If the
+   * {@link Mutation} violates the policy, this policy should throw a
+   * {@link SpaceLimitingException}.
+   *
+   * @throws SpaceLimitingException When the given mutation violates this policy.
+   */
+  void check(Mutation m) throws SpaceLimitingException;
+
+  /**
+   * Returns a logical name for the {@link SpaceViolationPolicy} that this enforcement is for.
+   */
+  String getPolicyName();
+
+  /**
+   * Returns whether or not compactions on this table should be disabled for this policy.
+   */
+  boolean areCompactionsDisabled();
+
+  /**
+   * Returns the {@link SpaceQuotaSnapshot} <code>this</code> was initialized with.
+   */
+  SpaceQuotaSnapshot getQuotaSnapshot();
+
+  /**
+   * Returns whether thet caller should verify any bulk loads against <code>this</code>.
+   */
+  boolean shouldCheckBulkLoads();
+
+  /**
+   * Checks the file at the given path against <code>this</code> policy and the current
+   * {@link SpaceQuotaSnapshot}. If the file would violate the policy, a
+   * {@link SpaceLimitingException} will be thrown.
+   *
+   * @param paths The paths in HDFS to files to be bulk loaded.
+   */
+  void checkBulkLoad(FileSystem fs, List<String> paths) throws SpaceLimitingException;
+
+}


[38/50] [abbrv] hbase git commit: HBASE-17025 Add shell commands for space quotas

Posted by el...@apache.org.
HBASE-17025 Add shell commands for space quotas


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

Branch: refs/heads/HBASE-16961
Commit: 7a1f9d34b89131f248f76d1fafde6b060bec2827
Parents: 0effca4
Author: Josh Elser <el...@apache.org>
Authored: Wed Jan 11 11:55:29 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:32 2017 -0400

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/hbase/quotas.rb       |  62 ++++++++-
 hbase-shell/src/main/ruby/hbase_constants.rb    |   1 +
 .../src/main/ruby/shell/commands/set_quota.rb   |  45 +++++-
 .../hadoop/hbase/client/AbstractTestShell.java  |   1 +
 hbase-shell/src/test/ruby/hbase/quotas_test.rb  | 137 +++++++++++++++++++
 hbase-shell/src/test/ruby/tests_runner.rb       |   1 +
 6 files changed, 242 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7a1f9d34/hbase-shell/src/main/ruby/hbase/quotas.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/quotas.rb b/hbase-shell/src/main/ruby/hbase/quotas.rb
index bf2dc63..d99fe72 100644
--- a/hbase-shell/src/main/ruby/hbase/quotas.rb
+++ b/hbase-shell/src/main/ruby/hbase/quotas.rb
@@ -24,14 +24,22 @@ java_import org.apache.hadoop.hbase.quotas.ThrottleType
 java_import org.apache.hadoop.hbase.quotas.QuotaFilter
 java_import org.apache.hadoop.hbase.quotas.QuotaRetriever
 java_import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory
+java_import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy
 
 module HBaseQuotasConstants
+  # RPC Quota constants
   GLOBAL_BYPASS = 'GLOBAL_BYPASS'
   THROTTLE_TYPE = 'THROTTLE_TYPE'
   THROTTLE = 'THROTTLE'
   REQUEST = 'REQUEST'
   WRITE = 'WRITE'
   READ = 'READ'
+  # Space quota constants
+  SPACE = 'SPACE'
+  NO_INSERTS = 'NO_INSERTS'
+  NO_WRITES = 'NO_WRITES'
+  NO_WRITES_COMPACTIONS = 'NO_WRITES_COMPACTIONS'
+  DISABLE = 'DISABLE'
 end
 
 module Hbase
@@ -107,6 +115,54 @@ module Hbase
       @admin.setQuota(settings)
     end
 
+    def limit_space(args)
+      raise(ArgumentError, 'Argument should be a Hash') unless (not args.nil? and args.kind_of?(Hash))
+      # Let the user provide a raw number
+      if args[LIMIT].is_a?(Numeric)
+        limit = args[LIMIT]
+      else
+        # Parse a string a 1K, 2G, etc.
+        limit = _parse_size(args[LIMIT])
+      end
+      # Extract the policy, failing if something bogus was provided
+      policy = SpaceViolationPolicy.valueOf(args[POLICY])
+      # Create a table or namespace quota
+      if args.key?(TABLE)
+        if args.key?(NAMESPACE)
+          raise(ArgumentError, "Only one of TABLE or NAMESPACE can be specified.")
+        end
+        settings = QuotaSettingsFactory.limitTableSpace(TableName.valueOf(args.delete(TABLE)), limit, policy)
+      elsif args.key?(NAMESPACE)
+        if args.key?(TABLE)
+          raise(ArgumentError, "Only one of TABLE or NAMESPACE can be specified.")
+        end
+        settings = QuotaSettingsFactory.limitNamespaceSpace(args.delete(NAMESPACE), limit, policy)
+      else
+        raise(ArgumentError, 'One of TABLE or NAMESPACE must be specified.')
+      end
+      # Apply the quota
+      @admin.setQuota(settings)
+    end
+
+    def remove_space_limit(args)
+      raise(ArgumentError, 'Argument should be a Hash') unless (not args.nil? and args.kind_of?(Hash))
+      if args.key?(TABLE)
+        if args.key?(NAMESPACE)
+          raise(ArgumentError, "Only one of TABLE or NAMESPACE can be specified.")
+        end
+        table = TableName.valueOf(args.delete(TABLE))
+        settings = QuotaSettingsFactory.removeTableSpaceLimit(table)
+      elsif args.key?(NAMESPACE)
+        if args.key?(TABLE)
+          raise(ArgumentError, "Only one of TABLE or NAMESPACE can be specified.")
+        end
+        settings = QuotaSettingsFactory.removeNamespaceSpaceLimit(args.delete(NAMESPACE))
+      else
+        raise(ArgumentError, 'One of TABLE or NAMESPACE must be specified.')
+      end
+      @admin.setQuota(settings)
+    end
+
     def set_global_bypass(bypass, args)
       raise(ArgumentError, "Arguments should be a Hash") unless args.kind_of?(Hash)
 
@@ -171,7 +227,7 @@ module Hbase
           return _size_from_str(match[1].to_i, match[2])
         end
       else
-        raise "Invalid size limit syntax"
+        raise(ArgumentError, "Invalid size limit syntax")
       end
     end
 
@@ -188,7 +244,7 @@ module Hbase
         end
 
         if limit <= 0
-          raise "Invalid throttle limit, must be greater then 0"
+          raise(ArgumentError, "Invalid throttle limit, must be greater then 0")
         end
 
         case match[3]
@@ -200,7 +256,7 @@ module Hbase
 
         return type, limit, time_unit
       else
-        raise "Invalid throttle limit syntax"
+        raise(ArgumentError, "Invalid throttle limit syntax")
       end
     end
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7a1f9d34/hbase-shell/src/main/ruby/hbase_constants.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase_constants.rb b/hbase-shell/src/main/ruby/hbase_constants.rb
index c02d5c6..a857403 100644
--- a/hbase-shell/src/main/ruby/hbase_constants.rb
+++ b/hbase-shell/src/main/ruby/hbase_constants.rb
@@ -81,6 +81,7 @@ module HBaseConstants
   NAMESPACES = 'NAMESPACES'
   CONFIG = 'CONFIG'
   DATA = 'DATA'
+  POLICY = 'POLICY'
 
   # Load constants from hbase java API
   def self.promote_constants(constants)

http://git-wip-us.apache.org/repos/asf/hbase/blob/7a1f9d34/hbase-shell/src/main/ruby/shell/commands/set_quota.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/set_quota.rb b/hbase-shell/src/main/ruby/shell/commands/set_quota.rb
index a638b93..06ed0ba 100644
--- a/hbase-shell/src/main/ruby/shell/commands/set_quota.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/set_quota.rb
@@ -52,6 +52,37 @@ For example:
     hbase> set_quota TYPE => THROTTLE, THROTTLE_TYPE => WRITE, USER => 'u1', LIMIT => NONE
 
     hbase> set_quota USER => 'u1', GLOBAL_BYPASS => true
+
+TYPE => SPACE
+Users can either set a quota on a table or a namespace. The quota is a limit on the target's
+size on the FileSystem and some action to take when the target exceeds that limit. The limit
+is in bytes and can expressed using standard metric suffixes (B, K, M, G, T, P), defaulting
+to bytes if not provided. Different quotas can be applied to one table at the table and namespace
+level; table-level quotas take priority over namespace-level quotas.
+
+There are a limited number of policies to take when a quota is violation, listed in order of
+least strict to most strict.
+
+  NO_INSERTS - No new data is allowed to be ingested (e.g. Put, Increment, Append).
+  NO_WRITES - Same as NO_INSERTS but Deletes are also disallowed.
+  NO_WRITES_COMPACTIONS - Same as NO_WRITES but compactions are also disallowed.
+  DISABLE - The table(s) are disabled.
+
+For example:
+
+  hbase> set_quota TYPE => SPACE, TABLE => 't1', LIMIT => '1G', POLICY => NO_INSERTS
+  hbase> set_quota TYPE => SPACE, TABLE => 't2', LIMIT => '50G', POLICY => DISABLE
+  hbase> set_quota TYPE => SPACE, TABLE => 't3', LIMIT => '2T', POLICY => NO_WRITES_COMPACTIONS
+  hbase> set_quota TYPE => SPACE, NAMESPACE => 'ns1', LIMIT => '50T', POLICY => NO_WRITES
+
+Space quotas can also be removed via this command. To remove a space quota, provide NONE
+for the limit.
+
+For example:
+
+  hbase> set_quota TYPE => SPACE, TABLE => 't1', LIMIT => NONE
+  hbase> set_quota TYPE => SPACE, NAMESPACE => 'ns1', LIMIT => NONE
+
 EOF
       end
 
@@ -66,8 +97,18 @@ EOF
               else
                 quotas_admin.throttle(args)
               end
-          else
-            raise "Invalid TYPE argument. got " + qtype
+            when SPACE
+              if args[LIMIT].eql? NONE
+                args.delete(LIMIT)
+                # Table/Namespace argument is verified in remove_space_limit
+                quotas_admin.remove_space_limit(args)
+              else
+                raise(ArgumentError, 'Expected a LIMIT to be provided') unless args.key?(LIMIT)
+                raise(ArgumentError, 'Expected a POLICY to be provided') unless args.key?(POLICY)
+                quotas_admin.limit_space(args)
+              end
+            else
+              raise "Invalid TYPE argument. got " + qtype
           end
         elsif args.has_key?(GLOBAL_BYPASS)
           quotas_admin.set_global_bypass(args.delete(GLOBAL_BYPASS), args)

http://git-wip-us.apache.org/repos/asf/hbase/blob/7a1f9d34/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/AbstractTestShell.java
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/AbstractTestShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/AbstractTestShell.java
index 1403805..53606e9 100644
--- a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/AbstractTestShell.java
+++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/AbstractTestShell.java
@@ -39,6 +39,7 @@ public abstract class AbstractTestShell {
     // Start mini cluster
     TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
     TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
+    TEST_UTIL.getConfiguration().setBoolean("hbase.quota.enabled", true);
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
     TEST_UTIL.getConfiguration().setBoolean(CoprocessorHost.ABORT_ON_ERROR_KEY, false);
     TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7a1f9d34/hbase-shell/src/test/ruby/hbase/quotas_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/quotas_test.rb b/hbase-shell/src/test/ruby/hbase/quotas_test.rb
new file mode 100644
index 0000000..78c889c
--- /dev/null
+++ b/hbase-shell/src/test/ruby/hbase/quotas_test.rb
@@ -0,0 +1,137 @@
+#
+#
+# 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.
+#
+
+require 'shell'
+require 'stringio'
+require 'hbase_constants'
+require 'hbase/hbase'
+require 'hbase/table'
+
+include HBaseConstants
+
+module Hbase
+  class SpaceQuotasTest < Test::Unit::TestCase
+    include TestHelpers
+
+    def setup
+      setup_hbase
+      # Create test table if it does not exist
+      @test_name = "hbase_shell_tests_table"
+      create_test_table(@test_name)
+    end
+
+    def teardown
+      shutdown
+    end
+
+    define_test 'limit_space errors on non Hash argument' do
+      qa = quotas_admin()
+      assert_raise(ArgumentError) do
+        qa.limit_space('foo')
+      end
+      assert_raise(ArgumentError) do
+        qa.limit_space()
+      end
+    end
+
+    define_test 'remove_space_limit errors on non Hash argument' do
+      qa = quotas_admin()
+      assert_raise(ArgumentError) do
+        qa.remove_space_limit('foo')
+      end
+      assert_raise(ArgumentError) do
+        qa.remove_space_limit()
+      end
+    end
+
+    define_test 'set quota with a non-numeric limit fails' do
+      assert_raise(ArgumentError) do
+        command(:set_quota, TYPE => SPACE, LIMIT => 'asdf', POLICY => NO_INSERTS, TABLE => @test_name)
+      end
+    end
+
+    define_test 'set quota without a limit fails' do
+      assert_raise(ArgumentError) do
+        command(:set_quota, TYPE => SPACE, POLICY => NO_INSERTS, TABLE => @test_name)
+      end
+    end
+
+    define_test 'set quota without a policy fails' do
+      assert_raise(ArgumentError) do
+        command(:set_quota, TYPE => SPACE, LIMIT => '1G', TABLE => @test_name)
+      end
+    end
+
+    define_test 'set quota without a table or namespace fails' do
+      assert_raise(ArgumentError) do
+        command(:set_quota, TYPE => SPACE, LIMIT => '1G', POLICY => NO_INSERTS)
+      end
+    end
+
+    define_test 'invalid violation policy specified' do
+      assert_raise(NameError) do
+        command(:set_quota, TYPE => SPACE, LIMIT => '1G', POLICY => FOO_BAR, TABLE => @test_name)
+      end
+    end
+
+    define_test 'table and namespace are mutually exclusive in set quota' do
+      assert_raise(ArgumentError) do
+        command(:set_quota, TYPE => SPACE, LIMIT => '1G', POLICY => NO_INSERTS, TABLE => @test_name, NAMESPACE => "foo")
+      end
+    end
+
+    define_test '_parse_size accepts various forms of byte shorthand' do
+      qa = quotas_admin()
+      KILO = 1024
+      MEGA = KILO * KILO
+      GIGA = MEGA * KILO
+      TERA = GIGA * KILO
+      PETA = TERA * KILO
+      assert_equal(1, qa._parse_size("1"))
+      assert_equal(1, qa._parse_size("1b"))
+      assert_equal(1, qa._parse_size("1B"))
+      assert_equal(KILO * 2, qa._parse_size("2k"))
+      assert_equal(KILO * 2, qa._parse_size("2K"))
+      assert_equal(MEGA * 5, qa._parse_size("5m"))
+      assert_equal(MEGA * 5, qa._parse_size("5M"))
+      assert_equal(GIGA * 3, qa._parse_size("3g"))
+      assert_equal(GIGA * 3, qa._parse_size("3G"))
+      assert_equal(TERA * 4, qa._parse_size("4t"))
+      assert_equal(TERA * 4, qa._parse_size("4T"))
+      assert_equal(PETA * 32, qa._parse_size("32p"))
+      assert_equal(PETA * 32, qa._parse_size("32P"))
+      assert_equal(GIGA * 4, qa._parse_size("4096m"))
+      assert_equal(GIGA * 4, qa._parse_size("4096M"))
+    end
+
+    define_test 'can set and remove quota' do
+      command(:set_quota, TYPE => SPACE, LIMIT => '1G', POLICY => NO_INSERTS, TABLE => @test_name)
+      output = capture_stdout{ command(:list_quotas) }
+      size = 1024 * 1024 * 1024
+      assert(output.include?("LIMIT => #{size}"))
+      assert(output.include?("VIOLATION_POLICY => NO_INSERTS"))
+      assert(output.include?("TYPE => SPACE"))
+      assert(output.include?("TABLE => #{@test_name}"))
+
+      command(:set_quota, TYPE => SPACE, LIMIT => NONE, TABLE => @test_name)
+      output = capture_stdout{ command(:list_quotas) }
+      assert(output.include?("0 row(s)"))
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/7a1f9d34/hbase-shell/src/test/ruby/tests_runner.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/tests_runner.rb b/hbase-shell/src/test/ruby/tests_runner.rb
index 74ddb48..54bf3f9 100644
--- a/hbase-shell/src/test/ruby/tests_runner.rb
+++ b/hbase-shell/src/test/ruby/tests_runner.rb
@@ -36,6 +36,7 @@ unless defined?($TEST_CLUSTER)
   $TEST_CLUSTER = HBaseTestingUtility.new
   $TEST_CLUSTER.configuration.setInt("hbase.regionserver.msginterval", 100)
   $TEST_CLUSTER.configuration.setInt("hbase.client.pause", 250)
+  $TEST_CLUSTER.configuration.set("hbase.quota.enabled", "true")
   $TEST_CLUSTER.configuration.setInt(org.apache.hadoop.hbase.HConstants::HBASE_CLIENT_RETRIES_NUMBER, 6)
   $TEST_CLUSTER.startMiniCluster
   @own_cluster = true


[40/50] [abbrv] hbase git commit: HBASE-17568 Better handle stale/missing region size reports

Posted by el...@apache.org.
HBASE-17568 Better handle stale/missing region size reports

* Expire region reports in the master after a timeout.
* Move regions in violation out of violation when insufficient
    region size reports are observed.


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

Branch: refs/heads/HBASE-16961
Commit: 71e14a3cccfbe7125284a32250e4aefc0d575b30
Parents: 4db44ad
Author: Josh Elser <el...@apache.org>
Authored: Fri Feb 3 16:33:47 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:44:00 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hbase/master/MasterRpcServices.java  |   4 +-
 .../hadoop/hbase/quotas/MasterQuotaManager.java |  86 ++++++-
 .../hadoop/hbase/quotas/QuotaObserverChore.java |  53 ++++-
 .../hbase/quotas/TestMasterQuotaManager.java    |  48 +++-
 .../TestQuotaObserverChoreRegionReports.java    | 233 +++++++++++++++++++
 5 files changed, 412 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/71e14a3c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 42f99b2..3d72a3e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -251,6 +251,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Updat
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.zookeeper.KeeperException;
@@ -2027,8 +2028,9 @@ public class MasterRpcServices extends RSRpcServices
         return RegionSpaceUseReportResponse.newBuilder().build();
       }
       MasterQuotaManager quotaManager = this.master.getMasterQuotaManager();
+      final long now = EnvironmentEdgeManager.currentTime();
       for (RegionSpaceUse report : request.getSpaceUseList()) {
-        quotaManager.addRegionSize(HRegionInfo.convert(report.getRegion()), report.getSize());
+        quotaManager.addRegionSize(HRegionInfo.convert(report.getRegion()), report.getSize(), now);
       }
       return RegionSpaceUseReportResponse.newBuilder().build();
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/71e14a3c/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index cb614ea..0622dba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -22,9 +22,12 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -47,6 +50,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Master Quota Manager.
  * It is responsible for initialize the quota table on the first-run and
@@ -68,7 +73,7 @@ public class MasterQuotaManager implements RegionStateListener {
   private NamedLock<String> userLocks;
   private boolean enabled = false;
   private NamespaceAuditor namespaceQuotaManager;
-  private ConcurrentHashMap<HRegionInfo, Long> regionSizes;
+  private ConcurrentHashMap<HRegionInfo, SizeSnapshotWithTimestamp> regionSizes;
 
   public MasterQuotaManager(final MasterServices masterServices) {
     this.masterServices = masterServices;
@@ -531,21 +536,88 @@ public class MasterQuotaManager implements RegionStateListener {
     }
   }
 
-  public void addRegionSize(HRegionInfo hri, long size) {
+  /**
+   * Holds the size of a region at the given time, millis since the epoch.
+   */
+  private static class SizeSnapshotWithTimestamp {
+    private final long size;
+    private final long time;
+
+    public SizeSnapshotWithTimestamp(long size, long time) {
+      this.size = size;
+      this.time = time;
+    }
+
+    public long getSize() {
+      return size;
+    }
+
+    public long getTime() {
+      return time;
+    }
+
+    public boolean equals(Object o) {
+      if (o instanceof SizeSnapshotWithTimestamp) {
+        SizeSnapshotWithTimestamp other = (SizeSnapshotWithTimestamp) o;
+        return size == other.size && time == other.time;
+      }
+      return false;
+    }
+
+    public int hashCode() {
+      HashCodeBuilder hcb = new HashCodeBuilder();
+      return hcb.append(size).append(time).toHashCode();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder(32);
+      sb.append("SizeSnapshotWithTimestamp={size=").append(size).append("B, ");
+      sb.append("time=").append(time).append("}");
+      return sb.toString();
+    }
+  }
+
+  @VisibleForTesting
+  void initializeRegionSizes() {
+    assert null == regionSizes;
+    this.regionSizes = new ConcurrentHashMap<>();
+  }
+
+  public void addRegionSize(HRegionInfo hri, long size, long time) {
     if (null == regionSizes) {
       return;
     }
-    // TODO Make proper API?
-    // TODO Prevent from growing indefinitely
-    regionSizes.put(hri, size);
+    regionSizes.put(hri, new SizeSnapshotWithTimestamp(size, time));
   }
 
   public Map<HRegionInfo, Long> snapshotRegionSizes() {
     if (null == regionSizes) {
       return EMPTY_MAP;
     }
-    // TODO Make proper API?
-    return new HashMap<>(regionSizes);
+
+    Map<HRegionInfo, Long> copy = new HashMap<>();
+    for (Entry<HRegionInfo, SizeSnapshotWithTimestamp> entry : regionSizes.entrySet()) {
+      copy.put(entry.getKey(), entry.getValue().getSize());
+    }
+    return copy;
+  }
+
+  int pruneEntriesOlderThan(long timeToPruneBefore) {
+    if (null == regionSizes) {
+      return 0;
+    }
+    int numEntriesRemoved = 0;
+    Iterator<Entry<HRegionInfo,SizeSnapshotWithTimestamp>> iterator =
+        regionSizes.entrySet().iterator();
+    while (iterator.hasNext()) {
+      long currentEntryTime = iterator.next().getValue().getTime();
+      if (currentEntryTime < timeToPruneBefore) {
+        iterator.remove();
+        numEntriesRemoved++;
+      }
+    }
+    return numEntriesRemoved;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/71e14a3c/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 7f894e4..79effe7 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
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.HashMultimap;
@@ -69,6 +70,11 @@ public class QuotaObserverChore extends ScheduledChore {
       "hbase.master.quotas.observer.report.percent";
   static final double QUOTA_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT= 0.95;
 
+  static final String REGION_REPORT_RETENTION_DURATION_KEY =
+      "hbase.master.quotas.region.report.retention.millis";
+  static final long REGION_REPORT_RETENTION_DURATION_DEFAULT =
+      1000 * 60 * 10; // 10 minutes
+
   private final Connection conn;
   private final Configuration conf;
   private final MasterQuotaManager quotaManager;
@@ -83,6 +89,9 @@ public class QuotaObserverChore extends ScheduledChore {
   private final Map<TableName,SpaceQuotaSnapshot> tableQuotaSnapshots;
   private final Map<String,SpaceQuotaSnapshot> namespaceQuotaSnapshots;
 
+  // The time, in millis, that region reports should be kept by the master
+  private final long regionReportLifetimeMillis;
+
   /*
    * Encapsulates logic for tracking the state of a table/namespace WRT space quotas
    */
@@ -108,6 +117,8 @@ public class QuotaObserverChore extends ScheduledChore {
     this.snapshotNotifier = Objects.requireNonNull(snapshotNotifier);
     this.tableQuotaSnapshots = new HashMap<>();
     this.namespaceQuotaSnapshots = new HashMap<>();
+    this.regionReportLifetimeMillis = conf.getLong(
+        REGION_REPORT_RETENTION_DURATION_KEY, REGION_REPORT_RETENTION_DURATION_DEFAULT);
   }
 
   @Override
@@ -136,18 +147,40 @@ public class QuotaObserverChore extends ScheduledChore {
       LOG.trace("Using " + reportedRegionSpaceUse.size() + " region space use reports");
     }
 
+    // Remove the "old" region reports
+    pruneOldRegionReports();
+
     // Create the stores to track table and namespace snapshots
     initializeSnapshotStores(reportedRegionSpaceUse);
 
     // Filter out tables for which we don't have adequate regionspace reports yet.
     // Important that we do this after we instantiate the stores above
-    tablesWithQuotas.filterInsufficientlyReportedTables(tableSnapshotStore);
+    // This gives us a set of Tables which may or may not be violating their quota.
+    // To be safe, we want to make sure that these are not in violation.
+    Set<TableName> tablesInLimbo = tablesWithQuotas.filterInsufficientlyReportedTables(
+        tableSnapshotStore);
 
     if (LOG.isTraceEnabled()) {
       LOG.trace("Filtered insufficiently reported tables, left with " +
           reportedRegionSpaceUse.size() + " regions reported");
     }
 
+    for (TableName tableInLimbo : tablesInLimbo) {
+      final SpaceQuotaSnapshot currentSnapshot = tableSnapshotStore.getCurrentState(tableInLimbo);
+      if (currentSnapshot.getQuotaStatus().isInViolation()) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Moving " + tableInLimbo + " out of violation because fewer region sizes were"
+              + " reported than required.");
+        }
+        SpaceQuotaSnapshot targetSnapshot = new SpaceQuotaSnapshot(
+            SpaceQuotaStatus.notInViolation(), currentSnapshot.getUsage(),
+            currentSnapshot.getLimit());
+        this.snapshotNotifier.transitionTable(tableInLimbo, targetSnapshot);
+        // Update it in the Table QuotaStore so that memory is consistent with no violation.
+        tableSnapshotStore.setCurrentState(tableInLimbo, targetSnapshot);
+      }
+    }
+
     // Transition each table to/from quota violation based on the current and target state.
     // Only table quotas are enacted.
     final Set<TableName> tablesWithTableQuotas = tablesWithQuotas.getTableQuotaTables();
@@ -347,6 +380,19 @@ public class QuotaObserverChore extends ScheduledChore {
   }
 
   /**
+   * Removes region reports over a certain age.
+   */
+  void pruneOldRegionReports() {
+    final long now = EnvironmentEdgeManager.currentTime();
+    final long pruneTime = now - regionReportLifetimeMillis;
+    final int numRemoved = quotaManager.pruneEntriesOlderThan(pruneTime);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Removed " + numRemoved + " old region size reports that were older than "
+          + pruneTime + ".");
+    }
+  }
+
+  /**
    * Computes the set of all tables that have quotas defined. This includes tables with quotas
    * explicitly set on them, in addition to tables that exist namespaces which have a quota
    * defined.
@@ -577,8 +623,8 @@ public class QuotaObserverChore extends ScheduledChore {
      * Filters out all tables for which the Master currently doesn't have enough region space
      * reports received from RegionServers yet.
      */
-    public void filterInsufficientlyReportedTables(QuotaSnapshotStore<TableName> tableStore)
-        throws IOException {
+    public Set<TableName> filterInsufficientlyReportedTables(
+        QuotaSnapshotStore<TableName> tableStore) throws IOException {
       final double percentRegionsReportedThreshold = getRegionReportPercent(getConfiguration());
       Set<TableName> tablesToRemove = new HashSet<>();
       for (TableName table : Iterables.concat(tablesWithTableQuotas, tablesWithNamespaceQuotas)) {
@@ -612,6 +658,7 @@ public class QuotaObserverChore extends ScheduledChore {
         tablesWithTableQuotas.remove(tableToRemove);
         tablesWithNamespaceQuotas.remove(tableToRemove);
       }
+      return tablesToRemove;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/71e14a3c/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java
index e383593..c024294 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java
@@ -16,9 +16,13 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+import static org.apache.hadoop.hbase.util.Bytes.toBytes;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.mockito.Mockito.mock;
 
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
@@ -31,7 +35,49 @@ public class TestMasterQuotaManager {
   public void testUninitializedQuotaManangerDoesNotFail() {
     MasterServices masterServices = mock(MasterServices.class);
     MasterQuotaManager manager = new MasterQuotaManager(masterServices);
-    manager.addRegionSize(null, 0);
+    manager.addRegionSize(null, 0, 0);
     assertNotNull(manager.snapshotRegionSizes());
   }
+
+  @Test
+  public void testOldEntriesRemoved() {
+    MasterServices masterServices = mock(MasterServices.class);
+    MasterQuotaManager manager = new MasterQuotaManager(masterServices);
+    manager.initializeRegionSizes();
+    // Mock out some regions
+    TableName tableName = TableName.valueOf("foo");
+    HRegionInfo region1 = new HRegionInfo(tableName, null, toBytes("a"));
+    HRegionInfo region2 = new HRegionInfo(tableName, toBytes("a"), toBytes("b"));
+    HRegionInfo region3 = new HRegionInfo(tableName, toBytes("b"), toBytes("c"));
+    HRegionInfo region4 = new HRegionInfo(tableName, toBytes("c"), toBytes("d"));
+    HRegionInfo region5 = new HRegionInfo(tableName, toBytes("d"), null);
+
+    final long size = 0;
+    long time1 = 10;
+    manager.addRegionSize(region1, size, time1);
+    manager.addRegionSize(region2, size, time1);
+
+    long time2 = 20;
+    manager.addRegionSize(region3, size, time2);
+    manager.addRegionSize(region4, size, time2);
+
+    long time3 = 30;
+    manager.addRegionSize(region5, size, time3);
+
+    assertEquals(5, manager.snapshotRegionSizes().size());
+
+    // Prune nothing
+    assertEquals(0, manager.pruneEntriesOlderThan(0));
+    assertEquals(5, manager.snapshotRegionSizes().size());
+    assertEquals(0, manager.pruneEntriesOlderThan(10));
+    assertEquals(5, manager.snapshotRegionSizes().size());
+
+    // Prune the elements at time1
+    assertEquals(2, manager.pruneEntriesOlderThan(15));
+    assertEquals(3, manager.snapshotRegionSizes().size());
+
+    // Prune the elements at time2
+    assertEquals(2, manager.pruneEntriesOlderThan(30));
+    assertEquals(1, manager.snapshotRegionSizes().size());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/71e14a3c/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
new file mode 100644
index 0000000..d7cdff9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
@@ -0,0 +1,233 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+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.HRegionInfo;
+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.Connection;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * A test case to verify that region reports are expired when they are not sent.
+ */
+@Category(LargeTests.class)
+public class TestQuotaObserverChoreRegionReports {
+  private static final Log LOG = LogFactory.getLog(TestQuotaObserverChoreRegionReports.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @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);
+    conf.setInt(QuotaObserverChore.REGION_REPORT_RETENTION_DURATION_KEY, 1000);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @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);
+    // Expire the reports after 5 seconds
+    conf.setInt(QuotaObserverChore.REGION_REPORT_RETENTION_DURATION_KEY, 5000);
+    TEST_UTIL.startMiniCluster(1);
+
+    final String FAM1 = "f1";
+    final HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
+    // Wait for the master to finish initialization.
+    while (null == master.getMasterQuotaManager()) {
+      LOG.debug("MasterQuotaManager is null, waiting...");
+      Thread.sleep(500);
+    }
+    final MasterQuotaManager quotaManager = master.getMasterQuotaManager();
+
+    // Create a table
+    final TableName tn = TableName.valueOf("reportExpiration");
+    HTableDescriptor tableDesc = new HTableDescriptor(tn);
+    tableDesc.addFamily(new HColumnDescriptor(FAM1));
+    TEST_UTIL.getAdmin().createTable(tableDesc);
+
+    // No reports right after we created this table.
+    assertEquals(0, getRegionReportsForTable(quotaManager.snapshotRegionSizes(), tn));
+
+    // Set a quota
+    final long sizeLimit = 100L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, violationPolicy);
+    TEST_UTIL.getAdmin().setQuota(settings);
+
+    // We should get one report for the one region we have.
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 45000, 1000, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        int numReports = getRegionReportsForTable(quotaManager.snapshotRegionSizes(), tn);
+        LOG.debug("Saw " + numReports + " reports for " + tn + " while waiting for 1");
+        return numReports == 1;
+      }
+    });
+
+    // We should then see no reports for the single region
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 15000, 1000, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        int numReports = getRegionReportsForTable(quotaManager.snapshotRegionSizes(), tn);
+        LOG.debug("Saw " + numReports + " reports for " + tn + " while waiting for none");
+        return numReports == 0;
+      }
+    });
+  }
+
+  @Test
+  public void testMissingReportsRemovesQuota() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    // Expire the reports after 5 seconds
+    conf.setInt(QuotaObserverChore.REGION_REPORT_RETENTION_DURATION_KEY, 5000);
+    TEST_UTIL.startMiniCluster(1);
+
+    final String FAM1 = "f1";
+
+    // Create a table
+    final TableName tn = TableName.valueOf("quotaAcceptanceWithoutReports");
+    HTableDescriptor tableDesc = new HTableDescriptor(tn);
+    tableDesc.addFamily(new HColumnDescriptor(FAM1));
+    TEST_UTIL.getAdmin().createTable(tableDesc);
+
+    // Set a quota
+    final long sizeLimit = 1L * SpaceQuotaHelperForTests.ONE_KILOBYTE;
+    final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, violationPolicy);
+    final Admin admin = TEST_UTIL.getAdmin();
+    admin.setQuota(settings);
+    final Connection conn = TEST_UTIL.getConnection();
+
+    // Write enough data to invalidate the quota
+    Put p = new Put(Bytes.toBytes("row1"));
+    byte[] bytes = new byte[10];
+    Arrays.fill(bytes, (byte) 2);
+    for (int i = 0; i < 200; i++) {
+      p.addColumn(Bytes.toBytes(FAM1), Bytes.toBytes("qual" + i), bytes);
+    }
+    conn.getTable(tn).put(p);
+    admin.flush(tn);
+
+    // Wait for the table to move into violation
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30000, 1000, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        SpaceQuotaSnapshot snapshot = getSnapshotForTable(conn, tn);
+        if (null == snapshot) {
+          return false;
+        }
+        return snapshot.getQuotaStatus().isInViolation();
+      }
+    });
+
+    // Close the region, prevent the server from sending new status reports.
+    List<HRegionInfo> regions = admin.getTableRegions(tn);
+    assertEquals(1, regions.size());
+    HRegionInfo hri = regions.get(0);
+    admin.closeRegion(TEST_UTIL.getMiniHBaseCluster().getRegionServer(0).getServerName(), hri);
+
+    // We should see this table move out of violation after the report expires.
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30000, 1000, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        SpaceQuotaSnapshot snapshot = getSnapshotForTable(conn, tn);
+        if (null == snapshot) {
+          return false;
+        }
+        return !snapshot.getQuotaStatus().isInViolation();
+      }
+    });
+
+    // The QuotaObserverChore's memory should also show it not in violation.
+    final HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
+    QuotaSnapshotStore<TableName> tableStore =
+        master.getQuotaObserverChore().getTableSnapshotStore();
+    SpaceQuotaSnapshot snapshot = tableStore.getCurrentState(tn);
+    assertFalse("Quota should not be in violation", snapshot.getQuotaStatus().isInViolation());
+  }
+
+  private SpaceQuotaSnapshot getSnapshotForTable(
+      Connection conn, TableName tn) throws IOException {
+    try (Table quotaTable = conn.getTable(QuotaUtil.QUOTA_TABLE_NAME);
+        ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaSnapshotScan())) {
+      Map<TableName,SpaceQuotaSnapshot> activeViolations = new HashMap<>();
+      for (Result result : scanner) {
+        try {
+          QuotaTableUtil.extractQuotaSnapshot(result, activeViolations);
+        } catch (IllegalArgumentException e) {
+          final String msg = "Failed to parse result for row " + Bytes.toString(result.getRow());
+          LOG.error(msg, e);
+          throw new IOException(msg, e);
+        }
+      }
+      return activeViolations.get(tn);
+    }
+  }
+
+  private int getRegionReportsForTable(Map<HRegionInfo,Long> reports, TableName tn) {
+    int numReports = 0;
+    for (Entry<HRegionInfo,Long> entry : reports.entrySet()) {
+      if (tn.equals(entry.getKey().getTable())) {
+        numReports++;
+      }
+    }
+    return numReports;
+  }
+}


[48/50] [abbrv] hbase git commit: HBASE-17002 JMX metrics and some UI additions for space quotas

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 3318a39..beb8f02 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -930,7 +930,11 @@ service MasterService {
   rpc removeDrainFromRegionServers(RemoveDrainFromRegionServersRequest)
     returns(RemoveDrainFromRegionServersResponse);
 
-  /** Fetches the Master's view of space quotas */
+  /** Fetches the Master's view of space utilization */
   rpc GetSpaceQuotaRegionSizes(GetSpaceQuotaRegionSizesRequest)
     returns(GetSpaceQuotaRegionSizesResponse);
+
+  /** Fetches the Master's view of quotas */
+  rpc GetQuotaStates(GetQuotaStatesRequest)
+    returns(GetQuotaStatesResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-protocol-shaded/src/main/protobuf/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Quota.proto b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
index 2d7e5f5..1a6d5ed 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Quota.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
@@ -119,6 +119,7 @@ message GetSpaceQuotaRegionSizesResponse {
   message RegionSizes {
     optional TableName table_name = 1;
     optional uint64 size = 2;
+
   }
   repeated RegionSizes sizes = 1;
 }
@@ -146,3 +147,19 @@ message GetSpaceQuotaEnforcementsResponse {
   }
   repeated TableViolationPolicy violation_policies = 1;
 }
+
+message GetQuotaStatesRequest {
+}
+
+message GetQuotaStatesResponse {
+  message TableQuotaSnapshot {
+    optional TableName table_name = 1;
+    optional SpaceQuotaSnapshot snapshot = 2;
+  }
+  message NamespaceQuotaSnapshot {
+    optional string namespace = 1;
+    optional SpaceQuotaSnapshot snapshot = 2;
+  }
+  repeated TableQuotaSnapshot table_snapshots = 1;
+  repeated NamespaceQuotaSnapshot ns_snapshots = 2;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/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 0243934..92c1461 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
@@ -908,7 +908,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       // Create the quota snapshot notifier
       spaceQuotaSnapshotNotifier = createQuotaSnapshotNotifier();
       spaceQuotaSnapshotNotifier.initialize(getClusterConnection());
-      this.quotaObserverChore = new QuotaObserverChore(this);
+      this.quotaObserverChore = new QuotaObserverChore(this, getMasterMetrics());
       // Start the chore to read the region FS space reports and act on them
       getChoreService().scheduleChore(quotaObserverChore);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 3d72a3e..8f796bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -62,7 +62,9 @@ import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
+import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -214,8 +216,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
@@ -2071,4 +2077,36 @@ public class MasterRpcServices extends RSRpcServices
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public GetQuotaStatesResponse getQuotaStates(
+      RpcController controller, GetQuotaStatesRequest request) throws ServiceException {
+    try {
+      master.checkInitialized();
+      QuotaObserverChore quotaChore = this.master.getQuotaObserverChore();
+      GetQuotaStatesResponse.Builder builder = GetQuotaStatesResponse.newBuilder();
+      if (null != quotaChore) {
+        // The "current" view of all tables with quotas
+        Map<TableName, SpaceQuotaSnapshot> tableSnapshots = quotaChore.getTableQuotaSnapshots();
+        for (Entry<TableName, SpaceQuotaSnapshot> entry : tableSnapshots.entrySet()) {
+          builder.addTableSnapshots(
+              TableQuotaSnapshot.newBuilder()
+                  .setTableName(ProtobufUtil.toProtoTableName(entry.getKey()))
+                  .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(entry.getValue())).build());
+        }
+        // The "current" view of all namespaces with quotas
+        Map<String, SpaceQuotaSnapshot> nsSnapshots = quotaChore.getNamespaceQuotaSnapshots();
+        for (Entry<String, SpaceQuotaSnapshot> entry : nsSnapshots.entrySet()) {
+          builder.addNsSnapshots(
+              NamespaceQuotaSnapshot.newBuilder()
+                  .setNamespace(entry.getKey())
+                  .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(entry.getValue())).build());
+        }
+        return builder.build();
+      }
+      return builder.build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/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 d055853..b5bc3d7 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
@@ -37,11 +37,14 @@ public class MetricsMaster {
   private static final Log LOG = LogFactory.getLog(MetricsMaster.class);
   private MetricsMasterSource masterSource;
   private MetricsMasterProcSource masterProcSource;
+  private MetricsMasterQuotaSource masterQuotaSource;
 
   public MetricsMaster(MetricsMasterWrapper masterWrapper) {
     masterSource = CompatibilitySingletonFactory.getInstance(MetricsMasterSourceFactory.class).create(masterWrapper);
     masterProcSource =
             CompatibilitySingletonFactory.getInstance(MetricsMasterProcSourceFactory.class).create(masterWrapper);
+    masterQuotaSource =
+            CompatibilitySingletonFactory.getInstance(MetricsMasterQuotaSourceFactory.class).create(masterWrapper);
   }
 
   // for unit-test usage
@@ -53,10 +56,49 @@ public class MetricsMaster {
     return masterProcSource;
   }
 
+  public MetricsMasterQuotaSource getMetricsQuotaSource() {
+    return masterQuotaSource;
+  }
+
   /**
    * @param inc How much to add to requests.
    */
   public void incrementRequests(final long inc) {
     masterSource.incRequests(inc);
   }
+
+  /**
+   * Sets the number of space quotas defined.
+   */
+  public void setNumSpaceQuotas(final long numSpaceQuotas) {
+    masterQuotaSource.updateNumSpaceQuotas(numSpaceQuotas);
+  }
+
+  /**
+   * Sets the number of table in violation of a space quota.
+   */
+  public void setNumTableInSpaceQuotaViolation(final long numTablesInViolation) {
+    masterQuotaSource.updateNumTablesInSpaceQuotaViolation(numTablesInViolation);
+  }
+
+  /**
+   * Sets the number of namespaces in violation of a space quota.
+   */
+  public void setNumNamespacesInSpaceQuotaViolation(final long numNamespacesInViolation) {
+    masterQuotaSource.updateNumNamespacesInSpaceQuotaViolation(numNamespacesInViolation);
+  }
+
+  /**
+   * Sets the number of region size reports the master has seen.
+   */
+  public void setNumRegionSizeReports(final long numRegionReports) {
+    masterQuotaSource.updateNumCurrentSpaceQuotaRegionSizeReports(numRegionReports);
+  }
+
+  /**
+   * Sets the execution time of a period of the QuotaObserverChore.
+   */
+  public void incrementQuotaObserverTime(final long executionTime) {
+    masterQuotaSource.incrementSpaceQuotaObserverChoreTime(executionTime);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
index 4cff28b..cbf7ba5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
@@ -17,9 +17,18 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
 /**
@@ -134,4 +143,35 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
     return master.getNumWALFiles();
   }
 
+  @Override
+  public Map<String,Entry<Long,Long>> getTableSpaceUtilization() {
+    QuotaObserverChore quotaChore = master.getQuotaObserverChore();
+    if (null == quotaChore) {
+      return Collections.emptyMap();
+    }
+    Map<TableName,SpaceQuotaSnapshot> tableSnapshots = quotaChore.getTableQuotaSnapshots();
+    Map<String,Entry<Long,Long>> convertedData = new HashMap<>();
+    for (Entry<TableName,SpaceQuotaSnapshot> entry : tableSnapshots.entrySet()) {
+      convertedData.put(entry.getKey().toString(), convertSnapshot(entry.getValue()));
+    }
+    return convertedData;
+  }
+
+  @Override
+  public Map<String,Entry<Long,Long>> getNamespaceSpaceUtilization() {
+    QuotaObserverChore quotaChore = master.getQuotaObserverChore();
+    if (null == quotaChore) {
+      return Collections.emptyMap();
+    }
+    Map<String,SpaceQuotaSnapshot> namespaceSnapshots = quotaChore.getNamespaceQuotaSnapshots();
+    Map<String,Entry<Long,Long>> convertedData = new HashMap<>();
+    for (Entry<String,SpaceQuotaSnapshot> entry : namespaceSnapshots.entrySet()) {
+      convertedData.put(entry.getKey(), convertSnapshot(entry.getValue()));
+    }
+    return convertedData;
+  }
+
+  Entry<Long,Long> convertSnapshot(SpaceQuotaSnapshot snapshot) {
+    return new SimpleImmutableEntry<Long,Long>(snapshot.getUsage(), snapshot.getLimit());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/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 79effe7..94c5c87 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
@@ -18,12 +18,12 @@ package org.apache.hadoop.hbase.quotas;
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MetricsMaster;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
@@ -78,6 +79,7 @@ public class QuotaObserverChore extends ScheduledChore {
   private final Connection conn;
   private final Configuration conf;
   private final MasterQuotaManager quotaManager;
+  private final MetricsMaster metrics;
   /*
    * Callback that changes in quota snapshots are passed to.
    */
@@ -87,7 +89,9 @@ public class QuotaObserverChore extends ScheduledChore {
    * Preserves the state of quota snapshots for tables and namespaces
    */
   private final Map<TableName,SpaceQuotaSnapshot> tableQuotaSnapshots;
+  private final Map<TableName,SpaceQuotaSnapshot> readOnlyTableQuotaSnapshots;
   private final Map<String,SpaceQuotaSnapshot> namespaceQuotaSnapshots;
+  private final Map<String,SpaceQuotaSnapshot> readOnlyNamespaceSnapshots;
 
   // The time, in millis, that region reports should be kept by the master
   private final long regionReportLifetimeMillis;
@@ -98,25 +102,28 @@ public class QuotaObserverChore extends ScheduledChore {
   private QuotaSnapshotStore<TableName> tableSnapshotStore;
   private QuotaSnapshotStore<String> namespaceSnapshotStore;
 
-  public QuotaObserverChore(HMaster master) {
+  public QuotaObserverChore(HMaster master, MetricsMaster metrics) {
     this(
         master.getConnection(), master.getConfiguration(),
         master.getSpaceQuotaSnapshotNotifier(), master.getMasterQuotaManager(),
-        master);
+        master, metrics);
   }
 
   QuotaObserverChore(
       Connection conn, Configuration conf, SpaceQuotaSnapshotNotifier snapshotNotifier,
-      MasterQuotaManager quotaManager, Stoppable stopper) {
+      MasterQuotaManager quotaManager, 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.quotaManager = quotaManager;
     this.snapshotNotifier = Objects.requireNonNull(snapshotNotifier);
-    this.tableQuotaSnapshots = new HashMap<>();
-    this.namespaceQuotaSnapshots = new HashMap<>();
+    this.tableQuotaSnapshots = new ConcurrentHashMap<>();
+    this.readOnlyTableQuotaSnapshots = Collections.unmodifiableMap(tableQuotaSnapshots);
+    this.namespaceQuotaSnapshots = new ConcurrentHashMap<>();
+    this.readOnlyNamespaceSnapshots = Collections.unmodifiableMap(namespaceQuotaSnapshots);
     this.regionReportLifetimeMillis = conf.getLong(
         REGION_REPORT_RETENTION_DURATION_KEY, REGION_REPORT_RETENTION_DURATION_DEFAULT);
   }
@@ -127,7 +134,11 @@ public class QuotaObserverChore extends ScheduledChore {
       if (LOG.isTraceEnabled()) {
         LOG.trace("Refreshing space quotas in RegionServer");
       }
+      long start = System.nanoTime();
       _chore();
+      if (null != metrics) {
+        metrics.incrementQuotaObserverTime((System.nanoTime() - start) / 1_000_000);
+      }
     } catch (IOException e) {
       LOG.warn("Failed to process quota reports and update quota state. Will retry.", e);
     }
@@ -141,6 +152,12 @@ public class QuotaObserverChore extends ScheduledChore {
       LOG.trace("Found following tables with quotas: " + tablesWithQuotas);
     }
 
+    if (null != metrics) {
+      // Set the number of namespaces and tables with quotas defined
+      metrics.setNumSpaceQuotas(tablesWithQuotas.getTableQuotaTables().size()
+          + tablesWithQuotas.getNamespacesWithQuotas().size());
+    }
+
     // The current "view" of region space use. Used henceforth.
     final Map<HRegionInfo,Long> reportedRegionSpaceUse = quotaManager.snapshotRegionSizes();
     if (LOG.isTraceEnabled()) {
@@ -152,6 +169,10 @@ public class QuotaObserverChore extends ScheduledChore {
 
     // Create the stores to track table and namespace snapshots
     initializeSnapshotStores(reportedRegionSpaceUse);
+    // Report the number of (non-expired) region size reports
+    if (null != metrics) {
+      metrics.setNumRegionSizeReports(reportedRegionSpaceUse.size());
+    }
 
     // Filter out tables for which we don't have adequate regionspace reports yet.
     // Important that we do this after we instantiate the stores above
@@ -215,6 +236,7 @@ public class QuotaObserverChore extends ScheduledChore {
    * @param tablesWithTableQuotas The HBase tables which have quotas defined
    */
   void processTablesWithQuotas(final Set<TableName> tablesWithTableQuotas) throws IOException {
+    long numTablesInViolation = 0L;
     for (TableName table : tablesWithTableQuotas) {
       final SpaceQuota spaceQuota = tableSnapshotStore.getSpaceQuota(table);
       if (null == spaceQuota) {
@@ -227,9 +249,18 @@ public class QuotaObserverChore extends ScheduledChore {
       final SpaceQuotaSnapshot currentSnapshot = tableSnapshotStore.getCurrentState(table);
       final SpaceQuotaSnapshot targetSnapshot = tableSnapshotStore.getTargetState(table, spaceQuota);
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Processing " + table + " with current=" + currentSnapshot + ", target=" + targetSnapshot);
+        LOG.trace("Processing " + table + " with current=" + currentSnapshot + ", target="
+            + targetSnapshot);
       }
       updateTableQuota(table, currentSnapshot, targetSnapshot);
+
+      if (targetSnapshot.getQuotaStatus().isInViolation()) {
+        numTablesInViolation++;
+      }
+    }
+    // Report the number of tables in violation
+    if (null != metrics) {
+      metrics.setNumTableInSpaceQuotaViolation(numTablesInViolation);
     }
   }
 
@@ -246,6 +277,7 @@ public class QuotaObserverChore extends ScheduledChore {
   void processNamespacesWithQuotas(
       final Set<String> namespacesWithQuotas,
       final Multimap<String,TableName> tablesByNamespace) throws IOException {
+    long numNamespacesInViolation = 0L;
     for (String namespace : namespacesWithQuotas) {
       // Get the quota definition for the namespace
       final SpaceQuota spaceQuota = namespaceSnapshotStore.getSpaceQuota(namespace);
@@ -257,8 +289,22 @@ public class QuotaObserverChore extends ScheduledChore {
         continue;
       }
       final SpaceQuotaSnapshot currentSnapshot = namespaceSnapshotStore.getCurrentState(namespace);
-      final SpaceQuotaSnapshot targetSnapshot = namespaceSnapshotStore.getTargetState(namespace, spaceQuota);
+      final SpaceQuotaSnapshot targetSnapshot = namespaceSnapshotStore.getTargetState(
+          namespace, spaceQuota);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Processing " + namespace + " with current=" + currentSnapshot + ", target="
+            + targetSnapshot);
+      }
       updateNamespaceQuota(namespace, currentSnapshot, targetSnapshot, tablesByNamespace);
+
+      if (targetSnapshot.getQuotaStatus().isInViolation()) {
+        numNamespacesInViolation++;
+      }
+    }
+
+    // Report the number of namespaces in violation
+    if (null != metrics) {
+      metrics.setNumNamespacesInSpaceQuotaViolation(numNamespacesInViolation);
     }
   }
 
@@ -285,14 +331,16 @@ public class QuotaObserverChore extends ScheduledChore {
         }
       } else if (LOG.isDebugEnabled()) {
         // We're either moving into violation or changing violation policies
-        LOG.debug(table + " moving into violation of table space quota with policy of " + targetStatus.getPolicy());
+        LOG.debug(table + " moving into violation of table space quota with policy of "
+            + targetStatus.getPolicy());
       }
 
       this.snapshotNotifier.transitionTable(table, targetSnapshot);
       // Update it in memory
       tableSnapshotStore.setCurrentState(table, targetSnapshot);
     } else if (LOG.isTraceEnabled()) {
-      // Policies are the same, so we have nothing to do except log this. Don't need to re-update the quota table
+      // Policies are the same, so we have nothing to do except log this. Don't need to re-update
+      // the quota table
       if (!currentStatus.isInViolation()) {
         LOG.trace(table + " remains in observance of quota.");
       } else {
@@ -347,11 +395,14 @@ public class QuotaObserverChore extends ScheduledChore {
             }
           } else {
             // No table quota present or a table quota present that is not in violation
-            LOG.info(tableInNS + " moving into violation of namespace space quota with policy " + targetStatus.getPolicy());
+            LOG.info(tableInNS + " moving into violation of namespace space quota with policy "
+                + targetStatus.getPolicy());
             this.snapshotNotifier.transitionTable(tableInNS, targetSnapshot);
           }
         }
       }
+      // Update the new state in memory for this namespace
+      namespaceSnapshotStore.setCurrentState(namespace, targetSnapshot);
     } else {
       // Policies are the same
       if (!targetStatus.isInViolation()) {
@@ -360,7 +411,8 @@ public class QuotaObserverChore extends ScheduledChore {
           LOG.trace(namespace + " remains in observance of quota.");
         }
       } else {
-        // Namespace quota is still in violation, need to enact if the table quota is not taking priority.
+        // Namespace quota is still in violation, need to enact if the table quota is not
+        // taking priority.
         for (TableName tableInNS : tablesByNamespace.get(namespace)) {
           // Does a table policy exist
           if (tableSnapshotStore.getCurrentState(tableInNS).getQuotaStatus().isInViolation()) {
@@ -451,6 +503,22 @@ public class QuotaObserverChore extends ScheduledChore {
   }
 
   /**
+   * Returns an unmodifiable view over the current {@link SpaceQuotaSnapshot} objects
+   * for each HBase table with a quota.
+   */
+  public Map<TableName,SpaceQuotaSnapshot> getTableQuotaSnapshots() {
+    return readOnlyTableQuotaSnapshots;
+  }
+
+  /**
+   * Returns an unmodifiable view over the current {@link SpaceQuotaSnapshot} objects
+   * for each HBase namespace with a quota.
+   */
+  public Map<String,SpaceQuotaSnapshot> getNamespaceQuotaSnapshots() {
+    return readOnlyNamespaceSnapshots;
+  }
+
+  /**
    * Fetches the {@link SpaceQuotaSnapshot} for the given table.
    */
   SpaceQuotaSnapshot getTableQuotaSnapshot(TableName table) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index 0f8a289..5c52601 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -42,10 +42,14 @@
   import="org.apache.hadoop.hbase.HConstants"
   import="org.apache.hadoop.hbase.master.HMaster"
   import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
+  import="org.apache.hadoop.hbase.quotas.QuotaTableUtil"
+  import="org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot"
   import="org.apache.hadoop.hbase.util.Bytes"
   import="org.apache.hadoop.hbase.util.FSUtils"
   import="org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos"
   import="org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos"
+  import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas"
+  import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota"
   import="org.apache.hadoop.hbase.TableName"
   import="org.apache.hadoop.hbase.HColumnDescriptor"
   import="org.apache.hadoop.hbase.HBaseConfiguration"
@@ -87,6 +91,7 @@
   if (showFragmentation) {
       frags = FSUtils.getTableFragmentation(master);
   }
+  boolean quotasEnabled = conf.getBoolean("hbase.quota.enabled", false);
   String action = request.getParameter("action");
   String key = request.getParameter("key");
   String left = request.getParameter("left");
@@ -328,6 +333,60 @@ if ( fqtn != null ) {
       <td>How fragmented is the table. After a major compaction it is 0%.</td>
   </tr>
 <%  } %>
+<%
+  if (quotasEnabled) {
+    TableName tn = TableName.valueOf(fqtn);
+    SpaceQuotaSnapshot masterSnapshot = null;
+    Quotas quota = QuotaTableUtil.getTableQuota(master.getConnection(), tn);
+    if (null == quota || !quota.hasSpace()) {
+      quota = QuotaTableUtil.getNamespaceQuota(master.getConnection(), tn.getNamespaceAsString());
+      if (null != quota) {
+        masterSnapshot = QuotaTableUtil.getCurrentSnapshot(master.getConnection(), tn.getNamespaceAsString());
+      }
+    } else {
+      masterSnapshot = QuotaTableUtil.getCurrentSnapshot(master.getConnection(), tn);
+    }
+    if (null != quota && quota.hasSpace()) {
+      SpaceQuota spaceQuota = quota.getSpace();
+%>
+  <tr>
+    <td>Space Quota</td>
+    <td>
+      <table>
+        <tr>
+          <th>Property</th>
+          <th>Value</th>
+        </tr>
+        <tr>
+          <td>Limit</td>
+          <td><%= StringUtils.byteDesc(spaceQuota.getSoftLimit()) %></td>
+        </tr>
+        <tr>
+          <td>Policy</td>
+          <td><%= spaceQuota.getViolationPolicy() %></td>
+        </tr>
+<%
+      if (null != masterSnapshot) {
+%>
+        <tr>
+          <td>Usage</td>
+          <td><%= StringUtils.byteDesc(masterSnapshot.getUsage()) %></td>
+        </tr>
+        <tr>
+          <td>State</td>
+          <td><%= masterSnapshot.getQuotaStatus().isInViolation() ? "In Violation" : "In Observance" %></td>
+        </tr>
+<%
+      }
+%>
+      </table>
+    </td>
+    <td>Information about a Space Quota on this table, if set.</td>
+  </tr>
+<%
+    }
+  }
+%>
 </table>
 <h2>Table Schema</h2>
 <table class="table table-striped">

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
index 02f3721..8c1138a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
@@ -19,9 +19,14 @@ package org.apache.hadoop.hbase.master;
 
 import static org.junit.Assert.*;
 
+import java.util.AbstractMap.SimpleImmutableEntry;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Threads;
@@ -77,4 +82,16 @@ public class TestMasterMetricsWrapper {
     assertEquals(1, info.getNumDeadRegionServers());
     assertEquals(1, info.getNumWALFiles());
   }
+
+  @Test
+  public void testQuotaSnapshotConversion() {
+    MetricsMasterWrapperImpl info = new MetricsMasterWrapperImpl(
+        TEST_UTIL.getHBaseCluster().getMaster());
+    assertEquals(new SimpleImmutableEntry<Long,Long>(1024L, 2048L),
+        info.convertSnapshot(new SpaceQuotaSnapshot(
+            SpaceQuotaStatus.notInViolation(), 1024L, 2048L)));
+    assertEquals(new SimpleImmutableEntry<Long,Long>(4096L, 2048L),
+        info.convertSnapshot(new SpaceQuotaSnapshot(
+            new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 4096L, 2048L)));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/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 5a00aaf..38dbf66 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
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -33,6 +34,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 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.master.HMaster;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -187,6 +189,87 @@ public class TestQuotaStatusRPCs {
     assertEquals(SpaceViolationPolicy.NO_INSERTS, policy);
   }
 
+  @Test
+  public void testQuotaStatusFromMaster() throws Exception {
+    final long sizeLimit = 1024L * 10L; // 10KB
+    final long tableSize = 1024L * 5; // 5KB
+    final long nsLimit = Long.MAX_VALUE;
+    final int numRegions = 10;
+    final TableName tn = helper.createTableWithRegions(numRegions);
+
+    // Define the quota
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, sizeLimit, SpaceViolationPolicy.NO_INSERTS);
+    TEST_UTIL.getAdmin().setQuota(settings);
+    QuotaSettings nsSettings = QuotaSettingsFactory.limitNamespaceSpace(
+        tn.getNamespaceAsString(), nsLimit, SpaceViolationPolicy.NO_INSERTS);
+    TEST_UTIL.getAdmin().setQuota(nsSettings);
+
+    // Write at least `tableSize` data
+    helper.writeData(tn, tableSize);
+
+    final Connection conn = TEST_UTIL.getConnection();
+    // Make sure the master has a snapshot for our table
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        SpaceQuotaSnapshot snapshot = QuotaTableUtil.getCurrentSnapshot(conn, tn);
+        LOG.info("Table snapshot after initial ingest: " + snapshot);
+        if (null == snapshot) {
+          return false;
+        }
+        return snapshot.getLimit() == sizeLimit && snapshot.getUsage() > 0L;
+      }
+    });
+    final AtomicReference<Long> nsUsage = new AtomicReference<>();
+    // If we saw the table snapshot, we should also see the namespace snapshot
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000 * 1000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        SpaceQuotaSnapshot snapshot = QuotaTableUtil.getCurrentSnapshot(
+            conn, tn.getNamespaceAsString());
+        LOG.debug("Namespace snapshot after initial ingest: " + snapshot);
+        if (null == snapshot) {
+          return false;
+        }
+        nsUsage.set(snapshot.getUsage());
+        return snapshot.getLimit() == nsLimit && snapshot.getUsage() > 0;
+      }
+    });
+
+    try {
+      helper.writeData(tn, tableSize * 2L);
+    } catch (SpaceLimitingException e) {
+      // Pass
+    }
+
+    // Wait for the status to move to violation
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        SpaceQuotaSnapshot snapshot = QuotaTableUtil.getCurrentSnapshot(conn, tn);
+        LOG.info("Table snapshot after second ingest: " + snapshot);
+        if (null == snapshot) {
+          return false;
+        }
+        return snapshot.getQuotaStatus().isInViolation();
+      }
+    });
+    // The namespace should still not be in violation, but have a larger usage than previously
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        SpaceQuotaSnapshot snapshot = QuotaTableUtil.getCurrentSnapshot(
+            conn, tn.getNamespaceAsString());
+        LOG.debug("Namespace snapshot after second ingest: " + snapshot);
+        if (null == snapshot) {
+          return false;
+        }
+        return snapshot.getUsage() > nsUsage.get() && !snapshot.getQuotaStatus().isInViolation();
+      }
+    });
+  }
+
   private int countRegionsForTable(TableName tn, Map<HRegionInfo,Long> regionSizes) {
     int size = 0;
     for (HRegionInfo regionInfo : regionSizes.keySet()) {


[28/50] [abbrv] hbase git commit: HBASE-17001 Enforce quota violation policies in the RegionServer

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/BaseViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/BaseViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/BaseViolationPolicyEnforcement.java
new file mode 100644
index 0000000..ec8f1bf
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/BaseViolationPolicyEnforcement.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public class BaseViolationPolicyEnforcement {
+
+  static final Append APPEND = new Append(Bytes.toBytes("foo"));
+  static final Delete DELETE = new Delete(Bytes.toBytes("foo"));
+  static final Increment INCREMENT = new Increment(Bytes.toBytes("foo"));
+  static final Put PUT = new Put(Bytes.toBytes("foo"));
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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
new file mode 100644
index 0000000..abe1b9d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestBulkLoadCheckingViolationPolicyEnforcement {
+
+  FileSystem fs;
+  RegionServerServices rss;
+  TableName tableName;
+  SpaceViolationPolicyEnforcement policy;
+
+  @Before
+  public void setup() {
+    fs = mock(FileSystem.class);
+    rss = mock(RegionServerServices.class);
+    tableName = TableName.valueOf("foo");
+    policy = new BulkLoadVerifyingViolationPolicyEnforcement();
+  }
+
+  @Test
+  public void testFilesUnderLimit() throws Exception {
+    final List<String> paths = new ArrayList<>();
+    final List<FileStatus> statuses = new ArrayList<>();
+    final long length = 100L * 1024L;
+    for (int i = 0; i < 5; i++) {
+      String path = "/" + i;
+      FileStatus status = mock(FileStatus.class);
+      when(fs.getFileStatus(new Path(path))).thenReturn(status);
+      when(status.getLen()).thenReturn(length);
+      when(status.isFile()).thenReturn(true);
+      paths.add(path);
+      statuses.add(status);
+    }
+
+    // Quota is not in violation now
+    SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, length * 6);
+
+    policy.initialize(rss, tableName, snapshot);
+
+    policy.checkBulkLoad(fs, paths);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testFileIsNotAFile() throws Exception {
+    final List<String> paths = new ArrayList<>();
+    String path = "/1";
+    FileStatus status = mock(FileStatus.class);
+    when(fs.getFileStatus(new Path(path))).thenReturn(status);
+    when(status.getLen()).thenReturn(1000L);
+    when(status.isFile()).thenReturn(false);
+    paths.add(path);
+
+    // Quota is not in violation now
+    SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, Long.MAX_VALUE);
+
+    policy.initialize(rss, tableName, snapshot);
+
+    // If the file to bulk load isn't a file, this should throw an exception
+    policy.checkBulkLoad(fs, paths);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testOneFileInBatchOverLimit() throws Exception {
+    final List<String> paths = new ArrayList<>();
+    final List<FileStatus> statuses = new ArrayList<>();
+    final long length = 1000L * 1024L;
+    for (int i = 0; i < 5; i++) {
+      String path = "/" + i;
+      FileStatus status = mock(FileStatus.class);
+      when(fs.getFileStatus(new Path(path))).thenReturn(status);
+      when(status.getLen()).thenReturn(length);
+      when(status.isFile()).thenReturn(true);
+      paths.add(path);
+      statuses.add(status);
+    }
+
+    // Quota is not in violation now
+    SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, 1024L);
+
+    policy.initialize(rss, tableName, snapshot);
+
+    policy.checkBulkLoad(fs, paths);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testSumOfFilesOverLimit() throws Exception {
+    final List<String> paths = new ArrayList<>();
+    final List<FileStatus> statuses = new ArrayList<>();
+    final long length = 1024L;
+    for (int i = 0; i < 5; i++) {
+      String path = "/" + i;
+      FileStatus status = mock(FileStatus.class);
+      when(fs.getFileStatus(new Path(path))).thenReturn(status);
+      when(status.getLen()).thenReturn(length);
+      when(status.isFile()).thenReturn(true);
+      paths.add(path);
+      statuses.add(status);
+    }
+
+    // Quota is not in violation now, but 5*1024 files would push us to violation
+    SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, 5000L);
+
+    policy.initialize(rss, tableName, snapshot);
+
+    policy.checkBulkLoad(fs, paths);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java
new file mode 100644
index 0000000..c42d866
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link DisableTableViolationPolicyEnforcement}.
+ */
+@Category(SmallTests.class)
+public class TestDisableTableViolationPolicyEnforcement extends BaseViolationPolicyEnforcement {
+
+  private DisableTableViolationPolicyEnforcement enforcement;
+
+  @Before
+  public void setup() {
+    enforcement = new DisableTableViolationPolicyEnforcement();
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckPut() throws SpaceLimitingException {
+    // If the policy is enacted, it will always throw an exception
+    // to avoid constantly re-checking the table state.
+    enforcement.check(PUT);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckAppend() throws SpaceLimitingException {
+    enforcement.check(APPEND);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckDelete() throws SpaceLimitingException {
+    enforcement.check(DELETE);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckIncrement() throws SpaceLimitingException {
+    enforcement.check(INCREMENT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java
new file mode 100644
index 0000000..1115fab
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link NoInsertsViolationPolicyEnforcement}.
+ */
+@Category(SmallTests.class)
+public class TestNoInsertsViolationPolicyEnforcement extends BaseViolationPolicyEnforcement {
+
+  private NoInsertsViolationPolicyEnforcement enforcement;
+
+  @Before
+  public void setup() {
+    enforcement = new NoInsertsViolationPolicyEnforcement();
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckAppend() throws Exception {
+    enforcement.check(APPEND);
+  }
+
+  @Test
+  public void testCheckDelete() throws Exception {
+    enforcement.check(DELETE);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckIncrement() throws Exception {
+    enforcement.check(INCREMENT);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckPut() throws Exception {
+    enforcement.check(PUT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java
new file mode 100644
index 0000000..1348eb7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link NoWritesCompactionsViolationPolicyEnforcement};
+ */
+@Category(SmallTests.class)
+public class TestNoWritesCompactionsViolationPolicyEnforcement
+    extends BaseViolationPolicyEnforcement {
+
+  private NoWritesCompactionsViolationPolicyEnforcement enforcement;
+
+  @Before
+  public void setup() {
+    enforcement = new NoWritesCompactionsViolationPolicyEnforcement();
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckAppend() throws Exception {
+    enforcement.check(APPEND);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckDelete() throws Exception {
+    enforcement.check(DELETE);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckIncrement() throws Exception {
+    enforcement.check(INCREMENT);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckPut() throws Exception {
+    enforcement.check(PUT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java
new file mode 100644
index 0000000..c032f26
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link NoWritesViolationPolicyEnforcement}.
+ */
+@Category(SmallTests.class)
+public class TestNoWritesViolationPolicyEnforcement extends BaseViolationPolicyEnforcement {
+
+  private NoWritesViolationPolicyEnforcement enforcement;
+
+  @Before
+  public void setup() {
+    enforcement = new NoWritesViolationPolicyEnforcement();
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckAppend() throws Exception {
+    enforcement.check(APPEND);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckDelete() throws Exception {
+    enforcement.check(DELETE);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckIncrement() throws Exception {
+    enforcement.check(INCREMENT);
+  }
+
+  @Test(expected = SpaceLimitingException.class)
+  public void testCheckPut() throws Exception {
+    enforcement.check(PUT);
+  }
+}


[04/50] [abbrv] hbase git commit: HBASE-17888: Added generic methods for updating metrics on submit and finish of a procedure execution

Posted by el...@apache.org.
HBASE-17888: Added generic methods for updating metrics on submit and finish of a procedure execution

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-16961
Commit: c8461456d0ae81b90d67d36e1e077ae1d01102e5
Parents: e2a7461
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Mon Apr 10 15:32:43 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Fri Apr 14 11:51:08 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   2 +-
 .../org/apache/hadoop/hbase/ProcedureInfo.java  |  20 +-
 .../master/MetricsAssignmentManagerSource.java  |   9 +-
 .../MetricsAssignmentManagerSourceImpl.java     |   9 +-
 .../hadoop/hbase/procedure2/Procedure.java      |  41 +-
 .../hbase/procedure2/ProcedureExecutor.java     |  11 +
 .../hadoop/hbase/procedure2/ProcedureUtil.java  |  10 +-
 .../hbase/procedure2/TestProcedureMetrics.java  | 254 ++++++++++
 .../procedure2/TestStateMachineProcedure.java   |   1 -
 .../shaded/protobuf/generated/MasterProtos.java | 490 +++++++++----------
 .../protobuf/generated/ProcedureProtos.java     | 146 +++---
 .../src/main/protobuf/Master.proto              |   2 +-
 .../src/main/protobuf/Procedure.proto           |   2 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   4 +-
 .../master/procedure/ServerCrashProcedure.java  |   2 +-
 .../hbase-webapps/master/procedures.jsp         |   2 +-
 .../main/ruby/shell/commands/list_procedures.rb |   6 +-
 17 files changed, 652 insertions(+), 359 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 155a272..cadd6cc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -2114,7 +2114,7 @@ public class HBaseAdmin implements Admin {
         procedureState, procProto.hasParentId() ? procProto.getParentId() : -1, nonceKey,
             procProto.hasException()?
                 ForeignExceptionUtil.toIOException(procProto.getException()): null,
-            procProto.getLastUpdate(), procProto.getStartTime(),
+            procProto.getLastUpdate(), procProto.getSubmittedTime(),
             procProto.hasResult()? procProto.getResult().toByteArray() : null);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
index bb8bb08..6104c22 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
@@ -39,7 +39,7 @@ public class ProcedureInfo implements Cloneable {
   private final NonceKey nonceKey;
   private final IOException exception;
   private final long lastUpdate;
-  private final long startTime;
+  private final long submittedTime;
   private final byte[] result;
 
   private long clientAckTime = -1;
@@ -54,7 +54,7 @@ public class ProcedureInfo implements Cloneable {
       final NonceKey nonceKey,
       final IOException exception,
       final long lastUpdate,
-      final long startTime,
+      final long submittedTime,
       final byte[] result) {
     this.procId = procId;
     this.procName = procName;
@@ -63,7 +63,7 @@ public class ProcedureInfo implements Cloneable {
     this.parentId = parentId;
     this.nonceKey = nonceKey;
     this.lastUpdate = lastUpdate;
-    this.startTime = startTime;
+    this.submittedTime = submittedTime;
 
     // If the procedure is completed, we should treat exception and result differently
     this.exception = exception;
@@ -74,7 +74,7 @@ public class ProcedureInfo implements Cloneable {
       justification="Intentional; calling super class clone doesn't make sense here.")
   public ProcedureInfo clone() {
     return new ProcedureInfo(procId, procName, procOwner, procState, parentId, nonceKey,
-      exception, lastUpdate, startTime, result);
+      exception, lastUpdate, submittedTime, result);
   }
 
   @Override
@@ -96,10 +96,10 @@ public class ProcedureInfo implements Cloneable {
     sb.append(procState);
 
     long now = EnvironmentEdgeManager.currentTime();
-    sb.append(", startTime=");
-    sb.append(StringUtils.formatTime(now - startTime));
+    sb.append(", submittedTime=");
+    sb.append(StringUtils.formatTime(now - submittedTime));
     sb.append(" ago, lastUpdate=");
-    sb.append(StringUtils.formatTime(now - startTime));
+    sb.append(StringUtils.formatTime(now - submittedTime));
     sb.append(" ago");
 
     if (isFailed()) {
@@ -168,8 +168,8 @@ public class ProcedureInfo implements Cloneable {
     return result;
   }
 
-  public long getStartTime() {
-    return startTime;
+  public long getSubmittedTime() {
+    return submittedTime;
   }
 
   public long getLastUpdate() {
@@ -177,7 +177,7 @@ public class ProcedureInfo implements Cloneable {
   }
 
   public long executionTime() {
-    return lastUpdate - startTime;
+    return lastUpdate - submittedTime;
   }
 
   @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
index f6c9cb8..fa7bbec 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
@@ -40,7 +40,7 @@ public interface MetricsAssignmentManagerSource extends BaseSource {
   /**
    * Description
    */
-  String METRICS_DESCRIPTION = "Metrics about HBase master assingment manager.";
+  String METRICS_DESCRIPTION = "Metrics about HBase master assignment manager.";
 
   String RIT_COUNT_NAME = "ritCount";
   String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold";
@@ -49,6 +49,13 @@ public interface MetricsAssignmentManagerSource extends BaseSource {
   String ASSIGN_TIME_NAME = "assign";
   String BULK_ASSIGN_TIME_NAME = "bulkAssign";
 
+  String RIT_COUNT_DESC = "Current number of Regions In Transition (Gauge).";
+  String RIT_COUNT_OVER_THRESHOLD_DESC =
+      "Current number of Regions In Transition over threshold time (Gauge).";
+  String RIT_OLDEST_AGE_DESC = "Timestamp in milliseconds of the oldest Region In Transition (Gauge).";
+  String RIT_DURATION_DESC =
+      "Total durations in milliseconds for all Regions in Transition (Histogram).";
+
   void updateAssignmentTime(long time);
 
   void updateBulkAssignTime(long time);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
index ab504f5..faae044 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
@@ -46,12 +46,13 @@ public class MetricsAssignmentManagerSourceImpl
   }
 
   public void init() {
-    ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, "", 0l);
-    ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME, "", 0l);
-    ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, "", 0l);
+    ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, RIT_COUNT_DESC, 0l);
+    ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME,
+        RIT_COUNT_OVER_THRESHOLD_DESC,0l);
+    ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, RIT_OLDEST_AGE_DESC, 0l);
     assignTimeHisto = metricsRegistry.newTimeHistogram(ASSIGN_TIME_NAME);
     bulkAssignTimeHisto = metricsRegistry.newTimeHistogram(BULK_ASSIGN_TIME_NAME);
-    ritDurationHisto = metricsRegistry.newTimeHistogram(RIT_DURATION_NAME);
+    ritDurationHisto = metricsRegistry.newTimeHistogram(RIT_DURATION_NAME, RIT_DURATION_DESC);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index 761ab3a..591c0d0 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -38,7 +38,7 @@ import com.google.common.annotations.VisibleForTesting;
 
 /**
  * Base Procedure class responsible to handle the Procedure Metadata
- * e.g. state, startTime, lastUpdate, stack-indexes, ...
+ * e.g. state, submittedTime, lastUpdate, stack-indexes, ...
  *
  * execute() is called each time the procedure is executed.
  * it may be called multiple times in case of failure and restart, so the
@@ -73,7 +73,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
   private long parentProcId = NO_PROC_ID;
   private long rootProcId = NO_PROC_ID;
   private long procId = NO_PROC_ID;
-  private long startTime;
+  private long submittedTime;
 
   // runtime state, updated every operation
   private ProcedureState state = ProcedureState.INITIALIZING;
@@ -240,6 +240,27 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
     return true;
   }
 
+  /**
+   * This function will be called just when procedure is submitted for execution. Override this
+   * method to update the metrics at the beginning of the procedure
+   */
+  protected void updateMetricsOnSubmit(final TEnvironment env) {}
+
+  /**
+   * This function will be called just after procedure execution is finished. Override this method
+   * to update metrics at the end of the procedure
+   *
+   * TODO: As any of the sub-procedures on failure rolls back all procedures in the stack,
+   * including successfully finished siblings, this function may get called twice in certain
+   * cases for certain procedures. Explore further if this can be called once.
+   *
+   * @param env
+   * @param runtime - Runtime of the procedure in milliseconds
+   * @param success - true if procedure is completed successfully
+   */
+  protected void updateMetricsOnFinish(final TEnvironment env, final long runtime,
+                                       boolean success) {}
+
   @Override
   public String toString() {
     // Return the simple String presentation of the procedure.
@@ -287,8 +308,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
   public String toStringDetails() {
     final StringBuilder sb = toStringSimpleSB();
 
-    sb.append(" startTime=");
-    sb.append(getStartTime());
+    sb.append(" submittedTime=");
+    sb.append(getSubmittedTime());
 
     sb.append(" lastUpdate=");
     sb.append(getLastUpdate());
@@ -353,8 +374,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
     return nonceKey;
   }
 
-  public long getStartTime() {
-    return startTime;
+  public long getSubmittedTime() {
+    return submittedTime;
   }
 
   public String getOwner() {
@@ -372,7 +393,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
   @InterfaceAudience.Private
   protected void setProcId(final long procId) {
     this.procId = procId;
-    this.startTime = EnvironmentEdgeManager.currentTime();
+    this.submittedTime = EnvironmentEdgeManager.currentTime();
     setState(ProcedureState.RUNNABLE);
   }
 
@@ -414,8 +435,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
    * the creation/deserialization.
    */
   @InterfaceAudience.Private
-  protected void setStartTime(final long startTime) {
-    this.startTime = startTime;
+  protected void setSubmittedTime(final long submittedTime) {
+    this.submittedTime = submittedTime;
   }
 
   // ==========================================================================
@@ -478,7 +499,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
    * @return the time elapsed between the last update and the start time of the procedure.
    */
   public long elapsedTime() {
-    return getLastUpdate() - getStartTime();
+    return getLastUpdate() - getSubmittedTime();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index 43f5839..1bb6118 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -831,6 +831,9 @@ public class ProcedureExecutor<TEnvironment> {
   private long pushProcedure(final Procedure proc) {
     final long currentProcId = proc.getProcId();
 
+    // Update metrics on start of a procedure
+    proc.updateMetricsOnSubmit(getEnvironment());
+
     // Create the rollback stack for the procedure
     RootProcedureState stack = new RootProcedureState();
     rollbackStack.put(currentProcId, stack);
@@ -1145,6 +1148,9 @@ public class ProcedureExecutor<TEnvironment> {
       }
 
       if (proc.isSuccess()) {
+        // update metrics on finishing the procedure
+        proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), true);
+
         if (LOG.isDebugEnabled()) {
           LOG.debug("Finished " + proc + " in " + StringUtils.humanTimeDiff(proc.elapsedTime()));
         }
@@ -1276,6 +1282,10 @@ public class ProcedureExecutor<TEnvironment> {
 
     if (proc.removeStackIndex()) {
       proc.setState(ProcedureState.ROLLEDBACK);
+
+      // update metrics on finishing the procedure (fail)
+      proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), false);
+
       if (proc.hasParent()) {
         store.delete(proc.getProcId());
         procedures.remove(proc.getProcId());
@@ -1444,6 +1454,7 @@ public class ProcedureExecutor<TEnvironment> {
   private void submitChildrenProcedures(final Procedure[] subprocs) {
     for (int i = 0; i < subprocs.length; ++i) {
       final Procedure subproc = subprocs[i];
+      subproc.updateMetricsOnSubmit(getEnvironment());
       assert !procedures.containsKey(subproc.getProcId());
       procedures.put(subproc.getProcId(), subproc);
       scheduler.addFront(subproc);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
index 372dcac..b4222c7 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
@@ -97,7 +97,7 @@ public final class ProcedureUtil {
       .setClassName(proc.getClass().getName())
       .setProcId(proc.getProcId())
       .setState(proc.getState())
-      .setStartTime(proc.getStartTime())
+      .setSubmittedTime(proc.getSubmittedTime())
       .setLastUpdate(proc.getLastUpdate());
 
     if (proc.hasParent()) {
@@ -164,7 +164,7 @@ public final class ProcedureUtil {
     // set fields
     proc.setProcId(proto.getProcId());
     proc.setState(proto.getState());
-    proc.setStartTime(proto.getStartTime());
+    proc.setSubmittedTime(proto.getSubmittedTime());
     proc.setLastUpdate(proto.getLastUpdate());
 
     if (proto.hasParentId()) {
@@ -217,7 +217,7 @@ public final class ProcedureUtil {
 
     builder.setClassName(procInfo.getProcName());
     builder.setProcId(procInfo.getProcId());
-    builder.setStartTime(procInfo.getStartTime());
+    builder.setSubmittedTime(procInfo.getSubmittedTime());
     builder.setState(ProcedureProtos.ProcedureState.valueOf(procInfo.getProcState().name()));
     builder.setLastUpdate(procInfo.getLastUpdate());
 
@@ -257,7 +257,7 @@ public final class ProcedureUtil {
         procProto.hasParentId() ? procProto.getParentId() : -1, nonceKey,
         procProto.hasException() ?
           ForeignExceptionUtil.toIOException(procProto.getException()) : null,
-        procProto.getLastUpdate(), procProto.getStartTime(),
+        procProto.getLastUpdate(), procProto.getSubmittedTime(),
         procProto.hasResult() ? procProto.getResult().toByteArray() : null);
   }
 
@@ -279,6 +279,6 @@ public final class ProcedureUtil {
         convertToProcedureState(proc.getState()),
         proc.hasParent() ? proc.getParentProcId() : -1, nonceKey,
         exception != null ? exception.unwrapRemoteIOException() : null,
-        proc.getLastUpdate(), proc.getStartTime(), proc.getResult());
+        proc.getLastUpdate(), proc.getSubmittedTime(), proc.getResult());
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java
new file mode 100644
index 0000000..0a57efa
--- /dev/null
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java
@@ -0,0 +1,254 @@
+/**
+ * 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.procedure2;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({MasterTests.class, SmallTests.class})
+public class TestProcedureMetrics {
+  private static final Log LOG = LogFactory.getLog(TestProcedureMetrics.class);
+
+  private static final int PROCEDURE_EXECUTOR_SLOTS = 1;
+
+  private TestProcEnv procEnv;
+  private static ProcedureExecutor<TestProcEnv> procExecutor;
+  private ProcedureStore procStore;
+
+  private HBaseCommonTestingUtility htu;
+  private FileSystem fs;
+  private Path testDir;
+  private Path logDir;
+
+  private static int beginCount = 0;
+  private static int successCount = 0;
+  private static int failedCount = 0;
+
+
+  @Before
+  public void setUp() throws IOException {
+    htu = new HBaseCommonTestingUtility();
+    testDir = htu.getDataTestDir();
+    fs = testDir.getFileSystem(htu.getConfiguration());
+    assertTrue(testDir.depth() > 1);
+
+    logDir = new Path(testDir, "proc-logs");
+    procEnv = new TestProcEnv();
+    procStore = ProcedureTestingUtility.createStore(htu.getConfiguration(), fs, logDir);
+    procExecutor = new ProcedureExecutor<TestProcEnv>(htu.getConfiguration(), procEnv, procStore);
+    procExecutor.testing = new ProcedureExecutor.Testing();
+    procStore.start(PROCEDURE_EXECUTOR_SLOTS);
+    procExecutor.start(PROCEDURE_EXECUTOR_SLOTS, true);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    procExecutor.stop();
+    procStore.stop(false);
+    fs.delete(logDir, true);
+  }
+
+  @Test
+  public void testMetricForSimpleProcedure() throws Exception {
+    // procedure that executes successfully
+    ProcedureMetrics proc = new ProcedureMetrics(true);
+    long id = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertNotEquals("ProcId zero!", 0, id);
+    beginCount++;
+    successCount++;
+    ProcedureTestingUtility.waitProcedure(procExecutor, proc);
+    assertEquals("beginCount doesn't match!", beginCount, proc.beginCount);
+    assertEquals("successCount doesn't match!", successCount, proc.successCount);
+    assertEquals("failedCont doesn't match!", failedCount, proc.failedCount);
+  }
+
+  @Test
+  public void testMetricsForFailedProcedure() throws Exception {
+    // procedure that fails
+    ProcedureMetrics proc = new ProcedureMetrics(false);
+    long id = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertNotEquals("ProcId zero!", 0, id);
+    beginCount++;
+    failedCount++;
+    ProcedureTestingUtility.waitProcedure(procExecutor, proc);
+    assertEquals("beginCount doesn't match!", beginCount, proc.beginCount);
+    assertEquals("successCount doesn't match!", successCount, proc.successCount);
+    assertEquals("failedCont doesn't match!", failedCount, proc.failedCount);
+  }
+
+  @Test
+  public void testMetricForYieldProcedure() throws Exception {
+    // procedure that yields
+    ProcedureMetrics proc = new ProcedureMetrics(true, true);
+    long id = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertNotEquals("ProcId zero!", 0, id);
+    beginCount++;
+    successCount++;
+    ProcedureTestingUtility.waitProcedure(procExecutor, proc);
+    assertEquals("beginCount doesn't match!", beginCount, proc.beginCount);
+    assertEquals("successCount doesn't match!", successCount, proc.successCount);
+    assertEquals("failedCont doesn't match!", failedCount, proc.failedCount);
+  }
+
+  @Test
+  public void testMetricForFailedYiledProcedure() {
+    // procedure that yields and fails
+    ProcedureMetrics proc = new ProcedureMetrics(false, true);
+    long id = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertNotEquals("ProcId zero!", 0, id);
+    beginCount++;
+    failedCount++;
+    ProcedureTestingUtility.waitProcedure(procExecutor, proc);
+    assertEquals("beginCount doesn't match!", beginCount, proc.beginCount);
+    assertEquals("successCount doesn't match!", successCount, proc.successCount);
+    assertEquals("failedCont doesn't match!", failedCount, proc.failedCount);
+  }
+
+  @Test
+  public void testMetricForProcedureWithChildren() throws Exception {
+    // Procedure that yileds with one of the sub-procedures that fail
+    int subProcCount = 10;
+    int failChildIndex = 2;
+    int yiledChildIndex = -1;
+    ProcedureMetrics[] subprocs = new ProcedureMetrics[subProcCount];
+    for (int i = 0; i < subProcCount; ++i) {
+      subprocs[i] = new ProcedureMetrics(failChildIndex != i, yiledChildIndex == i, 3);
+    }
+
+    ProcedureMetrics proc = new ProcedureMetrics(true, true, 3, subprocs);
+    long id = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertNotEquals("ProcId zero!", 0, id);
+    beginCount += subProcCount + 1;
+    successCount += subProcCount - (failChildIndex + 1);
+    if (failChildIndex >= 0) {
+      failedCount += subProcCount + 1;
+    } else {
+      successCount++;
+    }
+    ProcedureTestingUtility.waitProcedure(procExecutor, proc);
+    assertEquals("beginCount doesn't match!", beginCount, proc.beginCount);
+    assertEquals("successCount doesn't match!", successCount, proc.successCount);
+    assertEquals("failedCont doesn't match!", failedCount, proc.failedCount);
+  }
+
+  private static class TestProcEnv {
+    public boolean toggleKillBeforeStoreUpdate = false;
+    public boolean triggerRollbackOnChild = false;
+  }
+
+  public static class ProcedureMetrics extends SequentialProcedure<TestProcEnv> {
+    public static long beginCount = 0;
+    public static long successCount = 0;
+    public static long failedCount = 0;
+
+    private boolean success;
+    private boolean yield;
+    private int yieldCount;
+    private int yieldNum;
+
+    private ProcedureMetrics[] subprocs = null;
+
+    public ProcedureMetrics() {
+      this(true);
+    }
+
+    public ProcedureMetrics(boolean success) {
+      this(success, true);
+    }
+
+    public ProcedureMetrics(boolean success, boolean yield) {
+      this(success, yield, 1);
+    }
+
+    public ProcedureMetrics(boolean success, boolean yield, int yieldCount) {
+      this(success, yield, yieldCount, null);
+    }
+
+    public ProcedureMetrics(boolean success, ProcedureMetrics[] subprocs) {
+      this(success, false, 1, subprocs);
+    }
+
+    public ProcedureMetrics(boolean success, boolean yield, int yieldCount,
+                            ProcedureMetrics[] subprocs) {
+      this.success = success;
+      this.yield = yield;
+      this.yieldCount = yieldCount;
+      this.subprocs = subprocs;
+      yieldNum = 0;
+    }
+
+    @Override
+    protected void updateMetricsOnSubmit(TestProcEnv env) {
+      beginCount++;
+    }
+
+    @Override
+    protected Procedure[] execute(TestProcEnv env) throws ProcedureYieldException,
+        ProcedureSuspendedException, InterruptedException {
+      if (this.yield) {
+        if (yieldNum < yieldCount) {
+          yieldNum++;
+          throw new ProcedureYieldException();
+        }
+      }
+      if (!this.success) {
+        setFailure("Failed", new InterruptedException("Failed"));
+        return null;
+      }
+      return subprocs;
+    }
+
+    @Override
+    protected void rollback(TestProcEnv env) throws IOException, InterruptedException {
+
+    }
+
+    @Override
+    protected boolean abort(TestProcEnv env) {
+      return false;
+    }
+
+    @Override
+    protected void updateMetricsOnFinish(final TestProcEnv env, final long time,
+    boolean success) {
+      if (success) {
+        successCount++;
+      } else {
+        failedCount++;
+      }
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
index 4b36c76..82b767e 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.procedure2;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index 0071bef..8ff19b2 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -58948,13 +58948,13 @@ public final class MasterProtos {
     org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.State getState();
 
     /**
-     * <code>optional uint64 start_time = 2;</code>
+     * <code>optional uint64 submitted_time = 2;</code>
      */
-    boolean hasStartTime();
+    boolean hasSubmittedTime();
     /**
-     * <code>optional uint64 start_time = 2;</code>
+     * <code>optional uint64 submitted_time = 2;</code>
      */
-    long getStartTime();
+    long getSubmittedTime();
 
     /**
      * <code>optional uint64 last_update = 3;</code>
@@ -59000,7 +59000,7 @@ public final class MasterProtos {
     }
     private GetProcedureResultResponse() {
       state_ = 0;
-      startTime_ = 0L;
+      submittedTime_ = 0L;
       lastUpdate_ = 0L;
       result_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
@@ -59046,7 +59046,7 @@ public final class MasterProtos {
             }
             case 16: {
               bitField0_ |= 0x00000002;
-              startTime_ = input.readUInt64();
+              submittedTime_ = input.readUInt64();
               break;
             }
             case 24: {
@@ -59212,19 +59212,19 @@ public final class MasterProtos {
       return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.State.NOT_FOUND : result;
     }
 
-    public static final int START_TIME_FIELD_NUMBER = 2;
-    private long startTime_;
+    public static final int SUBMITTED_TIME_FIELD_NUMBER = 2;
+    private long submittedTime_;
     /**
-     * <code>optional uint64 start_time = 2;</code>
+     * <code>optional uint64 submitted_time = 2;</code>
      */
-    public boolean hasStartTime() {
+    public boolean hasSubmittedTime() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional uint64 start_time = 2;</code>
+     * <code>optional uint64 submitted_time = 2;</code>
      */
-    public long getStartTime() {
-      return startTime_;
+    public long getSubmittedTime() {
+      return submittedTime_;
     }
 
     public static final int LAST_UPDATE_FIELD_NUMBER = 3;
@@ -59298,7 +59298,7 @@ public final class MasterProtos {
         output.writeEnum(1, state_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeUInt64(2, startTime_);
+        output.writeUInt64(2, submittedTime_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeUInt64(3, lastUpdate_);
@@ -59323,7 +59323,7 @@ public final class MasterProtos {
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(2, startTime_);
+          .computeUInt64Size(2, submittedTime_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
@@ -59358,10 +59358,10 @@ public final class MasterProtos {
       if (hasState()) {
         result = result && state_ == other.state_;
       }
-      result = result && (hasStartTime() == other.hasStartTime());
-      if (hasStartTime()) {
-        result = result && (getStartTime()
-            == other.getStartTime());
+      result = result && (hasSubmittedTime() == other.hasSubmittedTime());
+      if (hasSubmittedTime()) {
+        result = result && (getSubmittedTime()
+            == other.getSubmittedTime());
       }
       result = result && (hasLastUpdate() == other.hasLastUpdate());
       if (hasLastUpdate()) {
@@ -59393,10 +59393,10 @@ public final class MasterProtos {
         hash = (37 * hash) + STATE_FIELD_NUMBER;
         hash = (53 * hash) + state_;
       }
-      if (hasStartTime()) {
-        hash = (37 * hash) + START_TIME_FIELD_NUMBER;
+      if (hasSubmittedTime()) {
+        hash = (37 * hash) + SUBMITTED_TIME_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getStartTime());
+            getSubmittedTime());
       }
       if (hasLastUpdate()) {
         hash = (37 * hash) + LAST_UPDATE_FIELD_NUMBER;
@@ -59532,7 +59532,7 @@ public final class MasterProtos {
         super.clear();
         state_ = 0;
         bitField0_ = (bitField0_ & ~0x00000001);
-        startTime_ = 0L;
+        submittedTime_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
         lastUpdate_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -59575,7 +59575,7 @@ public final class MasterProtos {
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.startTime_ = startTime_;
+        result.submittedTime_ = submittedTime_;
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000004;
         }
@@ -59637,8 +59637,8 @@ public final class MasterProtos {
         if (other.hasState()) {
           setState(other.getState());
         }
-        if (other.hasStartTime()) {
-          setStartTime(other.getStartTime());
+        if (other.hasSubmittedTime()) {
+          setSubmittedTime(other.getSubmittedTime());
         }
         if (other.hasLastUpdate()) {
           setLastUpdate(other.getLastUpdate());
@@ -59716,34 +59716,34 @@ public final class MasterProtos {
         return this;
       }
 
-      private long startTime_ ;
+      private long submittedTime_ ;
       /**
-       * <code>optional uint64 start_time = 2;</code>
+       * <code>optional uint64 submitted_time = 2;</code>
        */
-      public boolean hasStartTime() {
+      public boolean hasSubmittedTime() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional uint64 start_time = 2;</code>
+       * <code>optional uint64 submitted_time = 2;</code>
        */
-      public long getStartTime() {
-        return startTime_;
+      public long getSubmittedTime() {
+        return submittedTime_;
       }
       /**
-       * <code>optional uint64 start_time = 2;</code>
+       * <code>optional uint64 submitted_time = 2;</code>
        */
-      public Builder setStartTime(long value) {
+      public Builder setSubmittedTime(long value) {
         bitField0_ |= 0x00000002;
-        startTime_ = value;
+        submittedTime_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional uint64 start_time = 2;</code>
+       * <code>optional uint64 submitted_time = 2;</code>
        */
-      public Builder clearStartTime() {
+      public Builder clearSubmittedTime() {
         bitField0_ = (bitField0_ & ~0x00000002);
-        startTime_ = 0L;
+        submittedTime_ = 0L;
         onChanged();
         return this;
       }
@@ -76629,214 +76629,214 @@ public final class MasterProtos {
       "Description\"`\n\027IsProcedureDoneResponse\022\023" +
       "\n\004done\030\001 \001(\010:\005false\0220\n\010snapshot\030\002 \001(\0132\036.",
       "hbase.pb.ProcedureDescription\",\n\031GetProc" +
-      "edureResultRequest\022\017\n\007proc_id\030\001 \002(\004\"\371\001\n\032" +
+      "edureResultRequest\022\017\n\007proc_id\030\001 \002(\004\"\375\001\n\032" +
       "GetProcedureResultResponse\0229\n\005state\030\001 \002(" +
       "\0162*.hbase.pb.GetProcedureResultResponse." +
-      "State\022\022\n\nstart_time\030\002 \001(\004\022\023\n\013last_update" +
-      "\030\003 \001(\004\022\016\n\006result\030\004 \001(\014\0224\n\texception\030\005 \001(" +
-      "\0132!.hbase.pb.ForeignExceptionMessage\"1\n\005" +
-      "State\022\r\n\tNOT_FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n\010FIN" +
-      "ISHED\020\002\"M\n\025AbortProcedureRequest\022\017\n\007proc" +
-      "_id\030\001 \002(\004\022#\n\025mayInterruptIfRunning\030\002 \001(\010",
-      ":\004true\"6\n\026AbortProcedureResponse\022\034\n\024is_p" +
-      "rocedure_aborted\030\001 \002(\010\"\027\n\025ListProcedures" +
-      "Request\"@\n\026ListProceduresResponse\022&\n\tpro" +
-      "cedure\030\001 \003(\0132\023.hbase.pb.Procedure\"\315\001\n\017Se" +
-      "tQuotaRequest\022\021\n\tuser_name\030\001 \001(\t\022\022\n\nuser" +
-      "_group\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\'\n\ntable" +
-      "_name\030\004 \001(\0132\023.hbase.pb.TableName\022\022\n\nremo" +
-      "ve_all\030\005 \001(\010\022\026\n\016bypass_globals\030\006 \001(\010\022+\n\010" +
-      "throttle\030\007 \001(\0132\031.hbase.pb.ThrottleReques" +
-      "t\"\022\n\020SetQuotaResponse\"J\n\037MajorCompaction",
-      "TimestampRequest\022\'\n\ntable_name\030\001 \002(\0132\023.h" +
-      "base.pb.TableName\"U\n(MajorCompactionTime" +
-      "stampForRegionRequest\022)\n\006region\030\001 \002(\0132\031." +
-      "hbase.pb.RegionSpecifier\"@\n MajorCompact" +
-      "ionTimestampResponse\022\034\n\024compaction_times" +
-      "tamp\030\001 \002(\003\"\035\n\033SecurityCapabilitiesReques" +
-      "t\"\354\001\n\034SecurityCapabilitiesResponse\022G\n\014ca" +
-      "pabilities\030\001 \003(\01621.hbase.pb.SecurityCapa" +
-      "bilitiesResponse.Capability\"\202\001\n\nCapabili" +
-      "ty\022\031\n\025SIMPLE_AUTHENTICATION\020\000\022\031\n\025SECURE_",
-      "AUTHENTICATION\020\001\022\021\n\rAUTHORIZATION\020\002\022\026\n\022C" +
-      "ELL_AUTHORIZATION\020\003\022\023\n\017CELL_VISIBILITY\020\004" +
-      "\"\"\n ListDrainingRegionServersRequest\"N\n!" +
-      "ListDrainingRegionServersResponse\022)\n\013ser" +
-      "ver_name\030\001 \003(\0132\024.hbase.pb.ServerName\"F\n\031" +
-      "DrainRegionServersRequest\022)\n\013server_name" +
-      "\030\001 \003(\0132\024.hbase.pb.ServerName\"\034\n\032DrainReg" +
-      "ionServersResponse\"P\n#RemoveDrainFromReg" +
-      "ionServersRequest\022)\n\013server_name\030\001 \003(\0132\024" +
-      ".hbase.pb.ServerName\"&\n$RemoveDrainFromR",
-      "egionServersResponse*(\n\020MasterSwitchType" +
-      "\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\3013\n\rMasterService" +
-      "\022e\n\024GetSchemaAlterStatus\022%.hbase.pb.GetS" +
-      "chemaAlterStatusRequest\032&.hbase.pb.GetSc" +
-      "hemaAlterStatusResponse\022b\n\023GetTableDescr" +
-      "iptors\022$.hbase.pb.GetTableDescriptorsReq" +
-      "uest\032%.hbase.pb.GetTableDescriptorsRespo" +
-      "nse\022P\n\rGetTableNames\022\036.hbase.pb.GetTable" +
-      "NamesRequest\032\037.hbase.pb.GetTableNamesRes" +
-      "ponse\022Y\n\020GetClusterStatus\022!.hbase.pb.Get",
-      "ClusterStatusRequest\032\".hbase.pb.GetClust" +
-      "erStatusResponse\022V\n\017IsMasterRunning\022 .hb" +
-      "ase.pb.IsMasterRunningRequest\032!.hbase.pb" +
-      ".IsMasterRunningResponse\022D\n\tAddColumn\022\032." +
-      "hbase.pb.AddColumnRequest\032\033.hbase.pb.Add" +
-      "ColumnResponse\022M\n\014DeleteColumn\022\035.hbase.p" +
-      "b.DeleteColumnRequest\032\036.hbase.pb.DeleteC" +
-      "olumnResponse\022M\n\014ModifyColumn\022\035.hbase.pb" +
-      ".ModifyColumnRequest\032\036.hbase.pb.ModifyCo" +
-      "lumnResponse\022G\n\nMoveRegion\022\033.hbase.pb.Mo",
-      "veRegionRequest\032\034.hbase.pb.MoveRegionRes" +
-      "ponse\022\\\n\021MergeTableRegions\022\".hbase.pb.Me" +
-      "rgeTableRegionsRequest\032#.hbase.pb.MergeT" +
-      "ableRegionsResponse\022M\n\014AssignRegion\022\035.hb" +
-      "ase.pb.AssignRegionRequest\032\036.hbase.pb.As" +
-      "signRegionResponse\022S\n\016UnassignRegion\022\037.h" +
-      "base.pb.UnassignRegionRequest\032 .hbase.pb" +
-      ".UnassignRegionResponse\022P\n\rOfflineRegion" +
-      "\022\036.hbase.pb.OfflineRegionRequest\032\037.hbase" +
-      ".pb.OfflineRegionResponse\022J\n\013DeleteTable",
-      "\022\034.hbase.pb.DeleteTableRequest\032\035.hbase.p" +
-      "b.DeleteTableResponse\022P\n\rtruncateTable\022\036" +
-      ".hbase.pb.TruncateTableRequest\032\037.hbase.p" +
-      "b.TruncateTableResponse\022J\n\013EnableTable\022\034" +
-      ".hbase.pb.EnableTableRequest\032\035.hbase.pb." +
-      "EnableTableResponse\022M\n\014DisableTable\022\035.hb" +
-      "ase.pb.DisableTableRequest\032\036.hbase.pb.Di" +
-      "sableTableResponse\022J\n\013ModifyTable\022\034.hbas" +
-      "e.pb.ModifyTableRequest\032\035.hbase.pb.Modif" +
-      "yTableResponse\022J\n\013CreateTable\022\034.hbase.pb",
-      ".CreateTableRequest\032\035.hbase.pb.CreateTab" +
-      "leResponse\022A\n\010Shutdown\022\031.hbase.pb.Shutdo" +
-      "wnRequest\032\032.hbase.pb.ShutdownResponse\022G\n" +
-      "\nStopMaster\022\033.hbase.pb.StopMasterRequest" +
-      "\032\034.hbase.pb.StopMasterResponse\022h\n\031IsMast" +
-      "erInMaintenanceMode\022$.hbase.pb.IsInMaint" +
-      "enanceModeRequest\032%.hbase.pb.IsInMainten" +
-      "anceModeResponse\022>\n\007Balance\022\030.hbase.pb.B" +
-      "alanceRequest\032\031.hbase.pb.BalanceResponse" +
-      "\022_\n\022SetBalancerRunning\022#.hbase.pb.SetBal",
-      "ancerRunningRequest\032$.hbase.pb.SetBalanc" +
-      "erRunningResponse\022\\\n\021IsBalancerEnabled\022\"" +
-      ".hbase.pb.IsBalancerEnabledRequest\032#.hba" +
-      "se.pb.IsBalancerEnabledResponse\022k\n\026SetSp" +
-      "litOrMergeEnabled\022\'.hbase.pb.SetSplitOrM" +
-      "ergeEnabledRequest\032(.hbase.pb.SetSplitOr" +
-      "MergeEnabledResponse\022h\n\025IsSplitOrMergeEn" +
-      "abled\022&.hbase.pb.IsSplitOrMergeEnabledRe" +
-      "quest\032\'.hbase.pb.IsSplitOrMergeEnabledRe" +
-      "sponse\022D\n\tNormalize\022\032.hbase.pb.Normalize",
-      "Request\032\033.hbase.pb.NormalizeResponse\022e\n\024" +
-      "SetNormalizerRunning\022%.hbase.pb.SetNorma" +
-      "lizerRunningRequest\032&.hbase.pb.SetNormal" +
-      "izerRunningResponse\022b\n\023IsNormalizerEnabl" +
-      "ed\022$.hbase.pb.IsNormalizerEnabledRequest" +
-      "\032%.hbase.pb.IsNormalizerEnabledResponse\022" +
-      "S\n\016RunCatalogScan\022\037.hbase.pb.RunCatalogS" +
-      "canRequest\032 .hbase.pb.RunCatalogScanResp" +
-      "onse\022e\n\024EnableCatalogJanitor\022%.hbase.pb." +
-      "EnableCatalogJanitorRequest\032&.hbase.pb.E",
-      "nableCatalogJanitorResponse\022n\n\027IsCatalog" +
-      "JanitorEnabled\022(.hbase.pb.IsCatalogJanit" +
-      "orEnabledRequest\032).hbase.pb.IsCatalogJan" +
-      "itorEnabledResponse\022V\n\017RunCleanerChore\022 " +
-      ".hbase.pb.RunCleanerChoreRequest\032!.hbase" +
-      ".pb.RunCleanerChoreResponse\022k\n\026SetCleane" +
-      "rChoreRunning\022\'.hbase.pb.SetCleanerChore" +
-      "RunningRequest\032(.hbase.pb.SetCleanerChor" +
-      "eRunningResponse\022h\n\025IsCleanerChoreEnable" +
-      "d\022&.hbase.pb.IsCleanerChoreEnabledReques",
-      "t\032\'.hbase.pb.IsCleanerChoreEnabledRespon" +
-      "se\022^\n\021ExecMasterService\022#.hbase.pb.Copro" +
-      "cessorServiceRequest\032$.hbase.pb.Coproces" +
-      "sorServiceResponse\022A\n\010Snapshot\022\031.hbase.p" +
-      "b.SnapshotRequest\032\032.hbase.pb.SnapshotRes" +
-      "ponse\022h\n\025GetCompletedSnapshots\022&.hbase.p" +
-      "b.GetCompletedSnapshotsRequest\032\'.hbase.p" +
-      "b.GetCompletedSnapshotsResponse\022S\n\016Delet" +
-      "eSnapshot\022\037.hbase.pb.DeleteSnapshotReque" +
-      "st\032 .hbase.pb.DeleteSnapshotResponse\022S\n\016",
-      "IsSnapshotDone\022\037.hbase.pb.IsSnapshotDone" +
-      "Request\032 .hbase.pb.IsSnapshotDoneRespons" +
-      "e\022V\n\017RestoreSnapshot\022 .hbase.pb.RestoreS" +
-      "napshotRequest\032!.hbase.pb.RestoreSnapsho" +
-      "tResponse\022P\n\rExecProcedure\022\036.hbase.pb.Ex" +
-      "ecProcedureRequest\032\037.hbase.pb.ExecProced" +
-      "ureResponse\022W\n\024ExecProcedureWithRet\022\036.hb" +
-      "ase.pb.ExecProcedureRequest\032\037.hbase.pb.E" +
-      "xecProcedureResponse\022V\n\017IsProcedureDone\022" +
-      " .hbase.pb.IsProcedureDoneRequest\032!.hbas",
-      "e.pb.IsProcedureDoneResponse\022V\n\017ModifyNa" +
-      "mespace\022 .hbase.pb.ModifyNamespaceReques" +
-      "t\032!.hbase.pb.ModifyNamespaceResponse\022V\n\017" +
-      "CreateNamespace\022 .hbase.pb.CreateNamespa" +
-      "ceRequest\032!.hbase.pb.CreateNamespaceResp" +
-      "onse\022V\n\017DeleteNamespace\022 .hbase.pb.Delet" +
-      "eNamespaceRequest\032!.hbase.pb.DeleteNames" +
-      "paceResponse\022k\n\026GetNamespaceDescriptor\022\'" +
-      ".hbase.pb.GetNamespaceDescriptorRequest\032" +
-      "(.hbase.pb.GetNamespaceDescriptorRespons",
-      "e\022q\n\030ListNamespaceDescriptors\022).hbase.pb" +
-      ".ListNamespaceDescriptorsRequest\032*.hbase" +
-      ".pb.ListNamespaceDescriptorsResponse\022\206\001\n" +
-      "\037ListTableDescriptorsByNamespace\0220.hbase" +
-      ".pb.ListTableDescriptorsByNamespaceReque" +
-      "st\0321.hbase.pb.ListTableDescriptorsByName" +
-      "spaceResponse\022t\n\031ListTableNamesByNamespa" +
-      "ce\022*.hbase.pb.ListTableNamesByNamespaceR" +
-      "equest\032+.hbase.pb.ListTableNamesByNamesp" +
-      "aceResponse\022P\n\rGetTableState\022\036.hbase.pb.",
-      "GetTableStateRequest\032\037.hbase.pb.GetTable" +
-      "StateResponse\022A\n\010SetQuota\022\031.hbase.pb.Set" +
-      "QuotaRequest\032\032.hbase.pb.SetQuotaResponse" +
-      "\022x\n\037getLastMajorCompactionTimestamp\022).hb" +
-      "ase.pb.MajorCompactionTimestampRequest\032*" +
-      ".hbase.pb.MajorCompactionTimestampRespon" +
-      "se\022\212\001\n(getLastMajorCompactionTimestampFo" +
-      "rRegion\0222.hbase.pb.MajorCompactionTimest" +
-      "ampForRegionRequest\032*.hbase.pb.MajorComp" +
-      "actionTimestampResponse\022_\n\022getProcedureR",
-      "esult\022#.hbase.pb.GetProcedureResultReque" +
-      "st\032$.hbase.pb.GetProcedureResultResponse" +
-      "\022h\n\027getSecurityCapabilities\022%.hbase.pb.S" +
-      "ecurityCapabilitiesRequest\032&.hbase.pb.Se" +
-      "curityCapabilitiesResponse\022S\n\016AbortProce" +
-      "dure\022\037.hbase.pb.AbortProcedureRequest\032 ." +
-      "hbase.pb.AbortProcedureResponse\022S\n\016ListP" +
-      "rocedures\022\037.hbase.pb.ListProceduresReque" +
-      "st\032 .hbase.pb.ListProceduresResponse\022_\n\022" +
-      "AddReplicationPeer\022#.hbase.pb.AddReplica",
-      "tionPeerRequest\032$.hbase.pb.AddReplicatio" +
-      "nPeerResponse\022h\n\025RemoveReplicationPeer\022&" +
-      ".hbase.pb.RemoveReplicationPeerRequest\032\'" +
-      ".hbase.pb.RemoveReplicationPeerResponse\022" +
-      "h\n\025EnableReplicationPeer\022&.hbase.pb.Enab" +
-      "leReplicationPeerRequest\032\'.hbase.pb.Enab" +
-      "leReplicationPeerResponse\022k\n\026DisableRepl" +
-      "icationPeer\022\'.hbase.pb.DisableReplicatio" +
-      "nPeerRequest\032(.hbase.pb.DisableReplicati" +
-      "onPeerResponse\022q\n\030GetReplicationPeerConf",
-      "ig\022).hbase.pb.GetReplicationPeerConfigRe" +
-      "quest\032*.hbase.pb.GetReplicationPeerConfi" +
-      "gResponse\022z\n\033UpdateReplicationPeerConfig" +
-      "\022,.hbase.pb.UpdateReplicationPeerConfigR" +
-      "equest\032-.hbase.pb.UpdateReplicationPeerC" +
-      "onfigResponse\022e\n\024ListReplicationPeers\022%." +
-      "hbase.pb.ListReplicationPeersRequest\032&.h" +
-      "base.pb.ListReplicationPeersResponse\022t\n\031" +
-      "listDrainingRegionServers\022*.hbase.pb.Lis" +
-      "tDrainingRegionServersRequest\032+.hbase.pb",
-      ".ListDrainingRegionServersResponse\022_\n\022dr" +
-      "ainRegionServers\022#.hbase.pb.DrainRegionS" +
-      "erversRequest\032$.hbase.pb.DrainRegionServ" +
-      "ersResponse\022}\n\034removeDrainFromRegionServ" +
-      "ers\022-.hbase.pb.RemoveDrainFromRegionServ" +
-      "ersRequest\032..hbase.pb.RemoveDrainFromReg" +
-      "ionServersResponseBI\n1org.apache.hadoop." +
-      "hbase.shaded.protobuf.generatedB\014MasterP" +
-      "rotosH\001\210\001\001\240\001\001"
+      "State\022\026\n\016submitted_time\030\002 \001(\004\022\023\n\013last_up" +
+      "date\030\003 \001(\004\022\016\n\006result\030\004 \001(\014\0224\n\texception\030" +
+      "\005 \001(\0132!.hbase.pb.ForeignExceptionMessage" +
+      "\"1\n\005State\022\r\n\tNOT_FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n" +
+      "\010FINISHED\020\002\"M\n\025AbortProcedureRequest\022\017\n\007" +
+      "proc_id\030\001 \002(\004\022#\n\025mayInterruptIfRunning\030\002",
+      " \001(\010:\004true\"6\n\026AbortProcedureResponse\022\034\n\024" +
+      "is_procedure_aborted\030\001 \002(\010\"\027\n\025ListProced" +
+      "uresRequest\"@\n\026ListProceduresResponse\022&\n" +
+      "\tprocedure\030\001 \003(\0132\023.hbase.pb.Procedure\"\315\001" +
+      "\n\017SetQuotaRequest\022\021\n\tuser_name\030\001 \001(\t\022\022\n\n" +
+      "user_group\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\'\n\nt" +
+      "able_name\030\004 \001(\0132\023.hbase.pb.TableName\022\022\n\n" +
+      "remove_all\030\005 \001(\010\022\026\n\016bypass_globals\030\006 \001(\010" +
+      "\022+\n\010throttle\030\007 \001(\0132\031.hbase.pb.ThrottleRe" +
+      "quest\"\022\n\020SetQuotaResponse\"J\n\037MajorCompac",
+      "tionTimestampRequest\022\'\n\ntable_name\030\001 \002(\013" +
+      "2\023.hbase.pb.TableName\"U\n(MajorCompaction" +
+      "TimestampForRegionRequest\022)\n\006region\030\001 \002(" +
+      "\0132\031.hbase.pb.RegionSpecifier\"@\n MajorCom" +
+      "pactionTimestampResponse\022\034\n\024compaction_t" +
+      "imestamp\030\001 \002(\003\"\035\n\033SecurityCapabilitiesRe" +
+      "quest\"\354\001\n\034SecurityCapabilitiesResponse\022G" +
+      "\n\014capabilities\030\001 \003(\01621.hbase.pb.Security" +
+      "CapabilitiesResponse.Capability\"\202\001\n\nCapa" +
+      "bility\022\031\n\025SIMPLE_AUTHENTICATION\020\000\022\031\n\025SEC",
+      "URE_AUTHENTICATION\020\001\022\021\n\rAUTHORIZATION\020\002\022" +
+      "\026\n\022CELL_AUTHORIZATION\020\003\022\023\n\017CELL_VISIBILI" +
+      "TY\020\004\"\"\n ListDrainingRegionServersRequest" +
+      "\"N\n!ListDrainingRegionServersResponse\022)\n" +
+      "\013server_name\030\001 \003(\0132\024.hbase.pb.ServerName" +
+      "\"F\n\031DrainRegionServersRequest\022)\n\013server_" +
+      "name\030\001 \003(\0132\024.hbase.pb.ServerName\"\034\n\032Drai" +
+      "nRegionServersResponse\"P\n#RemoveDrainFro" +
+      "mRegionServersRequest\022)\n\013server_name\030\001 \003" +
+      "(\0132\024.hbase.pb.ServerName\"&\n$RemoveDrainF",
+      "romRegionServersResponse*(\n\020MasterSwitch" +
+      "Type\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\3013\n\rMasterSer" +
+      "vice\022e\n\024GetSchemaAlterStatus\022%.hbase.pb." +
+      "GetSchemaAlterStatusRequest\032&.hbase.pb.G" +
+      "etSchemaAlterStatusResponse\022b\n\023GetTableD" +
+      "escriptors\022$.hbase.pb.GetTableDescriptor" +
+      "sRequest\032%.hbase.pb.GetTableDescriptorsR" +
+      "esponse\022P\n\rGetTableNames\022\036.hbase.pb.GetT" +
+      "ableNamesRequest\032\037.hbase.pb.GetTableName" +
+      "sResponse\022Y\n\020GetClusterStatus\022!.hbase.pb",
+      ".GetClusterStatusRequest\032\".hbase.pb.GetC" +
+      "lusterStatusResponse\022V\n\017IsMasterRunning\022" +
+      " .hbase.pb.IsMasterRunningRequest\032!.hbas" +
+      "e.pb.IsMasterRunningResponse\022D\n\tAddColum" +
+      "n\022\032.hbase.pb.AddColumnRequest\032\033.hbase.pb" +
+      ".AddColumnResponse\022M\n\014DeleteColumn\022\035.hba" +
+      "se.pb.DeleteColumnRequest\032\036.hbase.pb.Del" +
+      "eteColumnResponse\022M\n\014ModifyColumn\022\035.hbas" +
+      "e.pb.ModifyColumnRequest\032\036.hbase.pb.Modi" +
+      "fyColumnResponse\022G\n\nMoveRegion\022\033.hbase.p",
+      "b.MoveRegionRequest\032\034.hbase.pb.MoveRegio" +
+      "nResponse\022\\\n\021MergeTableRegions\022\".hbase.p" +
+      "b.MergeTableRegionsRequest\032#.hbase.pb.Me" +
+      "rgeTableRegionsResponse\022M\n\014AssignRegion\022" +
+      "\035.hbase.pb.AssignRegionRequest\032\036.hbase.p" +
+      "b.AssignRegionResponse\022S\n\016UnassignRegion" +
+      "\022\037.hbase.pb.UnassignRegionRequest\032 .hbas" +
+      "e.pb.UnassignRegionResponse\022P\n\rOfflineRe" +
+      "gion\022\036.hbase.pb.OfflineRegionRequest\032\037.h" +
+      "base.pb.OfflineRegionResponse\022J\n\013DeleteT",
+      "able\022\034.hbase.pb.DeleteTableRequest\032\035.hba" +
+      "se.pb.DeleteTableResponse\022P\n\rtruncateTab" +
+      "le\022\036.hbase.pb.TruncateTableRequest\032\037.hba" +
+      "se.pb.TruncateTableResponse\022J\n\013EnableTab" +
+      "le\022\034.hbase.pb.EnableTableRequest\032\035.hbase" +
+      ".pb.EnableTableResponse\022M\n\014DisableTable\022" +
+      "\035.hbase.pb.DisableTableRequest\032\036.hbase.p" +
+      "b.DisableTableResponse\022J\n\013ModifyTable\022\034." +
+      "hbase.pb.ModifyTableRequest\032\035.hbase.pb.M" +
+      "odifyTableResponse\022J\n\013CreateTable\022\034.hbas",
+      "e.pb.CreateTableRequest\032\035.hbase.pb.Creat" +
+      "eTableResponse\022A\n\010Shutdown\022\031.hbase.pb.Sh" +
+      "utdownRequest\032\032.hbase.pb.ShutdownRespons" +
+      "e\022G\n\nStopMaster\022\033.hbase.pb.StopMasterReq" +
+      "uest\032\034.hbase.pb.StopMasterResponse\022h\n\031Is" +
+      "MasterInMaintenanceMode\022$.hbase.pb.IsInM" +
+      "aintenanceModeRequest\032%.hbase.pb.IsInMai" +
+      "ntenanceModeResponse\022>\n\007Balance\022\030.hbase." +
+      "pb.BalanceRequest\032\031.hbase.pb.BalanceResp" +
+      "onse\022_\n\022SetBalancerRunning\022#.hbase.pb.Se",
+      "tBalancerRunningRequest\032$.hbase.pb.SetBa" +
+      "lancerRunningResponse\022\\\n\021IsBalancerEnabl" +
+      "ed\022\".hbase.pb.IsBalancerEnabledRequest\032#" +
+      ".hbase.pb.IsBalancerEnabledResponse\022k\n\026S" +
+      "etSplitOrMergeEnabled\022\'.hbase.pb.SetSpli" +
+      "tOrMergeEnabledRequest\032(.hbase.pb.SetSpl" +
+      "itOrMergeEnabledResponse\022h\n\025IsSplitOrMer" +
+      "geEnabled\022&.hbase.pb.IsSplitOrMergeEnabl" +
+      "edRequest\032\'.hbase.pb.IsSplitOrMergeEnabl" +
+      "edResponse\022D\n\tNormalize\022\032.hbase.pb.Norma",
+      "lizeRequest\032\033.hbase.pb.NormalizeResponse" +
+      "\022e\n\024SetNormalizerRunning\022%.hbase.pb.SetN" +
+      "ormalizerRunningRequest\032&.hbase.pb.SetNo" +
+      "rmalizerRunningResponse\022b\n\023IsNormalizerE" +
+      "nabled\022$.hbase.pb.IsNormalizerEnabledReq" +
+      "uest\032%.hbase.pb.IsNormalizerEnabledRespo" +
+      "nse\022S\n\016RunCatalogScan\022\037.hbase.pb.RunCata" +
+      "logScanRequest\032 .hbase.pb.RunCatalogScan" +
+      "Response\022e\n\024EnableCatalogJanitor\022%.hbase" +
+      ".pb.EnableCatalogJanitorRequest\032&.hbase.",
+      "pb.EnableCatalogJanitorResponse\022n\n\027IsCat" +
+      "alogJanitorEnabled\022(.hbase.pb.IsCatalogJ" +
+      "anitorEnabledRequest\032).hbase.pb.IsCatalo" +
+      "gJanitorEnabledResponse\022V\n\017RunCleanerCho" +
+      "re\022 .hbase.pb.RunCleanerChoreRequest\032!.h" +
+      "base.pb.RunCleanerChoreResponse\022k\n\026SetCl" +
+      "eanerChoreRunning\022\'.hbase.pb.SetCleanerC" +
+      "horeRunningRequest\032(.hbase.pb.SetCleaner" +
+      "ChoreRunningResponse\022h\n\025IsCleanerChoreEn" +
+      "abled\022&.hbase.pb.IsCleanerChoreEnabledRe",
+      "quest\032\'.hbase.pb.IsCleanerChoreEnabledRe" +
+      "sponse\022^\n\021ExecMasterService\022#.hbase.pb.C" +
+      "oprocessorServiceRequest\032$.hbase.pb.Copr" +
+      "ocessorServiceResponse\022A\n\010Snapshot\022\031.hba" +
+      "se.pb.SnapshotRequest\032\032.hbase.pb.Snapsho" +
+      "tResponse\022h\n\025GetCompletedSnapshots\022&.hba" +
+      "se.pb.GetCompletedSnapshotsRequest\032\'.hba" +
+      "se.pb.GetCompletedSnapshotsResponse\022S\n\016D" +
+      "eleteSnapshot\022\037.hbase.pb.DeleteSnapshotR" +
+      "equest\032 .hbase.pb.DeleteSnapshotResponse",
+      "\022S\n\016IsSnapshotDone\022\037.hbase.pb.IsSnapshot" +
+      "DoneRequest\032 .hbase.pb.IsSnapshotDoneRes" +
+      "ponse\022V\n\017RestoreSnapshot\022 .hbase.pb.Rest" +
+      "oreSnapshotRequest\032!.hbase.pb.RestoreSna" +
+      "pshotResponse\022P\n\rExecProcedure\022\036.hbase.p" +
+      "b.ExecProcedureRequest\032\037.hbase.pb.ExecPr" +
+      "ocedureResponse\022W\n\024ExecProcedureWithRet\022" +
+      "\036.hbase.pb.ExecProcedureRequest\032\037.hbase." +
+      "pb.ExecProcedureResponse\022V\n\017IsProcedureD" +
+      "one\022 .hbase.pb.IsProcedureDoneRequest\032!.",
+      "hbase.pb.IsProcedureDoneResponse\022V\n\017Modi" +
+      "fyNamespace\022 .hbase.pb.ModifyNamespaceRe" +
+      "quest\032!.hbase.pb.ModifyNamespaceResponse" +
+      "\022V\n\017CreateNamespace\022 .hbase.pb.CreateNam" +
+      "espaceRequest\032!.hbase.pb.CreateNamespace" +
+      "Response\022V\n\017DeleteNamespace\022 .hbase.pb.D" +
+      "eleteNamespaceRequest\032!.hbase.pb.DeleteN" +
+      "amespaceResponse\022k\n\026GetNamespaceDescript" +
+      "or\022\'.hbase.pb.GetNamespaceDescriptorRequ" +
+      "est\032(.hbase.pb.GetNamespaceDescriptorRes",
+      "ponse\022q\n\030ListNamespaceDescriptors\022).hbas" +
+      "e.pb.ListNamespaceDescriptorsRequest\032*.h" +
+      "base.pb.ListNamespaceDescriptorsResponse" +
+      "\022\206\001\n\037ListTableDescriptorsByNamespace\0220.h" +
+      "base.pb.ListTableDescriptorsByNamespaceR" +
+      "equest\0321.hbase.pb.ListTableDescriptorsBy" +
+      "NamespaceResponse\022t\n\031ListTableNamesByNam" +
+      "espace\022*.hbase.pb.ListTableNamesByNamesp" +
+      "aceRequest\032+.hbase.pb.ListTableNamesByNa" +
+      "mespaceResponse\022P\n\rGetTableState\022\036.hbase",
+      ".pb.GetTableStateRequest\032\037.hbase.pb.GetT" +
+      "ableStateResponse\022A\n\010SetQuota\022\031.hbase.pb" +
+      ".SetQuotaRequest\032\032.hbase.pb.SetQuotaResp" +
+      "onse\022x\n\037getLastMajorCompactionTimestamp\022" +
+      ").hbase.pb.MajorCompactionTimestampReque" +
+      "st\032*.hbase.pb.MajorCompactionTimestampRe" +
+      "sponse\022\212\001\n(getLastMajorCompactionTimesta" +
+      "mpForRegion\0222.hbase.pb.MajorCompactionTi" +
+      "mestampForRegionRequest\032*.hbase.pb.Major" +
+      "CompactionTimestampResponse\022_\n\022getProced",
+      "ureResult\022#.hbase.pb.GetProcedureResultR" +
+      "equest\032$.hbase.pb.GetProcedureResultResp" +
+      "onse\022h\n\027getSecurityCapabilities\022%.hbase." +
+      "pb.SecurityCapabilitiesRequest\032&.hbase.p" +
+      "b.SecurityCapabilitiesResponse\022S\n\016AbortP" +
+      "rocedure\022\037.hbase.pb.AbortProcedureReques" +
+      "t\032 .hbase.pb.AbortProcedureResponse\022S\n\016L" +
+      "istProcedures\022\037.hbase.pb.ListProceduresR" +
+      "equest\032 .hbase.pb.ListProceduresResponse" +
+      "\022_\n\022AddReplicationPeer\022#.hbase.pb.AddRep",
+      "licationPeerRequest\032$.hbase.pb.AddReplic" +
+      "ationPeerResponse\022h\n\025RemoveReplicationPe" +
+      "er\022&.hbase.pb.RemoveReplicationPeerReque" +
+      "st\032\'.hbase.pb.RemoveReplicationPeerRespo" +
+      "nse\022h\n\025EnableReplicationPeer\022&.hbase.pb." +
+      "EnableReplicationPeerRequest\032\'.hbase.pb." +
+      "EnableReplicationPeerResponse\022k\n\026Disable" +
+      "ReplicationPeer\022\'.hbase.pb.DisableReplic" +
+      "ationPeerRequest\032(.hbase.pb.DisableRepli" +
+      "cationPeerResponse\022q\n\030GetReplicationPeer",
+      "Config\022).hbase.pb.GetReplicationPeerConf" +
+      "igRequest\032*.hbase.pb.GetReplicationPeerC" +
+      "onfigResponse\022z\n\033UpdateReplicationPeerCo" +
+      "nfig\022,.hbase.pb.UpdateReplicationPeerCon" +
+      "figRequest\032-.hbase.pb.UpdateReplicationP" +
+      "eerConfigResponse\022e\n\024ListReplicationPeer" +
+      "s\022%.hbase.pb.ListReplicationPeersRequest" +
+      "\032&.hbase.pb.ListReplicationPeersResponse" +
+      "\022t\n\031listDrainingRegionServers\022*.hbase.pb" +
+      ".ListDrainingRegionServersRequest\032+.hbas",
+      "e.pb.ListDrainingRegionServersResponse\022_" +
+      "\n\022drainRegionServers\022#.hbase.pb.DrainReg" +
+      "ionServersRequest\032$.hbase.pb.DrainRegion" +
+      "ServersResponse\022}\n\034removeDrainFromRegion" +
+      "Servers\022-.hbase.pb.RemoveDrainFromRegion" +
+      "ServersRequest\032..hbase.pb.RemoveDrainFro" +
+      "mRegionServersResponseBI\n1org.apache.had" +
+      "oop.hbase.shaded.protobuf.generatedB\014Mas" +
+      "terProtosH\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -77492,7 +77492,7 @@ public final class MasterProtos {
     internal_static_hbase_pb_GetProcedureResultResponse_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetProcedureResultResponse_descriptor,
-        new java.lang.String[] { "State", "StartTime", "LastUpdate", "Result", "Exception", });
+        new java.lang.String[] { "State", "SubmittedTime", "LastUpdate", "Result", "Exception", });
     internal_static_hbase_pb_AbortProcedureRequest_descriptor =
       getDescriptor().getMessageTypes().get(106);
     internal_static_hbase_pb_AbortProcedureRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
index 56d1e11..e6fe9b1 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
@@ -262,13 +262,13 @@ public final class ProcedureProtos {
     long getProcId();
 
     /**
-     * <code>required uint64 start_time = 4;</code>
+     * <code>required uint64 submitted_time = 4;</code>
      */
-    boolean hasStartTime();
+    boolean hasSubmittedTime();
     /**
-     * <code>required uint64 start_time = 4;</code>
+     * <code>required uint64 submitted_time = 4;</code>
      */
-    long getStartTime();
+    long getSubmittedTime();
 
     /**
      * <code>optional string owner = 5;</code>
@@ -449,7 +449,7 @@ public final class ProcedureProtos {
       className_ = "";
       parentId_ = 0L;
       procId_ = 0L;
-      startTime_ = 0L;
+      submittedTime_ = 0L;
       owner_ = "";
       state_ = 1;
       stackId_ = java.util.Collections.emptyList();
@@ -507,7 +507,7 @@ public final class ProcedureProtos {
             }
             case 32: {
               bitField0_ |= 0x00000008;
-              startTime_ = input.readUInt64();
+              submittedTime_ = input.readUInt64();
               break;
             }
             case 42: {
@@ -711,19 +711,19 @@ public final class ProcedureProtos {
       return procId_;
     }
 
-    public static final int START_TIME_FIELD_NUMBER = 4;
-    private long startTime_;
+    public static final int SUBMITTED_TIME_FIELD_NUMBER = 4;
+    private long submittedTime_;
     /**
-     * <code>required uint64 start_time = 4;</code>
+     * <code>required uint64 submitted_time = 4;</code>
      */
-    public boolean hasStartTime() {
+    public boolean hasSubmittedTime() {
       return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * <code>required uint64 start_time = 4;</code>
+     * <code>required uint64 submitted_time = 4;</code>
      */
-    public long getStartTime() {
-      return startTime_;
+    public long getSubmittedTime() {
+      return submittedTime_;
     }
 
     public static final int OWNER_FIELD_NUMBER = 5;
@@ -987,7 +987,7 @@ public final class ProcedureProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasStartTime()) {
+      if (!hasSubmittedTime()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -1015,7 +1015,7 @@ public final class ProcedureProtos {
         output.writeUInt64(3, procId_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeUInt64(4, startTime_);
+        output.writeUInt64(4, submittedTime_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 5, owner_);
@@ -1068,7 +1068,7 @@ public final class ProcedureProtos {
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(4, startTime_);
+          .computeUInt64Size(4, submittedTime_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(5, owner_);
@@ -1146,10 +1146,10 @@ public final class ProcedureProtos {
         result = result && (getProcId()
             == other.getProcId());
       }
-      result = result && (hasStartTime() == other.hasStartTime());
-      if (hasStartTime()) {
-        result = result && (getStartTime()
-            == other.getStartTime());
+      result = result && (hasSubmittedTime() == other.hasSubmittedTime());
+      if (hasSubmittedTime()) {
+        result = result && (getSubmittedTime()
+            == other.getSubmittedTime());
       }
       result = result && (hasOwner() == other.hasOwner());
       if (hasOwner()) {
@@ -1222,10 +1222,10 @@ public final class ProcedureProtos {
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
-      if (hasStartTime()) {
-        hash = (37 * hash) + START_TIME_FIELD_NUMBER;
+      if (hasSubmittedTime()) {
+        hash = (37 * hash) + SUBMITTED_TIME_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getStartTime());
+            getSubmittedTime());
       }
       if (hasOwner()) {
         hash = (37 * hash) + OWNER_FIELD_NUMBER;
@@ -1400,7 +1400,7 @@ public final class ProcedureProtos {
         bitField0_ = (bitField0_ & ~0x00000002);
         procId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
-        startTime_ = 0L;
+        submittedTime_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000008);
         owner_ = "";
         bitField0_ = (bitField0_ & ~0x00000010);
@@ -1465,7 +1465,7 @@ public final class ProcedureProtos {
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000008;
         }
-        result.startTime_ = startTime_;
+        result.submittedTime_ = submittedTime_;
         if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
           to_bitField0_ |= 0x00000010;
         }
@@ -1564,8 +1564,8 @@ public final class ProcedureProtos {
         if (other.hasProcId()) {
           setProcId(other.getProcId());
         }
-        if (other.hasStartTime()) {
-          setStartTime(other.getStartTime());
+        if (other.hasSubmittedTime()) {
+          setSubmittedTime(other.getSubmittedTime());
         }
         if (other.hasOwner()) {
           bitField0_ |= 0x00000010;
@@ -1618,7 +1618,7 @@ public final class ProcedureProtos {
         if (!hasProcId()) {
           return false;
         }
-        if (!hasStartTime()) {
+        if (!hasSubmittedTime()) {
           return false;
         }
         if (!hasState()) {
@@ -1829,34 +1829,34 @@ public final class ProcedureProtos {
         return this;
       }
 
-      private long startTime_ ;
+      private long submittedTime_ ;
       /**
-       * <code>required uint64 start_time = 4;</code>
+       * <code>required uint64 submitted_time = 4;</code>
        */
-      public boolean hasStartTime() {
+      public boolean hasSubmittedTime() {
         return ((bitField0_ & 0x00000008) == 0x00000008);
       }
       /**
-       * <code>required uint64 start_time = 4;</code>
+       * <code>required uint64 submitted_time = 4;</code>
        */
-      public long getStartTime() {
-        return startTime_;
+      public long getSubmittedTime() {
+        return submittedTime_;
       }
       /**
-       * <code>required uint64 start_time = 4;</code>
+       * <code>required uint64 submitted_time = 4;</code>
        */
-      public Builder setStartTime(long value) {
+      public Builder setSubmittedTime(long value) {
         bitField0_ |= 0x00000008;
-        startTime_ = value;
+        submittedTime_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required uint64 start_time = 4;</code>
+       * <code>required uint64 submitted_time = 4;</code>
        */
-      public Builder clearStartTime() {
+      public Builder clearSubmittedTime() {
         bitField0_ = (bitField0_ & ~0x00000008);
-        startTime_ = 0L;
+        submittedTime_ = 0L;
         onChanged();
         return this;
       }
@@ -7743,38 +7743,38 @@ public final class ProcedureProtos {
   static {
     java.lang.String[] descriptorData = {
       "\n\017Procedure.proto\022\010hbase.pb\032\023ErrorHandli" +
-      "ng.proto\"\313\002\n\tProcedure\022\022\n\nclass_name\030\001 \002" +
-      "(\t\022\021\n\tparent_id\030\002 \001(\004\022\017\n\007proc_id\030\003 \002(\004\022\022" +
-      "\n\nstart_time\030\004 \002(\004\022\r\n\005owner\030\005 \001(\t\022\'\n\005sta" +
-      "te\030\006 \002(\0162\030.hbase.pb.ProcedureState\022\020\n\010st" +
-      "ack_id\030\007 \003(\r\022\023\n\013last_update\030\010 \002(\004\022\017\n\007tim" +
-      "eout\030\t \001(\r\0224\n\texception\030\n \001(\0132!.hbase.pb" +
-      ".ForeignExceptionMessage\022\016\n\006result\030\013 \001(\014" +
-      "\022\022\n\nstate_data\030\014 \001(\014\022\026\n\013nonce_group\030\r \001(" +
-      "\004:\0010\022\020\n\005nonce\030\016 \001(\004:\0010\"+\n\027SequentialProc",
-      "edureData\022\020\n\010executed\030\001 \002(\010\"*\n\031StateMach" +
-      "ineProcedureData\022\r\n\005state\030\001 \003(\r\"X\n\022Proce" +
-      "dureWALHeader\022\017\n\007version\030\001 \002(\r\022\014\n\004type\030\002" +
-      " \002(\r\022\016\n\006log_id\030\003 \002(\004\022\023\n\013min_proc_id\030\004 \002(" +
-      "\004\";\n\023ProcedureWALTrailer\022\017\n\007version\030\001 \002(" +
-      "\r\022\023\n\013tracker_pos\030\002 \002(\004\"\225\001\n\025ProcedureStor" +
-      "eTracker\0229\n\004node\030\001 \003(\0132+.hbase.pb.Proced" +
-      "ureStoreTracker.TrackerNode\032A\n\013TrackerNo" +
-      "de\022\020\n\010start_id\030\001 \002(\004\022\017\n\007updated\030\002 \003(\004\022\017\n" +
-      "\007deleted\030\003 \003(\004\"\257\002\n\021ProcedureWALEntry\022.\n\004",
-      "type\030\001 \002(\0162 .hbase.pb.ProcedureWALEntry." +
-      "Type\022&\n\tprocedure\030\002 \003(\0132\023.hbase.pb.Proce" +
-      "dure\022\017\n\007proc_id\030\003 \001(\004\022\020\n\010child_id\030\004 \003(\004\"" +
-      "\236\001\n\004Type\022\025\n\021PROCEDURE_WAL_EOF\020\001\022\026\n\022PROCE" +
-      "DURE_WAL_INIT\020\002\022\030\n\024PROCEDURE_WAL_INSERT\020" +
-      "\003\022\030\n\024PROCEDURE_WAL_UPDATE\020\004\022\030\n\024PROCEDURE" +
-      "_WAL_DELETE\020\005\022\031\n\025PROCEDURE_WAL_COMPACT\020\006" +
-      "*{\n\016ProcedureState\022\020\n\014INITIALIZING\020\001\022\014\n\010" +
-      "RUNNABLE\020\002\022\013\n\007WAITING\020\003\022\023\n\017WAITING_TIMEO" +
-      "UT\020\004\022\016\n\nROLLEDBACK\020\005\022\013\n\007SUCCESS\020\006\022\n\n\006FAI",
-      "LED\020\007BL\n1org.apache.hadoop.hbase.shaded." +
-      "protobuf.generatedB\017ProcedureProtosH\001\210\001\001" +
-      "\240\001\001"
+      "ng.proto\"\317\002\n\tProcedure\022\022\n\nclass_name\030\001 \002" +
+      "(\t\022\021\n\tparent_id\030\002 \001(\004\022\017\n\007proc_id\030\003 \002(\004\022\026" +
+      "\n\016submitted_time\030\004 \002(\004\022\r\n\005owner\030\005 \001(\t\022\'\n" +
+      "\005state\030\006 \002(\0162\030.hbase.pb.ProcedureState\022\020" +
+      "\n\010stack_id\030\007 \003(\r\022\023\n\013last_update\030\010 \002(\004\022\017\n" +
+      "\007timeout\030\t \001(\r\0224\n\texception\030\n \001(\0132!.hbas" +
+      "e.pb.ForeignExceptionMessage\022\016\n\006result\030\013" +
+      " \001(\014\022\022\n\nstate_data\030\014 \001(\014\022\026\n\013nonce_group\030" +
+      "\r \001(\004:\0010\022\020\n\005nonce\030\016 \001(\004:\0010\"+\n\027Sequential",
+      "ProcedureData\022\020\n\010executed\030\001 \002(\010\"*\n\031State" +
+      "MachineProcedureData\022\r\n\005state\030\001 \003(\r\"X\n\022P" +
+      "rocedureWALHeader\022\017\n\007version\030\001 \002(\r\022\014\n\004ty" +
+      "pe\030\002 \002(\r\022\016\n\006log_id\030\003 \002(\004\022\023\n\013min_proc_id\030" +
+      "\004 \002(\004\";\n\023ProcedureWALTrailer\022\017\n\007version\030" +
+      "\001 \002(\r\022\023\n\013tracker_pos\030\002 \002(\004\"\225\001\n\025Procedure" +
+      "StoreTracker\0229\n\004node\030\001 \003(\0132+.hbase.pb.Pr" +
+      "ocedureStoreTracker.TrackerNode\032A\n\013Track" +
+      "erNode\022\020\n\010start_id\030\001 \002(\004\022\017\n\007updated\030\002 \003(" +
+      "\004\022\017\n\007deleted\030\003 \003(\004\"\257\002\n\021ProcedureWALEntry",
+      "\022.\n\004type\030\001 \002(\0162 .hbase.pb.ProcedureWALEn" +
+      "try.Type\022&\n\tprocedure\030\002 \003(\0132\023.hbase.pb.P" +
+      "rocedure\022\017\n\007proc_id\030\003 \001(\004\022\020\n\010child_id\030\004 " +
+      "\003(\004\"\236\001\n\004Type\022\025\n\021PROCEDURE_WAL_EOF\020\001\022\026\n\022P" +
+      "ROCEDURE_WAL_INIT\020\002\022\030\n\024PROCEDURE_WAL_INS" +
+      "ERT\020\003\022\030\n\024PROCEDURE_WAL_UPDATE\020\004\022\030\n\024PROCE" +
+      "DURE_WAL_DELETE\020\005\022\031\n\025PROCEDURE_WAL_COMPA" +
+      "CT\020\006*{\n\016ProcedureState\022\020\n\014INITIALIZING\020\001" +
+      "\022\014\n\010RUNNABLE\020\002\022\013\n\007WAITING\020\003\022\023\n\017WAITING_T" +
+      "IMEOUT\020\004\022\016\n\nROLLEDBACK\020\005\022\013\n\007SUCCESS\020\006\022\n\n",
+      "\006FAILED\020\007BL\n1org.apache.hadoop.hbase.sha" +
+      "ded.protobuf.generatedB\017ProcedureProtosH" +
+      "\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -7794,7 +7794,7 @@ public final class ProcedureProtos {
     internal_static_hbase_pb_Procedure_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Procedure_descriptor,
-        new java.lang.String[] { "ClassName", "ParentId", "ProcId", "StartTime", "Owner", "State", "StackId", "LastUpdate", "Timeout", "Exception", "Result", "StateData", "NonceGroup", "Nonce", });
+        new java.lang.String[] { "ClassName", "ParentId", "ProcId", "SubmittedTime", "Owner", "State", "StackId", "LastUpdate", "Timeout", "Exception", "Result", "StateData", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_SequentialProcedureData_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_SequentialProcedureData_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index e22695b..d7d51e2 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -512,7 +512,7 @@ message GetProcedureResultResponse {
   }
 
   required State state = 1;
-  optional uint64 start_time = 2;
+  optional uint64 submitted_time = 2;
   optional uint64 last_update = 3;
   optional bytes result = 4;
   optional ForeignExceptionMessage exception = 5;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
index 2fed24d..1a3ecf5 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
@@ -43,7 +43,7 @@ message Procedure {
   required string class_name = 1;        // full classname to be able to instantiate the procedure
   optional uint64 parent_id = 2;         // parent if not a root-procedure otherwise not set
   required uint64 proc_id = 3;
-  required uint64 start_time = 4;
+  required uint64 submitted_time = 4;
   optional string owner = 5;
 
   // internal "runtime" state

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index f86f800..9af8f45 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -1098,7 +1098,7 @@ public class MasterRpcServices extends RSRpcServices
       if (v.getFirst() != null) {
         ProcedureInfo result = v.getFirst();
         builder.setState(GetProcedureResultResponse.State.FINISHED);
-        builder.setStartTime(result.getStartTime());
+        builder.setSubmittedTime(result.getSubmittedTime());
         builder.setLastUpdate(result.getLastUpdate());
         if (result.isFailed()) {
           builder.setException(ForeignExceptionUtil.toProtoForeignException(result.getException()));
@@ -1113,7 +1113,7 @@ public class MasterRpcServices extends RSRpcServices
           builder.setState(GetProcedureResultResponse.State.NOT_FOUND);
         } else {
           builder.setState(GetProcedureResultResponse.State.RUNNING);
-          builder.setStartTime(proc.getStartTime());
+          builder.setSubmittedTime(proc.getSubmittedTime());
           builder.setLastUpdate(proc.getLastUpdate());
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index 484decc..2703947 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -166,7 +166,7 @@ implements ServerProcedureInterface {
 
   private void throwProcedureYieldException(final String msg) throws ProcedureYieldException {
     String logMsg = msg + "; cycle=" + this.cycles + ", running for " +
-        StringUtils.formatTimeDiff(System.currentTimeMillis(), getStartTime());
+        StringUtils.formatTimeDiff(System.currentTimeMillis(), getSubmittedTime());
     // The procedure executor logs ProcedureYieldException at trace level. For now, log these
     // yields for server crash processing at DEBUG. Revisit when stable.
     if (LOG.isDebugEnabled()) LOG.debug(logMsg);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp b/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
index 645f6fd..c841e61 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
@@ -124,7 +124,7 @@
         <td><%= escapeXml(procInfo.getProcState().toString()) %></a></td>
         <td><%= escapeXml(procInfo.getProcOwner()) %></a></td>
         <td><%= escapeXml(procInfo.getProcName()) %></a></td>
-        <td><%= new Date(procInfo.getStartTime()) %></a></td>
+        <td><%= new Date(procInfo.getSubmittedTime()) %></a></td>
         <td><%= new Date(procInfo.getLastUpdate()) %></a></td>
         <td><%= escapeXml(procInfo.isFailed() ? procInfo.getException().getMessage() : "") %></a></td>
       </tr>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb b/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
index 4358a96..64a3388 100644
--- a/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
@@ -29,13 +29,13 @@ EOF
       end
 
       def command()
-        formatter.header([ "Id", "Name", "State", "Start_Time", "Last_Update" ])
+        formatter.header([ "Id", "Name", "State", "Submitted_Time", "Last_Update" ])
 
         list = admin.list_procedures()
         list.each do |proc|
-          start_time = Time.at(proc.getStartTime / 1000).to_s
+          submitted_time = Time.at(proc.getSubmittedTime / 1000).to_s
           last_update = Time.at(proc.getLastUpdate / 1000).to_s
-          formatter.row([ proc.getProcId, proc.getProcName, proc.getProcState, start_time, last_update ])
+          formatter.row([ proc.getProcId, proc.getProcName, proc.getProcState, submitted_time, last_update ])
         end
 
         formatter.footer(list.size)


[14/50] [abbrv] hbase git commit: Revert "HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)"

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 095f4bd..d56d6ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -116,7 +116,6 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterAllFilter;
 import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.NullComparator;
@@ -4932,7 +4931,6 @@ public class TestHRegion {
       String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
       throws IOException {
     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
     final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);
     return initHRegion(tableName, startKey, stopKey, isReadOnly,

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index 6eed7df..0054642 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -153,7 +153,7 @@ public class TestHRegionReplayEvents {
     }
 
     time = System.currentTimeMillis();
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+
     primaryHri = new HRegionInfo(htd.getTableName(),
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
       false, time, 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
index 1768801..37a7664 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
@@ -48,30 +48,30 @@ import static org.junit.Assert.assertTrue;
 @Category({RegionServerTests.class, SmallTests.class})
 public class TestMemStoreChunkPool {
   private final static Configuration conf = new Configuration();
-  private static ChunkCreator chunkCreator;
+  private static MemStoreChunkPool chunkPool;
   private static boolean chunkPoolDisabledBeforeTest;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
     conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
-    chunkPoolDisabledBeforeTest = ChunkCreator.chunkPoolDisabled;
-    ChunkCreator.chunkPoolDisabled = false;
+    chunkPoolDisabledBeforeTest = MemStoreChunkPool.chunkPoolDisabled;
+    MemStoreChunkPool.chunkPoolDisabled = false;
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    chunkCreator = ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
-      globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
-    assertTrue(chunkCreator != null);
+    chunkPool = MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f,
+        MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
+    assertTrue(chunkPool != null);
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    ChunkCreator.chunkPoolDisabled = chunkPoolDisabledBeforeTest;
+    MemStoreChunkPool.chunkPoolDisabled = chunkPoolDisabledBeforeTest;
   }
 
   @Before
   public void tearDown() throws Exception {
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
   }
 
   @Test
@@ -90,7 +90,7 @@ public class TestMemStoreChunkPool {
       int size = KeyValueUtil.length(kv);
       ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
       if (newKv.getBuffer() != lastBuffer) {
-        expectedOff = 8;
+        expectedOff = 0;
         lastBuffer = newKv.getBuffer();
       }
       assertEquals(expectedOff, newKv.getOffset());
@@ -100,14 +100,14 @@ public class TestMemStoreChunkPool {
     }
     // chunks will be put back to pool after close
     mslab.close();
-    int chunkCount = chunkCreator.getPoolSize();
+    int chunkCount = chunkPool.getPoolSize();
     assertTrue(chunkCount > 0);
     // reconstruct mslab
     mslab = new MemStoreLABImpl(conf);
     // chunk should be got from the pool, so we can reuse it.
     KeyValue kv = new KeyValue(rk, cf, q, new byte[10]);
     mslab.copyCellInto(kv);
-    assertEquals(chunkCount - 1, chunkCreator.getPoolSize());
+    assertEquals(chunkCount - 1, chunkPool.getPoolSize());
   }
 
   @Test
@@ -143,7 +143,7 @@ public class TestMemStoreChunkPool {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkCreator.getPoolSize();
+    int chunkCount = chunkPool.getPoolSize();
     assertTrue(chunkCount > 0);
 
   }
@@ -189,16 +189,16 @@ public class TestMemStoreChunkPool {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkCreator.getPoolSize() == 0);
+    assertTrue(chunkPool.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
 
     // clear chunks
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
 
     // Creating another snapshot
     snapshot = memstore.snapshot();
@@ -218,20 +218,20 @@ public class TestMemStoreChunkPool {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
   }
 
   @Test
   public void testPutbackChunksMultiThreaded() throws Exception {
+    MemStoreChunkPool oldPool = MemStoreChunkPool.GLOBAL_INSTANCE;
     final int maxCount = 10;
     final int initialCount = 5;
-    final int chunkSize = 40;
+    final int chunkSize = 30;
     final int valSize = 7;
-    ChunkCreator oldCreator = ChunkCreator.getInstance();
-    ChunkCreator newCreator = new ChunkCreator(chunkSize, false, 400, 1, 0.5f, null);
-    assertEquals(initialCount, newCreator.getPoolSize());
-    assertEquals(maxCount, newCreator.getMaxCount());
-    ChunkCreator.INSTANCE = newCreator;// Replace the global ref with the new one we created.
+    MemStoreChunkPool pool = new MemStoreChunkPool(chunkSize, maxCount, initialCount, 1, false);
+    assertEquals(initialCount, pool.getPoolSize());
+    assertEquals(maxCount, pool.getMaxCount());
+    MemStoreChunkPool.GLOBAL_INSTANCE = pool;// Replace the global ref with the new one we created.
                                              // Used it for the testing. Later in finally we put
                                              // back the original
     final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
@@ -258,9 +258,9 @@ public class TestMemStoreChunkPool {
       t1.join();
       t2.join();
       t3.join();
-      assertTrue(newCreator.getPoolSize() <= maxCount);
+      assertTrue(pool.getPoolSize() <= maxCount);
     } finally {
-      ChunkCreator.INSTANCE = oldCreator;
+      MemStoreChunkPool.GLOBAL_INSTANCE = oldPool;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
index 6696e43..141b802 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
@@ -63,8 +63,8 @@ public class TestMemStoreLAB {
   public static void setUpBeforeClass() throws Exception {
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, globalMemStoreLimit,
-      0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
+    MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT,
+        MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
   }
 
   /**
@@ -76,7 +76,6 @@ public class TestMemStoreLAB {
     MemStoreLAB mslab = new MemStoreLABImpl();
     int expectedOff = 0;
     ByteBuffer lastBuffer = null;
-    long lastChunkId = -1;
     // 100K iterations by 0-1K alloc -> 50MB expected
     // should be reasonable for unit test and also cover wraparound
     // behavior
@@ -86,13 +85,8 @@ public class TestMemStoreLAB {
       int size = KeyValueUtil.length(kv);
       ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
       if (newKv.getBuffer() != lastBuffer) {
-        // since we add the chunkID at the 0th offset of the chunk and the
-        // chunkid is a long we need to account for those 8 bytes
-        expectedOff = Bytes.SIZEOF_LONG;
+        expectedOff = 0;
         lastBuffer = newKv.getBuffer();
-        long chunkId = newKv.getBuffer().getLong(0);
-        assertTrue("chunkid should be different", chunkId != lastChunkId);
-        lastChunkId = chunkId;
       }
       assertEquals(expectedOff, newKv.getOffset());
       assertTrue("Allocation overruns buffer",
@@ -142,21 +136,23 @@ public class TestMemStoreLAB {
       };
       ctx.addThread(t);
     }
-
+    
     ctx.startThreads();
     while (totalAllocated.get() < 50*1024*1024 && ctx.shouldRun()) {
       Thread.sleep(10);
     }
     ctx.stop();
+    
     // Partition the allocations by the actual byte[] they point into,
     // make sure offsets are unique for each chunk
     Map<ByteBuffer, Map<Integer, AllocRecord>> mapsByChunk =
       Maps.newHashMap();
-
+    
     int sizeCounted = 0;
     for (AllocRecord rec : Iterables.concat(allocations)) {
       sizeCounted += rec.size;
       if (rec.size == 0) continue;
+      
       Map<Integer, AllocRecord> mapForThisByteArray =
         mapsByChunk.get(rec.alloc);
       if (mapForThisByteArray == null) {
@@ -171,9 +167,7 @@ public class TestMemStoreLAB {
     
     // Now check each byte array to make sure allocations don't overlap
     for (Map<Integer, AllocRecord> allocsInChunk : mapsByChunk.values()) {
-      // since we add the chunkID at the 0th offset of the chunk and the
-      // chunkid is a long we need to account for those 8 bytes
-      int expectedOff = Bytes.SIZEOF_LONG;
+      int expectedOff = 0;
       for (AllocRecord alloc : allocsInChunk.values()) {
         assertEquals(expectedOff, alloc.offset);
         assertTrue("Allocation overruns buffer",
@@ -181,6 +175,7 @@ public class TestMemStoreLAB {
         expectedOff += alloc.size;
       }
     }
+
   }
 
   /**
@@ -199,7 +194,7 @@ public class TestMemStoreLAB {
     // set chunk size to default max alloc size, so we could easily trigger chunk retirement
     conf.setLong(MemStoreLABImpl.CHUNK_SIZE_KEY, MemStoreLABImpl.MAX_ALLOC_DEFAULT);
     // reconstruct mslab
-    ChunkCreator.clearDisableFlag();
+    MemStoreChunkPool.clearDisableFlag();
     mslab = new MemStoreLABImpl(conf);
     // launch multiple threads to trigger frequent chunk retirement
     List<Thread> threads = new ArrayList<>();
@@ -228,8 +223,6 @@ public class TestMemStoreLAB {
     }
     // close the mslab
     mslab.close();
-    // none of the chunkIds would have been returned back
-    assertTrue("All the chunks must have been cleared", ChunkCreator.INSTANCE.size() != 0);
     // make sure all chunks reclaimed or removed from chunk queue
     int queueLength = mslab.getPooledChunks().size();
     assertTrue("All chunks in chunk queue should be reclaimed or removed"

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
deleted file mode 100644
index f38a75e..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.lang.management.ManagementFactory;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ByteBufferKeyValue;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({RegionServerTests.class, SmallTests.class})
-public class TestMemstoreLABWithoutPool {
-  private final static Configuration conf = new Configuration();
-
-  private static final byte[] rk = Bytes.toBytes("r1");
-  private static final byte[] cf = Bytes.toBytes("f");
-  private static final byte[] q = Bytes.toBytes("q");
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
-        .getMax() * 0.8);
-    // disable pool
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT + Bytes.SIZEOF_LONG, false, globalMemStoreLimit,
-      0.0f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
-  }
-
-  /**
-   * Test a bunch of random allocations
-   */
-  @Test
-  public void testLABRandomAllocation() {
-    Random rand = new Random();
-    MemStoreLAB mslab = new MemStoreLABImpl();
-    int expectedOff = 0;
-    ByteBuffer lastBuffer = null;
-    long lastChunkId = -1;
-    // 100K iterations by 0-1K alloc -> 50MB expected
-    // should be reasonable for unit test and also cover wraparound
-    // behavior
-    for (int i = 0; i < 100000; i++) {
-      int valSize = rand.nextInt(1000);
-      KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
-      int size = KeyValueUtil.length(kv);
-      ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
-      if (newKv.getBuffer() != lastBuffer) {
-        // since we add the chunkID at the 0th offset of the chunk and the
-        // chunkid is a long we need to account for those 8 bytes
-        expectedOff = Bytes.SIZEOF_LONG;
-        lastBuffer = newKv.getBuffer();
-        long chunkId = newKv.getBuffer().getLong(0);
-        assertTrue("chunkid should be different", chunkId != lastChunkId);
-        lastChunkId = chunkId;
-      }
-      assertEquals(expectedOff, newKv.getOffset());
-      assertTrue("Allocation overruns buffer",
-          newKv.getOffset() + size <= newKv.getBuffer().capacity());
-      expectedOff += size;
-    }
-  }
-
-  /**
-   * Test frequent chunk retirement with chunk pool triggered by lots of threads, making sure
-   * there's no memory leak (HBASE-16195)
-   * @throws Exception if any error occurred
-   */
-  @Test
-  public void testLABChunkQueueWithMultipleMSLABs() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    MemStoreLABImpl[] mslab = new MemStoreLABImpl[10];
-    for (int i = 0; i < 10; i++) {
-      mslab[i] = new MemStoreLABImpl(conf);
-    }
-    // launch multiple threads to trigger frequent chunk retirement
-    List<Thread> threads = new ArrayList<>();
-    // create smaller sized kvs
-    final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
-        new byte[0]);
-    for (int i = 0; i < 10; i++) {
-      for (int j = 0; j < 10; j++) {
-        threads.add(getChunkQueueTestThread(mslab[i], "testLABChunkQueue-" + j, kv));
-      }
-    }
-    for (Thread thread : threads) {
-      thread.start();
-    }
-    // let it run for some time
-    Thread.sleep(3000);
-    for (Thread thread : threads) {
-      thread.interrupt();
-    }
-    boolean threadsRunning = true;
-    boolean alive = false;
-    while (threadsRunning) {
-      alive = false;
-      for (Thread thread : threads) {
-        if (thread.isAlive()) {
-          alive = true;
-          break;
-        }
-      }
-      if (!alive) {
-        threadsRunning = false;
-      }
-    }
-    // close the mslab
-    for (int i = 0; i < 10; i++) {
-      mslab[i].close();
-    }
-    // all of the chunkIds would have been returned back
-    assertTrue("All the chunks must have been cleared", ChunkCreator.INSTANCE.size() == 0);
-  }
-
-  private Thread getChunkQueueTestThread(final MemStoreLABImpl mslab, String threadName,
-      Cell cellToCopyInto) {
-    Thread thread = new Thread() {
-      boolean stopped = false;
-
-      @Override
-      public void run() {
-        while (!stopped) {
-          // keep triggering chunk retirement
-          mslab.copyCellInto(cellToCopyInto);
-        }
-      }
-
-      @Override
-      public void interrupt() {
-        this.stopped = true;
-      }
-    };
-    thread.setName(threadName);
-    thread.setDaemon(true);
-    return thread;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
index 7160e5e..4315bd4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
@@ -108,7 +108,6 @@ public class TestRecoveredEdits {
       }
     };
     Path hbaseRootDir = TEST_UTIL.getDataTestDir();
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
     Path tableDir = FSUtils.getTableDir(hbaseRootDir, htd.getTableName());
     HRegionFileSystem hrfs =

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
index ad56081..5d11c0e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
@@ -83,7 +83,6 @@ public class TestRegionIncrement {
   private HRegion getRegion(final Configuration conf, final String tableName) throws IOException {
     WAL wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDataTestDir(),
       TEST_UTIL.getDataTestDir().toString(), conf);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return (HRegion)TEST_UTIL.createLocalHRegion(Bytes.toBytes(tableName),
       HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, tableName, conf,
       false, Durability.SKIP_WAL, wal, INCREMENT_BYTES);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
index 5355c77..0d339b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
@@ -178,7 +178,6 @@ public class TestStore {
     } else {
       htd.addFamily(hcd);
     }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
index 99dd00d..3cdb227 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
@@ -111,7 +111,6 @@ public class TestStoreFileRefresherChore {
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + replicaId);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region =
         new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()),
             conf, htd, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 51260a6..4f247b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -588,7 +588,6 @@ public class TestWALLockup {
    */
   public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
   throws IOException {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.SYNC_WAL,
       wal, COLUMN_FAMILY_BYTES);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
index e63bad9..994779f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
@@ -98,7 +98,6 @@ public class TestWALMonotonicallyIncreasingSeqId {
     FSUtils.setRootDir(walConf, tableDir);
     this.walConf = walConf;
     wals = new WALFactory(walConf, null, "log_" + replicaId);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region = new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(),
         info.getTable().getNamespace()), conf, htd, null);
     region.initialize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
index 057b9bf..f976b49 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
@@ -37,9 +37,7 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -290,7 +288,6 @@ public class TestDurability {
           throw new IOException("Failed delete of " + path);
         }
       }
-      ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
       return HRegion.createHRegion(info, path, CONF, htd, log);
     }
 


[32/50] [abbrv] hbase git commit: HBASE-17001 Enforce quota violation policies in the RegionServer

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
index cc40536..d466e59 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
@@ -5778,6 +5778,1284 @@ public final class QuotaProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.SpaceLimitRequest)
   }
 
+  public interface SpaceQuotaStatusOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional .hbase.pb.SpaceViolationPolicy policy = 1;
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     */
+    boolean hasPolicy();
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getPolicy();
+
+    // optional bool in_violation = 2;
+    /**
+     * <code>optional bool in_violation = 2;</code>
+     */
+    boolean hasInViolation();
+    /**
+     * <code>optional bool in_violation = 2;</code>
+     */
+    boolean getInViolation();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.SpaceQuotaStatus}
+   *
+   * <pre>
+   * Represents the state of a quota on a table. Either the quota is not in violation
+   * or it is in violatino there is a violation policy which should be in effect.
+   * </pre>
+   */
+  public static final class SpaceQuotaStatus extends
+      com.google.protobuf.GeneratedMessage
+      implements SpaceQuotaStatusOrBuilder {
+    // Use SpaceQuotaStatus.newBuilder() to construct.
+    private SpaceQuotaStatus(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private SpaceQuotaStatus(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final SpaceQuotaStatus defaultInstance;
+    public static SpaceQuotaStatus getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public SpaceQuotaStatus getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SpaceQuotaStatus(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy value = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                policy_ = value;
+              }
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              inViolation_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaStatus_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.class, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<SpaceQuotaStatus> PARSER =
+        new com.google.protobuf.AbstractParser<SpaceQuotaStatus>() {
+      public SpaceQuotaStatus parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new SpaceQuotaStatus(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<SpaceQuotaStatus> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional .hbase.pb.SpaceViolationPolicy policy = 1;
+    public static final int POLICY_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy policy_;
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     */
+    public boolean hasPolicy() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getPolicy() {
+      return policy_;
+    }
+
+    // optional bool in_violation = 2;
+    public static final int IN_VIOLATION_FIELD_NUMBER = 2;
+    private boolean inViolation_;
+    /**
+     * <code>optional bool in_violation = 2;</code>
+     */
+    public boolean hasInViolation() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional bool in_violation = 2;</code>
+     */
+    public boolean getInViolation() {
+      return inViolation_;
+    }
+
+    private void initFields() {
+      policy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+      inViolation_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeEnum(1, policy_.getNumber());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(2, inViolation_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(1, policy_.getNumber());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(2, inViolation_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus other = (org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus) obj;
+
+      boolean result = true;
+      result = result && (hasPolicy() == other.hasPolicy());
+      if (hasPolicy()) {
+        result = result &&
+            (getPolicy() == other.getPolicy());
+      }
+      result = result && (hasInViolation() == other.hasInViolation());
+      if (hasInViolation()) {
+        result = result && (getInViolation()
+            == other.getInViolation());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasPolicy()) {
+        hash = (37 * hash) + POLICY_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getPolicy());
+      }
+      if (hasInViolation()) {
+        hash = (37 * hash) + IN_VIOLATION_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getInViolation());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.SpaceQuotaStatus}
+     *
+     * <pre>
+     * Represents the state of a quota on a table. Either the quota is not in violation
+     * or it is in violatino there is a violation policy which should be in effect.
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaStatus_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.class, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        policy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        inViolation_ = false;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaStatus_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus build() {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus result = new org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.policy_ = policy_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.inViolation_ = inViolation_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance()) return this;
+        if (other.hasPolicy()) {
+          setPolicy(other.getPolicy());
+        }
+        if (other.hasInViolation()) {
+          setInViolation(other.getInViolation());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional .hbase.pb.SpaceViolationPolicy policy = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy policy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       */
+      public boolean hasPolicy() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getPolicy() {
+        return policy_;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       */
+      public Builder setPolicy(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        policy_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       */
+      public Builder clearPolicy() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        policy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+        onChanged();
+        return this;
+      }
+
+      // optional bool in_violation = 2;
+      private boolean inViolation_ ;
+      /**
+       * <code>optional bool in_violation = 2;</code>
+       */
+      public boolean hasInViolation() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional bool in_violation = 2;</code>
+       */
+      public boolean getInViolation() {
+        return inViolation_;
+      }
+      /**
+       * <code>optional bool in_violation = 2;</code>
+       */
+      public Builder setInViolation(boolean value) {
+        bitField0_ |= 0x00000002;
+        inViolation_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool in_violation = 2;</code>
+       */
+      public Builder clearInViolation() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        inViolation_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SpaceQuotaStatus)
+    }
+
+    static {
+      defaultInstance = new SpaceQuotaStatus(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.SpaceQuotaStatus)
+  }
+
+  public interface SpaceQuotaSnapshotOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional .hbase.pb.SpaceQuotaStatus status = 1;
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    boolean hasStatus();
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus getStatus();
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getStatusOrBuilder();
+
+    // optional uint64 usage = 2;
+    /**
+     * <code>optional uint64 usage = 2;</code>
+     */
+    boolean hasUsage();
+    /**
+     * <code>optional uint64 usage = 2;</code>
+     */
+    long getUsage();
+
+    // optional uint64 limit = 3;
+    /**
+     * <code>optional uint64 limit = 3;</code>
+     */
+    boolean hasLimit();
+    /**
+     * <code>optional uint64 limit = 3;</code>
+     */
+    long getLimit();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.SpaceQuotaSnapshot}
+   *
+   * <pre>
+   * Message stored in the value of hbase:quota table to denote the status of a table WRT
+   * the quota applicable to it.
+   * </pre>
+   */
+  public static final class SpaceQuotaSnapshot extends
+      com.google.protobuf.GeneratedMessage
+      implements SpaceQuotaSnapshotOrBuilder {
+    // Use SpaceQuotaSnapshot.newBuilder() to construct.
+    private SpaceQuotaSnapshot(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private SpaceQuotaSnapshot(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final SpaceQuotaSnapshot defaultInstance;
+    public static SpaceQuotaSnapshot getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public SpaceQuotaSnapshot getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SpaceQuotaSnapshot(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = status_.toBuilder();
+              }
+              status_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(status_);
+                status_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              usage_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              limit_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.class, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<SpaceQuotaSnapshot> PARSER =
+        new com.google.protobuf.AbstractParser<SpaceQuotaSnapshot>() {
+      public SpaceQuotaSnapshot parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new SpaceQuotaSnapshot(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<SpaceQuotaSnapshot> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional .hbase.pb.SpaceQuotaStatus status = 1;
+    public static final int STATUS_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus status_;
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    public boolean hasStatus() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus getStatus() {
+      return status_;
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getStatusOrBuilder() {
+      return status_;
+    }
+
+    // optional uint64 usage = 2;
+    public static final int USAGE_FIELD_NUMBER = 2;
+    private long usage_;
+    /**
+     * <code>optional uint64 usage = 2;</code>
+     */
+    public boolean hasUsage() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional uint64 usage = 2;</code>
+     */
+    public long getUsage() {
+      return usage_;
+    }
+
+    // optional uint64 limit = 3;
+    public static final int LIMIT_FIELD_NUMBER = 3;
+    private long limit_;
+    /**
+     * <code>optional uint64 limit = 3;</code>
+     */
+    public boolean hasLimit() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional uint64 limit = 3;</code>
+     */
+    public long getLimit() {
+      return limit_;
+    }
+
+    private void initFields() {
+      status_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
+      usage_ = 0L;
+      limit_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, status_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, usage_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt64(3, limit_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, status_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, usage_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(3, limit_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot other = (org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot) obj;
+
+      boolean result = true;
+      result = result && (hasStatus() == other.hasStatus());
+      if (hasStatus()) {
+        result = result && getStatus()
+            .equals(other.getStatus());
+      }
+      result = result && (hasUsage() == other.hasUsage());
+      if (hasUsage()) {
+        result = result && (getUsage()
+            == other.getUsage());
+      }
+      result = result && (hasLimit() == other.hasLimit());
+      if (hasLimit()) {
+        result = result && (getLimit()
+            == other.getLimit());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasStatus()) {
+        hash = (37 * hash) + STATUS_FIELD_NUMBER;
+        hash = (53 * hash) + getStatus().hashCode();
+      }
+      if (hasUsage()) {
+        hash = (37 * hash) + USAGE_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getUsage());
+      }
+      if (hasLimit()) {
+        hash = (37 * hash) + LIMIT_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getLimit());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.SpaceQuotaSnapshot}
+     *
+     * <pre>
+     * Message stored in the value of hbase:quota table to denote the status of a table WRT
+     * the quota applicable to it.
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.class, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getStatusFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (statusBuilder_ == null) {
+          status_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
+        } else {
+          statusBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        usage_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        limit_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot build() {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot result = new org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (statusBuilder_ == null) {
+          result.status_ = status_;
+        } else {
+          result.status_ = statusBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.usage_ = usage_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.limit_ = limit_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.getDefaultInstance()) return this;
+        if (other.hasStatus()) {
+          mergeStatus(other.getStatus());
+        }
+        if (other.hasUsage()) {
+          setUsage(other.getUsage());
+        }
+        if (other.hasLimit()) {
+          setLimit(other.getLimit());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional .hbase.pb.SpaceQuotaStatus status = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus status_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder> statusBuilder_;
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public boolean hasStatus() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus getStatus() {
+        if (statusBuilder_ == null) {
+          return status_;
+        } else {
+          return statusBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public Builder setStatus(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
+        if (statusBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          status_ = value;
+          onChanged();
+        } else {
+          statusBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public Builder setStatus(
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder builderForValue) {
+        if (statusBuilder_ == null) {
+          status_ = builderForValue.build();
+          onChanged();
+        } else {
+          statusBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public Builder mergeStatus(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
+        if (statusBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              status_ != org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance()) {
+            status_ =
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.newBuilder(status_).mergeFrom(value).buildPartial();
+          } else {
+            status_ = value;
+          }
+          onChanged();
+        } else {
+          statusBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public Builder clearStatus() {
+        if (statusBuilder_ == null) {
+          status_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
+          onChanged();
+        } else {
+          statusBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder getStatusBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getStatusFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getStatusOrBuilder() {
+        if (statusBuilder_ != null) {
+          return statusBuilder_.getMessageOrBuilder();
+        } else {
+          return status_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder>
+          getStatusFieldBuilder() {
+        if (statusBuilder_ == null) {
+          statusBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder>(
+                  status_,
+                  getParentForChildren(),
+                  isClean());
+          status_ = null;
+        }
+        return statusBuilder_;
+      }
+
+      // optional uint64 usage = 2;
+      private long usage_ ;
+      /**
+       * <code>optional uint64 usage = 2;</code>
+       */
+      public boolean hasUsage() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional uint64 usage = 2;</code>
+       */
+      public long getUsage() {
+        return usage_;
+      }
+      /**
+       * <code>optional uint64 usage = 2;</code>
+       */
+      public Builder setUsage(long value) {
+        bitField0_ |= 0x00000002;
+        usage_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 usage = 2;</code>
+       */
+      public Builder clearUsage() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        usage_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional uint64 limit = 3;
+      private long limit_ ;
+      /**
+       * <code>optional uint64 limit = 3;</code>
+       */
+      public boolean hasLimit() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional uint64 limit = 3;</code>
+       */
+      public long getLimit() {
+        return limit_;
+      }
+      /**
+       * <code>optional uint64 limit = 3;</code>
+       */
+      public Builder setLimit(long value) {
+        bitField0_ |= 0x00000004;
+        limit_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 limit = 3;</code>
+       */
+      public Builder clearLimit() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        limit_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SpaceQuotaSnapshot)
+    }
+
+    static {
+      defaultInstance = new SpaceQuotaSnapshot(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.SpaceQuotaSnapshot)
+  }
+
   private static com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TimedQuota_descriptor;
   private static
@@ -5813,6 +7091,16 @@ public final class QuotaProtos {
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_SpaceQuotaStatus_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -5842,16 +7130,20 @@ public final class QuotaProtos {
       "limit\030\001 \001(\004\0228\n\020violation_policy\030\002 \001(\0162\036." +
       "hbase.pb.SpaceViolationPolicy\"8\n\021SpaceLi" +
       "mitRequest\022#\n\005quota\030\001 \001(\0132\024.hbase.pb.Spa",
-      "ceQuota*&\n\nQuotaScope\022\013\n\007CLUSTER\020\001\022\013\n\007MA" +
-      "CHINE\020\002*v\n\014ThrottleType\022\022\n\016REQUEST_NUMBE" +
-      "R\020\001\022\020\n\014REQUEST_SIZE\020\002\022\020\n\014WRITE_NUMBER\020\003\022" +
-      "\016\n\nWRITE_SIZE\020\004\022\017\n\013READ_NUMBER\020\005\022\r\n\tREAD" +
-      "_SIZE\020\006*$\n\tQuotaType\022\014\n\010THROTTLE\020\001\022\t\n\005SP" +
-      "ACE\020\002*]\n\024SpaceViolationPolicy\022\013\n\007DISABLE" +
-      "\020\001\022\031\n\025NO_WRITES_COMPACTIONS\020\002\022\r\n\tNO_WRIT" +
-      "ES\020\003\022\016\n\nNO_INSERTS\020\004BA\n*org.apache.hadoo" +
-      "p.hbase.protobuf.generatedB\013QuotaProtosH" +
-      "\001\210\001\001\240\001\001"
+      "ceQuota\"X\n\020SpaceQuotaStatus\022.\n\006policy\030\001 " +
+      "\001(\0162\036.hbase.pb.SpaceViolationPolicy\022\024\n\014i" +
+      "n_violation\030\002 \001(\010\"^\n\022SpaceQuotaSnapshot\022" +
+      "*\n\006status\030\001 \001(\0132\032.hbase.pb.SpaceQuotaSta" +
+      "tus\022\r\n\005usage\030\002 \001(\004\022\r\n\005limit\030\003 \001(\004*&\n\nQuo" +
+      "taScope\022\013\n\007CLUSTER\020\001\022\013\n\007MACHINE\020\002*v\n\014Thr" +
+      "ottleType\022\022\n\016REQUEST_NUMBER\020\001\022\020\n\014REQUEST" +
+      "_SIZE\020\002\022\020\n\014WRITE_NUMBER\020\003\022\016\n\nWRITE_SIZE\020" +
+      "\004\022\017\n\013READ_NUMBER\020\005\022\r\n\tREAD_SIZE\020\006*$\n\tQuo" +
+      "taType\022\014\n\010THROTTLE\020\001\022\t\n\005SPACE\020\002*]\n\024Space",
+      "ViolationPolicy\022\013\n\007DISABLE\020\001\022\031\n\025NO_WRITE" +
+      "S_COMPACTIONS\020\002\022\r\n\tNO_WRITES\020\003\022\016\n\nNO_INS" +
+      "ERTS\020\004BA\n*org.apache.hadoop.hbase.protob" +
+      "uf.generatedB\013QuotaProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -5900,6 +7192,18 @@ public final class QuotaProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_SpaceLimitRequest_descriptor,
               new java.lang.String[] { "Quota", });
+          internal_static_hbase_pb_SpaceQuotaStatus_descriptor =
+            getDescriptor().getMessageTypes().get(7);
+          internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_SpaceQuotaStatus_descriptor,
+              new java.lang.String[] { "Policy", "InViolation", });
+          internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor =
+            getDescriptor().getMessageTypes().get(8);
+          internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor,
+              new java.lang.String[] { "Status", "Usage", "Limit", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-protocol/src/main/protobuf/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Quota.proto b/hbase-protocol/src/main/protobuf/Quota.proto
index c12b997..02d825b 100644
--- a/hbase-protocol/src/main/protobuf/Quota.proto
+++ b/hbase-protocol/src/main/protobuf/Quota.proto
@@ -95,3 +95,18 @@ message SpaceQuota {
 message SpaceLimitRequest {
   optional SpaceQuota quota = 1;
 }
+
+// Represents the state of a quota on a table. Either the quota is not in violation
+// or it is in violatino there is a violation policy which should be in effect.
+message SpaceQuotaStatus {
+  optional SpaceViolationPolicy policy = 1;
+  optional bool in_violation = 2;
+}
+
+// Message stored in the value of hbase:quota table to denote the status of a table WRT
+// the quota applicable to it.
+message SpaceQuotaSnapshot {
+  optional SpaceQuotaStatus status = 1;
+  optional uint64 usage = 2;
+  optional uint64 limit = 3;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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 6f9dc8b..0243934 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
@@ -134,8 +134,8 @@ import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
-import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifier;
-import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifierFactory;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifier;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifierFactory;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -377,7 +377,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   // it is assigned after 'initialized' guard set to true, so should be volatile
   private volatile MasterQuotaManager quotaManager;
-  private SpaceQuotaViolationNotifier spaceQuotaViolationNotifier;
+  private SpaceQuotaSnapshotNotifier spaceQuotaSnapshotNotifier;
   private QuotaObserverChore quotaObserverChore;
 
   private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
@@ -904,10 +904,14 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     status.setStatus("Starting quota manager");
     initQuotaManager();
-    this.spaceQuotaViolationNotifier = createQuotaViolationNotifier();
-    this.quotaObserverChore = new QuotaObserverChore(this);
-    // Start the chore to read the region FS space reports and act on them
-    getChoreService().scheduleChore(quotaObserverChore);
+    if (QuotaUtil.isQuotaEnabled(conf)) {
+      // Create the quota snapshot notifier
+      spaceQuotaSnapshotNotifier = createQuotaSnapshotNotifier();
+      spaceQuotaSnapshotNotifier.initialize(getClusterConnection());
+      this.quotaObserverChore = new QuotaObserverChore(this);
+      // Start the chore to read the region FS space reports and act on them
+      getChoreService().scheduleChore(quotaObserverChore);
+    }
 
     // clear the dead servers with same host name and port of online server because we are not
     // removing dead server with same hostname and port of rs which is trying to check in before
@@ -995,10 +999,9 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.quotaManager = quotaManager;
   }
 
-  SpaceQuotaViolationNotifier createQuotaViolationNotifier() {
-    SpaceQuotaViolationNotifier notifier =
-        SpaceQuotaViolationNotifierFactory.getInstance().create(getConfiguration());
-    notifier.initialize(getClusterConnection());
+  SpaceQuotaSnapshotNotifier createQuotaSnapshotNotifier() {
+    SpaceQuotaSnapshotNotifier notifier =
+        SpaceQuotaSnapshotNotifierFactory.getInstance().create(getConfiguration());
     return notifier;
   }
 
@@ -3349,7 +3352,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     return this.quotaObserverChore;
   }
 
-  public SpaceQuotaViolationNotifier getSpaceQuotaViolationNotifier() {
-    return this.spaceQuotaViolationNotifier;
+  public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier() {
+    return this.spaceQuotaSnapshotNotifier;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
new file mode 100644
index 0000000..9408e6c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+
+/**
+ * A class to ease dealing with tables that have and do not have violation policies
+ * being enforced in a uniform manner. Immutable.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ActivePolicyEnforcement {
+  private final Map<TableName,SpaceViolationPolicyEnforcement> activePolicies;
+  private final Map<TableName,SpaceQuotaSnapshot> snapshots;
+  private final RegionServerServices rss;
+
+  public ActivePolicyEnforcement(Map<TableName,SpaceViolationPolicyEnforcement> activePolicies,
+      Map<TableName,SpaceQuotaSnapshot> snapshots, RegionServerServices rss) {
+    this.activePolicies = activePolicies;
+    this.snapshots = snapshots;
+    this.rss = rss;
+  }
+
+  /**
+   * Returns the proper {@link SpaceViolationPolicyEnforcement} implementation for the given table.
+   * If the given table does not have a violation policy enforced, a "no-op" policy will
+   * be returned which always allows an action.
+   *
+   * @see #getPolicyEnforcement(TableName)
+   */
+  public SpaceViolationPolicyEnforcement getPolicyEnforcement(Region r) {
+    return getPolicyEnforcement(Objects.requireNonNull(r).getTableDesc().getTableName());
+  }
+
+  /**
+   * Returns the proper {@link SpaceViolationPolicyEnforcement} implementation for the given table.
+   * If the given table does not have a violation policy enforced, a "no-op" policy will
+   * be returned which always allows an action.
+   *
+   * @param tableName The table to fetch the policy for.
+   * @return A non-null {@link SpaceViolationPolicyEnforcement} instance.
+   */
+  public SpaceViolationPolicyEnforcement getPolicyEnforcement(TableName tableName) {
+    SpaceViolationPolicyEnforcement policy = activePolicies.get(Objects.requireNonNull(tableName));
+    if (null == policy) {
+      synchronized (activePolicies) {
+        // If we've never seen a snapshot, assume no use, and infinite limit
+        SpaceQuotaSnapshot snapshot = snapshots.get(tableName);
+        if (null == snapshot) {
+          snapshot = SpaceQuotaSnapshot.getNoSuchSnapshot();
+        }
+        // Create the default policy and cache it
+        return SpaceViolationPolicyEnforcementFactory.getInstance().createWithoutViolation(
+            rss, tableName, snapshot);
+      }
+    }
+    return policy;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + ": " + activePolicies;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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
new file mode 100644
index 0000000..75550f3
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.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 java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+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.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+/**
+ * {@link QuotaSnapshotStore} implementation for namespaces.
+ */
+@InterfaceAudience.Private
+public class NamespaceQuotaSnapshotStore implements QuotaSnapshotStore<String> {
+  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  private final ReadLock rlock = lock.readLock();
+  private final WriteLock wlock = lock.writeLock();
+
+  private final Connection conn;
+  private final QuotaObserverChore chore;
+  private Map<HRegionInfo,Long> regionUsage;
+
+  public NamespaceQuotaSnapshotStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
+    this.conn = Objects.requireNonNull(conn);
+    this.chore = Objects.requireNonNull(chore);
+    this.regionUsage = Objects.requireNonNull(regionUsage);
+  }
+
+  @Override
+  public SpaceQuota getSpaceQuota(String namespace) throws IOException {
+    Quotas quotas = getQuotaForNamespace(namespace);
+    if (null != quotas && quotas.hasSpace()) {
+      return quotas.getSpace();
+    }
+    return null;
+  }
+
+  /**
+   * Fetches the namespace quota. Visible for mocking/testing.
+   */
+  Quotas getQuotaForNamespace(String namespace) throws IOException {
+    return QuotaTableUtil.getNamespaceQuota(conn, namespace);
+  }
+
+  @Override
+  public SpaceQuotaSnapshot getCurrentState(String namespace) {
+    // Defer the "current state" to the chore
+    return this.chore.getNamespaceQuotaSnapshot(namespace);
+  }
+
+  @Override
+  public SpaceQuotaSnapshot getTargetState(String subject, SpaceQuota spaceQuota) {
+    rlock.lock();
+    try {
+      final long sizeLimitInBytes = spaceQuota.getSoftLimit();
+      long sum = 0L;
+      for (Entry<HRegionInfo,Long> entry : filterBySubject(subject)) {
+        sum += entry.getValue();
+      }
+      // 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()));
+      return new SpaceQuotaSnapshot(status, sum, sizeLimitInBytes);
+    } finally {
+      rlock.unlock();
+    }
+  }
+
+  @Override
+  public Iterable<Entry<HRegionInfo,Long>> filterBySubject(String namespace) {
+    rlock.lock();
+    try {
+      return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<HRegionInfo,Long>>() {
+        @Override
+        public boolean apply(Entry<HRegionInfo,Long> input) {
+          return namespace.equals(input.getKey().getTable().getNamespaceAsString());
+        }
+      });
+    } finally {
+      rlock.unlock();
+    }
+  }
+
+  @Override
+  public void setCurrentState(String namespace, SpaceQuotaSnapshot snapshot) {
+    // Defer the "current state" to the chore
+    this.chore.setNamespaceQuotaSnapshot(namespace, snapshot);
+  }
+
+  @Override
+  public void setRegionUsage(Map<HRegionInfo,Long> regionUsage) {
+    wlock.lock();
+    try {
+      this.regionUsage = Objects.requireNonNull(regionUsage);
+    } finally {
+      wlock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaViolationStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaViolationStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaViolationStore.java
deleted file mode 100644
index 017ecec..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaViolationStore.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.quotas;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Map.Entry;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
-
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-
-/**
- * {@link QuotaViolationStore} implementation for namespaces.
- */
-@InterfaceAudience.Private
-public class NamespaceQuotaViolationStore implements QuotaViolationStore<String> {
-  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-  private final ReadLock rlock = lock.readLock();
-  private final WriteLock wlock = lock.writeLock();
-
-  private final Connection conn;
-  private final QuotaObserverChore chore;
-  private Map<HRegionInfo,Long> regionUsage;
-
-  public NamespaceQuotaViolationStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
-    this.conn = Objects.requireNonNull(conn);
-    this.chore = Objects.requireNonNull(chore);
-    this.regionUsage = Objects.requireNonNull(regionUsage);
-  }
-
-  @Override
-  public SpaceQuota getSpaceQuota(String namespace) throws IOException {
-    Quotas quotas = getQuotaForNamespace(namespace);
-    if (null != quotas && quotas.hasSpace()) {
-      return quotas.getSpace();
-    }
-    return null;
-  }
-
-  /**
-   * Fetches the namespace quota. Visible for mocking/testing.
-   */
-  Quotas getQuotaForNamespace(String namespace) throws IOException {
-    return QuotaTableUtil.getNamespaceQuota(conn, namespace);
-  }
-
-  @Override
-  public ViolationState getCurrentState(String namespace) {
-    // Defer the "current state" to the chore
-    return this.chore.getNamespaceQuotaViolation(namespace);
-  }
-
-  @Override
-  public ViolationState getTargetState(String subject, SpaceQuota spaceQuota) {
-    rlock.lock();
-    try {
-      final long sizeLimitInBytes = spaceQuota.getSoftLimit();
-      long sum = 0L;
-      for (Entry<HRegionInfo,Long> entry : filterBySubject(subject)) {
-        sum += entry.getValue();
-        if (sum > sizeLimitInBytes) {
-          // Short-circuit early
-          return ViolationState.IN_VIOLATION;
-        }
-      }
-      // Observance is defined as the size of the table being less than the limit
-      return sum <= sizeLimitInBytes ? ViolationState.IN_OBSERVANCE : ViolationState.IN_VIOLATION;
-    } finally {
-      rlock.unlock();
-    }
-  }
-
-  @Override
-  public Iterable<Entry<HRegionInfo,Long>> filterBySubject(String namespace) {
-    rlock.lock();
-    try {
-      return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<HRegionInfo,Long>>() {
-        @Override
-        public boolean apply(Entry<HRegionInfo,Long> input) {
-          return namespace.equals(input.getKey().getTable().getNamespaceAsString());
-        }
-      });
-    } finally {
-      rlock.unlock();
-    }
-  }
-
-  @Override
-  public void setCurrentState(String namespace, ViolationState state) {
-    // Defer the "current state" to the chore
-    this.chore.setNamespaceQuotaViolation(namespace, state);
-  }
-
-  @Override
-  public void setRegionUsage(Map<HRegionInfo,Long> regionUsage) {
-    wlock.lock();
-    try {
-      this.regionUsage = Objects.requireNonNull(regionUsage);
-    } finally {
-      wlock.unlock();
-    }
-  }
-}


[30/50] [abbrv] hbase git commit: HBASE-17001 Enforce quota violation policies in the RegionServer

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcementFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcementFactory.java
new file mode 100644
index 0000000..6b754b9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicyEnforcementFactory.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.hadoop.hbase.quotas.policies.BulkLoadVerifyingViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.DisableTableViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoInsertsViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesCompactionsViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+
+/**
+ * A factory class for instantiating {@link SpaceViolationPolicyEnforcement} instances.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SpaceViolationPolicyEnforcementFactory {
+
+  private static final SpaceViolationPolicyEnforcementFactory INSTANCE =
+      new SpaceViolationPolicyEnforcementFactory();
+
+  private SpaceViolationPolicyEnforcementFactory() {}
+
+  /**
+   * Returns an instance of this factory.
+   */
+  public static SpaceViolationPolicyEnforcementFactory getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Constructs the appropriate {@link SpaceViolationPolicyEnforcement} for tables that are
+   * in violation of their space quota.
+   */
+  public SpaceViolationPolicyEnforcement create(
+      RegionServerServices rss, TableName tableName, SpaceQuotaSnapshot snapshot) {
+    SpaceViolationPolicyEnforcement enforcement;
+    SpaceQuotaStatus status = snapshot.getQuotaStatus();
+    if (!status.isInViolation()) {
+      throw new IllegalArgumentException(tableName + " is not in violation. Snapshot=" + snapshot);
+    }
+    switch (status.getPolicy()) {
+      case DISABLE:
+        enforcement = new DisableTableViolationPolicyEnforcement();
+        break;
+      case NO_WRITES_COMPACTIONS:
+        enforcement = new NoWritesCompactionsViolationPolicyEnforcement();
+        break;
+      case NO_WRITES:
+        enforcement = new NoWritesViolationPolicyEnforcement();
+        break;
+      case NO_INSERTS:
+        enforcement = new NoInsertsViolationPolicyEnforcement();
+        break;
+      default:
+        throw new IllegalArgumentException("Unhandled SpaceViolationPolicy: " + status.getPolicy());
+    }
+    enforcement.initialize(rss, tableName, snapshot);
+    return enforcement;
+  }
+
+  /**
+   * Creates the "default" {@link SpaceViolationPolicyEnforcement} for a table that isn't in
+   * violation. This is used to have uniform policy checking for tables in and not quotas.
+   */
+  public SpaceViolationPolicyEnforcement createWithoutViolation(
+      RegionServerServices rss, TableName tableName, SpaceQuotaSnapshot snapshot) {
+    SpaceQuotaStatus status = snapshot.getQuotaStatus();
+    if (status.isInViolation()) {
+      throw new IllegalArgumentException(
+          tableName + " is in violation. Logic error. Snapshot=" + snapshot);
+    }
+    BulkLoadVerifyingViolationPolicyEnforcement enforcement = new BulkLoadVerifyingViolationPolicyEnforcement();
+    enforcement.initialize(rss, tableName, snapshot);
+    return enforcement;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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
new file mode 100644
index 0000000..e196354
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.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 java.io.IOException;
+import java.util.Map;
+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.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.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+/**
+ * {@link QuotaSnapshotStore} for tables.
+ */
+@InterfaceAudience.Private
+public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
+  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  private final ReadLock rlock = lock.readLock();
+  private final WriteLock wlock = lock.writeLock();
+
+  private final Connection conn;
+  private final QuotaObserverChore chore;
+  private Map<HRegionInfo,Long> regionUsage;
+
+  public TableQuotaSnapshotStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
+    this.conn = Objects.requireNonNull(conn);
+    this.chore = Objects.requireNonNull(chore);
+    this.regionUsage = Objects.requireNonNull(regionUsage);
+  }
+
+  @Override
+  public SpaceQuota getSpaceQuota(TableName subject) throws IOException {
+    Quotas quotas = getQuotaForTable(subject);
+    if (null != quotas && quotas.hasSpace()) {
+      return quotas.getSpace();
+    }
+    return null;
+  }
+  /**
+   * Fetches the table quota. Visible for mocking/testing.
+   */
+  Quotas getQuotaForTable(TableName table) throws IOException {
+    return QuotaTableUtil.getTableQuota(conn, table);
+  }
+
+  @Override
+  public SpaceQuotaSnapshot getCurrentState(TableName table) {
+    // Defer the "current state" to the chore
+    return chore.getTableQuotaSnapshot(table);
+  }
+
+  @Override
+  public SpaceQuotaSnapshot getTargetState(TableName table, SpaceQuota spaceQuota) {
+    rlock.lock();
+    try {
+      final long sizeLimitInBytes = spaceQuota.getSoftLimit();
+      long sum = 0L;
+      for (Entry<HRegionInfo,Long> entry : filterBySubject(table)) {
+        sum += entry.getValue();
+      }
+      // 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()));
+      return new SpaceQuotaSnapshot(status, sum, sizeLimitInBytes);
+    } finally {
+      rlock.unlock();
+    }
+  }
+
+  @Override
+  public Iterable<Entry<HRegionInfo,Long>> filterBySubject(TableName table) {
+    rlock.lock();
+    try {
+      return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<HRegionInfo,Long>>() {
+        @Override
+        public boolean apply(Entry<HRegionInfo,Long> input) {
+          return table.equals(input.getKey().getTable());
+        }
+      });
+    } finally {
+      rlock.unlock();
+    }
+  }
+
+  @Override
+  public void setCurrentState(TableName table, SpaceQuotaSnapshot snapshot) {
+    // Defer the "current state" to the chore
+    this.chore.setTableQuotaViolation(table, snapshot);
+  }
+
+  @Override
+  public void setRegionUsage(Map<HRegionInfo,Long> regionUsage) {
+    wlock.lock();
+    try {
+      this.regionUsage = Objects.requireNonNull(regionUsage);
+    } finally {
+      wlock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaViolationStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaViolationStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaViolationStore.java
deleted file mode 100644
index 6aba1cf..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaViolationStore.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.quotas;
-
-import java.io.IOException;
-import java.util.Map;
-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.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.shaded.protobuf.generated.QuotaProtos.Quotas;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-
-/**
- * {@link QuotaViolationStore} for tables.
- */
-@InterfaceAudience.Private
-public class TableQuotaViolationStore implements QuotaViolationStore<TableName> {
-  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-  private final ReadLock rlock = lock.readLock();
-  private final WriteLock wlock = lock.writeLock();
-
-  private final Connection conn;
-  private final QuotaObserverChore chore;
-  private Map<HRegionInfo,Long> regionUsage;
-
-  public TableQuotaViolationStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
-    this.conn = Objects.requireNonNull(conn);
-    this.chore = Objects.requireNonNull(chore);
-    this.regionUsage = Objects.requireNonNull(regionUsage);
-  }
-
-  @Override
-  public SpaceQuota getSpaceQuota(TableName subject) throws IOException {
-    Quotas quotas = getQuotaForTable(subject);
-    if (null != quotas && quotas.hasSpace()) {
-      return quotas.getSpace();
-    }
-    return null;
-  }
-  /**
-   * Fetches the table quota. Visible for mocking/testing.
-   */
-  Quotas getQuotaForTable(TableName table) throws IOException {
-    return QuotaTableUtil.getTableQuota(conn, table);
-  }
-
-  @Override
-  public ViolationState getCurrentState(TableName table) {
-    // Defer the "current state" to the chore
-    return chore.getTableQuotaViolation(table);
-  }
-
-  @Override
-  public ViolationState getTargetState(TableName table, SpaceQuota spaceQuota) {
-    rlock.lock();
-    try {
-      final long sizeLimitInBytes = spaceQuota.getSoftLimit();
-      long sum = 0L;
-      for (Entry<HRegionInfo,Long> entry : filterBySubject(table)) {
-        sum += entry.getValue();
-        if (sum > sizeLimitInBytes) {
-          // Short-circuit early
-          return ViolationState.IN_VIOLATION;
-        }
-      }
-      // Observance is defined as the size of the table being less than the limit
-      return sum <= sizeLimitInBytes ? ViolationState.IN_OBSERVANCE : ViolationState.IN_VIOLATION;
-    } finally {
-      rlock.unlock();
-    }
-  }
-
-  @Override
-  public Iterable<Entry<HRegionInfo,Long>> filterBySubject(TableName table) {
-    rlock.lock();
-    try {
-      return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<HRegionInfo,Long>>() {
-        @Override
-        public boolean apply(Entry<HRegionInfo,Long> input) {
-          return table.equals(input.getKey().getTable());
-        }
-      });
-    } finally {
-      rlock.unlock();
-    }
-  }
-
-  @Override
-  public void setCurrentState(TableName table, ViolationState state) {
-    // Defer the "current state" to the chore
-    this.chore.setTableQuotaViolation(table, state);
-  }
-
-  @Override
-  public void setRegionUsage(Map<HRegionInfo,Long> regionUsage) {
-    wlock.lock();
-    try {
-      this.regionUsage = Objects.requireNonNull(regionUsage);
-    } finally {
-      wlock.unlock();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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
new file mode 100644
index 0000000..548faf8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaSnapshotNotifier.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+
+/**
+ * A {@link SpaceQuotaSnapshotNotifier} which uses the hbase:quota table.
+ */
+public class TableSpaceQuotaSnapshotNotifier implements SpaceQuotaSnapshotNotifier {
+  private static final Log LOG = LogFactory.getLog(TableSpaceQuotaSnapshotNotifier.class);
+
+  private Connection conn;
+
+  @Override
+  public void transitionTable(
+      TableName tableName, SpaceQuotaSnapshot snapshot) throws IOException {
+    final Put p = QuotaTableUtil.createPutSpaceSnapshot(tableName, snapshot);
+    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Persisting a space quota snapshot " + snapshot + " for " + tableName);
+      }
+      quotaTable.put(p);
+    }
+  }
+
+  @Override
+  public void initialize(Connection conn) {
+    this.conn = conn;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java
deleted file mode 100644
index a8b1c55..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableSpaceQuotaViolationNotifier.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.quotas;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Table;
-
-/**
- * A {@link SpaceQuotaViolationNotifier} which uses the hbase:quota table.
- */
-public class TableSpaceQuotaViolationNotifier implements SpaceQuotaViolationNotifier {
-
-  private Connection conn;
-
-  @Override
-  public void transitionTableToViolation(
-      TableName tableName, SpaceViolationPolicy violationPolicy) throws IOException {
-    final Put p = QuotaTableUtil.createEnableViolationPolicyUpdate(tableName, violationPolicy);
-    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
-      quotaTable.put(p);
-    }
-  }
-
-  @Override
-  public void transitionTableToObservance(TableName tableName) throws IOException {
-    final Delete d = QuotaTableUtil.createRemoveViolationPolicyUpdate(tableName);
-    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
-      quotaTable.delete(d);
-    }
-  }
-
-  @Override
-  public void initialize(Connection conn) {
-    this.conn = conn;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/AbstractViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/AbstractViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/AbstractViolationPolicyEnforcement.java
new file mode 100644
index 0000000..2d34d45
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/AbstractViolationPolicyEnforcement.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+
+/**
+ * Abstract implementation for {@link SpaceViolationPolicyEnforcement}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class AbstractViolationPolicyEnforcement
+    implements SpaceViolationPolicyEnforcement {
+
+  RegionServerServices rss;
+  TableName tableName;
+  SpaceQuotaSnapshot quotaSnapshot;
+
+  public void setRegionServerServices(RegionServerServices rss) {
+    this.rss = Objects.requireNonNull(rss);
+  }
+
+  public void setTableName(TableName tableName) {
+    this.tableName = tableName;
+  }
+
+  public RegionServerServices getRegionServerServices() {
+    return this.rss;
+  }
+
+  public TableName getTableName() {
+    return this.tableName;
+  }
+
+  public void setQuotaSnapshot(SpaceQuotaSnapshot snapshot) {
+    this.quotaSnapshot = Objects.requireNonNull(snapshot);
+  }
+
+  @Override
+  public SpaceQuotaSnapshot getQuotaSnapshot() {
+    return this.quotaSnapshot;
+  }
+
+  @Override
+  public void initialize(RegionServerServices rss, TableName tableName, SpaceQuotaSnapshot snapshot) {
+    setRegionServerServices(rss);
+    setTableName(tableName);
+    setQuotaSnapshot(snapshot);
+  }
+
+  @Override
+  public boolean areCompactionsDisabled() {
+    return false;
+  }
+
+  @Override
+  public boolean shouldCheckBulkLoads() {
+    // Reference check. The singleton is used when no quota exists to check against
+    return SpaceQuotaSnapshot.getNoSuchSnapshot() != quotaSnapshot;
+  }
+
+  @Override
+  public void checkBulkLoad(FileSystem fs, List<String> paths) throws SpaceLimitingException {
+    // Compute the amount of space that could be used to save some arithmetic in the for-loop
+    final long sizeAvailableForBulkLoads = quotaSnapshot.getLimit() - quotaSnapshot.getUsage();
+    long size = 0L;
+    for (String path : paths) {
+      size += addSingleFile(fs, path);
+      if (size > sizeAvailableForBulkLoads) {
+        break;
+      }
+    }
+    if (size > sizeAvailableForBulkLoads) {
+      throw new SpaceLimitingException(getPolicyName(), "Bulk load of " + paths
+          + " is disallowed because the file(s) exceed the limits of a space quota.");
+    }
+  }
+
+  private long addSingleFile(FileSystem fs, String path) throws SpaceLimitingException {
+    final FileStatus status;
+    try {
+      status = fs.getFileStatus(new Path(Objects.requireNonNull(path)));
+    } catch (IOException e) {
+      throw new SpaceLimitingException(getPolicyName(), "Could not verify length of file to bulk load", e);
+    }
+    if (!status.isFile()) {
+      throw new IllegalArgumentException(path + " is not a file.");
+    }
+    return status.getLen();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/BulkLoadVerifyingViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/BulkLoadVerifyingViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/BulkLoadVerifyingViolationPolicyEnforcement.java
new file mode 100644
index 0000000..e4171ad
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/BulkLoadVerifyingViolationPolicyEnforcement.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+
+/**
+ * A {@link SpaceViolationPolicyEnforcement} instance which only checks for bulk loads. Useful for tables
+ * which have no violation policy. This is the default case for tables, as we want to make sure that
+ * a single bulk load call would violate the quota.
+ */
+@InterfaceAudience.Private
+public class BulkLoadVerifyingViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+
+  @Override
+  public void enable() {}
+
+  @Override
+  public void disable() {}
+
+  @Override
+  public String getPolicyName() {
+    return "BulkLoadVerifying";
+  }
+
+  @Override
+  public boolean areCompactionsDisabled() {
+    return false;
+  }
+
+  @Override
+  public void check(Mutation m) throws SpaceLimitingException {}
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java
new file mode 100644
index 0000000..0d6d886
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.TableNotEnabledException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+
+/**
+ * A {@link SpaceViolationPolicyEnforcement} which disables the table. The enforcement
+ * counterpart to {@link SpaceViolationPolicy#DISABLE}.
+ */
+@InterfaceAudience.Private
+public class DisableTableViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+  private static final Log LOG = LogFactory.getLog(DisableTableViolationPolicyEnforcement.class);
+
+  @Override
+  public void enable() throws IOException {
+    try {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Starting disable of " + getTableName());
+      }
+      getRegionServerServices().getClusterConnection().getAdmin().disableTable(getTableName());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Disable is complete for " + getTableName());
+      }
+    } catch (TableNotEnabledException tnee) {
+      // The state we wanted it to be in.
+    }
+  }
+
+  @Override
+  public void disable() throws IOException {
+    try {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Starting enable of " + getTableName());
+      }
+      getRegionServerServices().getClusterConnection().getAdmin().enableTable(getTableName());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Enable is complete for " + getTableName());
+      }
+    } catch (TableNotDisabledException tnde) {
+      // The state we wanted it to be in
+    }
+  }
+
+  @Override
+  public void check(Mutation m) throws SpaceLimitingException {
+    // If this policy is enacted, then the table is (or should be) disabled.
+    throw new SpaceLimitingException(
+        getPolicyName(), "This table is disabled due to violating a space quota.");
+  }
+
+  @Override
+  public String getPolicyName() {
+    return SpaceViolationPolicy.DISABLE.name();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoInsertsViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoInsertsViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoInsertsViolationPolicyEnforcement.java
new file mode 100644
index 0000000..a60cb45
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoInsertsViolationPolicyEnforcement.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+
+/**
+ * A {@link SpaceViolationPolicyEnforcement} which disallows any inserts to the table. The
+ * enforcement counterpart to {@link SpaceViolationPolicy#NO_INSERTS}.
+ */
+@InterfaceAudience.Private
+public class NoInsertsViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+
+  @Override
+  public void enable() {}
+
+  @Override
+  public void disable() {}
+
+  @Override
+  public void check(Mutation m) throws SpaceLimitingException {
+    // Disallow all "new" data flowing into HBase, but allow Deletes (even though we know they will
+    // temporarily increase utilization).
+    if (m instanceof Append  || m instanceof Increment || m instanceof Put) {
+      throw new SpaceLimitingException(getPolicyName(),
+          m.getClass().getSimpleName() + "s are disallowed due to a space quota.");
+    }
+  }
+
+  @Override
+  public String getPolicyName() {
+    return SpaceViolationPolicy.NO_INSERTS.name();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesCompactionsViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesCompactionsViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesCompactionsViolationPolicyEnforcement.java
new file mode 100644
index 0000000..e7f872c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesCompactionsViolationPolicyEnforcement.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+
+/**
+ * A {@link SpaceViolationPolicyEnforcement} implementation which disables all updates and
+ * compactions. The enforcement counterpart to {@link SpaceViolationPolicy#NO_WRITES_COMPACTIONS}.
+ */
+@InterfaceAudience.Private
+public class NoWritesCompactionsViolationPolicyEnforcement
+    extends NoWritesViolationPolicyEnforcement {
+  private static final Log LOG = LogFactory.getLog(
+      NoWritesCompactionsViolationPolicyEnforcement.class);
+
+  private AtomicBoolean disableCompactions = new AtomicBoolean(false);
+
+  @Override
+  public synchronized void enable() {
+    boolean ret = disableCompactions.compareAndSet(false, true);
+    if (!ret && LOG.isTraceEnabled()) {
+      LOG.trace("Compactions were already disabled upon enabling the policy");
+    }
+  }
+
+  @Override
+  public synchronized void disable() {
+    boolean ret = disableCompactions.compareAndSet(true, false);
+    if (!ret && LOG.isTraceEnabled()) {
+      LOG.trace("Compactions were already enabled upon disabling the policy");
+    }
+  }
+
+  @Override
+  public String getPolicyName() {
+    return SpaceViolationPolicy.NO_WRITES_COMPACTIONS.name();
+  }
+
+  @Override
+  public boolean areCompactionsDisabled() {
+    return disableCompactions.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesViolationPolicyEnforcement.java
new file mode 100644
index 0000000..a04f418
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/NoWritesViolationPolicyEnforcement.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+
+/**
+ * A {@link SpaceViolationPolicyEnforcement} implementation which disables all writes flowing
+ * into HBase. The enforcement counterpart to {@link SpaceViolationPolicy#NO_WRITES}.
+ */
+@InterfaceAudience.Private
+public class NoWritesViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+
+  @Override
+  public void enable() {}
+
+  @Override
+  public void disable() {}
+
+  @Override
+  public void check(Mutation m) throws SpaceLimitingException {
+    if (m instanceof Append || m instanceof Delete || m instanceof Increment || m instanceof Put) {
+      throw new SpaceLimitingException(getPolicyName(),
+          m.getClass().getSimpleName() + "s are disallowed due to a space quota.");
+    }
+  }
+
+  @Override
+  public String getPolicyName() {
+    return SpaceViolationPolicy.NO_WRITES.name();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
index eba984a..9aa0042 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
+import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
@@ -337,6 +338,17 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
       if (compaction == null) return null; // message logged inside
     }
 
+    final RegionServerSpaceQuotaManager spaceQuotaManager =
+      this.server.getRegionServerSpaceQuotaManager();
+    if (null != spaceQuotaManager && spaceQuotaManager.areCompactionsDisabled(
+        r.getTableDesc().getTableName())) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Ignoring compaction request for " + r + " as an active space quota violation "
+            + " policy disallows compactions.");
+      }
+      return null;
+    }
+
     // We assume that most compactions are small. So, put system compactions into small
     // pool; we will do selection there, and move to large pool if necessary.
     ThreadPoolExecutor pool = (selectNow && s.throttleCompaction(compaction.getRequest().getSize()))

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index fd9ca92..81b87d5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -96,8 +96,12 @@ import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
+import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement;
 import org.apache.hadoop.hbase.quotas.OperationQuota;
+import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.regionserver.Leases.Lease;
 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
@@ -195,7 +199,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescr
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
-import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.DNS;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -561,8 +564,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    */
   private boolean checkAndRowMutate(final Region region, final List<ClientProtos.Action> actions,
       final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, ByteArrayComparable comparator,
-                                    RegionActionResult.Builder builder) throws IOException {
+      CompareOp compareOp, ByteArrayComparable comparator, RegionActionResult.Builder builder,
+      ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
     if (!region.getRegionInfo().isMetaTable()) {
       regionServer.cacheFlusher.reclaimMemStoreMemory();
     }
@@ -581,10 +584,14 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       }
       switch (type) {
         case PUT:
-          rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner));
+          Put put = ProtobufUtil.toPut(action.getMutation(), cellScanner);
+          spaceQuotaEnforcement.getPolicyEnforcement(region).check(put);
+          rm.add(put);
           break;
         case DELETE:
-          rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
+          Delete del = ProtobufUtil.toDelete(action.getMutation(), cellScanner);
+          spaceQuotaEnforcement.getPolicyEnforcement(region).check(del);
+          rm.add(del);
           break;
         default:
           throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name());
@@ -611,10 +618,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * @throws IOException
    */
   private Result append(final Region region, final OperationQuota quota,
-      final MutationProto mutation, final CellScanner cellScanner, long nonceGroup)
+      final MutationProto mutation, final CellScanner cellScanner, long nonceGroup,
+      ActivePolicyEnforcement spaceQuota)
       throws IOException {
     long before = EnvironmentEdgeManager.currentTime();
     Append append = ProtobufUtil.toAppend(mutation, cellScanner);
+    spaceQuota.getPolicyEnforcement(region).check(append);
     quota.addMutation(append);
     Result r = null;
     if (region.getCoprocessorHost() != null) {
@@ -659,10 +668,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * @throws IOException
    */
   private Result increment(final Region region, final OperationQuota quota,
-      final MutationProto mutation, final CellScanner cells, long nonceGroup)
+      final MutationProto mutation, final CellScanner cells, long nonceGroup,
+      ActivePolicyEnforcement spaceQuota)
       throws IOException {
     long before = EnvironmentEdgeManager.currentTime();
     Increment increment = ProtobufUtil.toIncrement(mutation, cells);
+    spaceQuota.getPolicyEnforcement(region).check(increment);
     quota.addMutation(increment);
     Result r = null;
     if (region.getCoprocessorHost() != null) {
@@ -714,7 +725,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   private List<CellScannable> doNonAtomicRegionMutation(final Region region,
       final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner,
       final RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup,
-      final RegionScannersCloseCallBack closeCallBack, RpcCallContext context) {
+      final RegionScannersCloseCallBack closeCallBack, RpcCallContext context,
+      ActivePolicyEnforcement spaceQuotaEnforcement) {
     // Gather up CONTIGUOUS Puts and Deletes in this mutations List.  Idea is that rather than do
     // one at a time, we instead pass them in batch.  Be aware that the corresponding
     // ResultOrException instance that matches each Put or Delete is then added down in the
@@ -807,15 +819,17 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           if (type != MutationType.PUT && type != MutationType.DELETE && mutations != null &&
               !mutations.isEmpty()) {
             // Flush out any Puts or Deletes already collected.
-            doBatchOp(builder, region, quota, mutations, cellScanner);
+            doBatchOp(builder, region, quota, mutations, cellScanner, spaceQuotaEnforcement);
             mutations.clear();
           }
           switch (type) {
             case APPEND:
-              r = append(region, quota, action.getMutation(), cellScanner, nonceGroup);
+              r = append(region, quota, action.getMutation(), cellScanner, nonceGroup,
+                  spaceQuotaEnforcement);
               break;
             case INCREMENT:
-              r = increment(region, quota, action.getMutation(), cellScanner, nonceGroup);
+              r = increment(region, quota, action.getMutation(), cellScanner, nonceGroup,
+                  spaceQuotaEnforcement);
               break;
             case PUT:
             case DELETE:
@@ -866,7 +880,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     }
     // Finish up any outstanding mutations
     if (mutations != null && !mutations.isEmpty()) {
-      doBatchOp(builder, region, quota, mutations, cellScanner);
+      doBatchOp(builder, region, quota, mutations, cellScanner, spaceQuotaEnforcement);
     }
     return cellsToReturn;
   }
@@ -880,7 +894,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    */
   private void doBatchOp(final RegionActionResult.Builder builder, final Region region,
       final OperationQuota quota, final List<ClientProtos.Action> mutations,
-      final CellScanner cells) {
+      final CellScanner cells, ActivePolicyEnforcement spaceQuotaEnforcement) {
     Mutation[] mArray = new Mutation[mutations.size()];
     long before = EnvironmentEdgeManager.currentTime();
     boolean batchContainsPuts = false, batchContainsDelete = false;
@@ -897,6 +911,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           batchContainsDelete = true;
         }
         mArray[i++] = mutation;
+        // Check if a space quota disallows this mutation
+        spaceQuotaEnforcement.getPolicyEnforcement(region).check(mutation);
         quota.addMutation(mutation);
       }
 
@@ -1267,10 +1283,14 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return regionServer.getConfiguration();
   }
 
-  private RegionServerRpcQuotaManager getQuotaManager() {
+  private RegionServerRpcQuotaManager getRpcQuotaManager() {
     return regionServer.getRegionServerRpcQuotaManager();
   }
 
+  private RegionServerSpaceQuotaManager getSpaceQuotaManager() {
+    return regionServer.getRegionServerSpaceQuotaManager();
+  }
+
   void start() {
     rpcServer.start();
   }
@@ -1446,6 +1466,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       checkOpen();
       requestCount.increment();
       Region region = getRegion(request.getRegion());
+      if (QuotaUtil.isQuotaEnabled(getConfiguration()) &&
+          this.regionServer.getRegionServerSpaceQuotaManager().areCompactionsDisabled(
+              region.getTableDesc().getTableName())) {
+        throw new DoNotRetryIOException("Compactions on this region are "
+            + "disabled due to a space quota violation.");
+      }
       region.startRegionOperation(Operation.COMPACT_REGION);
       LOG.info("Compacting " + region.getRegionInfo().getRegionNameAsString());
       boolean major = false;
@@ -2132,6 +2158,21 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       boolean loaded = false;
       Map<byte[], List<Path>> map = null;
 
+      // Check to see if this bulk load would exceed the space quota for this table
+      if (QuotaUtil.isQuotaEnabled(getConfiguration())) {
+        ActivePolicyEnforcement activeSpaceQuotas = getSpaceQuotaManager().getActiveEnforcements();
+        SpaceViolationPolicyEnforcement enforcement = activeSpaceQuotas.getPolicyEnforcement(region);
+        if (null != enforcement) {
+          // Bulk loads must still be atomic. We must enact all or none.
+          List<String> filePaths = new ArrayList<>(request.getFamilyPathCount());
+          for (FamilyPath familyPath : request.getFamilyPathList()) {
+            filePaths.add(familyPath.getPath());
+          }
+          // Check if the batch of files exceeds the current quota
+          enforcement.checkBulkLoad(regionServer.getFileSystem(), filePaths);
+        }
+      }
+
       if (!request.hasBulkToken()) {
         // Old style bulk load. This will not be supported in future releases
         List<Pair<byte[], String>> familyPaths = new ArrayList<>(request.getFamilyPathCount());
@@ -2260,7 +2301,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       Boolean existence = null;
       Result r = null;
       RpcCallContext context = RpcServer.getCurrentCall();
-      quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.GET);
+      quota = getRpcQuotaManager().checkQuota(region, OperationQuota.OperationType.GET);
 
       Get clientGet = ProtobufUtil.toGet(get);
       if (get.getExistenceOnly() && region.getCoprocessorHost() != null) {
@@ -2398,6 +2439,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     this.rpcMultiRequestCount.increment();
     Map<RegionSpecifier, ClientProtos.RegionLoadStats> regionStats = new HashMap<>(request
       .getRegionActionCount());
+    ActivePolicyEnforcement spaceQuotaEnforcement = getSpaceQuotaManager().getActiveEnforcements();
     for (RegionAction regionAction : request.getRegionActionList()) {
       this.requestCount.add(regionAction.getActionCount());
       OperationQuota quota;
@@ -2406,7 +2448,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       RegionSpecifier regionSpecifier = regionAction.getRegion();
       try {
         region = getRegion(regionSpecifier);
-        quota = getQuotaManager().checkQuota(region, regionAction.getActionList());
+        quota = getRpcQuotaManager().checkQuota(region, regionAction.getActionList());
       } catch (IOException e) {
         rpcServer.getMetrics().exception(e);
         regionActionResultBuilder.setException(ResponseConverter.buildException(e));
@@ -2434,7 +2476,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                 ProtobufUtil.toComparator(condition.getComparator());
             processed = checkAndRowMutate(region, regionAction.getActionList(),
                   cellScanner, row, family, qualifier, compareOp,
-                  comparator, regionActionResultBuilder);
+                  comparator, regionActionResultBuilder, spaceQuotaEnforcement);
           } else {
             mutateRows(region, regionAction.getActionList(), cellScanner,
                 regionActionResultBuilder);
@@ -2455,7 +2497,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           context.setCallBack(closeCallBack);
         }
         cellsToReturn = doNonAtomicRegionMutation(region, quota, regionAction, cellScanner,
-            regionActionResultBuilder, cellsToReturn, nonceGroup, closeCallBack, context);
+            regionActionResultBuilder, cellsToReturn, nonceGroup, closeCallBack, context,
+            spaceQuotaEnforcement);
       }
       responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
       quota.close();
@@ -2522,6 +2565,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     CellScanner cellScanner = controller != null ? controller.cellScanner() : null;
     OperationQuota quota = null;
     RpcCallContext context = RpcServer.getCurrentCall();
+    ActivePolicyEnforcement spaceQuotaEnforcement = null;
     // Clear scanner so we are not holding on to reference across call.
     if (controller != null) {
       controller.setCellScanner(null);
@@ -2541,19 +2585,22 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       Boolean processed = null;
       MutationType type = mutation.getMutateType();
 
-      quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.MUTATE);
+      quota = getRpcQuotaManager().checkQuota(region, OperationQuota.OperationType.MUTATE);
+      spaceQuotaEnforcement = getSpaceQuotaManager().getActiveEnforcements();
 
       switch (type) {
       case APPEND:
         // TODO: this doesn't actually check anything.
-        r = append(region, quota, mutation, cellScanner, nonceGroup);
+        r = append(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement);
         break;
       case INCREMENT:
         // TODO: this doesn't actually check anything.
-        r = increment(region, quota, mutation, cellScanner, nonceGroup);
+        r = increment(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement);
         break;
       case PUT:
         Put put = ProtobufUtil.toPut(mutation, cellScanner);
+        // Throws an exception when violated
+        spaceQuotaEnforcement.getPolicyEnforcement(region).check(put);
         quota.addMutation(put);
         if (request.hasCondition()) {
           Condition condition = request.getCondition();
@@ -2583,6 +2630,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         break;
       case DELETE:
         Delete delete = ProtobufUtil.toDelete(mutation, cellScanner);
+        spaceQuotaEnforcement.getPolicyEnforcement(region).check(delete);
         quota.addMutation(delete);
         if (request.hasCondition()) {
           Condition condition = request.getCondition();
@@ -3042,7 +3090,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     }
     OperationQuota quota;
     try {
-      quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.SCAN);
+      quota = getRpcQuotaManager().checkQuota(region, OperationQuota.OperationType.SCAN);
     } catch (IOException e) {
       addScannerLeaseBack(lease);
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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
new file mode 100644
index 0000000..888978d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.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.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+
+@InterfaceAudience.Private
+public class SpaceQuotaHelperForTests {
+  private static final Log LOG = LogFactory.getLog(SpaceQuotaHelperForTests.class);
+
+  public static final int SIZE_PER_VALUE = 256;
+  public static final String F1 = "f1";
+  public static final long ONE_KILOBYTE = 1024L;
+  public static final long ONE_MEGABYTE = ONE_KILOBYTE * ONE_KILOBYTE;
+
+  private final HBaseTestingUtility testUtil;
+  private final TestName testName;
+  private final AtomicLong counter;
+
+  public SpaceQuotaHelperForTests(
+      HBaseTestingUtility testUtil, TestName testName, AtomicLong counter) {
+    this.testUtil = Objects.requireNonNull(testUtil);
+    this.testName = Objects.requireNonNull(testName);
+    this.counter = Objects.requireNonNull(counter);
+  }
+
+  //
+  // Helpers
+  //
+
+  void writeData(TableName tn, long sizeInBytes) throws IOException {
+    final Connection conn = testUtil.getConnection();
+    final Table table = conn.getTable(tn);
+    try {
+      List<Put> updates = new ArrayList<>();
+      long bytesToWrite = sizeInBytes;
+      long rowKeyId = 0L;
+      final StringBuilder sb = new StringBuilder();
+      final Random r = new Random();
+      while (bytesToWrite > 0L) {
+        sb.setLength(0);
+        sb.append(Long.toString(rowKeyId));
+        // Use the reverse counter as the rowKey to get even spread across all regions
+        Put p = new Put(Bytes.toBytes(sb.reverse().toString()));
+        byte[] value = new byte[SIZE_PER_VALUE];
+        r.nextBytes(value);
+        p.addColumn(Bytes.toBytes(F1), Bytes.toBytes("q1"), value);
+        updates.add(p);
+
+        // Batch ~13KB worth of updates
+        if (updates.size() > 50) {
+          table.put(updates);
+          updates.clear();
+        }
+
+        // Just count the value size, ignore the size of rowkey + column
+        bytesToWrite -= SIZE_PER_VALUE;
+        rowKeyId++;
+      }
+
+      // Write the final batch
+      if (!updates.isEmpty()) {
+        table.put(updates);
+      }
+
+      LOG.debug("Data was written to HBase");
+      // Push the data to disk.
+      testUtil.getAdmin().flush(tn);
+      LOG.debug("Data flushed to disk");
+    } finally {
+      table.close();
+    }
+  }
+
+  Multimap<TableName, QuotaSettings> createTablesWithSpaceQuotas() throws Exception {
+    final Admin admin = testUtil.getAdmin();
+    final Multimap<TableName, QuotaSettings> tablesWithQuotas = HashMultimap.create();
+
+    final TableName tn1 = createTable();
+    final TableName tn2 = createTable();
+
+    NamespaceDescriptor nd = NamespaceDescriptor.create("ns" + counter.getAndIncrement()).build();
+    admin.createNamespace(nd);
+    final TableName tn3 = createTableInNamespace(nd);
+    final TableName tn4 = createTableInNamespace(nd);
+    final TableName tn5 = createTableInNamespace(nd);
+
+    final long sizeLimit1 = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
+    final SpaceViolationPolicy violationPolicy1 = SpaceViolationPolicy.NO_WRITES;
+    QuotaSettings qs1 = QuotaSettingsFactory.limitTableSpace(tn1, sizeLimit1, violationPolicy1);
+    tablesWithQuotas.put(tn1, qs1);
+    admin.setQuota(qs1);
+
+    final long sizeLimit2 = 1024L * 1024L * 1024L * 200L; // 200GB
+    final SpaceViolationPolicy violationPolicy2 = SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
+    QuotaSettings qs2 = QuotaSettingsFactory.limitTableSpace(tn2, sizeLimit2, violationPolicy2);
+    tablesWithQuotas.put(tn2, qs2);
+    admin.setQuota(qs2);
+
+    final long sizeLimit3 = 1024L * 1024L * 1024L * 1024L * 100L; // 100TB
+    final SpaceViolationPolicy violationPolicy3 = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings qs3 = QuotaSettingsFactory.limitNamespaceSpace(
+        nd.getName(), sizeLimit3, violationPolicy3);
+    tablesWithQuotas.put(tn3, qs3);
+    tablesWithQuotas.put(tn4, qs3);
+    tablesWithQuotas.put(tn5, qs3);
+    admin.setQuota(qs3);
+
+    final long sizeLimit4 = 1024L * 1024L * 1024L * 5L; // 5GB
+    final SpaceViolationPolicy violationPolicy4 = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings qs4 = QuotaSettingsFactory.limitTableSpace(tn5, sizeLimit4, violationPolicy4);
+    // Override the ns quota for tn5, import edge-case to catch table quota taking
+    // precedence over ns quota.
+    tablesWithQuotas.put(tn5, qs4);
+    admin.setQuota(qs4);
+
+    return tablesWithQuotas;
+  }
+
+  TableName createTable() throws Exception {
+    return createTableWithRegions(1);
+  }
+
+  TableName createTableWithRegions(int numRegions) throws Exception {
+    return createTableWithRegions(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR, numRegions);
+  }
+
+  TableName createTableWithRegions(String namespace, int numRegions) throws Exception {
+    final Admin admin = testUtil.getAdmin();
+    final TableName tn = TableName.valueOf(
+        namespace, testName.getMethodName() + counter.getAndIncrement());
+
+    // Delete the old table
+    if (admin.tableExists(tn)) {
+      admin.disableTable(tn);
+      admin.deleteTable(tn);
+    }
+
+    // Create the table
+    HTableDescriptor tableDesc = new HTableDescriptor(tn);
+    tableDesc.addFamily(new HColumnDescriptor(F1));
+    if (numRegions == 1) {
+      admin.createTable(tableDesc);
+    } else {
+      admin.createTable(tableDesc, Bytes.toBytes("0"), Bytes.toBytes("9"), numRegions);
+    }
+    return tn;
+  }
+
+  TableName createTableInNamespace(NamespaceDescriptor nd) throws Exception {
+    final Admin admin = testUtil.getAdmin();
+    final TableName tn = TableName.valueOf(nd.getName(),
+        testName.getMethodName() + counter.getAndIncrement());
+
+    // Delete the old table
+    if (admin.tableExists(tn)) {
+      admin.disableTable(tn);
+      admin.deleteTable(tn);
+    }
+
+    // Create the table
+    HTableDescriptor tableDesc = new HTableDescriptor(tn);
+    tableDesc.addFamily(new HColumnDescriptor(F1));
+
+    admin.createTable(tableDesc);
+    return tn;
+  }
+
+  void partitionTablesByQuotaTarget(Multimap<TableName,QuotaSettings> quotas,
+      Set<TableName> tablesWithTableQuota, Set<TableName> tablesWithNamespaceQuota) {
+    // Partition the tables with quotas by table and ns quota
+    for (Entry<TableName, QuotaSettings> entry : quotas.entries()) {
+      SpaceLimitSettings settings = (SpaceLimitSettings) entry.getValue();
+      TableName tn = entry.getKey();
+      if (null != settings.getTableName()) {
+        tablesWithTableQuota.add(tn);
+      }
+      if (null != settings.getNamespace()) {
+        tablesWithNamespaceQuota.add(tn);
+      }
+
+      if (null == settings.getTableName() && null == settings.getNamespace()) {
+        fail("Unexpected table name with null tableName and namespace: " + tn);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifierForTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifierForTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifierForTest.java
new file mode 100644
index 0000000..0986e8c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifierForTest.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+
+/**
+ * A SpaceQuotaSnapshotNotifier implementation for testing.
+ */
+@InterfaceAudience.Private
+public class SpaceQuotaSnapshotNotifierForTest implements SpaceQuotaSnapshotNotifier {
+  private static final Log LOG = LogFactory.getLog(SpaceQuotaSnapshotNotifierForTest.class);
+
+  private final Map<TableName,SpaceQuotaSnapshot> tableQuotaSnapshots = new HashMap<>();
+
+  @Override
+  public void initialize(Connection conn) {}
+
+  @Override
+  public synchronized void transitionTable(TableName tableName, SpaceQuotaSnapshot snapshot) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Persisting " + tableName + "=>" + snapshot);
+    }
+    tableQuotaSnapshots.put(tableName, snapshot);
+  }
+
+  public synchronized Map<TableName,SpaceQuotaSnapshot> copySnapshots() {
+    return new HashMap<>(this.tableQuotaSnapshots);
+  }
+
+  public synchronized void clearSnapshots() {
+    this.tableQuotaSnapshots.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java
new file mode 100644
index 0000000..80363e8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java
@@ -0,0 +1,74 @@
+/*
+ * 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.assertTrue;
+import static org.mockito.Mockito.mock;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.quotas.policies.BulkLoadVerifyingViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link ActivePolicyEnforcement}.
+ */
+@Category(SmallTests.class)
+public class TestActivePolicyEnforcement {
+
+  @Test
+  public void testGetter() {
+    final TableName tableName = TableName.valueOf("table");
+    Map<TableName, SpaceViolationPolicyEnforcement> map = new HashMap<>();
+    map.put(tableName, new NoWritesViolationPolicyEnforcement());
+    ActivePolicyEnforcement ape = new ActivePolicyEnforcement(map, Collections.emptyMap(), null);
+    assertEquals(map.get(tableName), ape.getPolicyEnforcement(tableName));
+  }
+
+  @Test
+  public void testNoPolicyReturnsNoopEnforcement() {
+    ActivePolicyEnforcement ape = new ActivePolicyEnforcement(
+        new HashMap<>(), Collections.emptyMap(), mock(RegionServerServices.class));
+    SpaceViolationPolicyEnforcement enforcement = ape.getPolicyEnforcement(
+        TableName.valueOf("nonexistent"));
+    assertNotNull(enforcement);
+    assertTrue(
+        "Expected an instance of NoopViolationPolicyEnforcement",
+        enforcement instanceof BulkLoadVerifyingViolationPolicyEnforcement);
+  }
+
+  @Test
+  public void testNoBulkLoadChecksOnNoSnapshot() {
+    ActivePolicyEnforcement ape = new ActivePolicyEnforcement(
+        new HashMap<TableName, SpaceViolationPolicyEnforcement>(),
+        Collections.<TableName,SpaceQuotaSnapshot> emptyMap(),
+        mock(RegionServerServices.class));
+    SpaceViolationPolicyEnforcement enforcement = ape.getPolicyEnforcement(
+        TableName.valueOf("nonexistent"));
+    assertFalse("Should not check bulkloads", enforcement.shouldCheckBulkLoads());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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 ad98720..18e47af 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
@@ -144,6 +144,7 @@ public class TestFileSystemUtilizationChore {
     assertEquals(timeUnit, chore.getTimeUnit());
   }
 
+  @SuppressWarnings("unchecked")
   @Test
   public void testProcessingLeftoverRegions() {
     final Configuration conf = getDefaultHBaseConfiguration();
@@ -176,6 +177,7 @@ public class TestFileSystemUtilizationChore {
     chore.chore();
   }
 
+  @SuppressWarnings("unchecked")
   @Test
   public void testProcessingNowOfflineLeftoversAreIgnored() {
     final Configuration conf = getDefaultHBaseConfiguration();
@@ -185,7 +187,6 @@ public class TestFileSystemUtilizationChore {
     final List<Long> leftover1Sizes = Arrays.asList(1024L, 4096L);
     final long leftover1Sum = sum(leftover1Sizes);
     final List<Long> leftover2Sizes = Arrays.asList(2048L);
-    final long leftover2Sum = sum(leftover2Sizes);
 
     final Region lr1 = mockRegionWithSize(leftover1Sizes);
     final Region lr2 = mockRegionWithSize(leftover2Sizes);

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/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 8182513..4a7258f 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
@@ -31,7 +31,6 @@ 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.quotas.QuotaViolationStore.ViolationState;
 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;
@@ -45,7 +44,7 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 /**
- * Test class for {@link NamespaceQuotaViolationStore}.
+ * Test class for {@link NamespaceQuotaSnapshotStore}.
  */
 @Category(SmallTests.class)
 public class TestNamespaceQuotaViolationStore {
@@ -54,19 +53,19 @@ public class TestNamespaceQuotaViolationStore {
   private Connection conn;
   private QuotaObserverChore chore;
   private Map<HRegionInfo, Long> regionReports;
-  private NamespaceQuotaViolationStore store;
+  private NamespaceQuotaSnapshotStore store;
 
   @Before
   public void setup() {
     conn = mock(Connection.class);
     chore = mock(QuotaObserverChore.class);
     regionReports = new HashMap<>();
-    store = new NamespaceQuotaViolationStore(conn, chore, regionReports);
+    store = new NamespaceQuotaSnapshotStore(conn, chore, regionReports);
   }
 
   @Test
   public void testGetSpaceQuota() throws Exception {
-    NamespaceQuotaViolationStore mockStore = mock(NamespaceQuotaViolationStore.class);
+    NamespaceQuotaSnapshotStore mockStore = mock(NamespaceQuotaSnapshotStore.class);
     when(mockStore.getSpaceQuota(any(String.class))).thenCallRealMethod();
 
     Quotas quotaWithSpace = Quotas.newBuilder().setSpace(
@@ -113,17 +112,18 @@ public class TestNamespaceQuotaViolationStore {
     regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(1), Bytes.toBytes(2)), 1024L * 256L);
 
     // Below the quota
-    assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(NS, quota));
+    assertEquals(false, store.getTargetState(NS, quota).getQuotaStatus().isInViolation());
 
     regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(2), Bytes.toBytes(3)), 1024L * 256L);
 
     // Equal to the quota is still in observance
-    assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(NS, quota));
+    assertEquals(false, store.getTargetState(NS, quota).getQuotaStatus().isInViolation());
 
     regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(3), Bytes.toBytes(4)), 1024L);
 
     // Exceeds the quota, should be in violation
-    assertEquals(ViolationState.IN_VIOLATION, store.getTargetState(NS, quota));
+    assertEquals(true, store.getTargetState(NS, quota).getQuotaStatus().isInViolation());
+    assertEquals(SpaceViolationPolicy.DISABLE, store.getTargetState(NS, quota).getQuotaStatus().getPolicy());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d76d667/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java
index db549e4..da294c6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java
@@ -17,8 +17,6 @@
 package org.apache.hadoop.hbase.quotas;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -28,8 +26,6 @@ import java.util.Map;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Before;
@@ -50,8 +46,6 @@ public class TestQuotaObserverChore {
   public void setup() throws Exception {
     conn = mock(Connection.class);
     chore = mock(QuotaObserverChore.class);
-    // Set up some rules to call the real method on the mock.
-    when(chore.getViolationPolicy(any(SpaceQuota.class))).thenCallRealMethod();
   }
 
   @Test
@@ -76,31 +70,11 @@ public class TestQuotaObserverChore {
       regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i + 1)), 0L);
     }
 
-    TableQuotaViolationStore store = new TableQuotaViolationStore(conn, chore, regionReports);
-    when(chore.getTableViolationStore()).thenReturn(store);
+    TableQuotaSnapshotStore store = new TableQuotaSnapshotStore(conn, chore, regionReports);
+    when(chore.getTableSnapshotStore()).thenReturn(store);
 
     assertEquals(numTable1Regions, Iterables.size(store.filterBySubject(tn1)));
     assertEquals(numTable2Regions, Iterables.size(store.filterBySubject(tn2)));
     assertEquals(numTable3Regions, Iterables.size(store.filterBySubject(tn3)));
   }
-
-  @Test
-  public void testExtractViolationPolicy() {
-    for (SpaceViolationPolicy policy : SpaceViolationPolicy.values()) {
-      SpaceQuota spaceQuota = SpaceQuota.newBuilder()
-          .setSoftLimit(1024L)
-          .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(policy))
-          .build();
-      assertEquals(policy, chore.getViolationPolicy(spaceQuota));
-    }
-    SpaceQuota malformedQuota = SpaceQuota.newBuilder()
-        .setSoftLimit(1024L)
-        .build();
-    try {
-      chore.getViolationPolicy(malformedQuota);
-      fail("Should have thrown an IllegalArgumentException.");
-    } catch (IllegalArgumentException e) {
-      // Pass
-    }
-  }
 }
\ No newline at end of file


[13/50] [abbrv] hbase git commit: HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)

Posted by el...@apache.org.
HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)


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

Branch: refs/heads/HBASE-16961
Commit: c2c2178b2eebe4439eadec6b37fae2566944c16b
Parents: c8cd921
Author: Ramkrishna <ra...@intel.com>
Authored: Mon Apr 17 09:10:59 2017 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Mon Apr 17 09:28:24 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/CellUtil.java  |  24 --
 .../org/apache/hadoop/hbase/ExtendedCell.java   |  10 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   2 +
 .../hbase/regionserver/ByteBufferChunkCell.java |  48 +++
 .../apache/hadoop/hbase/regionserver/Chunk.java |  60 ++-
 .../hadoop/hbase/regionserver/ChunkCreator.java | 404 +++++++++++++++++++
 .../hbase/regionserver/HRegionServer.java       |  14 +-
 .../hbase/regionserver/MemStoreChunkPool.java   | 265 ------------
 .../hadoop/hbase/regionserver/MemStoreLAB.java  |   4 +-
 .../hbase/regionserver/MemStoreLABImpl.java     | 171 ++++----
 .../regionserver/NoTagByteBufferChunkCell.java  |  48 +++
 .../hadoop/hbase/regionserver/OffheapChunk.java |  31 +-
 .../hadoop/hbase/regionserver/OnheapChunk.java  |  32 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |   3 +
 .../coprocessor/TestCoprocessorInterface.java   |   4 +
 .../TestRegionObserverScannerOpenHook.java      |   3 +
 .../coprocessor/TestRegionObserverStacking.java |   3 +
 .../io/hfile/TestScannerFromBucketCache.java    |   3 +
 .../hadoop/hbase/master/TestCatalogJanitor.java |   7 +
 .../hadoop/hbase/regionserver/TestBulkLoad.java |   2 +-
 .../hbase/regionserver/TestCellFlatSet.java     |   2 +-
 .../regionserver/TestCompactingMemStore.java    |  37 +-
 .../TestCompactingToCellArrayMapMemStore.java   |  16 +-
 .../TestCompactionArchiveConcurrentClose.java   |   1 +
 .../TestCompactionArchiveIOException.java       |   1 +
 .../regionserver/TestCompactionPolicy.java      |   1 +
 .../hbase/regionserver/TestDefaultMemStore.java |  14 +-
 .../regionserver/TestFailedAppendAndSync.java   |   1 +
 .../hbase/regionserver/TestHMobStore.java       |   2 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |   2 +
 .../regionserver/TestHRegionReplayEvents.java   |   2 +-
 .../regionserver/TestMemStoreChunkPool.java     |  48 +--
 .../hbase/regionserver/TestMemStoreLAB.java     |  27 +-
 .../TestMemstoreLABWithoutPool.java             | 168 ++++++++
 .../hbase/regionserver/TestRecoveredEdits.java  |   1 +
 .../hbase/regionserver/TestRegionIncrement.java |   1 +
 .../hadoop/hbase/regionserver/TestStore.java    |   1 +
 .../TestStoreFileRefresherChore.java            |   1 +
 .../hbase/regionserver/TestWALLockup.java       |   1 +
 .../TestWALMonotonicallyIncreasingSeqId.java    |   1 +
 .../hbase/regionserver/wal/TestDurability.java  |   3 +
 41 files changed, 990 insertions(+), 479 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index e1bc969..56de21b 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -3135,28 +3135,4 @@ public final class CellUtil {
       return Type.DeleteFamily.getCode();
     }
   }
-
-  /**
-   * Clone the passed cell by copying its data into the passed buf.
-   */
-  public static Cell copyCellTo(Cell cell, ByteBuffer buf, int offset, int len) {
-    int tagsLen = cell.getTagsLength();
-    if (cell instanceof ExtendedCell) {
-      ((ExtendedCell) cell).write(buf, offset);
-    } else {
-      // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the
-      // other case also. The data fragments within Cell is copied into buf as in KeyValue
-      // serialization format only.
-      KeyValueUtil.appendTo(cell, buf, offset, true);
-    }
-    if (tagsLen == 0) {
-      // When tagsLen is 0, make a NoTagsByteBufferKeyValue version. This is an optimized class
-      // which directly return tagsLen as 0. So we avoid parsing many length components in
-      // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
-      // call getTagsLength().
-      return new NoTagsByteBufferKeyValue(buf, offset, len, cell.getSequenceId());
-    } else {
-      return new ByteBufferKeyValue(buf, offset, len, cell.getSequenceId());
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
index 517873f..10f20ca 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.io.HeapSize;
 public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestamp, HeapSize,
     Cloneable {
 
+  public static int CELL_NOT_BASED_ON_CHUNK = -1;
   /**
    * Write this cell to an OutputStream in a {@link KeyValue} format.
    * <br> KeyValue format <br>
@@ -73,4 +74,13 @@ public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestam
    * @return The deep cloned cell
    */
   Cell deepClone();
+
+  /**
+   * Extracts the id of the backing bytebuffer of this cell if it was obtained from fixed sized
+   * chunks as in case of MemstoreLAB
+   * @return the chunk id if the cell is backed by fixed sized Chunks, else return -1
+   */
+  default int getChunkId() {
+    return CELL_NOT_BASED_ON_CHUNK;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/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 bb9f282..f9670e1 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
@@ -748,6 +748,8 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     this.masterActiveTime = System.currentTimeMillis();
     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
+    // Initialize the chunkCreator
+    initializeMemStoreChunkCreator();
     this.fileSystemManager = new MasterFileSystem(this);
     this.walManager = new MasterWalManager(this);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
new file mode 100644
index 0000000..a8f1000
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
@@ -0,0 +1,48 @@
+/**
+ * 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.regionserver;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+/**
+ * ByteBuffer based cell which has the chunkid at the 0th offset
+ * @see MemStoreLAB
+ */
+//TODO : When moving this cell to CellChunkMap we will have the following things
+// to be serialized
+// chunkId (Integer) + offset (Integer) + length (Integer) + seqId (Long) = 20 bytes
+@InterfaceAudience.Private
+public class ByteBufferChunkCell extends ByteBufferKeyValue {
+  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
+    super(buf, offset, length);
+  }
+
+  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
+    super(buf, offset, length, seqId);
+  }
+
+  @Override
+  public int getChunkId() {
+    // The chunkId is embedded at the 0th offset of the bytebuffer
+    return ByteBufferUtils.toInt(buf, 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
index 2cbf0a3..fc4aa0b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
@@ -21,8 +21,10 @@ import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 /**
  * A chunk of memory out of which allocations are sliced.
@@ -46,13 +48,41 @@ public abstract class Chunk {
   /** Size of chunk in bytes */
   protected final int size;
 
+  // The unique id associated with the chunk.
+  private final int id;
+
+  // indicates if the chunk is formed by ChunkCreator#MemstorePool
+  private final boolean fromPool;
+
+  /**
+   * Create an uninitialized chunk. Note that memory is not allocated yet, so
+   * this is cheap.
+   * @param size in bytes
+   * @param id the chunk id
+   */
+  public Chunk(int size, int id) {
+    this(size, id, false);
+  }
+
   /**
-   * Create an uninitialized chunk. Note that memory is not allocated yet, so this is cheap.
-   *
+   * Create an uninitialized chunk. Note that memory is not allocated yet, so
+   * this is cheap.
    * @param size in bytes
+   * @param id the chunk id
+   * @param fromPool if the chunk is formed by pool
    */
-  Chunk(int size) {
+  public Chunk(int size, int id, boolean fromPool) {
     this.size = size;
+    this.id = id;
+    this.fromPool = fromPool;
+  }
+
+  int getId() {
+    return this.id;
+  }
+
+  boolean isFromPool() {
+    return this.fromPool;
   }
 
   /**
@@ -60,7 +90,24 @@ public abstract class Chunk {
    * constructed the chunk. It is thread-safe against other threads calling alloc(), who will block
    * until the allocation is complete.
    */
-  public abstract void init();
+  public void init() {
+    assert nextFreeOffset.get() == UNINITIALIZED;
+    try {
+      allocateDataBuffer();
+    } catch (OutOfMemoryError e) {
+      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
+      assert failInit; // should be true.
+      throw e;
+    }
+    // Mark that it's ready for use
+    // Move 8 bytes since the first 8 bytes are having the chunkid in it
+    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, Bytes.SIZEOF_LONG);
+    // We should always succeed the above CAS since only one thread
+    // calls init()!
+    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
+  }
+
+  abstract void allocateDataBuffer();
 
   /**
    * Reset the offset to UNINITIALIZED before before reusing an old chunk
@@ -74,7 +121,8 @@ public abstract class Chunk {
 
   /**
    * Try to allocate <code>size</code> bytes from the chunk.
-   *
+   * If a chunk is tried to get allocated before init() call, the thread doing the allocation
+   * will be in busy-wait state as it will keep looping till the nextFreeOffset is set.
    * @return the offset of the successful allocation, or -1 to indicate not-enough-space
    */
   public int alloc(int size) {
@@ -96,7 +144,7 @@ public abstract class Chunk {
       if (oldOffset + size > data.capacity()) {
         return -1; // alloc doesn't fit
       }
-
+      // TODO : If seqID is to be written add 8 bytes here for nextFreeOFfset
       // Try to atomically claim this chunk
       if (nextFreeOffset.compareAndSet(oldOffset, oldOffset + size)) {
         // we got the alloc

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
new file mode 100644
index 0000000..073fb25
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
@@ -0,0 +1,404 @@
+
+/**
+ * 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.regionserver;
+
+import java.lang.ref.SoftReference;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+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.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * Does the management of memstoreLAB chunk creations. A monotonically incrementing id is associated
+ * with every chunk
+ */
+@InterfaceAudience.Private
+public class ChunkCreator {
+  private final Log LOG = LogFactory.getLog(ChunkCreator.class);
+  // monotonically increasing chunkid
+  private AtomicInteger chunkID = new AtomicInteger(1);
+  // maps the chunk against the monotonically increasing chunk id. We need to preserve the
+  // natural ordering of the key
+  // CellChunkMap creation should convert the soft ref to hard reference
+  private Map<Integer, SoftReference<Chunk>> chunkIdMap =
+      new ConcurrentHashMap<Integer, SoftReference<Chunk>>();
+  private final int chunkSize;
+  private final boolean offheap;
+  @VisibleForTesting
+  static ChunkCreator INSTANCE;
+  @VisibleForTesting
+  static boolean chunkPoolDisabled = false;
+  private MemStoreChunkPool pool;
+
+  @VisibleForTesting
+  ChunkCreator(int chunkSize, boolean offheap, long globalMemStoreSize, float poolSizePercentage,
+      float initialCountPercentage, HeapMemoryManager heapMemoryManager) {
+    this.chunkSize = chunkSize;
+    this.offheap = offheap;
+    this.pool = initializePool(globalMemStoreSize, poolSizePercentage, initialCountPercentage);
+    if (heapMemoryManager != null && this.pool != null) {
+      // Register with Heap Memory manager
+      heapMemoryManager.registerTuneObserver(this.pool);
+    }
+  }
+
+  /**
+   * Initializes the instance of MSLABChunkCreator
+   * @param chunkSize the chunkSize
+   * @param offheap indicates if the chunk is to be created offheap or not
+   * @param globalMemStoreSize  the global memstore size
+   * @param poolSizePercentage pool size percentage
+   * @param initialCountPercentage the initial count of the chunk pool if any
+   * @param heapMemoryManager the heapmemory manager
+   * @return singleton MSLABChunkCreator
+   */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
+      justification = "Method is called by single thread at the starting of RS")
+  @VisibleForTesting
+  public static ChunkCreator initialize(int chunkSize, boolean offheap, long globalMemStoreSize,
+      float poolSizePercentage, float initialCountPercentage, HeapMemoryManager heapMemoryManager) {
+    if (INSTANCE != null) return INSTANCE;
+    INSTANCE = new ChunkCreator(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
+        initialCountPercentage, heapMemoryManager);
+    return INSTANCE;
+  }
+
+  static ChunkCreator getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Creates and inits a chunk.
+   * @return the chunk that was initialized
+   */
+  Chunk getChunk() {
+    Chunk chunk = null;
+    if (pool != null) {
+      //  the pool creates the chunk internally. The chunk#init() call happens here
+      chunk = this.pool.getChunk();
+      // the pool has run out of maxCount
+      if (chunk == null) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("The chunk pool is full. Reached maxCount= " + this.pool.getMaxCount()
+              + ". Creating chunk onheap.");
+        }
+      }
+    }
+    if (chunk == null) {
+      chunk = createChunk();
+    }
+    // put this chunk into the chunkIdMap
+    this.chunkIdMap.put(chunk.getId(), new SoftReference<>(chunk));
+    // now we need to actually do the expensive memory allocation step in case of a new chunk,
+    // else only the offset is set to the beginning of the chunk to accept allocations
+    chunk.init();
+    return chunk;
+  }
+
+  private Chunk createChunk() {
+    return createChunk(false);
+  }
+
+  /**
+   * Creates the chunk either onheap or offheap
+   * @param pool indicates if the chunks have to be created which will be used by the Pool
+   * @return the chunk
+   */
+  private Chunk createChunk(boolean pool) {
+    int id = chunkID.getAndIncrement();
+    assert id > 0;
+    // do not create offheap chunk on demand
+    if (pool && this.offheap) {
+      return new OffheapChunk(chunkSize, id, pool);
+    } else {
+      return new OnheapChunk(chunkSize, id, pool);
+    }
+  }
+
+  @VisibleForTesting
+  // TODO : To be used by CellChunkMap
+  Chunk getChunk(int id) {
+    SoftReference<Chunk> ref = chunkIdMap.get(id);
+    if (ref != null) {
+      return ref.get();
+    }
+    return null;
+  }
+
+  int getChunkSize() {
+    return this.chunkSize;
+  }
+
+  boolean isOffheap() {
+    return this.offheap;
+  }
+
+  private void removeChunks(Set<Integer> chunkIDs) {
+    this.chunkIdMap.keySet().removeAll(chunkIDs);
+  }
+
+  Chunk removeChunk(int chunkId) {
+    SoftReference<Chunk> ref = this.chunkIdMap.remove(chunkId);
+    if (ref != null) {
+      return ref.get();
+    }
+    return null;
+  }
+
+  @VisibleForTesting
+  int size() {
+    return this.chunkIdMap.size();
+  }
+
+  @VisibleForTesting
+  void clearChunkIds() {
+    this.chunkIdMap.clear();
+  }
+
+  /**
+   * A pool of {@link Chunk} instances.
+   *
+   * MemStoreChunkPool caches a number of retired chunks for reusing, it could
+   * decrease allocating bytes when writing, thereby optimizing the garbage
+   * collection on JVM.
+   */
+  private  class MemStoreChunkPool implements HeapMemoryTuneObserver {
+    private int maxCount;
+
+    // A queue of reclaimed chunks
+    private final BlockingQueue<Chunk> reclaimedChunks;
+    private final float poolSizePercentage;
+
+    /** Statistics thread schedule pool */
+    private final ScheduledExecutorService scheduleThreadPool;
+    /** Statistics thread */
+    private static final int statThreadPeriod = 60 * 5;
+    private final AtomicLong chunkCount = new AtomicLong();
+    private final AtomicLong reusedChunkCount = new AtomicLong();
+
+    MemStoreChunkPool(int maxCount, int initialCount, float poolSizePercentage) {
+      this.maxCount = maxCount;
+      this.poolSizePercentage = poolSizePercentage;
+      this.reclaimedChunks = new LinkedBlockingQueue<>();
+      for (int i = 0; i < initialCount; i++) {
+        Chunk chunk = createChunk(true);
+        chunk.init();
+        reclaimedChunks.add(chunk);
+      }
+      chunkCount.set(initialCount);
+      final String n = Thread.currentThread().getName();
+      scheduleThreadPool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
+          .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
+      this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(), statThreadPeriod,
+          statThreadPeriod, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Poll a chunk from the pool, reset it if not null, else create a new chunk to return if we have
+     * not yet created max allowed chunks count. When we have already created max allowed chunks and
+     * no free chunks as of now, return null. It is the responsibility of the caller to make a chunk
+     * then.
+     * Note: Chunks returned by this pool must be put back to the pool after its use.
+     * @return a chunk
+     * @see #putbackChunks(Set)
+     */
+    Chunk getChunk() {
+      Chunk chunk = reclaimedChunks.poll();
+      if (chunk != null) {
+        chunk.reset();
+        reusedChunkCount.incrementAndGet();
+      } else {
+        // Make a chunk iff we have not yet created the maxCount chunks
+        while (true) {
+          long created = this.chunkCount.get();
+          if (created < this.maxCount) {
+            if (this.chunkCount.compareAndSet(created, created + 1)) {
+              chunk = createChunk(true);
+              break;
+            }
+          } else {
+            break;
+          }
+        }
+      }
+      return chunk;
+    }
+
+    /**
+     * Add the chunks to the pool, when the pool achieves the max size, it will skip the remaining
+     * chunks
+     * @param chunks
+     */
+    private void putbackChunks(Set<Integer> chunks) {
+      int toAdd = Math.min(chunks.size(), this.maxCount - reclaimedChunks.size());
+      Iterator<Integer> iterator = chunks.iterator();
+      while (iterator.hasNext()) {
+        Integer chunkId = iterator.next();
+        // remove the chunks every time though they are from the pool or not
+        Chunk chunk = ChunkCreator.this.removeChunk(chunkId);
+        if (chunk != null) {
+          if (chunk.isFromPool() && toAdd > 0) {
+            reclaimedChunks.add(chunk);
+          }
+          toAdd--;
+        }
+      }
+    }
+
+    private class StatisticsThread extends Thread {
+      StatisticsThread() {
+        super("MemStoreChunkPool.StatisticsThread");
+        setDaemon(true);
+      }
+
+      @Override
+      public void run() {
+        logStats();
+      }
+
+      private void logStats() {
+        if (!LOG.isDebugEnabled()) return;
+        long created = chunkCount.get();
+        long reused = reusedChunkCount.get();
+        long total = created + reused;
+        LOG.debug("Stats: current pool size=" + reclaimedChunks.size()
+            + ",created chunk count=" + created
+            + ",reused chunk count=" + reused
+            + ",reuseRatio=" + (total == 0 ? "0" : StringUtils.formatPercent(
+                (float) reused / (float) total, 2)));
+      }
+    }
+
+    private int getMaxCount() {
+      return this.maxCount;
+    }
+
+    @Override
+    public void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize) {
+      // don't do any tuning in case of offheap memstore
+      if (isOffheap()) {
+        LOG.warn("Not tuning the chunk pool as it is offheap");
+        return;
+      }
+      int newMaxCount =
+          (int) (newMemstoreSize * poolSizePercentage / getChunkSize());
+      if (newMaxCount != this.maxCount) {
+        // We need an adjustment in the chunks numbers
+        if (newMaxCount > this.maxCount) {
+          // Max chunks getting increased. Just change the variable. Later calls to getChunk() would
+          // create and add them to Q
+          LOG.info("Max count for chunks increased from " + this.maxCount + " to " + newMaxCount);
+          this.maxCount = newMaxCount;
+        } else {
+          // Max chunks getting decreased. We may need to clear off some of the pooled chunks now
+          // itself. If the extra chunks are serving already, do not pool those when we get them back
+          LOG.info("Max count for chunks decreased from " + this.maxCount + " to " + newMaxCount);
+          this.maxCount = newMaxCount;
+          if (this.reclaimedChunks.size() > newMaxCount) {
+            synchronized (this) {
+              while (this.reclaimedChunks.size() > newMaxCount) {
+                this.reclaimedChunks.poll();
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @VisibleForTesting
+  static void clearDisableFlag() {
+    chunkPoolDisabled = false;
+  }
+
+  private MemStoreChunkPool initializePool(long globalMemStoreSize, float poolSizePercentage,
+      float initialCountPercentage) {
+    if (poolSizePercentage <= 0) {
+      LOG.info("PoolSizePercentage is less than 0. So not using pool");
+      return null;
+    }
+    if (chunkPoolDisabled) {
+      return null;
+    }
+    if (poolSizePercentage > 1.0) {
+      throw new IllegalArgumentException(
+          MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
+    }
+    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / getChunkSize());
+    if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
+      throw new IllegalArgumentException(
+          MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
+    }
+    int initialCount = (int) (initialCountPercentage * maxCount);
+    LOG.info("Allocating MemStoreChunkPool with chunk size "
+        + StringUtils.byteDesc(getChunkSize()) + ", max count " + maxCount
+        + ", initial count " + initialCount);
+    return new MemStoreChunkPool(maxCount, initialCount, poolSizePercentage);
+  }
+
+  @VisibleForTesting
+  int getMaxCount() {
+    if (pool != null) {
+      return pool.getMaxCount();
+    }
+    return 0;
+  }
+
+  @VisibleForTesting
+  int getPoolSize() {
+    if (pool != null) {
+      return pool.reclaimedChunks.size();
+    }
+    return 0;
+  }
+
+  /*
+   * Only used in testing
+   */
+  @VisibleForTesting
+  void clearChunksInPool() {
+    if (pool != null) {
+      pool.reclaimedChunks.clear();
+    }
+  }
+
+  synchronized void putbackChunks(Set<Integer> chunks) {
+    if (pool != null) {
+      pool.putbackChunks(chunks);
+    } else {
+      this.removeChunks(chunks);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index b3b5113..41eb0a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1472,7 +1472,7 @@ public class HRegionServer extends HasThread implements
       startServiceThreads();
       startHeapMemoryManager();
       // Call it after starting HeapMemoryManager.
-      initializeMemStoreChunkPool();
+      initializeMemStoreChunkCreator();
       LOG.info("Serving as " + this.serverName +
         ", RpcServer on " + rpcServices.isa +
         ", sessionid=0x" +
@@ -1492,7 +1492,7 @@ public class HRegionServer extends HasThread implements
     }
   }
 
-  private void initializeMemStoreChunkPool() {
+  protected void initializeMemStoreChunkCreator() {
     if (MemStoreLAB.isEnabled(conf)) {
       // MSLAB is enabled. So initialize MemStoreChunkPool
       // By this time, the MemstoreFlusher is already initialized. We can get the global limits from
@@ -1506,12 +1506,10 @@ public class HRegionServer extends HasThread implements
       float initialCountPercentage = conf.getFloat(MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY,
           MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT);
       int chunkSize = conf.getInt(MemStoreLAB.CHUNK_SIZE_KEY, MemStoreLAB.CHUNK_SIZE_DEFAULT);
-      MemStoreChunkPool pool = MemStoreChunkPool.initialize(globalMemStoreSize, poolSizePercentage,
-          initialCountPercentage, chunkSize, offheap);
-      if (pool != null && this.hMemManager != null) {
-        // Register with Heap Memory manager
-        this.hMemManager.registerTuneObserver(pool);
-      }
+      // init the chunkCreator
+      ChunkCreator chunkCreator =
+          ChunkCreator.initialize(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
+            initialCountPercentage, this.hMemManager);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
deleted file mode 100644
index b7ac212..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/**
- * Copyright The Apache Software Foundation
- *
- * 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.regionserver;
-
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * A pool of {@link Chunk} instances.
- * 
- * MemStoreChunkPool caches a number of retired chunks for reusing, it could
- * decrease allocating bytes when writing, thereby optimizing the garbage
- * collection on JVM.
- * 
- * The pool instance is globally unique and could be obtained through
- * {@link MemStoreChunkPool#initialize(long, float, float, int, boolean)}
- * 
- * {@link MemStoreChunkPool#getChunk()} is called when MemStoreLAB allocating
- * bytes, and {@link MemStoreChunkPool#putbackChunks(BlockingQueue)} is called
- * when MemStore clearing snapshot for flush
- */
-@SuppressWarnings("javadoc")
-@InterfaceAudience.Private
-public class MemStoreChunkPool implements HeapMemoryTuneObserver {
-  private static final Log LOG = LogFactory.getLog(MemStoreChunkPool.class);
-
-  // Static reference to the MemStoreChunkPool
-  static MemStoreChunkPool GLOBAL_INSTANCE;
-  /** Boolean whether we have disabled the memstore chunk pool entirely. */
-  static boolean chunkPoolDisabled = false;
-
-  private int maxCount;
-
-  // A queue of reclaimed chunks
-  private final BlockingQueue<Chunk> reclaimedChunks;
-  private final int chunkSize;
-  private final float poolSizePercentage;
-
-  /** Statistics thread schedule pool */
-  private final ScheduledExecutorService scheduleThreadPool;
-  /** Statistics thread */
-  private static final int statThreadPeriod = 60 * 5;
-  private final AtomicLong chunkCount = new AtomicLong();
-  private final AtomicLong reusedChunkCount = new AtomicLong();
-  private final boolean offheap;
-
-  MemStoreChunkPool(int chunkSize, int maxCount, int initialCount, float poolSizePercentage,
-      boolean offheap) {
-    this.maxCount = maxCount;
-    this.chunkSize = chunkSize;
-    this.poolSizePercentage = poolSizePercentage;
-    this.offheap = offheap;
-    this.reclaimedChunks = new LinkedBlockingQueue<>();
-    for (int i = 0; i < initialCount; i++) {
-      Chunk chunk = this.offheap ? new OffheapChunk(chunkSize) : new OnheapChunk(chunkSize);
-      chunk.init();
-      reclaimedChunks.add(chunk);
-    }
-    chunkCount.set(initialCount);
-    final String n = Thread.currentThread().getName();
-    scheduleThreadPool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
-        .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
-    this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(), statThreadPeriod,
-        statThreadPeriod, TimeUnit.SECONDS);
-  }
-
-  /**
-   * Poll a chunk from the pool, reset it if not null, else create a new chunk to return if we have
-   * not yet created max allowed chunks count. When we have already created max allowed chunks and
-   * no free chunks as of now, return null. It is the responsibility of the caller to make a chunk
-   * then.
-   * Note: Chunks returned by this pool must be put back to the pool after its use.
-   * @return a chunk
-   * @see #putbackChunk(Chunk)
-   * @see #putbackChunks(BlockingQueue)
-   */
-  Chunk getChunk() {
-    Chunk chunk = reclaimedChunks.poll();
-    if (chunk != null) {
-      chunk.reset();
-      reusedChunkCount.incrementAndGet();
-    } else {
-      // Make a chunk iff we have not yet created the maxCount chunks
-      while (true) {
-        long created = this.chunkCount.get();
-        if (created < this.maxCount) {
-          chunk = this.offheap ? new OffheapChunk(this.chunkSize) : new OnheapChunk(this.chunkSize);
-          if (this.chunkCount.compareAndSet(created, created + 1)) {
-            break;
-          }
-        } else {
-          break;
-        }
-      }
-    }
-    return chunk;
-  }
-
-  /**
-   * Add the chunks to the pool, when the pool achieves the max size, it will
-   * skip the remaining chunks
-   * @param chunks
-   */
-  synchronized void putbackChunks(BlockingQueue<Chunk> chunks) {
-    int toAdd = Math.min(chunks.size(), this.maxCount - reclaimedChunks.size());
-    Chunk chunk = null;
-    while ((chunk = chunks.poll()) != null && toAdd > 0) {
-      reclaimedChunks.add(chunk);
-      toAdd--;
-    }
-  }
-
-  /**
-   * Add the chunk to the pool, if the pool has achieved the max size, it will
-   * skip it
-   * @param chunk
-   */
-  synchronized void putbackChunk(Chunk chunk) {
-    if (reclaimedChunks.size() < this.maxCount) {
-      reclaimedChunks.add(chunk);
-    }
-  }
-
-  int getPoolSize() {
-    return this.reclaimedChunks.size();
-  }
-
-  /*
-   * Only used in testing
-   */
-  void clearChunks() {
-    this.reclaimedChunks.clear();
-  }
-
-  private class StatisticsThread extends Thread {
-    StatisticsThread() {
-      super("MemStoreChunkPool.StatisticsThread");
-      setDaemon(true);
-    }
-
-    @Override
-    public void run() {
-      logStats();
-    }
-
-    private void logStats() {
-      if (!LOG.isDebugEnabled()) return;
-      long created = chunkCount.get();
-      long reused = reusedChunkCount.get();
-      long total = created + reused;
-      LOG.debug("Stats: current pool size=" + reclaimedChunks.size()
-          + ",created chunk count=" + created
-          + ",reused chunk count=" + reused
-          + ",reuseRatio=" + (total == 0 ? "0" : StringUtils.formatPercent(
-              (float) reused / (float) total, 2)));
-    }
-  }
-
-  /**
-   * @return the global MemStoreChunkPool instance
-   */
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
-      justification = "Method is called by single thread at the starting of RS")
-  static MemStoreChunkPool initialize(long globalMemStoreSize, float poolSizePercentage,
-      float initialCountPercentage, int chunkSize, boolean offheap) {
-    if (GLOBAL_INSTANCE != null) return GLOBAL_INSTANCE;
-    if (chunkPoolDisabled) return null;
-
-    if (poolSizePercentage <= 0) {
-      chunkPoolDisabled = true;
-      return null;
-    }
-    if (poolSizePercentage > 1.0) {
-      throw new IllegalArgumentException(
-          MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
-    }
-    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / chunkSize);
-    if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
-      throw new IllegalArgumentException(
-          MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
-    }
-    int initialCount = (int) (initialCountPercentage * maxCount);
-    LOG.info("Allocating MemStoreChunkPool with chunk size " + StringUtils.byteDesc(chunkSize)
-        + ", max count " + maxCount + ", initial count " + initialCount);
-    GLOBAL_INSTANCE = new MemStoreChunkPool(chunkSize, maxCount, initialCount, poolSizePercentage,
-        offheap);
-    return GLOBAL_INSTANCE;
-  }
-
-  /**
-   * @return The singleton instance of this pool.
-   */
-  static MemStoreChunkPool getPool() {
-    return GLOBAL_INSTANCE;
-  }
-
-  int getMaxCount() {
-    return this.maxCount;
-  }
-
-  @VisibleForTesting
-  static void clearDisableFlag() {
-    chunkPoolDisabled = false;
-  }
-
-  @Override
-  public void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize) {
-    // don't do any tuning in case of offheap memstore
-    if (this.offheap) {
-      LOG.warn("Not tuning the chunk pool as it is offheap");
-      return;
-    }
-    int newMaxCount = (int) (newMemstoreSize * poolSizePercentage / chunkSize);
-    if (newMaxCount != this.maxCount) {
-      // We need an adjustment in the chunks numbers
-      if (newMaxCount > this.maxCount) {
-        // Max chunks getting increased. Just change the variable. Later calls to getChunk() would
-        // create and add them to Q
-        LOG.info("Max count for chunks increased from " + this.maxCount + " to " + newMaxCount);
-        this.maxCount = newMaxCount;
-      } else {
-        // Max chunks getting decreased. We may need to clear off some of the pooled chunks now
-        // itself. If the extra chunks are serving already, do not pool those when we get them back
-        LOG.info("Max count for chunks decreased from " + this.maxCount + " to " + newMaxCount);
-        this.maxCount = newMaxCount;
-        if (this.reclaimedChunks.size() > newMaxCount) {
-          synchronized (this) {
-            while (this.reclaimedChunks.size() > newMaxCount) {
-              this.reclaimedChunks.poll();
-            }
-          }
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
index f6d1607..72e937c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
  * <p>
  * The MemStoreLAB is basically a bump-the-pointer allocator that allocates big (2MB) chunks from
  * and then doles it out to threads that request slices into the array. These chunks can get pooled
- * as well. See {@link MemStoreChunkPool}.
+ * as well. See {@link ChunkCreator}.
  * <p>
  * The purpose of this is to combat heap fragmentation in the regionserver. By ensuring that all
  * Cells in a given memstore refer only to large chunks of contiguous memory, we ensure that
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
  * {@link #copyCellInto(Cell)} gets called. This allocates enough size in the chunk to hold this
  * cell's data and copies into this area and then recreate a Cell over this copied data.
  * <p>
- * @see MemStoreChunkPool
+ * @see ChunkCreator
  */
 @InterfaceAudience.Private
 public interface MemStoreLAB {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
index 4e87135..4fba82d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
@@ -18,23 +18,26 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import java.nio.ByteBuffer;
+import java.util.Set;
 import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
 
 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.CellUtil;
+import org.apache.hadoop.hbase.ExtendedCell;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-
 /**
  * A memstore-local allocation buffer.
  * <p>
@@ -55,8 +58,8 @@ import com.google.common.base.Preconditions;
  * would provide a performance improvement - probably would speed up the
  * Bytes.toLong/Bytes.toInt calls in KeyValue, but some of those are cached
  * anyway.
- * The chunks created by this MemStoreLAB can get pooled at {@link MemStoreChunkPool}.
- * When the Chunk comes pool, it can be either an on heap or an off heap backed chunk. The chunks,
+ * The chunks created by this MemStoreLAB can get pooled at {@link ChunkCreator}.
+ * When the Chunk comes from pool, it can be either an on heap or an off heap backed chunk. The chunks,
  * which this MemStoreLAB creates on its own (when no chunk available from pool), those will be
  * always on heap backed.
  */
@@ -66,14 +69,15 @@ public class MemStoreLABImpl implements MemStoreLAB {
   static final Log LOG = LogFactory.getLog(MemStoreLABImpl.class);
 
   private AtomicReference<Chunk> curChunk = new AtomicReference<>();
-  // A queue of chunks from pool contained by this memstore LAB
-  // TODO: in the future, it would be better to have List implementation instead of Queue,
-  // as FIFO order is not so important here
+  // Lock to manage multiple handlers requesting for a chunk
+  private ReentrantLock lock = new ReentrantLock();
+
+  // A set of chunks contained by this memstore LAB
   @VisibleForTesting
-  BlockingQueue<Chunk> pooledChunkQueue = null;
+  Set<Integer> chunks = new ConcurrentSkipListSet<Integer>();
   private final int chunkSize;
   private final int maxAlloc;
-  private final MemStoreChunkPool chunkPool;
+  private final ChunkCreator chunkCreator;
 
   // This flag is for closing this instance, its set when clearing snapshot of
   // memstore
@@ -92,20 +96,12 @@ public class MemStoreLABImpl implements MemStoreLAB {
   public MemStoreLABImpl(Configuration conf) {
     chunkSize = conf.getInt(CHUNK_SIZE_KEY, CHUNK_SIZE_DEFAULT);
     maxAlloc = conf.getInt(MAX_ALLOC_KEY, MAX_ALLOC_DEFAULT);
-    this.chunkPool = MemStoreChunkPool.getPool();
-    // currently chunkQueue is only used for chunkPool
-    if (this.chunkPool != null) {
-      // set queue length to chunk pool max count to avoid keeping reference of
-      // too many non-reclaimable chunks
-      pooledChunkQueue = new LinkedBlockingQueue<>(chunkPool.getMaxCount());
-    }
-
+    this.chunkCreator = ChunkCreator.getInstance();
     // if we don't exclude allocations >CHUNK_SIZE, we'd infiniteloop on one!
     Preconditions.checkArgument(maxAlloc <= chunkSize,
         MAX_ALLOC_KEY + " must be less than " + CHUNK_SIZE_KEY);
   }
 
-
   @Override
   public Cell copyCellInto(Cell cell) {
     int size = KeyValueUtil.length(cell);
@@ -118,19 +114,52 @@ public class MemStoreLABImpl implements MemStoreLAB {
     Chunk c = null;
     int allocOffset = 0;
     while (true) {
+      // Try to get the chunk
       c = getOrMakeChunk();
+      // we may get null because the some other thread succeeded in getting the lock
+      // and so the current thread has to try again to make its chunk or grab the chunk
+      // that the other thread created
       // Try to allocate from this chunk
-      allocOffset = c.alloc(size);
-      if (allocOffset != -1) {
-        // We succeeded - this is the common case - small alloc
-        // from a big buffer
-        break;
+      if (c != null) {
+        allocOffset = c.alloc(size);
+        if (allocOffset != -1) {
+          // We succeeded - this is the common case - small alloc
+          // from a big buffer
+          break;
+        }
+        // not enough space!
+        // try to retire this chunk
+        tryRetireChunk(c);
       }
-      // not enough space!
-      // try to retire this chunk
-      tryRetireChunk(c);
     }
-    return CellUtil.copyCellTo(cell, c.getData(), allocOffset, size);
+    return copyToChunkCell(cell, c.getData(), allocOffset, size);
+  }
+
+  /**
+   * Clone the passed cell by copying its data into the passed buf and create a cell with a chunkid
+   * out of it
+   */
+  private Cell copyToChunkCell(Cell cell, ByteBuffer buf, int offset, int len) {
+    int tagsLen = cell.getTagsLength();
+    if (cell instanceof ExtendedCell) {
+      ((ExtendedCell) cell).write(buf, offset);
+    } else {
+      // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the
+      // other case also. The data fragments within Cell is copied into buf as in KeyValue
+      // serialization format only.
+      KeyValueUtil.appendTo(cell, buf, offset, true);
+    }
+    // TODO : write the seqid here. For writing seqId we should create a new cell type so
+    // that seqId is not used as the state
+    if (tagsLen == 0) {
+      // When tagsLen is 0, make a NoTagsByteBufferKeyValue version. This is an optimized class
+      // which directly return tagsLen as 0. So we avoid parsing many length components in
+      // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
+      // call getTagsLength().
+      return new NoTagByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
+    } else {
+      return new ByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
+    }
   }
 
   /**
@@ -142,9 +171,9 @@ public class MemStoreLABImpl implements MemStoreLAB {
     this.closed = true;
     // We could put back the chunks to pool for reusing only when there is no
     // opening scanner which will read their data
-    if (chunkPool != null && openScannerCount.get() == 0
-        && reclaimed.compareAndSet(false, true)) {
-      chunkPool.putbackChunks(this.pooledChunkQueue);
+    int count  = openScannerCount.get();
+    if(count == 0) {
+      recycleChunks();
     }
   }
 
@@ -162,9 +191,14 @@ public class MemStoreLABImpl implements MemStoreLAB {
   @Override
   public void decScannerCount() {
     int count = this.openScannerCount.decrementAndGet();
-    if (this.closed && chunkPool != null && count == 0
-        && reclaimed.compareAndSet(false, true)) {
-      chunkPool.putbackChunks(this.pooledChunkQueue);
+    if (this.closed && count == 0) {
+      recycleChunks();
+    }
+  }
+
+  private void recycleChunks() {
+    if (reclaimed.compareAndSet(false, true)) {
+      chunkCreator.putbackChunks(chunks);
     }
   }
 
@@ -190,45 +224,33 @@ public class MemStoreLABImpl implements MemStoreLAB {
    * allocate a new one from the JVM.
    */
   private Chunk getOrMakeChunk() {
-    while (true) {
-      // Try to get the chunk
-      Chunk c = curChunk.get();
-      if (c != null) {
-        return c;
-      }
-
-      // No current chunk, so we want to allocate one. We race
-      // against other allocators to CAS in an uninitialized chunk
-      // (which is cheap to allocate)
-      if (chunkPool != null) {
-        c = chunkPool.getChunk();
-      }
-      boolean pooledChunk = false;
-      if (c != null) {
-        // This is chunk from pool
-        pooledChunk = true;
-      } else {
-        c = new OnheapChunk(chunkSize);// When chunk is not from pool, always make it as on heap.
-      }
-      if (curChunk.compareAndSet(null, c)) {
-        // we won race - now we need to actually do the expensive
-        // allocation step
-        c.init();
-        if (pooledChunk) {
-          if (!this.closed && !this.pooledChunkQueue.offer(c)) {
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Chunk queue is full, won't reuse this new chunk. Current queue size: "
-                  + pooledChunkQueue.size());
-            }
-          }
+    // Try to get the chunk
+    Chunk c = curChunk.get();
+    if (c != null) {
+      return c;
+    }
+    // No current chunk, so we want to allocate one. We race
+    // against other allocators to CAS in an uninitialized chunk
+    // (which is cheap to allocate)
+    if (lock.tryLock()) {
+      try {
+        // once again check inside the lock
+        c = curChunk.get();
+        if (c != null) {
+          return c;
         }
-        return c;
-      } else if (pooledChunk) {
-        chunkPool.putbackChunk(c);
+        c = this.chunkCreator.getChunk();
+        if (c != null) {
+          // set the curChunk. No need of CAS as only one thread will be here
+          curChunk.set(c);
+          chunks.add(c.getId());
+          return c;
+        }
+      } finally {
+        lock.unlock();
       }
-      // someone else won race - that's fine, we'll try to grab theirs
-      // in the next iteration of the loop.
     }
+    return null;
   }
 
   @VisibleForTesting
@@ -236,8 +258,15 @@ public class MemStoreLABImpl implements MemStoreLAB {
     return this.curChunk.get();
   }
 
-
+  @VisibleForTesting
   BlockingQueue<Chunk> getPooledChunks() {
-    return this.pooledChunkQueue;
+    BlockingQueue<Chunk> pooledChunks = new LinkedBlockingQueue<>();
+    for (Integer id : this.chunks) {
+      Chunk chunk = chunkCreator.getChunk(id);
+      if (chunk != null && chunk.isFromPool()) {
+        pooledChunks.add(chunk);
+      }
+    }
+    return pooledChunks;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
new file mode 100644
index 0000000..a8ba50c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
@@ -0,0 +1,48 @@
+/**
+ * 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.regionserver;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+
+/**
+ * ByteBuffer based cell which has the chunkid at the 0th offset and with no tags
+ * @see MemStoreLAB
+ */
+@InterfaceAudience.Private
+public class NoTagByteBufferChunkCell extends NoTagsByteBufferKeyValue {
+
+  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
+    super(buf, offset, length);
+  }
+
+  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
+    super(buf, offset, length, seqId);
+  }
+
+  @Override
+  public int getChunkId() {
+    // The chunkId is embedded at the 0th offset of the bytebuffer
+    return ByteBufferUtils.toInt(buf, 0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
index ed98cfa..e244a33 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
@@ -21,34 +21,27 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
-import com.google.common.base.Preconditions;
-
 /**
  * An off heap chunk implementation.
  */
 @InterfaceAudience.Private
 public class OffheapChunk extends Chunk {
 
-  OffheapChunk(int size) {
-    super(size);
+  OffheapChunk(int size, int id) {
+    // better if this is always created fromPool. This should not be called
+    super(size, id);
+  }
+
+  OffheapChunk(int size, int id, boolean fromPool) {
+    super(size, id, fromPool);
+    assert fromPool == true;
   }
 
   @Override
-  public void init() {
-    assert nextFreeOffset.get() == UNINITIALIZED;
-    try {
-      if (data == null) {
-        data = ByteBuffer.allocateDirect(this.size);
-      }
-    } catch (OutOfMemoryError e) {
-      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
-      assert failInit; // should be true.
-      throw e;
+  void allocateDataBuffer() {
+    if (data == null) {
+      data = ByteBuffer.allocateDirect(this.size);
+      data.putLong(0, this.getId());
     }
-    // Mark that it's ready for use
-    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
-    // We should always succeed the above CAS since only one thread
-    // calls init()!
-    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
index bd33cb5..da34e24 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
@@ -21,33 +21,25 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
-import com.google.common.base.Preconditions;
-
 /**
  * An on heap chunk implementation.
  */
 @InterfaceAudience.Private
 public class OnheapChunk extends Chunk {
 
-  OnheapChunk(int size) {
-    super(size);
+  OnheapChunk(int size, int id) {
+    super(size, id);
+  }
+
+  OnheapChunk(int size, int id, boolean fromPool) {
+    super(size, id, fromPool);
   }
 
-  public void init() {
-    assert nextFreeOffset.get() == UNINITIALIZED;
-    try {
-      if (data == null) {
-        data = ByteBuffer.allocate(this.size);
-      }
-    } catch (OutOfMemoryError e) {
-      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
-      assert failInit; // should be true.
-      throw e;
+  @Override
+  void allocateDataBuffer() {
+    if (data == null) {
+      data = ByteBuffer.allocate(this.size);
+      data.putLong(0, this.getId());
     }
-    // Mark that it's ready for use
-    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
-    // We should always succeed the above CAS since only one thread
-    // calls init()!
-    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 82c2eab..6563122 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -96,6 +96,8 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -2426,6 +2428,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
       final Configuration conf, final HTableDescriptor htd, boolean initialize)
       throws IOException {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     WAL wal = createWal(conf, rootDir, info);
     return HRegion.createHRegion(info, rootDir, conf, htd, wal, initialize);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
index 422c54b..8d8b6df 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
@@ -27,6 +27,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -49,8 +50,10 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
@@ -397,6 +400,7 @@ public class TestCoprocessorInterface {
     for(byte [] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(tableName, null, null, false);
     Path path = new Path(DIR + callingMethod);
     Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
index 80d0e3a..b99087d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
@@ -47,10 +47,12 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -152,6 +154,7 @@ public class TestRegionObserverScannerOpenHook {
     for (byte[] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     Path path = new Path(DIR + callingMethod);
     WAL wal = HBaseTestingUtility.createWal(conf, path, info);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
index 2e44dee..15d449d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
@@ -34,7 +34,9 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -100,6 +102,7 @@ public class TestRegionObserverStacking extends TestCase {
     for(byte [] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     Path path = new Path(DIR + callingMethod);
     HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
index f1775d0..fae7247 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
@@ -40,8 +40,10 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -342,6 +344,7 @@ public class TestScannerFromBucketCache {
   private static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
       String callingMethod, Configuration conf, HBaseTestingUtility test_util, boolean isReadOnly,
       byte[]... families) throws IOException {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     Path logDir = test_util.getDataTestDirOnTestFS(callingMethod + ".log");
     HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
     final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index cc73d9d..32bce26 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -65,6 +65,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActi
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
 import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -73,6 +75,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.Triple;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -88,6 +91,10 @@ public class TestCatalogJanitor {
   @Rule
   public TestName name = new TestName();
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+  }
   /**
    * Mock MasterServices for tests below.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
index 418aadf..096c5ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
@@ -241,7 +241,7 @@ public class TestBulkLoad {
     for (byte[] family : families) {
       hTableDescriptor.addFamily(new HColumnDescriptor(family));
     }
-
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     // TODO We need a way to do this without creating files
     return HRegion.createHRegion(hRegionInfo,
         new Path(testFolder.newFolder().toURI()),

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
index 3b4d068..09877b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
@@ -73,7 +73,7 @@ public class TestCellFlatSet extends TestCase {
     descCbOnHeap = new CellArrayMap(CellComparator.COMPARATOR,descCells,0,NUM_OF_CELLS,true);
     CONF.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
     CONF.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
-    MemStoreChunkPool.chunkPoolDisabled = false;
+    ChunkCreator.chunkPoolDisabled = false;
   }
 
   /* Create and test CellSet based on CellArrayMap */

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
index a888c45..9e90f3e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -50,7 +51,7 @@ import static org.junit.Assert.assertTrue;
 public class TestCompactingMemStore extends TestDefaultMemStore {
 
   private static final Log LOG = LogFactory.getLog(TestCompactingMemStore.class);
-  protected static MemStoreChunkPool chunkPool;
+  protected static ChunkCreator chunkCreator;
   protected HRegion region;
   protected RegionServicesForStores regionServicesForStores;
   protected HStore store;
@@ -65,7 +66,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
 
   @After
   public void tearDown() throws Exception {
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
   }
 
   @Override
@@ -84,15 +85,21 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000);
     HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf);
     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
-    this.region = hbaseUtility.createTestRegion("foobar", hcd);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("foobar"));
+    htd.addFamily(hcd);
+    HRegionInfo info =
+        new HRegionInfo(TableName.valueOf("foobar"), null, null, false);
+    WAL wal = hbaseUtility.createWal(conf, hbaseUtility.getDataTestDir(), info);
+    this.region = HRegion.createHRegion(info, hbaseUtility.getDataTestDir(), conf, htd, wal, true);
+    //this.region = hbaseUtility.createTestRegion("foobar", hcd);
     this.regionServicesForStores = region.getRegionServicesForStores();
     this.store = new HStore(region, hcd, conf);
 
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    chunkPool = MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f,
-        MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
-    assertTrue(chunkPool != null);
+    chunkCreator = ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
+      globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
+    assertTrue(chunkCreator != null);
   }
 
   /**
@@ -390,7 +397,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkPool.getPoolSize();
+    int chunkCount = chunkCreator.getPoolSize();
     assertTrue(chunkCount > 0);
 
   }
@@ -434,16 +441,16 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkPool.getPoolSize() == 0);
+    assertTrue(chunkCreator.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
 
     // clear chunks
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
 
     // Creating another snapshot
 
@@ -464,7 +471,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
   }
 
   @Test
@@ -516,16 +523,16 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     memstore.add(new KeyValue(row, fam, qf1, 3, val), null);
     assertEquals(3, memstore.getActive().getCellsCount());
 
-    assertTrue(chunkPool.getPoolSize() == 0);
+    assertTrue(chunkCreator.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
 
     // clear chunks
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
 
     // Creating another snapshot
 
@@ -553,7 +560,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
   }
 
   //////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
index 5a48455..66e107a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
@@ -44,17 +44,13 @@ import java.util.List;
 public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore {
 
   private static final Log LOG = LogFactory.getLog(TestCompactingToCellArrayMapMemStore.class);
-  //private static MemStoreChunkPool chunkPool;
-  //private HRegion region;
-  //private RegionServicesForStores regionServicesForStores;
-  //private HStore store;
 
   //////////////////////////////////////////////////////////////////////////////
   // Helpers
   //////////////////////////////////////////////////////////////////////////////
 
   @Override public void tearDown() throws Exception {
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
   }
 
   @Override public void setUp() throws Exception {
@@ -408,16 +404,16 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkPool.getPoolSize() == 0);
+    assertTrue(chunkCreator.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
 
     // clear chunks
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
 
     // Creating another snapshot
 
@@ -438,7 +434,7 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
   }
 
   @Test
@@ -472,7 +468,7 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkPool.getPoolSize();
+    int chunkCount = chunkCreator.getPoolSize();
     assertTrue(chunkCount > 0);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
index 8e85730..e320368 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
@@ -164,6 +164,7 @@ public class TestCompactionArchiveConcurrentClose {
 
     HRegionFileSystem fs = new WaitingHRegionFileSystem(conf, tableDir.getFileSystem(conf),
         tableDir, info);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + info.getEncodedName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
index 89b2368..e7fcf18 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
@@ -174,6 +174,7 @@ public class TestCompactionArchiveIOException {
   private HRegion initHRegion(HTableDescriptor htd, HRegionInfo info)
       throws IOException {
     Configuration conf = testUtil.getConfiguration();
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
     Path regionDir = new Path(tableDir, info.getEncodedName());
     Path storeDir = new Path(regionDir, htd.getColumnFamilies()[0].getNameAsString());

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
index 7154511..bff5bec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
@@ -104,6 +104,7 @@ public class TestCompactionPolicy {
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
 
     hlog = new FSHLog(fs, basedir, logName, conf);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     region = HRegion.createHRegion(info, basedir, conf, htd, hlog);
     region.close();
     Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index 7434eb1..41b304b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -84,6 +85,7 @@ public class TestDefaultMemStore {
   protected static final byte[] FAMILY = Bytes.toBytes("column");
   protected MultiVersionConcurrencyControl mvcc;
   protected AtomicLong startSeqNum = new AtomicLong(0);
+  protected ChunkCreator chunkCreator;
 
   private String getName() {
     return this.name.getMethodName();
@@ -92,9 +94,17 @@ public class TestDefaultMemStore {
   @Before
   public void setUp() throws Exception {
     internalSetUp();
+    // no pool
+    this.chunkCreator =
+        ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     this.memstore = new DefaultMemStore();
   }
 
+  @AfterClass
+  public static void tearDownClass() throws Exception {
+    ChunkCreator.getInstance().clearChunkIds();
+  }
+
   protected void internalSetUp() throws Exception {
     this.mvcc = new MultiVersionConcurrencyControl();
   }
@@ -129,7 +139,9 @@ public class TestDefaultMemStore {
       assertEquals(Segment.getCellLength(kv), sizeChangeForSecondCell.getDataSize());
       // make sure chunk size increased even when writing the same cell, if using MSLAB
       if (msLab instanceof MemStoreLABImpl) {
-        assertEquals(2 * Segment.getCellLength(kv),
+        // since we add the chunkID at the 0th offset of the chunk and the
+        // chunkid is a long we need to account for those 8 bytes
+        assertEquals(2 * Segment.getCellLength(kv) + Bytes.SIZEOF_LONG,
           ((MemStoreLABImpl) msLab).getCurrentChunk().getNextFreeOffset());
       }
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
index 73fb9cf..24e850d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
@@ -266,6 +266,7 @@ public class TestFailedAppendAndSync {
    */
   public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
   throws IOException {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.SYNC_WAL,
       wal, COLUMN_FAMILY_BYTES);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index b416c7d..0f24a24 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -153,7 +153,7 @@ public class TestHMobStore {
 
     htd.addFamily(hcd);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
-
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);
     final WALFactory wals = new WALFactory(walConf, null, methodName);


[19/50] [abbrv] hbase git commit: HBASE-17000 Implement computation of online region sizes and report to the Master

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dea6764/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
index 1c373ee..23ddd43 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -141,6 +141,22 @@ message SplitTableRegionResponse {
   optional uint64 proc_id = 1;
 }
 
+message RegionSpaceUse {
+  optional RegionInfo region = 1; // A region identifier
+  optional uint64 size = 2; // The size in bytes of the region
+}
+
+/**
+ * Reports filesystem usage for regions.
+ */
+message RegionSpaceUseReportRequest {
+  repeated RegionSpaceUse space_use = 1;
+}
+
+message RegionSpaceUseReportResponse {
+
+}
+
 service RegionServerStatusService {
   /** Called when a region server first starts. */
   rpc RegionServerStartup(RegionServerStartupRequest)
@@ -182,4 +198,10 @@ service RegionServerStatusService {
    */
   rpc getProcedureResult(GetProcedureResultRequest)
     returns(GetProcedureResultResponse);
+
+  /**
+   * Reports Region filesystem space use
+   */
+  rpc ReportRegionSpaceUse(RegionSpaceUseReportRequest)
+    returns(RegionSpaceUseReportResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dea6764/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 9af8f45..f454248 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -216,6 +217,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
@@ -2006,4 +2010,19 @@ public class MasterRpcServices extends RSRpcServices
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public RegionSpaceUseReportResponse reportRegionSpaceUse(RpcController controller,
+      RegionSpaceUseReportRequest request) throws ServiceException {
+    try {
+      master.checkInitialized();
+      MasterQuotaManager quotaManager = this.master.getMasterQuotaManager();
+      for (RegionSpaceUse report : request.getSpaceUseList()) {
+        quotaManager.addRegionSize(HRegionInfo.convert(report.getRegion()), report.getSize());
+      }
+      return RegionSpaceUseReportResponse.newBuilder().build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dea6764/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
new file mode 100644
index 0000000..01540eb
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+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.util.EnvironmentEdgeManager;
+
+/**
+ * A chore which computes the size of each {@link HRegion} on the FileSystem hosted by the given {@link HRegionServer}.
+ */
+@InterfaceAudience.Private
+public class FileSystemUtilizationChore extends ScheduledChore {
+  private static final Log LOG = LogFactory.getLog(FileSystemUtilizationChore.class);
+  static final String FS_UTILIZATION_CHORE_PERIOD_KEY = "hbase.regionserver.quotas.fs.utilization.chore.period";
+  static final int FS_UTILIZATION_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
+
+  static final String FS_UTILIZATION_CHORE_DELAY_KEY = "hbase.regionserver.quotas.fs.utilization.chore.delay";
+  static final long FS_UTILIZATION_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
+
+  static final String FS_UTILIZATION_CHORE_TIMEUNIT_KEY = "hbase.regionserver.quotas.fs.utilization.chore.timeunit";
+  static final String FS_UTILIZATION_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
+
+  static final String FS_UTILIZATION_MAX_ITERATION_DURATION_KEY = "hbase.regionserver.quotas.fs.utilization.chore.max.iteration.millis";
+  static final long FS_UTILIZATION_MAX_ITERATION_DURATION_DEFAULT = 5000L;
+
+  private final HRegionServer rs;
+  private final long maxIterationMillis;
+  private Iterator<Region> leftoverRegions;
+
+  public FileSystemUtilizationChore(HRegionServer rs) {
+    super(FileSystemUtilizationChore.class.getSimpleName(), rs, getPeriod(rs.getConfiguration()),
+        getInitialDelay(rs.getConfiguration()), getTimeUnit(rs.getConfiguration()));
+    this.rs = rs;
+    this.maxIterationMillis = rs.getConfiguration().getLong(
+        FS_UTILIZATION_MAX_ITERATION_DURATION_KEY, FS_UTILIZATION_MAX_ITERATION_DURATION_DEFAULT);
+  }
+
+  @Override
+  protected void chore() {
+    final Map<HRegionInfo,Long> onlineRegionSizes = new HashMap<>();
+    final Set<Region> onlineRegions = new HashSet<>(rs.getOnlineRegions());
+    // Process the regions from the last run if we have any. If we are somehow having difficulty
+    // processing the Regions, we want to avoid creating a backlog in memory of Region objs.
+    Iterator<Region> oldRegionsToProcess = getLeftoverRegions();
+    final Iterator<Region> iterator;
+    final boolean processingLeftovers;
+    if (null == oldRegionsToProcess) {
+      iterator = onlineRegions.iterator();
+      processingLeftovers = false;
+    } else {
+      iterator = oldRegionsToProcess;
+      processingLeftovers = true;
+    }
+    // Reset the leftoverRegions and let the loop re-assign if necessary.
+    setLeftoverRegions(null);
+    long regionSizesCalculated = 0L;
+    long offlineRegionsSkipped = 0L;
+    long skippedSplitParents = 0L;
+    long skippedRegionReplicas = 0L;
+    final long start = EnvironmentEdgeManager.currentTime();
+    while (iterator.hasNext()) {
+      // Make sure this chore doesn't hog the thread.
+      long timeRunning = EnvironmentEdgeManager.currentTime() - start;
+      if (timeRunning > maxIterationMillis) {
+        LOG.debug("Preempting execution of FileSystemUtilizationChore because it exceeds the"
+            + " maximum iteration configuration value. Will process remaining iterators"
+            + " on a subsequent invocation.");
+        setLeftoverRegions(iterator);
+        break;
+      }
+
+      final Region region = iterator.next();
+      // If we're processing leftover regions, the region may no-longer be online.
+      // If so, we can skip it.
+      if (processingLeftovers && !onlineRegions.contains(region)) {
+        offlineRegionsSkipped++;
+        continue;
+      }
+      // Avoid computing the size of regions which are the parent of split.
+      if (region.getRegionInfo().isSplitParent()) {
+        skippedSplitParents++;
+        continue;
+      }
+      // Avoid computing the size of region replicas.
+      if (HRegionInfo.DEFAULT_REPLICA_ID != region.getRegionInfo().getReplicaId()) {
+        skippedRegionReplicas++;
+        continue;
+      }
+      final long sizeInBytes = computeSize(region);
+      onlineRegionSizes.put(region.getRegionInfo(), sizeInBytes);
+      regionSizesCalculated++;
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Computed the size of " + regionSizesCalculated + " Regions. Skipped computation"
+          + " of " + offlineRegionsSkipped + " regions due to not being online on this RS, "
+          + skippedSplitParents + " regions due to being the parent of a split, and"
+          + skippedRegionReplicas + " regions due to being region replicas.");
+    }
+    reportRegionSizesToMaster(onlineRegionSizes);
+  }
+
+  /**
+   * Returns an {@link Iterator} over the Regions which were skipped last invocation of the chore.
+   *
+   * @return Regions from the previous invocation to process, or null.
+   */
+  Iterator<Region> getLeftoverRegions() {
+    return leftoverRegions;
+  }
+
+  /**
+   * Sets a new collection of Regions as leftovers.
+   */
+  void setLeftoverRegions(Iterator<Region> newLeftovers) {
+    this.leftoverRegions = newLeftovers;
+  }
+
+  /**
+   * Computes total FileSystem size for the given {@link Region}.
+   *
+   * @param r The region
+   * @return The size, in bytes, of the Region.
+   */
+  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();
+    }
+    return regionSize;
+  }
+
+  /**
+   * Reports the computed region sizes to the currently active Master.
+   *
+   * @param onlineRegionSizes The computed region sizes to report.
+   */
+  void reportRegionSizesToMaster(Map<HRegionInfo,Long> onlineRegionSizes) {
+    this.rs.reportRegionSizesForQuotas(onlineRegionSizes);
+  }
+
+  /**
+   * 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(FS_UTILIZATION_CHORE_PERIOD_KEY, FS_UTILIZATION_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(FS_UTILIZATION_CHORE_DELAY_KEY, FS_UTILIZATION_CHORE_DELAY_DEFAULT);
+  }
+
+  /**
+   * Extracts the time unit for the chore period and initial delay from the configuration. The
+   * configuration value for {@link #FS_UTILIZATION_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(FS_UTILIZATION_CHORE_TIMEUNIT_KEY,
+        FS_UTILIZATION_CHORE_TIMEUNIT_DEFAULT));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dea6764/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index 1469268..37ccdc0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -19,7 +19,10 @@
 package org.apache.hadoop.hbase.quotas;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -62,6 +65,7 @@ public class MasterQuotaManager implements RegionStateListener {
   private NamedLock<String> userLocks;
   private boolean enabled = false;
   private NamespaceAuditor namespaceQuotaManager;
+  private ConcurrentHashMap<HRegionInfo, Long> regionSizes;
 
   public MasterQuotaManager(final MasterServices masterServices) {
     this.masterServices = masterServices;
@@ -85,6 +89,7 @@ public class MasterQuotaManager implements RegionStateListener {
     namespaceLocks = new NamedLock<>();
     tableLocks = new NamedLock<>();
     userLocks = new NamedLock<>();
+    regionSizes = new ConcurrentHashMap<>();
 
     namespaceQuotaManager = new NamespaceAuditor(masterServices);
     namespaceQuotaManager.start();
@@ -515,5 +520,15 @@ public class MasterQuotaManager implements RegionStateListener {
       this.namespaceQuotaManager.removeRegionFromNamespaceUsage(hri);
     }
   }
+
+  public void addRegionSize(HRegionInfo hri, long size) {
+    // TODO Make proper API
+    regionSizes.put(hri, size);
+  }
+
+  public Map<HRegionInfo, Long> snapshotRegionSizes() {
+    // TODO Make proper API
+    return new HashMap<>(regionSizes);
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dea6764/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index b3b5113..9be4131 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -36,6 +36,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedMap;
@@ -72,6 +73,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HealthCheckChore;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Stoppable;
@@ -115,6 +117,7 @@ import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.mob.MobCacheConfig;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
+import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore;
 import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
@@ -150,12 +153,15 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpeci
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
@@ -510,6 +516,8 @@ public class HRegionServer extends HasThread implements
 
   protected SecureBulkLoadManager secureBulkLoadManager;
 
+  protected FileSystemUtilizationChore fsUtilizationChore;
+
   /**
    * Starts a HRegionServer at the default location.
    */
@@ -921,6 +929,8 @@ public class HRegionServer extends HasThread implements
     // Setup the Quota Manager
     rsQuotaManager = new RegionServerQuotaManager(this);
 
+    this.fsUtilizationChore = new FileSystemUtilizationChore(this);
+
     // Setup RPC client for master communication
     rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
         rpcServices.isa.getAddress(), 0), clusterConnection.getConnectionMetrics());
@@ -1234,6 +1244,66 @@ public class HRegionServer extends HasThread implements
     }
   }
 
+  /**
+   * Reports the given map of Regions and their size on the filesystem to the active Master.
+   *
+   * @param onlineRegionSizes A map of region info to size in bytes
+   */
+  public void reportRegionSizesForQuotas(final Map<HRegionInfo, Long> onlineRegionSizes) {
+    RegionServerStatusService.BlockingInterface rss = rssStub;
+    if (rss == null) {
+      // the current server could be stopping.
+      LOG.trace("Skipping Region size report to HMaster as stub is null");
+      return;
+    }
+    try {
+      RegionSpaceUseReportRequest request = buildRegionSpaceUseReportRequest(
+          Objects.requireNonNull(onlineRegionSizes));
+      rss.reportRegionSpaceUse(null, request);
+    } catch (ServiceException se) {
+      IOException ioe = ProtobufUtil.getRemoteException(se);
+      if (ioe instanceof PleaseHoldException) {
+        LOG.trace("Failed to report region sizes to Master because it is initializing. This will be retried.", ioe);
+        // The Master is coming up. Will retry the report later. Avoid re-creating the stub.
+        return;
+      }
+      LOG.debug("Failed to report region sizes to Master. This will be retried.", ioe);
+      if (rssStub == rss) {
+        rssStub = null;
+      }
+      createRegionServerStatusStub(true);
+    }
+  }
+
+  /**
+   * Builds a {@link RegionSpaceUseReportRequest} protobuf message from the region size map.
+   *
+   * @param regionSizes Map of region info to size in bytes.
+   * @return The corresponding protocol buffer message.
+   */
+  RegionSpaceUseReportRequest buildRegionSpaceUseReportRequest(Map<HRegionInfo,Long> regionSizes) {
+    RegionSpaceUseReportRequest.Builder request = RegionSpaceUseReportRequest.newBuilder();
+    for (Entry<HRegionInfo, Long> entry : Objects.requireNonNull(regionSizes).entrySet()) {
+      request.addSpaceUse(convertRegionSize(entry.getKey(), entry.getValue()));
+    }
+    return request.build();
+  }
+
+  /**
+   * Converts a pair of {@link HRegionInfo} and {@code long} into a {@link RegionSpaceUse}
+   * protobuf message.
+   *
+   * @param regionInfo The HRegionInfo
+   * @param sizeInBytes The size in bytes of the Region
+   * @return The protocol buffer
+   */
+  RegionSpaceUse convertRegionSize(HRegionInfo regionInfo, Long sizeInBytes) {
+    return RegionSpaceUse.newBuilder()
+        .setRegion(HRegionInfo.convert(Objects.requireNonNull(regionInfo)))
+        .setSize(Objects.requireNonNull(sizeInBytes))
+        .build();
+  }
+
   ClusterStatusProtos.ServerLoad buildServerLoad(long reportStartTime, long reportEndTime)
       throws IOException {
     // We're getting the MetricsRegionServerWrapper here because the wrapper computes requests
@@ -1816,6 +1886,7 @@ public class HRegionServer extends HasThread implements
     if (this.nonceManagerChore != null) choreService.scheduleChore(nonceManagerChore);
     if (this.storefileRefresher != null) choreService.scheduleChore(storefileRefresher);
     if (this.movedRegionsCleaner != null) choreService.scheduleChore(movedRegionsCleaner);
+    if (this.fsUtilizationChore != null) choreService.scheduleChore(fsUtilizationChore);
 
     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
     // an unhandled exception, it will just exit.
@@ -2325,6 +2396,7 @@ public class HRegionServer extends HasThread implements
     if (this.healthCheckChore != null) healthCheckChore.cancel(true);
     if (this.storefileRefresher != null) storefileRefresher.cancel(true);
     if (this.movedRegionsCleaner != null) movedRegionsCleaner.cancel(true);
+    if (this.fsUtilizationChore != null) fsUtilizationChore.cancel(true);
 
     if (this.cacheFlusher != null) {
       this.cacheFlusher.join();

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dea6764/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
new file mode 100644
index 0000000..ad98720
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
@@ -0,0 +1,357 @@
+/*
+ * 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.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+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.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Test class for {@link FileSystemUtilizationChore}.
+ */
+@Category(SmallTests.class)
+public class TestFileSystemUtilizationChore {
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testNoOnlineRegions() {
+    // One region with a store size of one.
+    final List<Long> regionSizes = Collections.emptyList();
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+    final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
+    doAnswer(new ExpectedRegionSizeSummationAnswer(sum(regionSizes)))
+        .when(rs)
+        .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+
+    final Region region = mockRegionWithSize(regionSizes);
+    when(rs.getOnlineRegions()).thenReturn(Arrays.asList(region));
+    chore.chore();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testRegionSizes() {
+    // One region with a store size of one.
+    final List<Long> regionSizes = Arrays.asList(1024L);
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+    final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
+    doAnswer(new ExpectedRegionSizeSummationAnswer(sum(regionSizes)))
+        .when(rs)
+        .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+
+    final Region region = mockRegionWithSize(regionSizes);
+    when(rs.getOnlineRegions()).thenReturn(Arrays.asList(region));
+    chore.chore();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testMultipleRegionSizes() {
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+
+    // Three 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 long r2Sum = sum(r2Sizes);
+    final List<Long> r3Sizes = Arrays.asList(10L * 1024L * 1024L);
+    final long r3Sum = sum(r3Sizes);
+
+    final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
+    doAnswer(new ExpectedRegionSizeSummationAnswer(sum(Arrays.asList(r1Sum, r2Sum, r3Sum))))
+        .when(rs)
+        .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+
+    final Region r1 = mockRegionWithSize(r1Sizes);
+    final Region r2 = mockRegionWithSize(r2Sizes);
+    final Region r3 = mockRegionWithSize(r3Sizes);
+    when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2, r3));
+    chore.chore();
+  }
+
+  @Test
+  public void testDefaultConfigurationProperties() {
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+    final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
+    // Verify that the expected default values are actually represented.
+    assertEquals(
+        FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_DEFAULT, chore.getPeriod());
+    assertEquals(
+        FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_DEFAULT, chore.getInitialDelay());
+    assertEquals(
+        TimeUnit.valueOf(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_TIMEUNIT_DEFAULT),
+        chore.getTimeUnit());
+  }
+
+  @Test
+  public void testNonDefaultConfigurationProperties() {
+    final Configuration conf = getDefaultHBaseConfiguration();
+    // Override the default values
+    final int period = 60 * 10;
+    final long delay = 30L;
+    final TimeUnit timeUnit = TimeUnit.SECONDS;
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, period);
+    conf.setLong(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_KEY, delay);
+    conf.set(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_TIMEUNIT_KEY, timeUnit.name());
+
+    // Verify that the chore reports these non-default values
+    final HRegionServer rs = mockRegionServer(conf);
+    final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
+    assertEquals(period, chore.getPeriod());
+    assertEquals(delay, chore.getInitialDelay());
+    assertEquals(timeUnit, chore.getTimeUnit());
+  }
+
+  @Test
+  public void testProcessingLeftoverRegions() {
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+
+    // Some leftover regions from a previous chore()
+    final List<Long> leftover1Sizes = Arrays.asList(1024L, 4096L);
+    final long leftover1Sum = sum(leftover1Sizes);
+    final List<Long> leftover2Sizes = Arrays.asList(2048L);
+    final long leftover2Sum = sum(leftover2Sizes);
+
+    final Region lr1 = mockRegionWithSize(leftover1Sizes);
+    final Region lr2 = mockRegionWithSize(leftover2Sizes);
+    final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs) {
+      @Override
+      Iterator<Region> getLeftoverRegions() {
+        return Arrays.asList(lr1, lr2).iterator();
+      }
+    };
+    doAnswer(new ExpectedRegionSizeSummationAnswer(sum(Arrays.asList(leftover1Sum, leftover2Sum))))
+        .when(rs)
+        .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+
+    // We shouldn't compute all of these region sizes, just the leftovers
+    final Region r1 = mockRegionWithSize(Arrays.asList(1024L, 2048L));
+    final Region r2 = mockRegionWithSize(Arrays.asList(1024L * 1024L));
+    final Region r3 = mockRegionWithSize(Arrays.asList(10L * 1024L * 1024L));
+    when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2, r3, lr1, lr2));
+
+    chore.chore();
+  }
+
+  @Test
+  public void testProcessingNowOfflineLeftoversAreIgnored() {
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+
+    // Some leftover regions from a previous chore()
+    final List<Long> leftover1Sizes = Arrays.asList(1024L, 4096L);
+    final long leftover1Sum = sum(leftover1Sizes);
+    final List<Long> leftover2Sizes = Arrays.asList(2048L);
+    final long leftover2Sum = sum(leftover2Sizes);
+
+    final Region lr1 = mockRegionWithSize(leftover1Sizes);
+    final Region lr2 = mockRegionWithSize(leftover2Sizes);
+    final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs) {
+      @Override
+      Iterator<Region> getLeftoverRegions() {
+        return Arrays.asList(lr1, lr2).iterator();
+      }
+    };
+    doAnswer(new ExpectedRegionSizeSummationAnswer(sum(Arrays.asList(leftover1Sum))))
+        .when(rs)
+        .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+
+    // We shouldn't compute all of these region sizes, just the leftovers
+    final Region r1 = mockRegionWithSize(Arrays.asList(1024L, 2048L));
+    final Region r2 = mockRegionWithSize(Arrays.asList(1024L * 1024L));
+    final Region r3 = mockRegionWithSize(Arrays.asList(10L * 1024L * 1024L));
+    // lr2 is no longer online, so it should be ignored
+    when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2, r3, lr1));
+
+    chore.chore();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testIgnoreSplitParents() {
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+
+    // Three 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))))
+        .when(rs)
+        .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+
+    final Region r1 = mockRegionWithSize(r1Sizes);
+    final Region r2 = mockSplitParentRegionWithSize(r2Sizes);
+    when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2));
+    chore.chore();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testIgnoreRegionReplicas() {
+    final Configuration conf = getDefaultHBaseConfiguration();
+    final HRegionServer rs = mockRegionServer(conf);
+
+    // Three 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))))
+        .when(rs)
+        .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+
+    final Region r1 = mockRegionWithSize(r1Sizes);
+    final Region r2 = mockRegionReplicaWithSize(r2Sizes);
+    when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2));
+    chore.chore();
+  }
+
+  /**
+   * Creates an HBase Configuration object for the default values.
+   */
+  private Configuration getDefaultHBaseConfiguration() {
+    final Configuration conf = HBaseConfiguration.create();
+    conf.addResource("hbase-default.xml");
+    return conf;
+  }
+
+  /**
+   * Creates an HRegionServer using the given Configuration.
+   */
+  private HRegionServer mockRegionServer(Configuration conf) {
+    final HRegionServer rs = mock(HRegionServer.class);
+    when(rs.getConfiguration()).thenReturn(conf);
+    return rs;
+  }
+
+  /**
+   * Sums the collection of non-null numbers.
+   */
+  private long sum(Collection<Long> values) {
+    long sum = 0L;
+    for (Long value : values) {
+      assertNotNull(value);
+      sum += value;
+    }
+    return sum;
+  }
+
+  /**
+   * Creates a region with a number of Stores equal to the length of {@code storeSizes}. Each
+   * {@link Store} will have a reported size corresponding to the element in {@code storeSizes}.
+   *
+   * @param storeSizes A list of sizes for each Store.
+   * @return A mocked Region.
+   */
+  private Region mockRegionWithSize(Collection<Long> storeSizes) {
+    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);
+    for (Long storeSize : storeSizes) {
+      final Store s = mock(Store.class);
+      stores.add(s);
+      when(s.getStorefilesSize()).thenReturn(storeSize);
+    }
+    return r;
+  }
+
+  /**
+   * Creates a region which is the parent of a split.
+   *
+   * @param storeSizes A list of sizes for each Store.
+   * @return A mocked Region.
+   */
+  private Region mockSplitParentRegionWithSize(Collection<Long> storeSizes) {
+    final Region r = mockRegionWithSize(storeSizes);
+    final HRegionInfo info = r.getRegionInfo();
+    when(info.isSplitParent()).thenReturn(true);
+    return r;
+  }
+
+  /**
+   * Creates a region who has a replicaId of <code>1</code>.
+   *
+   * @param storeSizes A list of sizes for each Store.
+   * @return A mocked Region.
+   */
+  private Region mockRegionReplicaWithSize(Collection<Long> storeSizes) {
+    final Region r = mockRegionWithSize(storeSizes);
+    final HRegionInfo info = r.getRegionInfo();
+    when(info.getReplicaId()).thenReturn(1);
+    return r;
+  }
+
+  /**
+   * An Answer implementation which verifies the sum of the Region sizes to report is as expected.
+   */
+  private static class ExpectedRegionSizeSummationAnswer implements Answer<Void> {
+    private final long expectedSize;
+
+    public ExpectedRegionSizeSummationAnswer(long expectedSize) {
+      this.expectedSize = expectedSize;
+    }
+
+    @Override
+    public Void answer(InvocationOnMock invocation) throws Throwable {
+      Object[] args = invocation.getArguments();
+      assertEquals(1, args.length);
+      @SuppressWarnings("unchecked")
+      Map<HRegionInfo,Long> regionSizes = (Map<HRegionInfo,Long>) args[0];
+      long sum = 0L;
+      for (Long regionSize : regionSizes.values()) {
+        sum += regionSize;
+      }
+      assertEquals(expectedSize, sum);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dea6764/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
new file mode 100644
index 0000000..ed8a2f3
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java
@@ -0,0 +1,194 @@
+/*
+ * 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.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+
+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.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test class which verifies that region sizes are reported to the master.
+ */
+@Category(MediumTests.class)
+public class TestRegionSizeUse {
+  private static final Log LOG = LogFactory.getLog(TestRegionSizeUse.class);
+  private static final int SIZE_PER_VALUE = 256;
+  private static final int NUM_SPLITS = 10;
+  private static final String F1 = "f1";
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private MiniHBaseCluster cluster;
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @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);
+    cluster = TEST_UTIL.startMiniCluster(2);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testBasicRegionSizeReports() throws Exception {
+    final long bytesWritten = 5L * 1024L * 1024L; // 5MB
+    final TableName tn = writeData(bytesWritten);
+    LOG.debug("Data was written to HBase");
+    final Admin admin = TEST_UTIL.getAdmin();
+    // Push the data to disk.
+    admin.flush(tn);
+    LOG.debug("Data flushed to disk");
+    // Get the final region distribution
+    final List<HRegionInfo> regions = TEST_UTIL.getAdmin().getTableRegions(tn);
+
+    HMaster master = cluster.getMaster();
+    MasterQuotaManager quotaManager = master.getMasterQuotaManager();
+    Map<HRegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
+    // Wait until we get all of the region reports for our table
+    // The table may split, so make sure we have at least as many as expected right after we
+    // finished writing the data.
+    int observedRegions = numRegionsForTable(tn, regionSizes);
+    while (observedRegions < regions.size()) {
+      LOG.debug("Expecting more regions. Saw " + observedRegions
+          + " region sizes reported, expected at least " + regions.size());
+      Thread.sleep(1000);
+      regionSizes = quotaManager.snapshotRegionSizes();
+      observedRegions = numRegionsForTable(tn, regionSizes);
+    }
+
+    LOG.debug("Observed region sizes by the HMaster: " + regionSizes);
+    long totalRegionSize = 0L;
+    for (Long regionSize : regionSizes.values()) {
+      totalRegionSize += regionSize;
+    }
+    assertTrue("Expected region size report to exceed " + bytesWritten + ", but was "
+        + totalRegionSize + ". RegionSizes=" + regionSizes, bytesWritten < totalRegionSize);
+  }
+
+  /**
+   * Writes at least {@code sizeInBytes} bytes of data to HBase and returns the TableName used.
+   *
+   * @param sizeInBytes The amount of data to write in bytes.
+   * @return The table the data was written to
+   */
+  private TableName writeData(long sizeInBytes) throws IOException {
+    final Connection conn = TEST_UTIL.getConnection();
+    final Admin admin = TEST_UTIL.getAdmin();
+    final TableName tn = TableName.valueOf(testName.getMethodName());
+
+    // Delete the old table
+    if (admin.tableExists(tn)) {
+      admin.disableTable(tn);
+      admin.deleteTable(tn);
+    }
+
+    // Create the table
+    HTableDescriptor tableDesc = new HTableDescriptor(tn);
+    tableDesc.addFamily(new HColumnDescriptor(F1));
+    admin.createTable(tableDesc, Bytes.toBytes("1"), Bytes.toBytes("9"), NUM_SPLITS);
+
+    final Table table = conn.getTable(tn);
+    try {
+      List<Put> updates = new ArrayList<>();
+      long bytesToWrite = sizeInBytes;
+      long rowKeyId = 0L;
+      final StringBuilder sb = new StringBuilder();
+      final Random r = new Random();
+      while (bytesToWrite > 0L) {
+        sb.setLength(0);
+        sb.append(Long.toString(rowKeyId));
+        // Use the reverse counter as the rowKey to get even spread across all regions
+        Put p = new Put(Bytes.toBytes(sb.reverse().toString()));
+        byte[] value = new byte[SIZE_PER_VALUE];
+        r.nextBytes(value);
+        p.addColumn(Bytes.toBytes(F1), Bytes.toBytes("q1"), value);
+        updates.add(p);
+
+        // Batch 50K worth of updates
+        if (updates.size() > 50) {
+          table.put(updates);
+          updates.clear();
+        }
+
+        // Just count the value size, ignore the size of rowkey + column
+        bytesToWrite -= SIZE_PER_VALUE;
+        rowKeyId++;
+      }
+
+      // Write the final batch
+      if (!updates.isEmpty()) {
+        table.put(updates);
+      }
+
+      return tn;
+    } finally {
+      table.close();
+    }
+  }
+
+  /**
+   * Computes the number of regions for the given table that have a positive size.
+   *
+   * @param tn The TableName in question
+   * @param regions A collection of region sizes
+   * @return The number of regions for the given table.
+   */
+  private int numRegionsForTable(TableName tn, Map<HRegionInfo,Long> regions) {
+    int sum = 0;
+    for (Entry<HRegionInfo,Long> entry : regions.entrySet()) {
+      if (tn.equals(entry.getKey().getTable()) && 0 < entry.getValue()) {
+        sum++;
+      }
+    }
+    return sum;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dea6764/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
new file mode 100644
index 0000000..3244681
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
@@ -0,0 +1,99 @@
+/*
+ * 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.regionserver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+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.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for isolated (non-cluster) tests surrounding the report
+ * of Region space use to the Master by RegionServers.
+ */
+@Category(SmallTests.class)
+public class TestRegionServerRegionSpaceUseReport {
+
+  @Test
+  public void testConversion() {
+    TableName tn = TableName.valueOf("table1");
+    HRegionInfo hri1 = new HRegionInfo(tn, Bytes.toBytes("a"), Bytes.toBytes("b"));
+    HRegionInfo hri2 = new HRegionInfo(tn, Bytes.toBytes("b"), Bytes.toBytes("c"));
+    HRegionInfo hri3 = new HRegionInfo(tn, Bytes.toBytes("c"), Bytes.toBytes("d"));
+    Map<HRegionInfo,Long> sizes = new HashMap<>();
+    sizes.put(hri1, 1024L * 1024L);
+    sizes.put(hri2, 1024L * 1024L * 8L);
+    sizes.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(Map.class));
+    doCallRealMethod().when(rs).convertRegionSize(any(HRegionInfo.class), anyLong());
+
+    RegionSpaceUseReportRequest requests = rs.buildRegionSpaceUseReportRequest(sizes);
+    assertEquals(sizes.size(), requests.getSpaceUseCount());
+    for (RegionSpaceUse spaceUse : requests.getSpaceUseList()) {
+      RegionInfo ri = spaceUse.getRegion();
+      HRegionInfo hri = HRegionInfo.convert(ri);
+      Long expectedSize = sizes.remove(hri);
+      assertNotNull("Could not find size for HRI: " + hri, expectedSize);
+      assertEquals(expectedSize.longValue(), spaceUse.getSize());
+    }
+    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(Map.class));
+    doCallRealMethod().when(rs).convertRegionSize(any(HRegionInfo.class), anyLong());
+
+    rs.buildRegionSpaceUseReportRequest(null);
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testMalformedMap() {
+    TableName tn = TableName.valueOf("table1");
+    HRegionInfo hri1 = new HRegionInfo(tn, Bytes.toBytes("a"), Bytes.toBytes("b"));
+    Map<HRegionInfo,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(Map.class));
+    doCallRealMethod().when(rs).convertRegionSize(any(HRegionInfo.class), anyLong());
+
+    rs.buildRegionSpaceUseReportRequest(sizes);
+  }
+}


[50/50] [abbrv] hbase git commit: HBASE-17002 JMX metrics and some UI additions for space quotas

Posted by el...@apache.org.
HBASE-17002 JMX metrics and some UI additions for space quotas


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

Branch: refs/heads/HBASE-16961
Commit: 96c6b8fa8c7c038ee9b0134fb92061b0e61a5fd4
Parents: 71e14a3
Author: Josh Elser <el...@apache.org>
Authored: Wed Feb 15 14:24:57 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:47:49 2017 -0400

----------------------------------------------------------------------
 .../hbase/client/ConnectionImplementation.java  |    8 +
 .../hadoop/hbase/client/QuotaStatusCalls.java   |   39 +-
 .../client/ShortCircuitMasterConnection.java    |    8 +
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |   41 +
 .../hbase/shaded/protobuf/RequestConverter.java |   11 +
 .../hbase/master/MetricsMasterQuotaSource.java  |   75 +
 .../master/MetricsMasterQuotaSourceFactory.java |   26 +
 .../hbase/master/MetricsMasterWrapper.java      |   13 +
 .../MetricsRegionServerQuotaSource.java         |   54 +
 .../MetricsMasterQuotaSourceFactoryImpl.java    |   36 +
 .../master/MetricsMasterQuotaSourceImpl.java    |  129 +
 ...hadoop.hbase.master.MetricsMasterQuotaSource |   18 +
 ...hbase.master.MetricsMasterQuotaSourceFactory |   18 +
 .../shaded/protobuf/generated/MasterProtos.java |   93 +-
 .../shaded/protobuf/generated/QuotaProtos.java  | 3099 +++++++++++++++++-
 .../src/main/protobuf/Master.proto              |    6 +-
 .../src/main/protobuf/Quota.proto               |   17 +
 .../org/apache/hadoop/hbase/master/HMaster.java |    2 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   38 +
 .../hadoop/hbase/master/MetricsMaster.java      |   42 +
 .../hbase/master/MetricsMasterWrapperImpl.java  |   42 +-
 .../hadoop/hbase/quotas/QuotaObserverChore.java |   92 +-
 .../resources/hbase-webapps/master/table.jsp    |   59 +
 .../hbase/master/TestMasterMetricsWrapper.java  |   17 +
 .../hbase/quotas/TestQuotaStatusRPCs.java       |   83 +
 25 files changed, 4032 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 3f27e1c..d9219f3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -94,6 +94,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
@@ -1740,6 +1742,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
           throws ServiceException {
         return stub.getSpaceQuotaRegionSizes(controller, request);
       }
+
+      @Override
+      public GetQuotaStatesResponse getQuotaStates(
+          RpcController controller, GetQuotaStatesRequest request) throws ServiceException {
+        return stub.getQuotaStates(controller, request);
+      }
     };
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
index f0f385d..af36d1e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
@@ -36,7 +37,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuo
 public class QuotaStatusCalls {
 
   /**
-   * {@link #getMasterRegionSizes(Connection, RpcControllerFactory, RpcRetryingCallerFactory, int)}
+   * See {@link #getMasterRegionSizes(Connection, RpcControllerFactory, RpcRetryingCallerFactory, int)}
    */
   public static GetSpaceQuotaRegionSizesResponse getMasterRegionSizes(
       ClusterConnection clusterConn, int timeout) throws IOException {
@@ -68,7 +69,39 @@ public class QuotaStatusCalls {
   }
 
   /**
-   * {@link #getRegionServerQuotaSnapshot(ClusterConnection, RpcControllerFactory, int, ServerName)}
+   * See {@link #getMasterQuotaStates(Connection, RpcControllerFactory, RpcRetryingCallerFactory, int)}
+   */
+  public static GetQuotaStatesResponse getMasterQuotaStates(
+      ClusterConnection clusterConn, int timeout) throws IOException {
+    RpcControllerFactory rpcController = clusterConn.getRpcControllerFactory();
+    RpcRetryingCallerFactory rpcCaller = clusterConn.getRpcRetryingCallerFactory();
+    return getMasterQuotaStates(clusterConn, rpcController, rpcCaller, timeout);
+  }
+
+  /**
+   * Executes an RPC tot he HBase master to fetch its view on space quotas.
+   */
+  public static GetQuotaStatesResponse getMasterQuotaStates(
+      Connection conn, RpcControllerFactory factory, RpcRetryingCallerFactory rpcCaller,
+      int timeout) throws IOException {
+    MasterCallable<GetQuotaStatesResponse> callable =
+        new MasterCallable<GetQuotaStatesResponse>(conn, factory) {
+      @Override
+      protected GetQuotaStatesResponse rpcCall() throws Exception {
+        return master.getQuotaStates(
+            getRpcController(), RequestConverter.buildGetQuotaStatesRequest());
+      }
+    };
+    RpcRetryingCaller<GetQuotaStatesResponse> caller = rpcCaller.newCaller();
+    try {
+      return caller.callWithoutRetries(callable, timeout);
+    } finally {
+      callable.close();
+    }
+  }
+
+  /**
+   * See {@link #getRegionServerQuotaSnapshot(ClusterConnection, RpcControllerFactory, int, ServerName)}
    */
   public static GetSpaceQuotaSnapshotsResponse getRegionServerQuotaSnapshot(
       ClusterConnection clusterConn, int timeout, ServerName sn) throws IOException {
@@ -96,7 +129,7 @@ public class QuotaStatusCalls {
   }
 
   /**
-   * {@link #getRegionServerSpaceQuotaEnforcements(ClusterConnection, RpcControllerFactory, int, ServerName)}
+   * See {@link #getRegionServerSpaceQuotaEnforcements(ClusterConnection, RpcControllerFactory, int, ServerName)}
    */
   public static GetSpaceQuotaEnforcementsResponse getRegionServerSpaceQuotaEnforcements(
       ClusterConnection clusterConn, int timeout, ServerName sn) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
index 30ae79c..9fcfbaf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
@@ -23,6 +23,8 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.*;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
@@ -485,4 +487,10 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
       GetSpaceQuotaRegionSizesRequest request) throws ServiceException {
     return stub.getSpaceQuotaRegionSizes(controller, request);
   }
+
+  @Override
+  public GetQuotaStatesResponse getQuotaStates(RpcController controller,
+      GetQuotaStatesRequest request) throws ServiceException {
+    return stub.getQuotaStates(controller, request);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/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 560f40c..ad59517 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
@@ -56,7 +56,9 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
@@ -472,6 +474,45 @@ public class QuotaTableUtil {
     return policies;
   }
 
+  /**
+   * Returns the Master's view of a quota on the given {@code tableName} or null if the
+   * Master has no quota information on that table.
+   */
+  public static SpaceQuotaSnapshot getCurrentSnapshot(
+      Connection conn, TableName tn) throws IOException {
+    if (!(conn instanceof ClusterConnection)) {
+      throw new IllegalArgumentException("Expected a ClusterConnection");
+    }
+    ClusterConnection clusterConn = (ClusterConnection) conn;
+    GetQuotaStatesResponse resp = QuotaStatusCalls.getMasterQuotaStates(clusterConn, 0);
+    HBaseProtos.TableName protoTableName = ProtobufUtil.toProtoTableName(tn);
+    for (GetQuotaStatesResponse.TableQuotaSnapshot tableSnapshot : resp.getTableSnapshotsList()) {
+      if (protoTableName.equals(tableSnapshot.getTableName())) {
+        return SpaceQuotaSnapshot.toSpaceQuotaSnapshot(tableSnapshot.getSnapshot());
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Returns the Master's view of a quota on the given {@code namespace} or null if the
+   * Master has no quota information on that namespace.
+   */
+  public static SpaceQuotaSnapshot getCurrentSnapshot(
+      Connection conn, String namespace) throws IOException {
+    if (!(conn instanceof ClusterConnection)) {
+      throw new IllegalArgumentException("Expected a ClusterConnection");
+    }
+    ClusterConnection clusterConn = (ClusterConnection) conn;
+    GetQuotaStatesResponse resp = QuotaStatusCalls.getMasterQuotaStates(clusterConn, 0);
+    for (GetQuotaStatesResponse.NamespaceQuotaSnapshot nsSnapshot : resp.getNsSnapshotsList()) {
+      if (namespace.equals(nsSnapshot.getNamespace())) {
+        return SpaceQuotaSnapshot.toSpaceQuotaSnapshot(nsSnapshot.getSnapshot());
+      }
+    }
+    return null;
+  }
+
   /* =========================================================================
    *  Quotas protobuf helpers
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index b37be68..63724bf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -116,6 +116,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormali
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
@@ -1728,4 +1729,14 @@ public final class RequestConverter {
   public static GetSpaceQuotaEnforcementsRequest buildGetSpaceQuotaEnforcementsRequest() {
     return GET_SPACE_QUOTA_ENFORCEMENTS_REQUEST;
   }
+
+  private static final GetQuotaStatesRequest GET_QUOTA_STATES_REQUEST =
+      GetQuotaStatesRequest.newBuilder().build();
+
+  /**
+   * Returns a {@link GetQuotaStatesRequest} object.
+   */
+  public static GetQuotaStatesRequest buildGetQuotaStatesRequest() {
+    return GET_QUOTA_STATES_REQUEST;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/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
new file mode 100644
index 0000000..be579be
--- /dev/null
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
@@ -0,0 +1,75 @@
+/*
+ * 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.master;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+
+/**
+ * A collection of exposed metrics for HBase quotas from the HBase Master.
+ */
+public interface MetricsMasterQuotaSource extends BaseSource {
+
+  String METRICS_NAME = "Quotas";
+  String METRICS_CONTEXT = "master";
+  String METRICS_JMX_CONTEXT = "Master,sub=" + METRICS_NAME;
+  String METRICS_DESCRIPTION = "Metrics about HBase Quotas by the Master";
+
+  String NUM_SPACE_QUOTAS_NAME = "numSpaceQuotas";
+  String NUM_SPACE_QUOTAS_DESC = "Number of space quotas defined";
+  String NUM_TABLES_QUOTA_VIOLATIONS_NAME = "numTablesInQuotaViolation";
+  String NUM_TABLES_QUOTA_VIOLATIONS_DESC = "Number of tables violating space quotas";
+  String NUM_NS_QUOTA_VIOLATIONS_NAME = "numNamespaceInQuotaViolation";
+  String NUM_NS_QUOTA_VIOLATIONS_DESC = "Number of namespaces violating space quotas";
+  String NUM_REGION_SIZE_REPORTS_NAME = "numRegionSizeReports";
+  String NUM_REGION_SIZE_REPORTS_DESC = "Number of Region sizes reported";
+  String QUOTA_OBSERVER_CHORE_TIME_NAME = "quotaObserverChoreTime";
+  String QUOTA_OBSERVER_CHORE_TIME_DESC =
+      "Histogram for the time in millis for the QuotaObserverChore";
+  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";
+  String NS_QUOTA_USAGE_DESC = "A JSON summary of the usage of all namespaces with space quotas";
+
+  /**
+   * Updates the metric tracking the number of space quotas defined in the system.
+   */
+  void updateNumSpaceQuotas(long numSpaceQuotas);
+
+  /**
+   * Updates the metric tracking the number of tables the master has computed to be in
+   * violation of their space quota.
+   */
+  void updateNumTablesInSpaceQuotaViolation(long numTablesInViolation);
+
+  /**
+   * Updates the metric tracking the number of namespaces the master has computed to be in
+   * violation of their space quota.
+   */
+  void updateNumNamespacesInSpaceQuotaViolation(long numNamespacesInViolation);
+
+  /**
+   * Updates the metric tracking the number of region size reports the master is currently
+   * retaining in memory.
+   */
+  void updateNumCurrentSpaceQuotaRegionSizeReports(long numCurrentRegionSizeReports);
+
+  /**
+   * Updates the metric tracking the amount of time taken by the {@code QuotaObserverChore}
+   * which runs periodically.
+   */
+  void incrementSpaceQuotaObserverChoreTime(long time);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactory.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactory.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactory.java
new file mode 100644
index 0000000..6e10746
--- /dev/null
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactory.java
@@ -0,0 +1,26 @@
+/*
+ * 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.master;
+
+/**
+ * Interface of a factory to create MetricsMasterQuotaSource when given a MetricsMasterWrapper.
+ */
+public interface MetricsMasterQuotaSourceFactory {
+
+  MetricsMasterQuotaSource create(MetricsMasterWrapper masterWrapper);
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
index 64e0a8a..4789283 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.hbase.master;
 
+import java.util.Map;
+import java.util.Map.Entry;
+
 /**
  * This is the interface that will expose information to hadoop1/hadoop2 implementations of the
  * MetricsMasterSource.
@@ -121,4 +124,14 @@ public interface MetricsMasterWrapper {
    * Get the number of region merge plans executed.
    */
   long getMergePlanCount();
+
+  /**
+   * Gets the space usage and limit for each table.
+   */
+  Map<String,Entry<Long,Long>> getTableSpaceUtilization();
+
+  /**
+   * Gets the space usage and limit for each namespace.
+   */
+  Map<String,Entry<Long,Long>> getNamespaceSpaceUtilization();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerQuotaSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerQuotaSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerQuotaSource.java
new file mode 100644
index 0000000..12fa66d
--- /dev/null
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerQuotaSource.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+
+/**
+ * A collection of exposed metrics for HBase quotas from an HBase RegionServer.
+ */
+public interface MetricsRegionServerQuotaSource extends BaseSource {
+
+  String METRICS_NAME = "Quotas";
+  String METRICS_CONTEXT = "regionserver";
+  String METRICS_DESCRIPTION = "Metrics about HBase RegionServer Quotas";
+  String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
+
+  /**
+   * Updates the metric tracking how many tables this RegionServer has marked as in violation
+   * of their space quota.
+   */
+  void updateNumTablesInSpaceQuotaViolation(long tablesInViolation);
+
+  /**
+   * Updates the metric tracking how many tables this RegionServer has received
+   * {@code SpaceQuotaSnapshot}s for.
+   */
+  void updateNumTableSpaceQuotaSnapshots(long numSnapshots);
+
+  /**
+   * Updates the metric tracking how much time was spent scanning the filesystem to compute
+   * the size of each region hosted by this RegionServer.
+   */
+  void incrementSpaceQuotaFileSystemScannerChoreTime(long time);
+
+  /**
+   * Updates the metric tracking how much time was spent updating the RegionServer with the
+   * lastest information on space quotas from the {@code hbase:quota} table.
+   */
+  void incrementSpaceQuotaRefresherChoreTime(long time);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactoryImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactoryImpl.java
new file mode 100644
index 0000000..96a57c4
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactoryImpl.java
@@ -0,0 +1,36 @@
+/*
+ * 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.master;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Factory to create MetricsMasterQuotaSource instances when given a MetricsMasterWrapper.
+ */
+@InterfaceAudience.Private
+public class MetricsMasterQuotaSourceFactoryImpl implements MetricsMasterQuotaSourceFactory {
+
+  private MetricsMasterQuotaSource quotaSource;
+
+  @Override
+  public synchronized MetricsMasterQuotaSource create(MetricsMasterWrapper masterWrapper) {
+    if (null == quotaSource) {
+      quotaSource = new MetricsMasterQuotaSourceImpl(masterWrapper);
+    }
+    return quotaSource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/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
new file mode 100644
index 0000000..064f7fc
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
@@ -0,0 +1,129 @@
+/*
+ * 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.master;
+
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.MetricHistogram;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+
+@InterfaceAudience.Private
+public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements MetricsMasterQuotaSource {
+  private final MetricsMasterWrapper wrapper;
+  private final MutableGaugeLong spaceQuotasGauge;
+  private final MutableGaugeLong tablesViolatingQuotasGauge;
+  private final MutableGaugeLong namespacesViolatingQuotasGauge;
+  private final MutableGaugeLong regionSpaceReportsGauge;
+  private final MetricHistogram quotaObserverTimeHisto;
+
+  public MetricsMasterQuotaSourceImpl(MetricsMasterWrapper wrapper) {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, wrapper);
+  }
+
+  public MetricsMasterQuotaSourceImpl(
+      String metricsName, String metricsDescription, String metricsContext,
+      String metricsJmxContext, MetricsMasterWrapper wrapper) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+    this.wrapper = wrapper;
+
+    spaceQuotasGauge = getMetricsRegistry().newGauge(
+        NUM_SPACE_QUOTAS_NAME, NUM_SPACE_QUOTAS_DESC, 0L);
+    tablesViolatingQuotasGauge = getMetricsRegistry().newGauge(
+        NUM_TABLES_QUOTA_VIOLATIONS_NAME, NUM_TABLES_QUOTA_VIOLATIONS_DESC, 0L);
+    namespacesViolatingQuotasGauge = getMetricsRegistry().newGauge(
+        NUM_NS_QUOTA_VIOLATIONS_NAME, NUM_NS_QUOTA_VIOLATIONS_DESC, 0L);
+    regionSpaceReportsGauge = getMetricsRegistry().newGauge(
+        NUM_REGION_SIZE_REPORTS_NAME, NUM_REGION_SIZE_REPORTS_DESC, 0L);
+
+    quotaObserverTimeHisto = getMetricsRegistry().newTimeHistogram(
+        QUOTA_OBSERVER_CHORE_TIME_NAME, QUOTA_OBSERVER_CHORE_TIME_DESC);
+  }
+
+  @Override
+  public void updateNumSpaceQuotas(long numSpaceQuotas) {
+    spaceQuotasGauge.set(numSpaceQuotas);
+  }
+
+  @Override
+  public void updateNumTablesInSpaceQuotaViolation(long numTablesInViolation) {
+    tablesViolatingQuotasGauge.set(numTablesInViolation);
+  }
+
+  @Override
+  public void updateNumNamespacesInSpaceQuotaViolation(long numNamespacesInViolation) {
+    namespacesViolatingQuotasGauge.set(numNamespacesInViolation);
+  }
+
+  @Override
+  public void updateNumCurrentSpaceQuotaRegionSizeReports(long numCurrentRegionSizeReports) {
+    regionSpaceReportsGauge.set(numCurrentRegionSizeReports);
+  }
+
+  @Override
+  public void incrementSpaceQuotaObserverChoreTime(long time) {
+    quotaObserverTimeHisto.add(time);
+  }
+
+  @Override
+  public void getMetrics(MetricsCollector metricsCollector, boolean all) {
+    MetricsRecordBuilder record = metricsCollector.addRecord(metricsRegistry.info());
+    if (null != wrapper) {
+      // Summarize the tables
+      Map<String,Entry<Long,Long>> tableUsages = wrapper.getTableSpaceUtilization();
+      String tableSummary = "[]";
+      if (null != tableUsages && !tableUsages.isEmpty()) {
+        tableSummary = generateJsonQuotaSummary(tableUsages.entrySet(), "table");
+      }
+      record.tag(Interns.info(TABLE_QUOTA_USAGE_NAME, TABLE_QUOTA_USAGE_DESC), tableSummary);
+
+      // Summarize the namespaces
+      String nsSummary = "[]";
+      Map<String,Entry<Long,Long>> namespaceUsages = wrapper.getNamespaceSpaceUtilization();
+      if (null != namespaceUsages && !namespaceUsages.isEmpty()) {
+        nsSummary = generateJsonQuotaSummary(namespaceUsages.entrySet(), "namespace");
+      }
+      record.tag(Interns.info(NS_QUOTA_USAGE_NAME, NS_QUOTA_USAGE_DESC), nsSummary);
+    }
+    metricsRegistry.snapshot(record, all);
+  }
+
+  /**
+   * Summarizes the usage and limit for many targets (table or namespace) into JSON.
+   */
+  private String generateJsonQuotaSummary(
+      Iterable<Entry<String,Entry<Long,Long>>> data, String target) {
+    StringBuilder sb = new StringBuilder();
+    for (Entry<String,Entry<Long,Long>> tableUsage : data) {
+      String tableName = tableUsage.getKey();
+      long usage = tableUsage.getValue().getKey();
+      long limit = tableUsage.getValue().getValue();
+      if (sb.length() > 0) {
+        sb.append(", ");
+      }
+      sb.append("{").append(target).append("=").append(tableName).append(", usage=").append(usage)
+          .append(", limit=").append(limit).append("}");
+    }
+    sb.insert(0, "[").append("]");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterQuotaSource
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterQuotaSource b/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterQuotaSource
new file mode 100644
index 0000000..d3891d2
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterQuotaSource
@@ -0,0 +1,18 @@
+# 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.
+#
+org.apache.hadoop.hbase.master.MetricsMasterQuotaSourceImpl

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterQuotaSourceFactory
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterQuotaSourceFactory b/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterQuotaSourceFactory
new file mode 100644
index 0000000..5bbcc8a
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterQuotaSourceFactory
@@ -0,0 +1,18 @@
+# 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.
+#
+org.apache.hadoop.hbase.master.MetricsMasterQuotaSourceFactoryImpl

http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index b81599b..be6baa4 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -71161,7 +71161,7 @@ public final class MasterProtos {
 
       /**
        * <pre>
-       ** Fetches the Master's view of space quotas 
+       ** Fetches the Master's view of space utilization 
        * </pre>
        *
        * <code>rpc GetSpaceQuotaRegionSizes(.hbase.pb.GetSpaceQuotaRegionSizesRequest) returns (.hbase.pb.GetSpaceQuotaRegionSizesResponse);</code>
@@ -71171,6 +71171,18 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse> done);
 
+      /**
+       * <pre>
+       ** Fetches the Master's view of quotas 
+       * </pre>
+       *
+       * <code>rpc GetQuotaStates(.hbase.pb.GetQuotaStatesRequest) returns (.hbase.pb.GetQuotaStatesResponse);</code>
+       */
+      public abstract void getQuotaStates(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse> done);
+
     }
 
     public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Service newReflectiveService(
@@ -71744,6 +71756,14 @@ public final class MasterProtos {
           impl.getSpaceQuotaRegionSizes(controller, request, done);
         }
 
+        @java.lang.Override
+        public  void getQuotaStates(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse> done) {
+          impl.getQuotaStates(controller, request, done);
+        }
+
       };
     }
 
@@ -71908,6 +71928,8 @@ public final class MasterProtos {
               return impl.removeDrainFromRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest)request);
             case 70:
               return impl.getSpaceQuotaRegionSizes(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest)request);
+            case 71:
+              return impl.getQuotaStates(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -72064,6 +72086,8 @@ public final class MasterProtos {
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest.getDefaultInstance();
             case 70:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest.getDefaultInstance();
+            case 71:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -72220,6 +72244,8 @@ public final class MasterProtos {
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance();
             case 70:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.getDefaultInstance();
+            case 71:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -73101,7 +73127,7 @@ public final class MasterProtos {
 
     /**
      * <pre>
-     ** Fetches the Master's view of space quotas 
+     ** Fetches the Master's view of space utilization 
      * </pre>
      *
      * <code>rpc GetSpaceQuotaRegionSizes(.hbase.pb.GetSpaceQuotaRegionSizesRequest) returns (.hbase.pb.GetSpaceQuotaRegionSizesResponse);</code>
@@ -73111,6 +73137,18 @@ public final class MasterProtos {
         org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest request,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse> done);
 
+    /**
+     * <pre>
+     ** Fetches the Master's view of quotas 
+     * </pre>
+     *
+     * <code>rpc GetQuotaStates(.hbase.pb.GetQuotaStatesRequest) returns (.hbase.pb.GetQuotaStatesResponse);</code>
+     */
+    public abstract void getQuotaStates(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest request,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse> done);
+
     public static final
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
@@ -73488,6 +73526,11 @@ public final class MasterProtos {
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse>specializeCallback(
               done));
           return;
+        case 71:
+          this.getQuotaStates(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest)request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse>specializeCallback(
+              done));
+          return;
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -73644,6 +73687,8 @@ public final class MasterProtos {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest.getDefaultInstance();
         case 70:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest.getDefaultInstance();
+        case 71:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -73800,6 +73845,8 @@ public final class MasterProtos {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance();
         case 70:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.getDefaultInstance();
+        case 71:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -74885,6 +74932,21 @@ public final class MasterProtos {
             org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.getDefaultInstance()));
       }
+
+      public  void getQuotaStates(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(71),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.getDefaultInstance(),
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.class,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.getDefaultInstance()));
+      }
     }
 
     public static BlockingInterface newBlockingStub(
@@ -75247,6 +75309,11 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse getQuotaStates(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
@@ -76107,6 +76174,18 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.getDefaultInstance());
       }
 
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse getQuotaStates(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(71),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.getDefaultInstance());
+      }
+
     }
 
     // @@protoc_insertion_point(class_scope:hbase.pb.MasterService)
@@ -76948,7 +77027,7 @@ public final class MasterProtos {
       "veDrainFromRegionServersRequest\022)\n\013serve",
       "r_name\030\001 \003(\0132\024.hbase.pb.ServerName\"&\n$Re" +
       "moveDrainFromRegionServersResponse*(\n\020Ma" +
-      "sterSwitchType\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\2644\n" +
+      "sterSwitchType\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\2115\n" +
       "\rMasterService\022e\n\024GetSchemaAlterStatus\022%" +
       ".hbase.pb.GetSchemaAlterStatusRequest\032&." +
       "hbase.pb.GetSchemaAlterStatusResponse\022b\n" +
@@ -77116,9 +77195,11 @@ public final class MasterProtos {
       "veDrainFromRegionServersResponse\022q\n\030GetS" +
       "paceQuotaRegionSizes\022).hbase.pb.GetSpace" +
       "QuotaRegionSizesRequest\032*.hbase.pb.GetSp",
-      "aceQuotaRegionSizesResponseBI\n1org.apach" +
-      "e.hadoop.hbase.shaded.protobuf.generated" +
-      "B\014MasterProtosH\001\210\001\001\240\001\001"
+      "aceQuotaRegionSizesResponse\022S\n\016GetQuotaS" +
+      "tates\022\037.hbase.pb.GetQuotaStatesRequest\032 " +
+      ".hbase.pb.GetQuotaStatesResponseBI\n1org." +
+      "apache.hadoop.hbase.shaded.protobuf.gene" +
+      "ratedB\014MasterProtosH\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {


[10/50] [abbrv] hbase git commit: HBASE-17366 Run TestHFile#testReaderWithoutBlockCache failes

Posted by el...@apache.org.
HBASE-17366 Run TestHFile#testReaderWithoutBlockCache failes

Signed-off-by: CHIA-PING TSAI <ch...@gmail.com>


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

Branch: refs/heads/HBASE-16961
Commit: c1ac3f7739f8c9e20f6aed428558128339467d04
Parents: 363f627
Author: huaxiang sun <hs...@cloudera.com>
Authored: Mon Apr 17 10:32:17 2017 +0800
Committer: CHIA-PING TSAI <ch...@gmail.com>
Committed: Mon Apr 17 10:34:17 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/StoreFileWriter.java   | 9 +++++++++
 .../java/org/apache/hadoop/hbase/io/hfile/TestHFile.java    | 2 +-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c1ac3f77/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
index ccfd735..88cba75 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
@@ -384,6 +384,15 @@ public class StoreFileWriter implements CellSink, ShipperListener {
     }
 
     /**
+     * Creates Builder with cache configuration disabled
+     */
+    public Builder(Configuration conf, FileSystem fs) {
+      this.conf = conf;
+      this.cacheConf = CacheConfig.DISABLED;
+      this.fs = fs;
+    }
+
+    /**
      * @param trt A premade TimeRangeTracker to use rather than build one per append (building one
      * of these is expensive so good to pass one in if you have one).
      * @return this (for chained invocation)

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1ac3f77/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
index 7074c9d..4db459a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
@@ -115,7 +115,7 @@ public class TestHFile  {
     Path storeFileParentDir = new Path(TEST_UTIL.getDataTestDir(), "TestHFile");
     HFileContext meta = new HFileContextBuilder().withBlockSize(64 * 1024).build();
     StoreFileWriter sfw =
-        new StoreFileWriter.Builder(conf, cacheConf, fs).withOutputDir(storeFileParentDir)
+        new StoreFileWriter.Builder(conf, fs).withOutputDir(storeFileParentDir)
             .withComparator(CellComparator.COMPARATOR).withFileContext(meta).build();
 
     final int rowLen = 32;


[16/50] [abbrv] hbase git commit: HBASE-17557 HRegionServer#reportRegionSizesForQuotas() should respond to UnsupportedOperationException

Posted by el...@apache.org.
HBASE-17557 HRegionServer#reportRegionSizesForQuotas() should respond to UnsupportedOperationException


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

Branch: refs/heads/HBASE-16961
Commit: d77ec498c23de9c75390237ce028fe6e32f3c8b3
Parents: 2dea676
Author: tedyu <yu...@gmail.com>
Authored: Mon Jan 30 07:47:40 2017 -0800
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:31 2017 -0400

----------------------------------------------------------------------
 .../quotas/FileSystemUtilizationChore.java      | 20 +++++++++++++---
 .../hbase/regionserver/HRegionServer.java       | 24 ++++++++++++++++----
 2 files changed, 36 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d77ec498/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 01540eb..efc17ff 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
@@ -53,6 +53,9 @@ public class FileSystemUtilizationChore extends ScheduledChore {
   static final String FS_UTILIZATION_MAX_ITERATION_DURATION_KEY = "hbase.regionserver.quotas.fs.utilization.chore.max.iteration.millis";
   static final long FS_UTILIZATION_MAX_ITERATION_DURATION_DEFAULT = 5000L;
 
+  private int numberOfCyclesToSkip = 0, prevNumberOfCyclesToSkip = 0;
+  private static final int CYCLE_UPPER_BOUND = 32;
+
   private final HRegionServer rs;
   private final long maxIterationMillis;
   private Iterator<Region> leftoverRegions;
@@ -67,6 +70,10 @@ public class FileSystemUtilizationChore extends ScheduledChore {
 
   @Override
   protected void chore() {
+    if (numberOfCyclesToSkip > 0) {
+      numberOfCyclesToSkip--;
+      return;
+    }
     final Map<HRegionInfo,Long> onlineRegionSizes = new HashMap<>();
     final Set<Region> onlineRegions = new HashSet<>(rs.getOnlineRegions());
     // Process the regions from the last run if we have any. If we are somehow having difficulty
@@ -126,7 +133,14 @@ public class FileSystemUtilizationChore extends ScheduledChore {
           + skippedSplitParents + " regions due to being the parent of a split, and"
           + skippedRegionReplicas + " regions due to being region replicas.");
     }
-    reportRegionSizesToMaster(onlineRegionSizes);
+    if (!reportRegionSizesToMaster(onlineRegionSizes)) {
+      // backoff reporting
+      numberOfCyclesToSkip = prevNumberOfCyclesToSkip > 0 ? 2 * prevNumberOfCyclesToSkip : 1;
+      if (numberOfCyclesToSkip > CYCLE_UPPER_BOUND) {
+        numberOfCyclesToSkip = CYCLE_UPPER_BOUND;
+      }
+      prevNumberOfCyclesToSkip = numberOfCyclesToSkip;
+    }
   }
 
   /**
@@ -166,8 +180,8 @@ public class FileSystemUtilizationChore extends ScheduledChore {
    *
    * @param onlineRegionSizes The computed region sizes to report.
    */
-  void reportRegionSizesToMaster(Map<HRegionInfo,Long> onlineRegionSizes) {
-    this.rs.reportRegionSizesForQuotas(onlineRegionSizes);
+  boolean reportRegionSizesToMaster(Map<HRegionInfo,Long> onlineRegionSizes) {
+    return this.rs.reportRegionSizesForQuotas(onlineRegionSizes);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d77ec498/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 9be4131..053e4ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.ClockOutOfSyncException;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
@@ -1248,13 +1249,14 @@ public class HRegionServer extends HasThread implements
    * Reports the given map of Regions and their size on the filesystem to the active Master.
    *
    * @param onlineRegionSizes A map of region info to size in bytes
+   * @return false if FileSystemUtilizationChore should pause reporting to master. true otherwise
    */
-  public void reportRegionSizesForQuotas(final Map<HRegionInfo, Long> onlineRegionSizes) {
+  public boolean reportRegionSizesForQuotas(final Map<HRegionInfo, Long> onlineRegionSizes) {
     RegionServerStatusService.BlockingInterface rss = rssStub;
     if (rss == null) {
       // the current server could be stopping.
       LOG.trace("Skipping Region size report to HMaster as stub is null");
-      return;
+      return true;
     }
     try {
       RegionSpaceUseReportRequest request = buildRegionSpaceUseReportRequest(
@@ -1263,16 +1265,28 @@ public class HRegionServer extends HasThread implements
     } catch (ServiceException se) {
       IOException ioe = ProtobufUtil.getRemoteException(se);
       if (ioe instanceof PleaseHoldException) {
-        LOG.trace("Failed to report region sizes to Master because it is initializing. This will be retried.", ioe);
+        LOG.trace("Failed to report region sizes to Master because it is initializing."
+            + " This will be retried.", ioe);
         // The Master is coming up. Will retry the report later. Avoid re-creating the stub.
-        return;
+        return true;
       }
-      LOG.debug("Failed to report region sizes to Master. This will be retried.", ioe);
       if (rssStub == rss) {
         rssStub = null;
       }
       createRegionServerStatusStub(true);
+      if (ioe instanceof DoNotRetryIOException) {
+        DoNotRetryIOException doNotRetryEx = (DoNotRetryIOException) ioe;
+        if (doNotRetryEx.getCause() != null) {
+          Throwable t = doNotRetryEx.getCause();
+          if (t instanceof UnsupportedOperationException) {
+            LOG.debug("master doesn't support ReportRegionSpaceUse, pause before retrying");
+            return false;
+          }
+        }
+      }
+      LOG.debug("Failed to report region sizes to Master. This will be retried.", ioe);
     }
+    return true;
   }
 
   /**


[11/50] [abbrv] hbase git commit: HBASE-16875 Changed try-with-resources in the docs to recommended way

Posted by el...@apache.org.
HBASE-16875 Changed try-with-resources in the docs to recommended way

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/HBASE-16961
Commit: c8cd921bededa67b2b0de823005830d750534d93
Parents: c1ac3f7
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Sat Mar 4 10:04:02 2017 +0100
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Mon Apr 17 10:59:46 2017 +0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/architecture.adoc |  7 +++---
 src/main/asciidoc/_chapters/security.adoc     | 29 ++++++++--------------
 2 files changed, 13 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c8cd921b/src/main/asciidoc/_chapters/architecture.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 27aebd9..7f9ba07 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -219,10 +219,9 @@ For applications which require high-end multithreaded access (e.g., web-servers
 ----
 // Create a connection to the cluster.
 Configuration conf = HBaseConfiguration.create();
-try (Connection connection = ConnectionFactory.createConnection(conf)) {
-  try (Table table = connection.getTable(TableName.valueOf(tablename)) {
-    // use table as needed, the table returned is lightweight
-  }
+try (Connection connection = ConnectionFactory.createConnection(conf);
+     Table table = connection.getTable(TableName.valueOf(tablename))) {
+  // use table as needed, the table returned is lightweight
 }
 ----
 ====

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8cd921b/src/main/asciidoc/_chapters/security.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/security.adoc b/src/main/asciidoc/_chapters/security.adoc
index 0ed9ba2..ccb5adb 100644
--- a/src/main/asciidoc/_chapters/security.adoc
+++ b/src/main/asciidoc/_chapters/security.adoc
@@ -202,10 +202,9 @@ Set it in the `Configuration` supplied to `Table`:
 Configuration conf = HBaseConfiguration.create();
 Connection connection = ConnectionFactory.createConnection(conf);
 conf.set("hbase.rpc.protection", "privacy");
-try (Connection connection = ConnectionFactory.createConnection(conf)) {
-  try (Table table = connection.getTable(TableName.valueOf(tablename)) {
+try (Connection connection = ConnectionFactory.createConnection(conf);
+     Table table = connection.getTable(TableName.valueOf(tablename))) {
   .... do your stuff
-  }
 }
 ----
 
@@ -1014,24 +1013,16 @@ public static void grantOnTable(final HBaseTestingUtility util, final String use
   SecureTestUtil.updateACLs(util, new Callable<Void>() {
     @Override
     public Void call() throws Exception {
-      Configuration conf = HBaseConfiguration.create();
-      Connection connection = ConnectionFactory.createConnection(conf);
-      try (Connection connection = ConnectionFactory.createConnection(conf)) {
-        try (Table table = connection.getTable(TableName.valueOf(tablename)) {
-          AccessControlLists.ACL_TABLE_NAME);
-          try {
-            BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
-            AccessControlService.BlockingInterface protocol =
-                AccessControlService.newBlockingStub(service);
-            ProtobufUtil.grant(protocol, user, table, family, qualifier, actions);
-          } finally {
-            acl.close();
-          }
-          return null;
-        }
+      try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration());
+           Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) {
+        BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
+        AccessControlService.BlockingInterface protocol =
+          AccessControlService.newBlockingStub(service);
+        AccessControlUtil.grant(null, protocol, user, table, family, qualifier, false, actions);
       }
+      return null;
     }
-  }
+  });
 }
 ----
 


[12/50] [abbrv] hbase git commit: HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index d56d6ec..095f4bd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -116,6 +116,7 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterAllFilter;
 import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.NullComparator;
@@ -4931,6 +4932,7 @@ public class TestHRegion {
       String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
       throws IOException {
     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
     final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);
     return initHRegion(tableName, startKey, stopKey, isReadOnly,

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index 0054642..6eed7df 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -153,7 +153,7 @@ public class TestHRegionReplayEvents {
     }
 
     time = System.currentTimeMillis();
-
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     primaryHri = new HRegionInfo(htd.getTableName(),
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
       false, time, 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
index 37a7664..1768801 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
@@ -48,30 +48,30 @@ import static org.junit.Assert.assertTrue;
 @Category({RegionServerTests.class, SmallTests.class})
 public class TestMemStoreChunkPool {
   private final static Configuration conf = new Configuration();
-  private static MemStoreChunkPool chunkPool;
+  private static ChunkCreator chunkCreator;
   private static boolean chunkPoolDisabledBeforeTest;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
     conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
-    chunkPoolDisabledBeforeTest = MemStoreChunkPool.chunkPoolDisabled;
-    MemStoreChunkPool.chunkPoolDisabled = false;
+    chunkPoolDisabledBeforeTest = ChunkCreator.chunkPoolDisabled;
+    ChunkCreator.chunkPoolDisabled = false;
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    chunkPool = MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f,
-        MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
-    assertTrue(chunkPool != null);
+    chunkCreator = ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
+      globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
+    assertTrue(chunkCreator != null);
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    MemStoreChunkPool.chunkPoolDisabled = chunkPoolDisabledBeforeTest;
+    ChunkCreator.chunkPoolDisabled = chunkPoolDisabledBeforeTest;
   }
 
   @Before
   public void tearDown() throws Exception {
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
   }
 
   @Test
@@ -90,7 +90,7 @@ public class TestMemStoreChunkPool {
       int size = KeyValueUtil.length(kv);
       ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
       if (newKv.getBuffer() != lastBuffer) {
-        expectedOff = 0;
+        expectedOff = 8;
         lastBuffer = newKv.getBuffer();
       }
       assertEquals(expectedOff, newKv.getOffset());
@@ -100,14 +100,14 @@ public class TestMemStoreChunkPool {
     }
     // chunks will be put back to pool after close
     mslab.close();
-    int chunkCount = chunkPool.getPoolSize();
+    int chunkCount = chunkCreator.getPoolSize();
     assertTrue(chunkCount > 0);
     // reconstruct mslab
     mslab = new MemStoreLABImpl(conf);
     // chunk should be got from the pool, so we can reuse it.
     KeyValue kv = new KeyValue(rk, cf, q, new byte[10]);
     mslab.copyCellInto(kv);
-    assertEquals(chunkCount - 1, chunkPool.getPoolSize());
+    assertEquals(chunkCount - 1, chunkCreator.getPoolSize());
   }
 
   @Test
@@ -143,7 +143,7 @@ public class TestMemStoreChunkPool {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkPool.getPoolSize();
+    int chunkCount = chunkCreator.getPoolSize();
     assertTrue(chunkCount > 0);
 
   }
@@ -189,16 +189,16 @@ public class TestMemStoreChunkPool {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkPool.getPoolSize() == 0);
+    assertTrue(chunkCreator.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
 
     // clear chunks
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
 
     // Creating another snapshot
     snapshot = memstore.snapshot();
@@ -218,20 +218,20 @@ public class TestMemStoreChunkPool {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
   }
 
   @Test
   public void testPutbackChunksMultiThreaded() throws Exception {
-    MemStoreChunkPool oldPool = MemStoreChunkPool.GLOBAL_INSTANCE;
     final int maxCount = 10;
     final int initialCount = 5;
-    final int chunkSize = 30;
+    final int chunkSize = 40;
     final int valSize = 7;
-    MemStoreChunkPool pool = new MemStoreChunkPool(chunkSize, maxCount, initialCount, 1, false);
-    assertEquals(initialCount, pool.getPoolSize());
-    assertEquals(maxCount, pool.getMaxCount());
-    MemStoreChunkPool.GLOBAL_INSTANCE = pool;// Replace the global ref with the new one we created.
+    ChunkCreator oldCreator = ChunkCreator.getInstance();
+    ChunkCreator newCreator = new ChunkCreator(chunkSize, false, 400, 1, 0.5f, null);
+    assertEquals(initialCount, newCreator.getPoolSize());
+    assertEquals(maxCount, newCreator.getMaxCount());
+    ChunkCreator.INSTANCE = newCreator;// Replace the global ref with the new one we created.
                                              // Used it for the testing. Later in finally we put
                                              // back the original
     final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
@@ -258,9 +258,9 @@ public class TestMemStoreChunkPool {
       t1.join();
       t2.join();
       t3.join();
-      assertTrue(pool.getPoolSize() <= maxCount);
+      assertTrue(newCreator.getPoolSize() <= maxCount);
     } finally {
-      MemStoreChunkPool.GLOBAL_INSTANCE = oldPool;
+      ChunkCreator.INSTANCE = oldCreator;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
index 141b802..6696e43 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
@@ -63,8 +63,8 @@ public class TestMemStoreLAB {
   public static void setUpBeforeClass() throws Exception {
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT,
-        MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, globalMemStoreLimit,
+      0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
   }
 
   /**
@@ -76,6 +76,7 @@ public class TestMemStoreLAB {
     MemStoreLAB mslab = new MemStoreLABImpl();
     int expectedOff = 0;
     ByteBuffer lastBuffer = null;
+    long lastChunkId = -1;
     // 100K iterations by 0-1K alloc -> 50MB expected
     // should be reasonable for unit test and also cover wraparound
     // behavior
@@ -85,8 +86,13 @@ public class TestMemStoreLAB {
       int size = KeyValueUtil.length(kv);
       ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
       if (newKv.getBuffer() != lastBuffer) {
-        expectedOff = 0;
+        // since we add the chunkID at the 0th offset of the chunk and the
+        // chunkid is a long we need to account for those 8 bytes
+        expectedOff = Bytes.SIZEOF_LONG;
         lastBuffer = newKv.getBuffer();
+        long chunkId = newKv.getBuffer().getLong(0);
+        assertTrue("chunkid should be different", chunkId != lastChunkId);
+        lastChunkId = chunkId;
       }
       assertEquals(expectedOff, newKv.getOffset());
       assertTrue("Allocation overruns buffer",
@@ -136,23 +142,21 @@ public class TestMemStoreLAB {
       };
       ctx.addThread(t);
     }
-    
+
     ctx.startThreads();
     while (totalAllocated.get() < 50*1024*1024 && ctx.shouldRun()) {
       Thread.sleep(10);
     }
     ctx.stop();
-    
     // Partition the allocations by the actual byte[] they point into,
     // make sure offsets are unique for each chunk
     Map<ByteBuffer, Map<Integer, AllocRecord>> mapsByChunk =
       Maps.newHashMap();
-    
+
     int sizeCounted = 0;
     for (AllocRecord rec : Iterables.concat(allocations)) {
       sizeCounted += rec.size;
       if (rec.size == 0) continue;
-      
       Map<Integer, AllocRecord> mapForThisByteArray =
         mapsByChunk.get(rec.alloc);
       if (mapForThisByteArray == null) {
@@ -167,7 +171,9 @@ public class TestMemStoreLAB {
     
     // Now check each byte array to make sure allocations don't overlap
     for (Map<Integer, AllocRecord> allocsInChunk : mapsByChunk.values()) {
-      int expectedOff = 0;
+      // since we add the chunkID at the 0th offset of the chunk and the
+      // chunkid is a long we need to account for those 8 bytes
+      int expectedOff = Bytes.SIZEOF_LONG;
       for (AllocRecord alloc : allocsInChunk.values()) {
         assertEquals(expectedOff, alloc.offset);
         assertTrue("Allocation overruns buffer",
@@ -175,7 +181,6 @@ public class TestMemStoreLAB {
         expectedOff += alloc.size;
       }
     }
-
   }
 
   /**
@@ -194,7 +199,7 @@ public class TestMemStoreLAB {
     // set chunk size to default max alloc size, so we could easily trigger chunk retirement
     conf.setLong(MemStoreLABImpl.CHUNK_SIZE_KEY, MemStoreLABImpl.MAX_ALLOC_DEFAULT);
     // reconstruct mslab
-    MemStoreChunkPool.clearDisableFlag();
+    ChunkCreator.clearDisableFlag();
     mslab = new MemStoreLABImpl(conf);
     // launch multiple threads to trigger frequent chunk retirement
     List<Thread> threads = new ArrayList<>();
@@ -223,6 +228,8 @@ public class TestMemStoreLAB {
     }
     // close the mslab
     mslab.close();
+    // none of the chunkIds would have been returned back
+    assertTrue("All the chunks must have been cleared", ChunkCreator.INSTANCE.size() != 0);
     // make sure all chunks reclaimed or removed from chunk queue
     int queueLength = mslab.getPooledChunks().size();
     assertTrue("All chunks in chunk queue should be reclaimed or removed"

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
new file mode 100644
index 0000000..f38a75e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
@@ -0,0 +1,168 @@
+/**
+ *
+ * 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.regionserver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.lang.management.ManagementFactory;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({RegionServerTests.class, SmallTests.class})
+public class TestMemstoreLABWithoutPool {
+  private final static Configuration conf = new Configuration();
+
+  private static final byte[] rk = Bytes.toBytes("r1");
+  private static final byte[] cf = Bytes.toBytes("f");
+  private static final byte[] q = Bytes.toBytes("q");
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
+        .getMax() * 0.8);
+    // disable pool
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT + Bytes.SIZEOF_LONG, false, globalMemStoreLimit,
+      0.0f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
+  }
+
+  /**
+   * Test a bunch of random allocations
+   */
+  @Test
+  public void testLABRandomAllocation() {
+    Random rand = new Random();
+    MemStoreLAB mslab = new MemStoreLABImpl();
+    int expectedOff = 0;
+    ByteBuffer lastBuffer = null;
+    long lastChunkId = -1;
+    // 100K iterations by 0-1K alloc -> 50MB expected
+    // should be reasonable for unit test and also cover wraparound
+    // behavior
+    for (int i = 0; i < 100000; i++) {
+      int valSize = rand.nextInt(1000);
+      KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
+      int size = KeyValueUtil.length(kv);
+      ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
+      if (newKv.getBuffer() != lastBuffer) {
+        // since we add the chunkID at the 0th offset of the chunk and the
+        // chunkid is a long we need to account for those 8 bytes
+        expectedOff = Bytes.SIZEOF_LONG;
+        lastBuffer = newKv.getBuffer();
+        long chunkId = newKv.getBuffer().getLong(0);
+        assertTrue("chunkid should be different", chunkId != lastChunkId);
+        lastChunkId = chunkId;
+      }
+      assertEquals(expectedOff, newKv.getOffset());
+      assertTrue("Allocation overruns buffer",
+          newKv.getOffset() + size <= newKv.getBuffer().capacity());
+      expectedOff += size;
+    }
+  }
+
+  /**
+   * Test frequent chunk retirement with chunk pool triggered by lots of threads, making sure
+   * there's no memory leak (HBASE-16195)
+   * @throws Exception if any error occurred
+   */
+  @Test
+  public void testLABChunkQueueWithMultipleMSLABs() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    MemStoreLABImpl[] mslab = new MemStoreLABImpl[10];
+    for (int i = 0; i < 10; i++) {
+      mslab[i] = new MemStoreLABImpl(conf);
+    }
+    // launch multiple threads to trigger frequent chunk retirement
+    List<Thread> threads = new ArrayList<>();
+    // create smaller sized kvs
+    final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
+        new byte[0]);
+    for (int i = 0; i < 10; i++) {
+      for (int j = 0; j < 10; j++) {
+        threads.add(getChunkQueueTestThread(mslab[i], "testLABChunkQueue-" + j, kv));
+      }
+    }
+    for (Thread thread : threads) {
+      thread.start();
+    }
+    // let it run for some time
+    Thread.sleep(3000);
+    for (Thread thread : threads) {
+      thread.interrupt();
+    }
+    boolean threadsRunning = true;
+    boolean alive = false;
+    while (threadsRunning) {
+      alive = false;
+      for (Thread thread : threads) {
+        if (thread.isAlive()) {
+          alive = true;
+          break;
+        }
+      }
+      if (!alive) {
+        threadsRunning = false;
+      }
+    }
+    // close the mslab
+    for (int i = 0; i < 10; i++) {
+      mslab[i].close();
+    }
+    // all of the chunkIds would have been returned back
+    assertTrue("All the chunks must have been cleared", ChunkCreator.INSTANCE.size() == 0);
+  }
+
+  private Thread getChunkQueueTestThread(final MemStoreLABImpl mslab, String threadName,
+      Cell cellToCopyInto) {
+    Thread thread = new Thread() {
+      boolean stopped = false;
+
+      @Override
+      public void run() {
+        while (!stopped) {
+          // keep triggering chunk retirement
+          mslab.copyCellInto(cellToCopyInto);
+        }
+      }
+
+      @Override
+      public void interrupt() {
+        this.stopped = true;
+      }
+    };
+    thread.setName(threadName);
+    thread.setDaemon(true);
+    return thread;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
index 4315bd4..7160e5e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
@@ -108,6 +108,7 @@ public class TestRecoveredEdits {
       }
     };
     Path hbaseRootDir = TEST_UTIL.getDataTestDir();
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
     Path tableDir = FSUtils.getTableDir(hbaseRootDir, htd.getTableName());
     HRegionFileSystem hrfs =

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
index 5d11c0e..ad56081 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
@@ -83,6 +83,7 @@ public class TestRegionIncrement {
   private HRegion getRegion(final Configuration conf, final String tableName) throws IOException {
     WAL wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDataTestDir(),
       TEST_UTIL.getDataTestDir().toString(), conf);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return (HRegion)TEST_UTIL.createLocalHRegion(Bytes.toBytes(tableName),
       HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, tableName, conf,
       false, Durability.SKIP_WAL, wal, INCREMENT_BYTES);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
index 0d339b1..5355c77 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
@@ -178,6 +178,7 @@ public class TestStore {
     } else {
       htd.addFamily(hcd);
     }
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
index 3cdb227..99dd00d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
@@ -111,6 +111,7 @@ public class TestStoreFileRefresherChore {
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + replicaId);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region =
         new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()),
             conf, htd, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 4f247b0..51260a6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -588,6 +588,7 @@ public class TestWALLockup {
    */
   public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
   throws IOException {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.SYNC_WAL,
       wal, COLUMN_FAMILY_BYTES);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
index 994779f..e63bad9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
@@ -98,6 +98,7 @@ public class TestWALMonotonicallyIncreasingSeqId {
     FSUtils.setRootDir(walConf, tableDir);
     this.walConf = walConf;
     wals = new WALFactory(walConf, null, "log_" + replicaId);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region = new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(),
         info.getTable().getNamespace()), conf, htd, null);
     region.initialize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
index f976b49..057b9bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
@@ -37,7 +37,9 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -288,6 +290,7 @@ public class TestDurability {
           throw new IOException("Failed delete of " + path);
         }
       }
+      ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
       return HRegion.createHRegion(info, path, CONF, htd, log);
     }
 


[39/50] [abbrv] hbase git commit: HBASE-17602 Reduce some quota chore periods/delays

Posted by el...@apache.org.
HBASE-17602 Reduce some quota chore periods/delays


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

Branch: refs/heads/HBASE-16961
Commit: 4db44ad66497729118b695e22fd2e0f4c7787cc9
Parents: 48332ee
Author: Josh Elser <el...@apache.org>
Authored: Tue Feb 7 11:21:08 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:44:00 2017 -0400

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java  | 4 ++--
 .../org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java | 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4db44ad6/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 b9f4592..7f894e4 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
@@ -55,11 +55,11 @@ public class QuotaObserverChore extends ScheduledChore {
   private static final Log LOG = LogFactory.getLog(QuotaObserverChore.class);
   static final String QUOTA_OBSERVER_CHORE_PERIOD_KEY =
       "hbase.master.quotas.observer.chore.period";
-  static final int QUOTA_OBSERVER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
+  static final int QUOTA_OBSERVER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 1; // 1 minutes in millis
 
   static final String QUOTA_OBSERVER_CHORE_DELAY_KEY =
       "hbase.master.quotas.observer.chore.delay";
-  static final long QUOTA_OBSERVER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
+  static final long QUOTA_OBSERVER_CHORE_DELAY_DEFAULT = 1000L * 15L; // 15 seconds in millis
 
   static final String QUOTA_OBSERVER_CHORE_TIMEUNIT_KEY =
       "hbase.master.quotas.observer.chore.timeunit";

http://git-wip-us.apache.org/repos/asf/hbase/blob/4db44ad6/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
index e1a2693..8587e79 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
@@ -44,11 +44,11 @@ public class SpaceQuotaRefresherChore extends ScheduledChore {
 
   static final String POLICY_REFRESHER_CHORE_PERIOD_KEY =
       "hbase.regionserver.quotas.policy.refresher.chore.period";
-  static final int POLICY_REFRESHER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
+  static final int POLICY_REFRESHER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 1; // 1 minute in millis
 
   static final String POLICY_REFRESHER_CHORE_DELAY_KEY =
       "hbase.regionserver.quotas.policy.refresher.chore.delay";
-  static final long POLICY_REFRESHER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
+  static final long POLICY_REFRESHER_CHORE_DELAY_DEFAULT = 1000L * 15L; // 15 seconds in millis
 
   static final String POLICY_REFRESHER_CHORE_TIMEUNIT_KEY =
       "hbase.regionserver.quotas.policy.refresher.chore.timeunit";


[47/50] [abbrv] hbase git commit: HBASE-17794 Swap "violation" for "snapshot" where appropriate

Posted by el...@apache.org.
HBASE-17794 Swap "violation" for "snapshot" where appropriate

A couple of variables and comments in which violation is incorrectly
used to describe what the code is doing. This was a hold over from early
implementation -- need to scrub these out for clarity.


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

Branch: refs/heads/HBASE-16961
Commit: abe1c065ab803ba60fa28b17432746e756c31ed7
Parents: 5b3926b
Author: Josh Elser <el...@apache.org>
Authored: Thu Mar 16 19:26:14 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:47:49 2017 -0400

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java    | 4 ++--
 hbase-protocol-shaded/src/main/protobuf/Quota.proto            | 2 +-
 .../org/apache/hadoop/hbase/quotas/QuotaObserverChore.java     | 6 +++---
 .../apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java    | 2 +-
 4 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/abe1c065/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 ad59517..c008702 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
@@ -228,7 +228,7 @@ public class QuotaTableUtil {
   }
 
   /**
-   * Creates a {@link Scan} which returns only quota violations from the quota table.
+   * Creates a {@link Scan} which returns only quota snapshots from the quota table.
    */
   public static Scan makeQuotaSnapshotScan() {
     Scan s = new Scan();
@@ -246,7 +246,7 @@ public class QuotaTableUtil {
    * will throw an {@link IllegalArgumentException}.
    *
    * @param result A row from the quota table.
-   * @param snapshots A map of violations to add the result of this method into.
+   * @param snapshots A map of snapshots to add the result of this method into.
    */
   public static void extractQuotaSnapshot(
       Result result, Map<TableName,SpaceQuotaSnapshot> snapshots) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/abe1c065/hbase-protocol-shaded/src/main/protobuf/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Quota.proto b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
index 1a6d5ed..364c58b 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Quota.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
@@ -98,7 +98,7 @@ message SpaceLimitRequest {
 }
 
 // Represents the state of a quota on a table. Either the quota is not in violation
-// or it is in violatino there is a violation policy which should be in effect.
+// or it is in violation there is a violation policy which should be in effect.
 message SpaceQuotaStatus {
   optional SpaceViolationPolicy policy = 1;
   optional bool in_violation = 2;

http://git-wip-us.apache.org/repos/asf/hbase/blob/abe1c065/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 94c5c87..254f2a1 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
@@ -532,9 +532,9 @@ public class QuotaObserverChore extends ScheduledChore {
   }
 
   /**
-   * Stores the quota violation state for the given table.
+   * Stores the quota state for the given table.
    */
-  void setTableQuotaViolation(TableName table, SpaceQuotaSnapshot snapshot) {
+  void setTableQuotaSnapshot(TableName table, SpaceQuotaSnapshot snapshot) {
     this.tableQuotaSnapshots.put(table, snapshot);
   }
 
@@ -552,7 +552,7 @@ public class QuotaObserverChore extends ScheduledChore {
   }
 
   /**
-   * Stores the quota violation state for the given namespace.
+   * Stores the quota state for the given namespace.
    */
   void setNamespaceQuotaSnapshot(String namespace, SpaceQuotaSnapshot snapshot) {
     this.namespaceQuotaSnapshots.put(namespace, snapshot);

http://git-wip-us.apache.org/repos/asf/hbase/blob/abe1c065/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 e196354..82d3684 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
@@ -112,7 +112,7 @@ public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
   @Override
   public void setCurrentState(TableName table, SpaceQuotaSnapshot snapshot) {
     // Defer the "current state" to the chore
-    this.chore.setTableQuotaViolation(table, snapshot);
+    this.chore.setTableQuotaSnapshot(table, snapshot);
   }
 
   @Override


[36/50] [abbrv] hbase git commit: HBASE-16999 Implement master and regionserver synchronization of quota state

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/dccfc846/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java
new file mode 100644
index 0000000..4a7000c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.NavigableMap;
+import java.util.Objects;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentMatcher;
+
+/**
+ * Test case for {@link TableSpaceQuotaViolationNotifier}.
+ */
+@Category(SmallTests.class)
+public class TestTableSpaceQuotaViolationNotifier {
+
+  private TableSpaceQuotaViolationNotifier notifier;
+  private Connection conn;
+
+  @Before
+  public void setup() throws Exception {
+    notifier = new TableSpaceQuotaViolationNotifier();
+    conn = mock(Connection.class);
+    notifier.initialize(conn);
+  }
+
+  @Test
+  public void testToViolation() throws Exception {
+    final TableName tn = TableName.valueOf("inviolation");
+    final SpaceViolationPolicy policy = SpaceViolationPolicy.NO_INSERTS;
+    final Table quotaTable = mock(Table.class);
+    when(conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
+
+    final Put expectedPut = new Put(Bytes.toBytes("t." + tn.getNameAsString()));
+    final SpaceQuota protoQuota = SpaceQuota.newBuilder()
+        .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(policy))
+        .build();
+    expectedPut.addColumn(Bytes.toBytes("u"), Bytes.toBytes("v"), protoQuota.toByteArray());
+
+    notifier.transitionTableToViolation(tn, policy);
+
+    verify(quotaTable).put(argThat(new SingleCellPutMatcher(expectedPut)));
+  }
+
+  @Test
+  public void testToObservance() throws Exception {
+    final TableName tn = TableName.valueOf("notinviolation");
+    final Table quotaTable = mock(Table.class);
+    when(conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
+
+    final Delete expectedDelete = new Delete(Bytes.toBytes("t." + tn.getNameAsString()));
+    expectedDelete.addColumn(Bytes.toBytes("u"), Bytes.toBytes("v"));
+
+    notifier.transitionTableToObservance(tn);
+
+    verify(quotaTable).delete(argThat(new SingleCellDeleteMatcher(expectedDelete)));
+  }
+
+  /**
+   * Parameterized for Puts.
+   */
+  private static class SingleCellPutMatcher extends SingleCellMutationMatcher<Put> {
+    private SingleCellPutMatcher(Put expected) {
+      super(expected);
+    }
+  }
+
+  /**
+   * Parameterized for Deletes.
+   */
+  private static class SingleCellDeleteMatcher extends SingleCellMutationMatcher<Delete> {
+    private SingleCellDeleteMatcher(Delete expected) {
+      super(expected);
+    }
+  }
+
+  /**
+   * Quick hack to verify a Mutation with one column.
+   */
+  private static class SingleCellMutationMatcher<T> extends ArgumentMatcher<T> {
+    private final Mutation expected;
+
+    private SingleCellMutationMatcher(Mutation expected) {
+      this.expected = expected;
+    }
+
+    @Override
+    public boolean matches(Object argument) {
+      if (!expected.getClass().isAssignableFrom(argument.getClass())) {
+        return false;
+      }
+      Mutation actual = (Mutation) argument;
+      if (!Arrays.equals(expected.getRow(), actual.getRow())) {
+        return false;
+      }
+      if (expected.size() != actual.size()) {
+        return false;
+      }
+      NavigableMap<byte[],List<Cell>> expectedCells = expected.getFamilyCellMap();
+      NavigableMap<byte[],List<Cell>> actualCells = actual.getFamilyCellMap();
+      Entry<byte[],List<Cell>> expectedEntry = expectedCells.entrySet().iterator().next();
+      Entry<byte[],List<Cell>> actualEntry = actualCells.entrySet().iterator().next();
+      if (!Arrays.equals(expectedEntry.getKey(), actualEntry.getKey())) {
+        return false;
+      }
+      return Objects.equals(expectedEntry.getValue(), actualEntry.getValue());
+    }
+  }
+}


[21/50] [abbrv] hbase git commit: HBASE-17000 Implement computation of online region sizes and report to the Master

Posted by el...@apache.org.
HBASE-17000 Implement computation of online region sizes and report to the Master

Includes a trivial implementation of the Master-side collection to
avoid. Only enough to write a test to verify RS collection.


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

Branch: refs/heads/HBASE-16961
Commit: 2dea67644982aa18d174de32f454f526354eea5c
Parents: a29abe6
Author: Josh Elser <el...@apache.org>
Authored: Mon Nov 7 13:46:42 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:31 2017 -0400

----------------------------------------------------------------------
 .../generated/RegionServerStatusProtos.java     | 2071 +++++++++++++++++-
 .../src/main/protobuf/RegionServerStatus.proto  |   22 +
 .../hadoop/hbase/master/MasterRpcServices.java  |   19 +
 .../quotas/FileSystemUtilizationChore.java      |  205 ++
 .../hadoop/hbase/quotas/MasterQuotaManager.java |   15 +
 .../hbase/regionserver/HRegionServer.java       |   72 +
 .../quotas/TestFileSystemUtilizationChore.java  |  357 +++
 .../hadoop/hbase/quotas/TestRegionSizeUse.java  |  194 ++
 .../TestRegionServerRegionSpaceUseReport.java   |   99 +
 9 files changed, 3032 insertions(+), 22 deletions(-)
----------------------------------------------------------------------



[08/50] [abbrv] hbase git commit: HBASE-17903 Corrected the alias for the link of HBASE-6580

Posted by el...@apache.org.
HBASE-17903 Corrected the alias for the link of HBASE-6580

Signed-off-by: CHIA-PING TSAI <ch...@gmail.com>


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

Branch: refs/heads/HBASE-16961
Commit: 918aa4655c4109159f27b6d78460bd3681c11f06
Parents: 8db9760
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Sun Apr 16 17:02:47 2017 +0200
Committer: CHIA-PING TSAI <ch...@gmail.com>
Committed: Mon Apr 17 10:22:25 2017 +0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/architecture.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/918aa465/src/main/asciidoc/_chapters/architecture.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 773d237..27aebd9 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -230,7 +230,7 @@ try (Connection connection = ConnectionFactory.createConnection(conf)) {
 .`HTablePool` is Deprecated
 [WARNING]
 ====
-Previous versions of this guide discussed `HTablePool`, which was deprecated in HBase 0.94, 0.95, and 0.96, and removed in 0.98.1, by link:https://issues.apache.org/jira/browse/HBASE-6580[HBASE-6500], or `HConnection`, which is deprecated in HBase 1.0 by `Connection`.
+Previous versions of this guide discussed `HTablePool`, which was deprecated in HBase 0.94, 0.95, and 0.96, and removed in 0.98.1, by link:https://issues.apache.org/jira/browse/HBASE-6580[HBASE-6580], or `HConnection`, which is deprecated in HBase 1.0 by `Connection`.
 Please use link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Connection.html[Connection] instead.
 ====
 


[25/50] [abbrv] hbase git commit: HBASE-16995 Build client Java API and client protobuf messages - addendum fixes white spaces (Josh Elser)

Posted by el...@apache.org.
HBASE-16995 Build client Java API and client protobuf messages - addendum fixes white spaces (Josh Elser)


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

Branch: refs/heads/HBASE-16961
Commit: eaeef44e2fbd4f13a7f2d8dc5934eda3c54c529f
Parents: 990062a
Author: tedyu <yu...@gmail.com>
Authored: Thu Nov 17 10:42:18 2016 -0800
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:31 2017 -0400

----------------------------------------------------------------------
 .../hbase/quotas/TestQuotaSettingsFactory.java    |  2 +-
 .../shaded/protobuf/generated/MasterProtos.java   |  2 +-
 .../shaded/protobuf/generated/QuotaProtos.java    | 18 +++++++++---------
 .../hbase/protobuf/generated/QuotaProtos.java     |  4 ++--
 4 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eaeef44e/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
index 17015d6..e0012a7 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
@@ -44,7 +44,7 @@ import org.junit.experimental.categories.Category;
  */
 @Category(SmallTests.class)
 public class TestQuotaSettingsFactory {
-  
+
   @Test
   public void testAllQuotasAddedToList() {
     final SpaceQuota spaceQuota = SpaceQuota.newBuilder()

http://git-wip-us.apache.org/repos/asf/hbase/blob/eaeef44e/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index 0c3248c..bbc6d1d 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -63752,7 +63752,7 @@ public final class MasterProtos {
        * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder>
           getSpaceLimitFieldBuilder() {
         if (spaceLimitBuilder_ == null) {
           spaceLimitBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<

http://git-wip-us.apache.org/repos/asf/hbase/blob/eaeef44e/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
index e3c6bfd..0ab2576 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
@@ -4362,7 +4362,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>
           getSpaceFieldBuilder() {
         if (spaceBuilder_ == null) {
           spaceBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -5957,7 +5957,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>
           getQuotaFieldBuilder() {
         if (quotaBuilder_ == null) {
           quotaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -6020,37 +6020,37 @@ public final class QuotaProtos {
 
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TimedQuota_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TimedQuota_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Throttle_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Throttle_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ThrottleRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Quotas_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Quotas_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_QuotaUsage_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_QuotaUsage_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SpaceQuota_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SpaceQuota_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SpaceLimitRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/eaeef44e/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
index 1925828..cc40536 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
@@ -4258,7 +4258,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>
           getSpaceFieldBuilder() {
         if (spaceBuilder_ == null) {
           spaceBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -5754,7 +5754,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>
           getQuotaFieldBuilder() {
         if (quotaBuilder_ == null) {
           quotaBuilder_ = new com.google.protobuf.SingleFieldBuilder<


[18/50] [abbrv] hbase git commit: HBASE-16998 Implement Master-side analysis of region space reports

Posted by el...@apache.org.
HBASE-16998 Implement Master-side analysis of region space reports

Adds a new Chore to the Master that analyzes the reports that are
sent by RegionServers. The Master must then, for all tables with
quotas, determine the tables that are violating quotas and move
those tables into violation. Similarly, tables no longer violating
the quota can be moved out of violation.

The Chore is the "stateful" bit, managing which tables are and
are not in violation. Everything else is just performing
computation and informing the Chore on the updated state.

Added InterfaceAudience annotations and clean up the QuotaObserverChore
constructor. Cleaned up some javadoc and QuotaObserverChore. Reuse
the QuotaViolationStore impl objects.


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

Branch: refs/heads/HBASE-16961
Commit: bcf6da40514f0858c8f63cb9ac379dbcd28d1058
Parents: d77ec49
Author: Josh Elser <el...@apache.org>
Authored: Tue Nov 8 18:55:12 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:31 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hbase/quotas/QuotaRetriever.java     |  27 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  20 +
 .../hadoop/hbase/quotas/MasterQuotaManager.java |   1 +
 .../quotas/NamespaceQuotaViolationStore.java    | 127 ++++
 .../hadoop/hbase/quotas/QuotaObserverChore.java | 618 +++++++++++++++++++
 .../hbase/quotas/QuotaViolationStore.java       |  89 +++
 .../quotas/SpaceQuotaViolationNotifier.java     |  44 ++
 .../SpaceQuotaViolationNotifierForTest.java     |  50 ++
 .../hbase/quotas/TableQuotaViolationStore.java  | 127 ++++
 .../TestNamespaceQuotaViolationStore.java       | 156 +++++
 .../hbase/quotas/TestQuotaObserverChore.java    | 106 ++++
 .../TestQuotaObserverChoreWithMiniCluster.java  | 596 ++++++++++++++++++
 .../hadoop/hbase/quotas/TestQuotaTableUtil.java |   4 -
 .../quotas/TestTableQuotaViolationStore.java    | 151 +++++
 .../hbase/quotas/TestTablesWithQuotas.java      | 198 ++++++
 15 files changed, 2305 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
index 0f7baa5..4482693 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
@@ -22,6 +22,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.Objects;
 import java.util.Queue;
 
 import org.apache.commons.logging.Log;
@@ -54,11 +55,23 @@ public class QuotaRetriever implements Closeable, Iterable<QuotaSettings> {
   private Connection connection;
   private Table table;
 
-  private QuotaRetriever() {
+  /**
+   * Should QutoaRetriever manage the state of the connection, or leave it be.
+   */
+  private boolean isManagedConnection = false;
+
+  QuotaRetriever() {
   }
 
   void init(final Configuration conf, final Scan scan) throws IOException {
-    this.connection = ConnectionFactory.createConnection(conf);
+    // Set this before creating the connection and passing it down to make sure
+    // it's cleaned up if we fail to construct the Scanner.
+    this.isManagedConnection = true;
+    init(ConnectionFactory.createConnection(conf), scan);
+  }
+
+  void init(final Connection conn, final Scan scan) throws IOException {
+    this.connection = Objects.requireNonNull(conn);
     this.table = this.connection.getTable(QuotaTableUtil.QUOTA_TABLE_NAME);
     try {
       scanner = table.getScanner(scan);
@@ -77,10 +90,14 @@ public class QuotaRetriever implements Closeable, Iterable<QuotaSettings> {
       this.table.close();
       this.table = null;
     }
-    if (this.connection != null) {
-      this.connection.close();
-      this.connection = null;
+    // Null out the connection on close() even if we didn't explicitly close it
+    // to maintain typical semantics.
+    if (isManagedConnection) {
+      if (this.connection != null) {
+        this.connection.close();
+      }
     }
+    this.connection = null;
   }
 
   public QuotaSettings next() throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/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 bb9f282..5b2d89c 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
@@ -132,6 +132,9 @@ import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
+import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifier;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifierForTest;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -370,6 +373,8 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   // it is assigned after 'initialized' guard set to true, so should be volatile
   private volatile MasterQuotaManager quotaManager;
+  private SpaceQuotaViolationNotifier spaceQuotaViolationNotifier;
+  private QuotaObserverChore quotaObserverChore;
 
   private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
   private WALProcedureStore procedureStore;
@@ -895,6 +900,10 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     status.setStatus("Starting quota manager");
     initQuotaManager();
+    this.spaceQuotaViolationNotifier = new SpaceQuotaViolationNotifierForTest();
+    this.quotaObserverChore = new QuotaObserverChore(this);
+    // Start the chore to read the region FS space reports and act on them
+    getChoreService().scheduleChore(quotaObserverChore);
 
     // clear the dead servers with same host name and port of online server because we are not
     // removing dead server with same hostname and port of rs which is trying to check in before
@@ -1193,6 +1202,9 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (this.periodicDoMetricsChore != null) {
       periodicDoMetricsChore.cancel();
     }
+    if (this.quotaObserverChore != null) {
+      quotaObserverChore.cancel();
+    }
   }
 
   /**
@@ -3301,4 +3313,12 @@ public class HMaster extends HRegionServer implements MasterServices {
   public LockManager getLockManager() {
     return lockManager;
   }
+
+  public QuotaObserverChore getQuotaObserverChore() {
+    return this.quotaObserverChore;
+  }
+
+  public SpaceQuotaViolationNotifier getSpaceQuotaViolationNotifier() {
+    return this.spaceQuotaViolationNotifier;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index 37ccdc0..206d81d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -523,6 +523,7 @@ public class MasterQuotaManager implements RegionStateListener {
 
   public void addRegionSize(HRegionInfo hri, long size) {
     // TODO Make proper API
+    // TODO Prevent from growing indefinitely
     regionSizes.put(hri, size);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaViolationStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaViolationStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaViolationStore.java
new file mode 100644
index 0000000..017ecec
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaViolationStore.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 java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+/**
+ * {@link QuotaViolationStore} implementation for namespaces.
+ */
+@InterfaceAudience.Private
+public class NamespaceQuotaViolationStore implements QuotaViolationStore<String> {
+  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  private final ReadLock rlock = lock.readLock();
+  private final WriteLock wlock = lock.writeLock();
+
+  private final Connection conn;
+  private final QuotaObserverChore chore;
+  private Map<HRegionInfo,Long> regionUsage;
+
+  public NamespaceQuotaViolationStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
+    this.conn = Objects.requireNonNull(conn);
+    this.chore = Objects.requireNonNull(chore);
+    this.regionUsage = Objects.requireNonNull(regionUsage);
+  }
+
+  @Override
+  public SpaceQuota getSpaceQuota(String namespace) throws IOException {
+    Quotas quotas = getQuotaForNamespace(namespace);
+    if (null != quotas && quotas.hasSpace()) {
+      return quotas.getSpace();
+    }
+    return null;
+  }
+
+  /**
+   * Fetches the namespace quota. Visible for mocking/testing.
+   */
+  Quotas getQuotaForNamespace(String namespace) throws IOException {
+    return QuotaTableUtil.getNamespaceQuota(conn, namespace);
+  }
+
+  @Override
+  public ViolationState getCurrentState(String namespace) {
+    // Defer the "current state" to the chore
+    return this.chore.getNamespaceQuotaViolation(namespace);
+  }
+
+  @Override
+  public ViolationState getTargetState(String subject, SpaceQuota spaceQuota) {
+    rlock.lock();
+    try {
+      final long sizeLimitInBytes = spaceQuota.getSoftLimit();
+      long sum = 0L;
+      for (Entry<HRegionInfo,Long> entry : filterBySubject(subject)) {
+        sum += entry.getValue();
+        if (sum > sizeLimitInBytes) {
+          // Short-circuit early
+          return ViolationState.IN_VIOLATION;
+        }
+      }
+      // Observance is defined as the size of the table being less than the limit
+      return sum <= sizeLimitInBytes ? ViolationState.IN_OBSERVANCE : ViolationState.IN_VIOLATION;
+    } finally {
+      rlock.unlock();
+    }
+  }
+
+  @Override
+  public Iterable<Entry<HRegionInfo,Long>> filterBySubject(String namespace) {
+    rlock.lock();
+    try {
+      return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<HRegionInfo,Long>>() {
+        @Override
+        public boolean apply(Entry<HRegionInfo,Long> input) {
+          return namespace.equals(input.getKey().getTable().getNamespaceAsString());
+        }
+      });
+    } finally {
+      rlock.unlock();
+    }
+  }
+
+  @Override
+  public void setCurrentState(String namespace, ViolationState state) {
+    // Defer the "current state" to the chore
+    this.chore.setNamespaceQuotaViolation(namespace, state);
+  }
+
+  @Override
+  public void setRegionUsage(Map<HRegionInfo,Long> regionUsage) {
+    wlock.lock();
+    try {
+      this.regionUsage = Objects.requireNonNull(regionUsage);
+    } finally {
+      wlock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/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
new file mode 100644
index 0000000..88a6149
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
@@ -0,0 +1,618 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+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.Connection;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.quotas.QuotaViolationStore.ViolationState;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Multimap;
+
+/**
+ * Reads the currently received Region filesystem-space use reports and acts on those which
+ * violate a defined quota.
+ */
+@InterfaceAudience.Private
+public class QuotaObserverChore extends ScheduledChore {
+  private static final Log LOG = LogFactory.getLog(QuotaObserverChore.class);
+  static final String VIOLATION_OBSERVER_CHORE_PERIOD_KEY =
+      "hbase.master.quotas.violation.observer.chore.period";
+  static final int VIOLATION_OBSERVER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
+
+  static final String VIOLATION_OBSERVER_CHORE_DELAY_KEY =
+      "hbase.master.quotas.violation.observer.chore.delay";
+  static final long VIOLATION_OBSERVER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
+
+  static final String VIOLATION_OBSERVER_CHORE_TIMEUNIT_KEY =
+      "hbase.master.quotas.violation.observer.chore.timeunit";
+  static final String VIOLATION_OBSERVER_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
+
+  static final String VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_KEY =
+      "hbase.master.quotas.violation.observer.report.percent";
+  static final double VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT= 0.95;
+
+  private final Connection conn;
+  private final Configuration conf;
+  private final MasterQuotaManager quotaManager;
+  /*
+   * Callback that changes in quota violation are passed to.
+   */
+  private final SpaceQuotaViolationNotifier violationNotifier;
+
+  /*
+   * Preserves the state of quota violations for tables and namespaces
+   */
+  private final Map<TableName,ViolationState> tableQuotaViolationStates;
+  private final Map<String,ViolationState> namespaceQuotaViolationStates;
+
+  /*
+   * Encapsulates logic for moving tables/namespaces into or out of quota violation
+   */
+  private QuotaViolationStore<TableName> tableViolationStore;
+  private QuotaViolationStore<String> namespaceViolationStore;
+
+  public QuotaObserverChore(HMaster master) {
+    this(
+        master.getConnection(), master.getConfiguration(),
+        master.getSpaceQuotaViolationNotifier(), master.getMasterQuotaManager(),
+        master);
+  }
+
+  QuotaObserverChore(
+      Connection conn, Configuration conf, SpaceQuotaViolationNotifier violationNotifier,
+      MasterQuotaManager quotaManager, Stoppable stopper) {
+    super(
+        QuotaObserverChore.class.getSimpleName(), stopper, getPeriod(conf),
+        getInitialDelay(conf), getTimeUnit(conf));
+    this.conn = conn;
+    this.conf = conf;
+    this.quotaManager = quotaManager;
+    this.violationNotifier = violationNotifier;
+    this.tableQuotaViolationStates = new HashMap<>();
+    this.namespaceQuotaViolationStates = new HashMap<>();
+  }
+
+  @Override
+  protected void chore() {
+    try {
+      _chore();
+    } catch (IOException e) {
+      LOG.warn("Failed to process quota reports and update quota violation state. Will retry.", e);
+    }
+  }
+
+  void _chore() throws IOException {
+    // Get the total set of tables that have quotas defined. Includes table quotas
+    // and tables included by namespace quotas.
+    TablesWithQuotas tablesWithQuotas = fetchAllTablesWithQuotasDefined();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Found following tables with quotas: " + tablesWithQuotas);
+    }
+
+    // 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");
+    }
+
+    // Create the stores to track table and namespace violations
+    initializeViolationStores(reportedRegionSpaceUse);
+
+    // Filter out tables for which we don't have adequate regionspace reports yet.
+    // Important that we do this after we instantiate the stores above
+    tablesWithQuotas.filterInsufficientlyReportedTables(tableViolationStore);
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Filtered insufficiently reported tables, left with " +
+          reportedRegionSpaceUse.size() + " regions reported");
+    }
+
+    // Transition each table to/from quota violation based on the current and target state.
+    // Only table quotas are enacted.
+    final Set<TableName> tablesWithTableQuotas = tablesWithQuotas.getTableQuotaTables();
+    processTablesWithQuotas(tablesWithTableQuotas);
+
+    // For each Namespace quota, transition each table in the namespace in or out of violation
+    // only if a table quota violation policy has not already been applied.
+    final Set<String> namespacesWithQuotas = tablesWithQuotas.getNamespacesWithQuotas();
+    final Multimap<String,TableName> tablesByNamespace = tablesWithQuotas.getTablesByNamespace();
+    processNamespacesWithQuotas(namespacesWithQuotas, tablesByNamespace);
+  }
+
+  void initializeViolationStores(Map<HRegionInfo,Long> regionSizes) {
+    Map<HRegionInfo,Long> immutableRegionSpaceUse = Collections.unmodifiableMap(regionSizes);
+    if (null == tableViolationStore) {
+      tableViolationStore = new TableQuotaViolationStore(conn, this, immutableRegionSpaceUse);
+    } else {
+      tableViolationStore.setRegionUsage(immutableRegionSpaceUse);
+    }
+    if (null == namespaceViolationStore) {
+      namespaceViolationStore = new NamespaceQuotaViolationStore(
+          conn, this, immutableRegionSpaceUse);
+    } else {
+      namespaceViolationStore.setRegionUsage(immutableRegionSpaceUse);
+    }
+  }
+
+  /**
+   * Processes each {@code TableName} which has a quota defined and moves it in or out of
+   * violation based on the space use.
+   *
+   * @param tablesWithTableQuotas The HBase tables which have quotas defined
+   */
+  void processTablesWithQuotas(final Set<TableName> tablesWithTableQuotas) throws IOException {
+    for (TableName table : tablesWithTableQuotas) {
+      final SpaceQuota spaceQuota = tableViolationStore.getSpaceQuota(table);
+      if (null == spaceQuota) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Unexpectedly did not find a space quota for " + table
+              + ", maybe it was recently deleted.");
+        }
+        continue;
+      }
+      final ViolationState currentState = tableViolationStore.getCurrentState(table);
+      final ViolationState targetState = tableViolationStore.getTargetState(table, spaceQuota);
+
+      if (currentState == ViolationState.IN_VIOLATION) {
+        if (targetState == ViolationState.IN_OBSERVANCE) {
+          LOG.info(table + " moving into observance of table space quota.");
+          transitionTableToObservance(table);
+          tableViolationStore.setCurrentState(table, ViolationState.IN_OBSERVANCE);
+        } else if (targetState == ViolationState.IN_VIOLATION) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(table + " remains in violation of quota.");
+          }
+          tableViolationStore.setCurrentState(table, ViolationState.IN_VIOLATION);
+        }
+      } else if (currentState == ViolationState.IN_OBSERVANCE) {
+        if (targetState == ViolationState.IN_VIOLATION) {
+          LOG.info(table + " moving into violation of table space quota.");
+          transitionTableToViolation(table, getViolationPolicy(spaceQuota));
+          tableViolationStore.setCurrentState(table, ViolationState.IN_VIOLATION);
+        } else if (targetState == ViolationState.IN_OBSERVANCE) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(table + " remains in observance of quota.");
+          }
+          tableViolationStore.setCurrentState(table, ViolationState.IN_OBSERVANCE);
+        }
+      }
+    }
+  }
+
+  /**
+   * Processes each namespace which has a quota defined and moves all of the tables contained
+   * in that namespace into or out of violation of the quota. Tables which are already in
+   * violation of a quota at the table level which <em>also</em> have a reside in a namespace
+   * with a violated quota will not have the namespace quota enacted. The table quota takes
+   * priority over the namespace quota.
+   *
+   * @param namespacesWithQuotas The set of namespaces that have quotas defined
+   * @param tablesByNamespace A mapping of namespaces and the tables contained in those namespaces
+   */
+  void processNamespacesWithQuotas(
+      final Set<String> namespacesWithQuotas,
+      final Multimap<String,TableName> tablesByNamespace) throws IOException {
+    for (String namespace : namespacesWithQuotas) {
+      // Get the quota definition for the namespace
+      final SpaceQuota spaceQuota = namespaceViolationStore.getSpaceQuota(namespace);
+      if (null == spaceQuota) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Could not get Namespace space quota for " + namespace
+              + ", maybe it was recently deleted.");
+        }
+        continue;
+      }
+      final ViolationState currentState = namespaceViolationStore.getCurrentState(namespace);
+      final ViolationState targetState = namespaceViolationStore.getTargetState(namespace, spaceQuota);
+      // When in observance, check if we need to move to violation.
+      if (ViolationState.IN_OBSERVANCE == currentState) {
+        if (ViolationState.IN_VIOLATION == targetState) {
+          for (TableName tableInNS : tablesByNamespace.get(namespace)) {
+            if (ViolationState.IN_VIOLATION == tableViolationStore.getCurrentState(tableInNS)) {
+              // Table-level quota violation policy is being applied here.
+              if (LOG.isTraceEnabled()) {
+                LOG.trace("Not activating Namespace violation policy because Table violation"
+                    + " policy is already in effect for " + tableInNS);
+              }
+              continue;
+            } else {
+              LOG.info(tableInNS + " moving into violation of namespace space quota");
+              transitionTableToViolation(tableInNS, getViolationPolicy(spaceQuota));
+            }
+          }
+        } else {
+          // still in observance
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(namespace + " remains in observance of quota.");
+          }
+        }
+      } else if (ViolationState.IN_VIOLATION == currentState) {
+        // When in violation, check if we need to move to observance.
+        if (ViolationState.IN_OBSERVANCE == targetState) {
+          for (TableName tableInNS : tablesByNamespace.get(namespace)) {
+            if (ViolationState.IN_VIOLATION == tableViolationStore.getCurrentState(tableInNS)) {
+              // Table-level quota violation policy is being applied here.
+              if (LOG.isTraceEnabled()) {
+                LOG.trace("Not activating Namespace violation policy because Table violation"
+                    + " policy is already in effect for " + tableInNS);
+              }
+              continue;
+            } else {
+              LOG.info(tableInNS + " moving into observance of namespace space quota");
+              transitionTableToObservance(tableInNS);
+            }
+          }
+        } else {
+          // Remains in violation
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(namespace + " remains in violation of quota.");
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Computes the set of all tables that have quotas defined. This includes tables with quotas
+   * explicitly set on them, in addition to tables that exist namespaces which have a quota
+   * defined.
+   */
+  TablesWithQuotas fetchAllTablesWithQuotasDefined() throws IOException {
+    final Scan scan = QuotaTableUtil.makeScan(null);
+    final QuotaRetriever scanner = new QuotaRetriever();
+    final TablesWithQuotas tablesWithQuotas = new TablesWithQuotas(conn, conf);
+    try {
+      scanner.init(conn, scan);
+      for (QuotaSettings quotaSettings : scanner) {
+        // Only one of namespace and tablename should be 'null'
+        final String namespace = quotaSettings.getNamespace();
+        final TableName tableName = quotaSettings.getTableName();
+        if (QuotaType.SPACE != quotaSettings.getQuotaType()) {
+          continue;
+        }
+
+        if (null != namespace) {
+          assert null == tableName;
+          // Collect all of the tables in the namespace
+          TableName[] tablesInNS = conn.getAdmin()
+              .listTableNamesByNamespace(namespace);
+          for (TableName tableUnderNs : tablesInNS) {
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Adding " + tableUnderNs + " under " +  namespace
+                  + " as having a namespace quota");
+            }
+            tablesWithQuotas.addNamespaceQuotaTable(tableUnderNs);
+          }
+        } else {
+          assert null != tableName;
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Adding " + tableName + " as having table quota.");
+          }
+          // namespace is already null, must be a non-null tableName
+          tablesWithQuotas.addTableQuotaTable(tableName);
+        }
+      }
+      return tablesWithQuotas;
+    } finally {
+      if (null != scanner) {
+        scanner.close();
+      }
+    }
+  }
+
+  @VisibleForTesting
+  QuotaViolationStore<TableName> getTableViolationStore() {
+    return tableViolationStore;
+  }
+
+  @VisibleForTesting
+  QuotaViolationStore<String> getNamespaceViolationStore() {
+    return namespaceViolationStore;
+  }
+
+  /**
+   * Transitions the given table to violation of its quota, enabling the violation policy.
+   */
+  private void transitionTableToViolation(TableName table, SpaceViolationPolicy violationPolicy) {
+    this.violationNotifier.transitionTableToViolation(table, violationPolicy);
+  }
+
+  /**
+   * Transitions the given table to observance of its quota, disabling the violation policy.
+   */
+  private void transitionTableToObservance(TableName table) {
+    this.violationNotifier.transitionTableToObservance(table);
+  }
+
+  /**
+   * Fetch the {@link ViolationState} for the given table.
+   */
+  ViolationState getTableQuotaViolation(TableName table) {
+    // TODO Can one instance of a Chore be executed concurrently?
+    ViolationState state = this.tableQuotaViolationStates.get(table);
+    if (null == state) {
+      // No tracked state implies observance.
+      return ViolationState.IN_OBSERVANCE;
+    }
+    return state;
+  }
+
+  /**
+   * Stores the quota violation state for the given table.
+   */
+  void setTableQuotaViolation(TableName table, ViolationState state) {
+    this.tableQuotaViolationStates.put(table, state);
+  }
+
+  /**
+   * Fetches the {@link ViolationState} for the given namespace.
+   */
+  ViolationState getNamespaceQuotaViolation(String namespace) {
+    // TODO Can one instance of a Chore be executed concurrently?
+    ViolationState state = this.namespaceQuotaViolationStates.get(namespace);
+    if (null == state) {
+      // No tracked state implies observance.
+      return ViolationState.IN_OBSERVANCE;
+    }
+    return state;
+  }
+
+  /**
+   * Stores the quota violation state for the given namespace.
+   */
+  void setNamespaceQuotaViolation(String namespace, ViolationState state) {
+    this.namespaceQuotaViolationStates.put(namespace, state);
+  }
+
+  /**
+   * Extracts the {@link SpaceViolationPolicy} from the serialized {@link Quotas} protobuf.
+   * @throws IllegalArgumentException If the SpaceQuota lacks a ViolationPolicy
+   */
+  SpaceViolationPolicy getViolationPolicy(SpaceQuota spaceQuota) {
+    if (!spaceQuota.hasViolationPolicy()) {
+      throw new IllegalArgumentException("SpaceQuota had no associated violation policy: "
+          + spaceQuota);
+    }
+    return ProtobufUtil.toViolationPolicy(spaceQuota.getViolationPolicy());
+  }
+
+  /**
+   * 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(VIOLATION_OBSERVER_CHORE_PERIOD_KEY,
+        VIOLATION_OBSERVER_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(VIOLATION_OBSERVER_CHORE_DELAY_KEY,
+        VIOLATION_OBSERVER_CHORE_DELAY_DEFAULT);
+  }
+
+  /**
+   * Extracts the time unit for the chore period and initial delay from the configuration. The
+   * configuration value for {@link #VIOLATION_OBSERVER_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(VIOLATION_OBSERVER_CHORE_TIMEUNIT_KEY,
+        VIOLATION_OBSERVER_CHORE_TIMEUNIT_DEFAULT));
+  }
+
+  /**
+   * Extracts the percent of Regions for a table to have been reported to enable quota violation
+   * state change.
+   *
+   * @param conf The configuration object.
+   * @return The percent of regions reported to use.
+   */
+  static Double getRegionReportPercent(Configuration conf) {
+    return conf.getDouble(VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_KEY,
+        VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT);
+  }
+
+  /**
+   * A container which encapsulates the tables which have a table quota and the tables which
+   * are contained in a namespace which have a namespace quota.
+   */
+  static class TablesWithQuotas {
+    private final Set<TableName> tablesWithTableQuotas = new HashSet<>();
+    private final Set<TableName> tablesWithNamespaceQuotas = new HashSet<>();
+    private final Connection conn;
+    private final Configuration conf;
+
+    public TablesWithQuotas(Connection conn, Configuration conf) {
+      this.conn = Objects.requireNonNull(conn);
+      this.conf = Objects.requireNonNull(conf);
+    }
+
+    Configuration getConfiguration() {
+      return conf;
+    }
+
+    /**
+     * Adds a table with a table quota.
+     */
+    public void addTableQuotaTable(TableName tn) {
+      tablesWithTableQuotas.add(tn);
+    }
+
+    /**
+     * Adds a table with a namespace quota.
+     */
+    public void addNamespaceQuotaTable(TableName tn) {
+      tablesWithNamespaceQuotas.add(tn);
+    }
+
+    /**
+     * Returns true if the given table has a table quota.
+     */
+    public boolean hasTableQuota(TableName tn) {
+      return tablesWithTableQuotas.contains(tn);
+    }
+
+    /**
+     * Returns true if the table exists in a namespace with a namespace quota.
+     */
+    public boolean hasNamespaceQuota(TableName tn) {
+      return tablesWithNamespaceQuotas.contains(tn);
+    }
+
+    /**
+     * Returns an unmodifiable view of all tables with table quotas.
+     */
+    public Set<TableName> getTableQuotaTables() {
+      return Collections.unmodifiableSet(tablesWithTableQuotas);
+    }
+
+    /**
+     * Returns an unmodifiable view of all tables in namespaces that have
+     * namespace quotas.
+     */
+    public Set<TableName> getNamespaceQuotaTables() {
+      return Collections.unmodifiableSet(tablesWithNamespaceQuotas);
+    }
+
+    public Set<String> getNamespacesWithQuotas() {
+      Set<String> namespaces = new HashSet<>();
+      for (TableName tn : tablesWithNamespaceQuotas) {
+        namespaces.add(tn.getNamespaceAsString());
+      }
+      return namespaces;
+    }
+
+    /**
+     * Returns a view of all tables that reside in a namespace with a namespace
+     * quota, grouped by the namespace itself.
+     */
+    public Multimap<String,TableName> getTablesByNamespace() {
+      Multimap<String,TableName> tablesByNS = HashMultimap.create();
+      for (TableName tn : tablesWithNamespaceQuotas) {
+        tablesByNS.put(tn.getNamespaceAsString(), tn);
+      }
+      return tablesByNS;
+    }
+
+    /**
+     * Filters out all tables for which the Master currently doesn't have enough region space
+     * reports received from RegionServers yet.
+     */
+    public void filterInsufficientlyReportedTables(QuotaViolationStore<TableName> tableStore)
+        throws IOException {
+      final double percentRegionsReportedThreshold = getRegionReportPercent(getConfiguration());
+      Set<TableName> tablesToRemove = new HashSet<>();
+      for (TableName table : Iterables.concat(tablesWithTableQuotas, tablesWithNamespaceQuotas)) {
+        // Don't recompute a table we've already computed
+        if (tablesToRemove.contains(table)) {
+          continue;
+        }
+        final int numRegionsInTable = getNumRegions(table);
+        // If the table doesn't exist (no regions), bail out.
+        if (0 == numRegionsInTable) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Filtering " + table + " because no regions were reported");
+          }
+          tablesToRemove.add(table);
+          continue;
+        }
+        final int reportedRegionsInQuota = getNumReportedRegions(table, tableStore);
+        final double ratioReported = ((double) reportedRegionsInQuota) / numRegionsInTable;
+        if (ratioReported < percentRegionsReportedThreshold) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Filtering " + table + " because " + reportedRegionsInQuota  + " of " +
+                numRegionsInTable + " were reported.");
+          }
+          tablesToRemove.add(table);
+        } else if (LOG.isTraceEnabled()) {
+          LOG.trace("Retaining " + table + " because " + reportedRegionsInQuota  + " of " +
+              numRegionsInTable + " were reported.");
+        }
+      }
+      for (TableName tableToRemove : tablesToRemove) {
+        tablesWithTableQuotas.remove(tableToRemove);
+        tablesWithNamespaceQuotas.remove(tableToRemove);
+      }
+    }
+
+    /**
+     * Computes the total number of regions in a table.
+     */
+    int getNumRegions(TableName table) throws IOException {
+      List<HRegionInfo> regions = this.conn.getAdmin().getTableRegions(table);
+      if (null == regions) {
+        return 0;
+      }
+      return regions.size();
+    }
+
+    /**
+     * Computes the number of regions reported for a table.
+     */
+    int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore)
+        throws IOException {
+      return Iterables.size(tableStore.filterBySubject(table));
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(32);
+      sb.append(getClass().getSimpleName())
+          .append(": tablesWithTableQuotas=")
+          .append(this.tablesWithTableQuotas)
+          .append(", tablesWithNamespaceQuotas=")
+          .append(this.tablesWithNamespaceQuotas);
+      return sb.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaViolationStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaViolationStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaViolationStore.java
new file mode 100644
index 0000000..381ac8e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaViolationStore.java
@@ -0,0 +1,89 @@
+/*
+ * 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.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+
+/**
+ * A common interface for computing and storing space quota observance/violation for entities.
+ *
+ * An entity is presently a table or a namespace.
+ */
+@InterfaceAudience.Private
+public interface QuotaViolationStore<T> {
+
+  /**
+   * The current state of a table with respect to the policy set forth by a quota.
+   */
+  @InterfaceAudience.Private
+  public enum ViolationState {
+    IN_VIOLATION,
+    IN_OBSERVANCE,
+  }
+
+  /**
+   * Fetch the Quota for the given {@code subject}. May be null.
+   *
+   * @param subject The object for which the quota should be fetched
+   */
+  SpaceQuota getSpaceQuota(T subject) throws IOException;
+
+  /**
+   * Returns the current {@link ViolationState} for the given {@code subject}.
+   *
+   * @param subject The object which the quota violation state should be fetched
+   */
+  ViolationState getCurrentState(T subject);
+
+  /**
+   * Computes the target {@link ViolationState} for the given {@code subject} and
+   * {@code spaceQuota}.
+   *
+   * @param subject The object which to determine the target quota violation state of
+   * @param spaceQuota The quota "definition" for the {@code subject}
+   */
+  ViolationState getTargetState(T subject, SpaceQuota spaceQuota);
+
+  /**
+   * Filters the provided <code>regions</code>, returning those which match the given
+   * <code>subject</code>.
+   *
+   * @param subject The filter criteria. Only regions belonging to this parameter will be returned
+   */
+  Iterable<Entry<HRegionInfo,Long>> filterBySubject(T subject);
+
+  /**
+   * Persists the current {@link ViolationState} for the {@code subject}.
+   *
+   * @param subject The object which the {@link ViolationState} is being persisted for
+   * @param state The current {@link ViolationState} of the {@code subject}
+   */
+  void setCurrentState(T subject, ViolationState state);
+
+  /**
+   * Updates {@code this} with the latest snapshot of filesystem use by region.
+   *
+   * @param regionUsage A map of {@code HRegionInfo} objects to their filesystem usage in bytes
+   */
+  void setRegionUsage(Map<HRegionInfo,Long> regionUsage);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
new file mode 100644
index 0000000..bccf519
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifier.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * An interface which abstract away the action taken to enable or disable
+ * a space quota violation policy across the HBase cluster.
+ */
+@InterfaceAudience.Private
+public interface SpaceQuotaViolationNotifier {
+
+  /**
+   * Instructs the cluster that the given table is in violation of a space quota. The
+   * provided violation policy is the action which should be taken on the table.
+   *
+   * @param tableName The name of the table in violation of the quota.
+   * @param violationPolicy The policy which should be enacted on the table.
+   */
+  void transitionTableToViolation(TableName tableName, SpaceViolationPolicy violationPolicy);
+
+  /**
+   * Instructs the cluster that the given table is in observance of any applicable space quota.
+   *
+   * @param tableName The name of the table in observance.
+   */
+  void transitionTableToObservance(TableName tableName);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
new file mode 100644
index 0000000..4ab9834
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaViolationNotifierForTest.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A SpaceQuotaViolationNotifier implementation for verifying testing.
+ */
+@InterfaceAudience.Private
+public class SpaceQuotaViolationNotifierForTest implements SpaceQuotaViolationNotifier {
+
+  private final Map<TableName,SpaceViolationPolicy> tablesInViolation = new HashMap<>();
+
+  @Override
+  public void transitionTableToViolation(TableName tableName, SpaceViolationPolicy violationPolicy) {
+    tablesInViolation.put(tableName, violationPolicy);
+  }
+
+  @Override
+  public void transitionTableToObservance(TableName tableName) {
+    tablesInViolation.remove(tableName);
+  }
+
+  public Map<TableName,SpaceViolationPolicy> snapshotTablesInViolation() {
+    return new HashMap<>(this.tablesInViolation);
+  }
+
+  public void clearTableViolations() {
+    this.tablesInViolation.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaViolationStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaViolationStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaViolationStore.java
new file mode 100644
index 0000000..6aba1cf
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaViolationStore.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 java.io.IOException;
+import java.util.Map;
+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.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.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+/**
+ * {@link QuotaViolationStore} for tables.
+ */
+@InterfaceAudience.Private
+public class TableQuotaViolationStore implements QuotaViolationStore<TableName> {
+  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  private final ReadLock rlock = lock.readLock();
+  private final WriteLock wlock = lock.writeLock();
+
+  private final Connection conn;
+  private final QuotaObserverChore chore;
+  private Map<HRegionInfo,Long> regionUsage;
+
+  public TableQuotaViolationStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
+    this.conn = Objects.requireNonNull(conn);
+    this.chore = Objects.requireNonNull(chore);
+    this.regionUsage = Objects.requireNonNull(regionUsage);
+  }
+
+  @Override
+  public SpaceQuota getSpaceQuota(TableName subject) throws IOException {
+    Quotas quotas = getQuotaForTable(subject);
+    if (null != quotas && quotas.hasSpace()) {
+      return quotas.getSpace();
+    }
+    return null;
+  }
+  /**
+   * Fetches the table quota. Visible for mocking/testing.
+   */
+  Quotas getQuotaForTable(TableName table) throws IOException {
+    return QuotaTableUtil.getTableQuota(conn, table);
+  }
+
+  @Override
+  public ViolationState getCurrentState(TableName table) {
+    // Defer the "current state" to the chore
+    return chore.getTableQuotaViolation(table);
+  }
+
+  @Override
+  public ViolationState getTargetState(TableName table, SpaceQuota spaceQuota) {
+    rlock.lock();
+    try {
+      final long sizeLimitInBytes = spaceQuota.getSoftLimit();
+      long sum = 0L;
+      for (Entry<HRegionInfo,Long> entry : filterBySubject(table)) {
+        sum += entry.getValue();
+        if (sum > sizeLimitInBytes) {
+          // Short-circuit early
+          return ViolationState.IN_VIOLATION;
+        }
+      }
+      // Observance is defined as the size of the table being less than the limit
+      return sum <= sizeLimitInBytes ? ViolationState.IN_OBSERVANCE : ViolationState.IN_VIOLATION;
+    } finally {
+      rlock.unlock();
+    }
+  }
+
+  @Override
+  public Iterable<Entry<HRegionInfo,Long>> filterBySubject(TableName table) {
+    rlock.lock();
+    try {
+      return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<HRegionInfo,Long>>() {
+        @Override
+        public boolean apply(Entry<HRegionInfo,Long> input) {
+          return table.equals(input.getKey().getTable());
+        }
+      });
+    } finally {
+      rlock.unlock();
+    }
+  }
+
+  @Override
+  public void setCurrentState(TableName table, ViolationState state) {
+    // Defer the "current state" to the chore
+    this.chore.setTableQuotaViolation(table, state);
+  }
+
+  @Override
+  public void setRegionUsage(Map<HRegionInfo,Long> regionUsage) {
+    wlock.lock();
+    try {
+      this.regionUsage = Objects.requireNonNull(regionUsage);
+    } finally {
+      wlock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/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
new file mode 100644
index 0000000..8182513
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java
@@ -0,0 +1,156 @@
+/*
+ * 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 com.google.common.collect.Iterables.size;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+
+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.quotas.QuotaViolationStore.ViolationState;
+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;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Test class for {@link NamespaceQuotaViolationStore}.
+ */
+@Category(SmallTests.class)
+public class TestNamespaceQuotaViolationStore {
+  private static final long ONE_MEGABYTE = 1024L * 1024L;
+
+  private Connection conn;
+  private QuotaObserverChore chore;
+  private Map<HRegionInfo, Long> regionReports;
+  private NamespaceQuotaViolationStore store;
+
+  @Before
+  public void setup() {
+    conn = mock(Connection.class);
+    chore = mock(QuotaObserverChore.class);
+    regionReports = new HashMap<>();
+    store = new NamespaceQuotaViolationStore(conn, chore, regionReports);
+  }
+
+  @Test
+  public void testGetSpaceQuota() throws Exception {
+    NamespaceQuotaViolationStore mockStore = mock(NamespaceQuotaViolationStore.class);
+    when(mockStore.getSpaceQuota(any(String.class))).thenCallRealMethod();
+
+    Quotas quotaWithSpace = Quotas.newBuilder().setSpace(
+        SpaceQuota.newBuilder()
+            .setSoftLimit(1024L)
+            .setViolationPolicy(QuotaProtos.SpaceViolationPolicy.DISABLE)
+            .build())
+        .build();
+    Quotas quotaWithoutSpace = Quotas.newBuilder().build();
+
+    AtomicReference<Quotas> quotaRef = new AtomicReference<>();
+    when(mockStore.getQuotaForNamespace(any(String.class))).then(new Answer<Quotas>() {
+      @Override
+      public Quotas answer(InvocationOnMock invocation) throws Throwable {
+        return quotaRef.get();
+      }
+    });
+
+    quotaRef.set(quotaWithSpace);
+    assertEquals(quotaWithSpace.getSpace(), mockStore.getSpaceQuota("ns"));
+    quotaRef.set(quotaWithoutSpace);
+    assertNull(mockStore.getSpaceQuota("ns"));
+  }
+
+  @Test
+  public void testTargetViolationState() {
+    final String NS = "ns";
+    TableName tn1 = TableName.valueOf(NS, "tn1");
+    TableName tn2 = TableName.valueOf(NS, "tn2");
+    TableName tn3 = TableName.valueOf("tn3");
+    SpaceQuota quota = SpaceQuota.newBuilder()
+        .setSoftLimit(ONE_MEGABYTE)
+        .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(SpaceViolationPolicy.DISABLE))
+        .build();
+
+    // Create some junk data to filter. Makes sure it's so large that it would
+    // immediately violate the quota.
+    for (int i = 0; i < 3; i++) {
+      regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i + 1)),
+          5L * ONE_MEGABYTE);
+    }
+
+    regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(0), Bytes.toBytes(1)), 1024L * 512L);
+    regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(1), Bytes.toBytes(2)), 1024L * 256L);
+
+    // Below the quota
+    assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(NS, quota));
+
+    regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(2), Bytes.toBytes(3)), 1024L * 256L);
+
+    // Equal to the quota is still in observance
+    assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(NS, quota));
+
+    regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(3), Bytes.toBytes(4)), 1024L);
+
+    // Exceeds the quota, should be in violation
+    assertEquals(ViolationState.IN_VIOLATION, store.getTargetState(NS, quota));
+  }
+
+  @Test
+  public void testFilterRegionsByNamespace() {
+    TableName tn1 = TableName.valueOf("foo");
+    TableName tn2 = TableName.valueOf("sn", "bar");
+    TableName tn3 = TableName.valueOf("ns", "foo");
+    TableName tn4 = TableName.valueOf("ns", "bar");
+
+    assertEquals(0, size(store.filterBySubject("asdf")));
+
+    for (int i = 0; i < 5; i++) {
+      regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+    }
+    for (int i = 0; i < 3; i++) {
+      regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+    }
+    for (int i = 0; i < 10; i++) {
+      regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+    }
+    for (int i = 0; i < 8; i++) {
+      regionReports.put(new HRegionInfo(tn4, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+    }
+    assertEquals(26, regionReports.size());
+    assertEquals(5, size(store.filterBySubject(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR)));
+    assertEquals(3, size(store.filterBySubject("sn")));
+    assertEquals(18, size(store.filterBySubject("ns")));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bcf6da40/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java
new file mode 100644
index 0000000..db549e4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java
@@ -0,0 +1,106 @@
+/*
+ * 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.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * Non-HBase cluster unit tests for {@link QuotaObserverChore}.
+ */
+@Category(SmallTests.class)
+public class TestQuotaObserverChore {
+  private Connection conn;
+  private QuotaObserverChore chore;
+
+  @Before
+  public void setup() throws Exception {
+    conn = mock(Connection.class);
+    chore = mock(QuotaObserverChore.class);
+    // Set up some rules to call the real method on the mock.
+    when(chore.getViolationPolicy(any(SpaceQuota.class))).thenCallRealMethod();
+  }
+
+  @Test
+  public void testNumRegionsForTable() {
+    TableName tn1 = TableName.valueOf("t1");
+    TableName tn2 = TableName.valueOf("t2");
+    TableName tn3 = TableName.valueOf("t3");
+
+    final int numTable1Regions = 10;
+    final int numTable2Regions = 15;
+    final int numTable3Regions = 8;
+    Map<HRegionInfo,Long> regionReports = new HashMap<>();
+    for (int i = 0; i < numTable1Regions; i++) {
+      regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(i), Bytes.toBytes(i + 1)), 0L);
+    }
+
+    for (int i = 0; i < numTable2Regions; i++) {
+      regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(i), Bytes.toBytes(i + 1)), 0L);
+    }
+
+    for (int i = 0; i < numTable3Regions; i++) {
+      regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i + 1)), 0L);
+    }
+
+    TableQuotaViolationStore store = new TableQuotaViolationStore(conn, chore, regionReports);
+    when(chore.getTableViolationStore()).thenReturn(store);
+
+    assertEquals(numTable1Regions, Iterables.size(store.filterBySubject(tn1)));
+    assertEquals(numTable2Regions, Iterables.size(store.filterBySubject(tn2)));
+    assertEquals(numTable3Regions, Iterables.size(store.filterBySubject(tn3)));
+  }
+
+  @Test
+  public void testExtractViolationPolicy() {
+    for (SpaceViolationPolicy policy : SpaceViolationPolicy.values()) {
+      SpaceQuota spaceQuota = SpaceQuota.newBuilder()
+          .setSoftLimit(1024L)
+          .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(policy))
+          .build();
+      assertEquals(policy, chore.getViolationPolicy(spaceQuota));
+    }
+    SpaceQuota malformedQuota = SpaceQuota.newBuilder()
+        .setSoftLimit(1024L)
+        .build();
+    try {
+      chore.getViolationPolicy(malformedQuota);
+      fail("Should have thrown an IllegalArgumentException.");
+    } catch (IllegalArgumentException e) {
+      // Pass
+    }
+  }
+}
\ No newline at end of file


[46/50] [abbrv] hbase git commit: HBASE-17003 Documentation updates for space quotas

Posted by el...@apache.org.
HBASE-17003 Documentation updates for space quotas


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

Branch: refs/heads/HBASE-16961
Commit: 5b3926b941fcc0acd026c9892d6a72345659e1f5
Parents: 96c6b8f
Author: Josh Elser <el...@apache.org>
Authored: Thu Mar 16 16:21:14 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:47:49 2017 -0400

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/ops_mgt.adoc | 64 ++++++++++++++++++++++++++-
 1 file changed, 63 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5b3926b9/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index e4c077f..f9009f3 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -1705,7 +1705,7 @@ handling multiple workloads:
 
 [[quota]]
 === Quotas
-HBASE-11598 introduces quotas, which allow you to throttle requests based on
+HBASE-11598 introduces RPC quotas, which allow you to throttle requests based on
 the following limits:
 
 . <<request-quotas,The number or size of requests(read, write, or read+write) in a given timeframe>>
@@ -1885,6 +1885,68 @@ at the same time and that fewer scans can be executed at the same time. A value
 `0.9` will give more queue/handlers to scans, so the number of scans executed will
 increase and the number of gets will decrease.
 
+[[space-quotas]]
+=== Space Quotas
+
+link:https://issues.apache.org/jira/browse/HBASE-16961[HBASE-16961] introduces a new type of
+quotas for HBase to leverage: filesystem quotas. These "space" quotas limit the amount of space
+on the filesystem that HBase namespaces and tables can consume. If a user, malicious or ignorant,
+has the ability to write data into HBase, with enough time, that user can effectively crash HBase
+(or worse HDFS) by consuming all available space. When there is no filesystem space available,
+HBase crashes because it can no longer create/sync data to the write-ahead log.
+
+This feature allows a for a limit to be set on the size of a table or namespace. When a space quota is set
+on a namespace, the quota's limit applies to the sum of usage of all tables in that namespace.
+When a table with a quota exists in a namespace with a quota, the table quota takes priority
+over the namespace quota. This allows for a scenario where a large limit can be placed on
+a collection of tables, but a single table in that collection can have a fine-grained limit set.
+
+The existing `set_quota` and `list_quota` HBase shell commands can be used to interact with
+space quotas. Space quotas are quotas with a `TYPE` of `SPACE` and have `LIMIT` and `POLICY`
+attributes. The `LIMIT` is a string that refers to the amount of space on the filesystem
+that the quota subject (e.g. the table or namespace) may consume. For example, valid values
+of `LIMIT` are `'10G'`, `'2T'`, or `'256M'`. The `POLICY` refers to the action that HBase will
+take when the quota subject's usage exceeds the `LIMIT`. The following are valid `POLICY` values.
+
+* `NO_INSERTS` - No new data may be written (e.g. `Put`, `Increment`, `Append`).
+* `NO_WRITES` - Same as `NO_INSERTS` but `Deletes` are also disallowed.
+* `NO_WRITES_COMPACTIONS` - Same as `NO_WRITES` but compactions are also disallowed.
+* `DISABLE` - The table(s) are disabled, preventing all read/write access.
+
+.Setting simple space quotas
+----
+# Sets a quota on the table 't1' with a limit of 1GB, disallowing Puts/Increments/Appends when the table exceeds 1GB
+hbase> set_quota TYPE => SPACE, TABLE => 't1', LIMIT => '1G', POLICY => NO_INSERTS
+
+# Sets a quota on the namespace 'ns1' with a limit of 50TB, disallowing Puts/Increments/Appends/Deletes
+hbase> set_quota TYPE => SPACE, NAMESPACE => 'ns1', LIMIT => '50T', POLICY => NO_WRITES
+
+# Sets a quota on the table 't3' with a limit of 2TB, disallowing any writes and compactions when the table exceeds 2TB.
+hbase> set_quota TYPE => SPACE, TABLE => 't3', LIMIT => '2T', POLICY => NO_WRITES_COMPACTIONS
+
+# Sets a quota on the table 't2' with a limit of 50GB, disabling the table when it exceeds 50GB
+hbase> set_quota TYPE => SPACE, TABLE => 't2', LIMIT => '50G', POLICY => DISABLE
+----
+
+Consider the following scenario to set up quotas on a namespace, overriding the quota on tables in that namespace
+
+.Table and Namespace space quotas
+----
+hbase> create_namespace 'ns1'
+hbase> create 'ns1:t1'
+hbase> create 'ns1:t2'
+hbase> create 'ns1:t3'
+hbase> set_quota TYPE => SPACE, NAMESPACE => 'ns1', LIMIT => '100T', POLICY => NO_INSERTS
+hbase> set_quota TYPE => SPACE, TABLE => 'ns1:t2', LIMIT => '200G', POLICY => NO_WRITES
+hbase> set_quota TYPE => SPACE, TABLE => 'ns1:t3', LIMIT => '20T', POLICY => NO_WRITES
+----
+
+In the above scenario, the tables in the namespace `ns1` will not be allowed to consume more than
+100TB of space on the filesystem among each other. The table 'ns1:t2' is only allowed to be 200GB in size, and will
+disallow all writes when the usage exceeds this limit. The table 'ns1:t3' is allowed to grow to 20TB in size
+and also will disallow all writes then the usage exceeds this limit. Because there is no table quota
+on 'ns1:t1', this table can grow up to 100TB, but only if 'ns1:t2' and 'ns1:t3' have a usage of zero bytes.
+Practically, it's limit is 100TB less the current usage of 'ns1:t2' and 'ns1:t3'.
 
 [[ops.backup]]
 == HBase Backup


[03/50] [abbrv] hbase git commit: HBASE-17090 Redundant exclusion of jruby-complete in pom of hbase-spark

Posted by el...@apache.org.
HBASE-17090 Redundant exclusion of jruby-complete in pom of hbase-spark

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-16961
Commit: e2a746152ca8c02c18214f0b5180ed8dcc84e947
Parents: 9dd5cda
Author: Xiang Li <wa...@gmail.com>
Authored: Fri Apr 14 16:15:42 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Apr 14 08:08:42 2017 -0700

----------------------------------------------------------------------
 hbase-spark/pom.xml | 24 ------------------------
 1 file changed, 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e2a74615/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index a7997f1..1afae85 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -290,10 +290,6 @@
                     <artifactId>thrift</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.slf4j</groupId>
                     <artifactId>slf4j-log4j12</artifactId>
                 </exclusion>
@@ -338,10 +334,6 @@
                     <artifactId>jasper-compiler</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.jboss.netty</groupId>
                     <artifactId>netty</artifactId>
                 </exclusion>
@@ -382,10 +374,6 @@
                     <artifactId>thrift</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.slf4j</groupId>
                     <artifactId>slf4j-log4j12</artifactId>
                 </exclusion>
@@ -430,10 +418,6 @@
                     <artifactId>jasper-compiler</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.jboss.netty</groupId>
                     <artifactId>netty</artifactId>
                 </exclusion>
@@ -460,10 +444,6 @@
                     <artifactId>thrift</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.slf4j</groupId>
                     <artifactId>slf4j-log4j12</artifactId>
                 </exclusion>
@@ -508,10 +488,6 @@
                     <artifactId>jasper-compiler</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.jboss.netty</groupId>
                     <artifactId>netty</artifactId>
                 </exclusion>


[09/50] [abbrv] hbase git commit: HBASE-15535 Correct link to Trafodion

Posted by el...@apache.org.
HBASE-15535 Correct link to Trafodion

Signed-off-by: CHIA-PING TSAI <ch...@gmail.com>


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

Branch: refs/heads/HBASE-16961
Commit: 363f62751c760cc8056a2b1be40a410281e634f7
Parents: 918aa46
Author: G�bor Lipt�k <gl...@gmail.com>
Authored: Sat Apr 15 11:43:38 2017 -0400
Committer: CHIA-PING TSAI <ch...@gmail.com>
Committed: Mon Apr 17 10:26:28 2017 +0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/sql.adoc      | 2 +-
 src/main/site/xdoc/supportingprojects.xml | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/363f6275/src/main/asciidoc/_chapters/sql.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/sql.adoc b/src/main/asciidoc/_chapters/sql.adoc
index b47104c..b1ad063 100644
--- a/src/main/asciidoc/_chapters/sql.adoc
+++ b/src/main/asciidoc/_chapters/sql.adoc
@@ -37,6 +37,6 @@ link:http://phoenix.apache.org[Apache Phoenix]
 
 === Trafodion
 
-link:https://wiki.trafodion.org/[Trafodion: Transactional SQL-on-HBase]
+link:http://trafodion.incubator.apache.org/[Trafodion: Transactional SQL-on-HBase]
 
 :numbered:

http://git-wip-us.apache.org/repos/asf/hbase/blob/363f6275/src/main/site/xdoc/supportingprojects.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/supportingprojects.xml b/src/main/site/xdoc/supportingprojects.xml
index f349c7f..f949a57 100644
--- a/src/main/site/xdoc/supportingprojects.xml
+++ b/src/main/site/xdoc/supportingprojects.xml
@@ -46,9 +46,9 @@ under the License.
     for HBase.</dd>
    <dt><a href="https://github.com/juwi/HBase-TAggregator">HBase TAggregator</a></dt>
    <dd>An HBase coprocessor for timeseries-based aggregations.</dd>
-   <dt><a href="http://www.trafodion.org">Trafodion</a></dt>
-   <dd>Trafodion is an HP-sponsored Apache-licensed open source SQL on HBase
-    DBMS with full-ACID distributed transaction support.</dd>
+   <dt><a href="http://trafodion.incubator.apache.org/">Apache Trafodion</a></dt>
+   <dd>Apache Trafodion is a webscale SQL-on-Hadoop solution enabling
+    transactional or operational workloads on Hadoop.</dd>
    <dt><a href="http://phoenix.apache.org/">Apache Phoenix</a></dt>
    <dd>Apache Phoenix is a relational database layer over HBase delivered as a
     client-embedded JDBC driver targeting low latency queries over HBase data.</dd>


[02/50] [abbrv] hbase git commit: HBASE-17906 When a huge amount of data writing to hbase through thrift2, there will be a deadlock error. (Albert Lee)

Posted by el...@apache.org.
HBASE-17906 When a huge amount of data writing to hbase through thrift2, there will be a deadlock error. (Albert Lee)


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

Branch: refs/heads/HBASE-16961
Commit: 9dd5cda01747ffb91ac084792fa4a8670859e810
Parents: da5fb27
Author: Michael Stack <st...@apache.org>
Authored: Thu Apr 13 21:59:11 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu Apr 13 21:59:11 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java   | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9dd5cda0/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
index 560ae64..8f56b10 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
@@ -432,9 +432,6 @@ public class ThriftServer extends Configured implements Tool {
       throw new RuntimeException("Could not parse the value provided for the port option", e);
     }
 
-    // Thrift's implementation uses '0' as a placeholder for 'use the default.'
-    int backlog = conf.getInt(BACKLOG_CONF_KEY, 0);
-
     // Local hostname and user name,
     // used only if QOP is configured.
     String host = null;


[34/50] [abbrv] hbase git commit: HBASE-17478 Avoid reporting FS use when quotas are disabled

Posted by el...@apache.org.
HBASE-17478 Avoid reporting FS use when quotas are disabled

Also, gracefully produce responses when quotas are disabled.


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

Branch: refs/heads/HBASE-16961
Commit: efd6edc44193155fabbf8dfe8d7af360087023c1
Parents: 7a1f9d3
Author: Josh Elser <el...@apache.org>
Authored: Tue Jan 17 14:41:45 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:32 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hbase/master/MasterRpcServices.java  |  4 +++
 .../hadoop/hbase/quotas/MasterQuotaManager.java | 13 +++++--
 .../hbase/regionserver/HRegionServer.java       |  5 ++-
 .../hbase/quotas/TestMasterQuotaManager.java    | 37 ++++++++++++++++++++
 4 files changed, 56 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/efd6edc4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index f454248..de437fd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
+import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -2016,6 +2017,9 @@ public class MasterRpcServices extends RSRpcServices
       RegionSpaceUseReportRequest request) throws ServiceException {
     try {
       master.checkInitialized();
+      if (!QuotaUtil.isQuotaEnabled(master.getConfiguration())) {
+        return RegionSpaceUseReportResponse.newBuilder().build();
+      }
       MasterQuotaManager quotaManager = this.master.getMasterQuotaManager();
       for (RegionSpaceUse report : request.getSpaceUseList()) {
         quotaManager.addRegionSize(HRegionInfo.convert(report.getRegion()), report.getSize());

http://git-wip-us.apache.org/repos/asf/hbase/blob/efd6edc4/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index a5832f9..cb614ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.quotas;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -58,6 +59,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
 @InterfaceStability.Evolving
 public class MasterQuotaManager implements RegionStateListener {
   private static final Log LOG = LogFactory.getLog(MasterQuotaManager.class);
+  private static final Map<HRegionInfo, Long> EMPTY_MAP = Collections.unmodifiableMap(
+      new HashMap<>());
 
   private final MasterServices masterServices;
   private NamedLock<String> namespaceLocks;
@@ -529,13 +532,19 @@ public class MasterQuotaManager implements RegionStateListener {
   }
 
   public void addRegionSize(HRegionInfo hri, long size) {
-    // TODO Make proper API
+    if (null == regionSizes) {
+      return;
+    }
+    // TODO Make proper API?
     // TODO Prevent from growing indefinitely
     regionSizes.put(hri, size);
   }
 
   public Map<HRegionInfo, Long> snapshotRegionSizes() {
-    // TODO Make proper API
+    if (null == regionSizes) {
+      return EMPTY_MAP;
+    }
+    // TODO Make proper API?
     return new HashMap<>(regionSizes);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/efd6edc4/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 79e2c46..81c58c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -119,6 +119,7 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.mob.MobCacheConfig;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
 import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore;
+import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
@@ -933,7 +934,9 @@ public class HRegionServer extends HasThread implements
     rsQuotaManager = new RegionServerRpcQuotaManager(this);
     rsSpaceQuotaManager = new RegionServerSpaceQuotaManager(this);
 
-    this.fsUtilizationChore = new FileSystemUtilizationChore(this);
+    if (QuotaUtil.isQuotaEnabled(conf)) {
+      this.fsUtilizationChore = new FileSystemUtilizationChore(this);
+    }
 
     // Setup RPC client for master communication
     rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(

http://git-wip-us.apache.org/repos/asf/hbase/blob/efd6edc4/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java
new file mode 100644
index 0000000..e383593
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java
@@ -0,0 +1,37 @@
+/*
+ * 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.assertNotNull;
+import static org.mockito.Mockito.mock;
+
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestMasterQuotaManager {
+
+  @Test
+  public void testUninitializedQuotaManangerDoesNotFail() {
+    MasterServices masterServices = mock(MasterServices.class);
+    MasterQuotaManager manager = new MasterQuotaManager(masterServices);
+    manager.addRegionSize(null, 0);
+    assertNotNull(manager.snapshotRegionSizes());
+  }
+}


[45/50] [abbrv] hbase git commit: HBASE-17447 Implement a MasterObserver for automatically deleting space quotas

Posted by el...@apache.org.
HBASE-17447 Implement a MasterObserver for automatically deleting space quotas

When a table or namespace is deleted, it would be nice to automatically
delete the quota on said table/NS. It's possible that not all people
would want this functionality so we can leave it up to the user to
configure this Observer.


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

Branch: refs/heads/HBASE-16961
Commit: 2524f8cd26422ff7a44cca2aa6c913f713c1d530
Parents: abe1c06
Author: Josh Elser <el...@apache.org>
Authored: Thu Mar 16 18:54:01 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:47:49 2017 -0400

----------------------------------------------------------------------
 .../hbase/quotas/MasterSpaceQuotaObserver.java  |  85 ++++++++++
 .../quotas/TestMasterSpaceQuotaObserver.java    | 169 +++++++++++++++++++
 src/main/asciidoc/_chapters/ops_mgt.adoc        |  17 ++
 3 files changed, 271 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2524f8cd/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.java
new file mode 100644
index 0000000..a3abf32
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+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.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+
+/**
+ * An observer to automatically delete space quotas when a table/namespace
+ * are deleted.
+ */
+@InterfaceAudience.Private
+public class MasterSpaceQuotaObserver implements MasterObserver {
+  private CoprocessorEnvironment cpEnv;
+  private Configuration conf;
+  private boolean quotasEnabled = false;
+
+  @Override
+  public void start(CoprocessorEnvironment ctx) throws IOException {
+    this.cpEnv = ctx;
+    this.conf = cpEnv.getConfiguration();
+    this.quotasEnabled = QuotaUtil.isQuotaEnabled(conf);
+  }
+
+  @Override
+  public void postDeleteTable(
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException {
+    // Do nothing if quotas aren't enabled
+    if (!quotasEnabled) {
+      return;
+    }
+    final MasterServices master = ctx.getEnvironment().getMasterServices();
+    final Connection conn = master.getConnection();
+    Quotas quotas = QuotaUtil.getTableQuota(master.getConnection(), tableName);
+    if (null != quotas && quotas.hasSpace()) {
+      QuotaSettings settings = QuotaSettingsFactory.removeTableSpaceLimit(tableName);
+      try (Admin admin = conn.getAdmin()) {
+        admin.setQuota(settings);
+      }
+    }
+  }
+
+  @Override
+  public void postDeleteNamespace(
+      ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException {
+    // Do nothing if quotas aren't enabled
+    if (!quotasEnabled) {
+      return;
+    }
+    final MasterServices master = ctx.getEnvironment().getMasterServices();
+    final Connection conn = master.getConnection();
+    Quotas quotas = QuotaUtil.getNamespaceQuota(master.getConnection(), namespace);
+    if (null != quotas && quotas.hasSpace()) {
+      QuotaSettings settings = QuotaSettingsFactory.removeNamespaceSpaceLimit(namespace);
+      try (Admin admin = conn.getAdmin()) {
+        admin.setQuota(settings);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2524f8cd/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java
new file mode 100644
index 0000000..a1eee4f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java
@@ -0,0 +1,169 @@
+/*
+ * 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 java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.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;
+
+/**
+ * Test class for {@link MasterSpaceQuotaObserver}.
+ */
+@Category(MediumTests.class)
+public class TestMasterSpaceQuotaObserver {
+  private static final Log LOG = LogFactory.getLog(TestSpaceQuotas.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, MasterSpaceQuotaObserver.class.getName());
+    conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void removeAllQuotas() throws Exception {
+    final Connection conn = TEST_UTIL.getConnection();
+    // Wait for the quota table to be created
+    if (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME)) {
+      do {
+        LOG.debug("Quota table does not yet exist");
+        Thread.sleep(1000);
+      } while (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME));
+    } else {
+      // Or, clean up any quotas from previous test runs.
+      QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration());
+      for (QuotaSettings quotaSettings : scanner) {
+        final String namespace = quotaSettings.getNamespace();
+        final TableName tableName = quotaSettings.getTableName();
+        if (null != namespace) {
+          LOG.debug("Deleting quota for namespace: " + namespace);
+          QuotaUtil.deleteNamespaceQuota(conn, namespace);
+        } else {
+          assert null != tableName;
+          LOG.debug("Deleting quota for table: "+ tableName);
+          QuotaUtil.deleteTableQuota(conn, tableName);
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testTableQuotaRemoved() throws Exception {
+    final Connection conn = TEST_UTIL.getConnection();
+    final Admin admin = conn.getAdmin();
+    final TableName tn = TableName.valueOf(testName.getMethodName());
+    // Drop the table if it somehow exists
+    if (admin.tableExists(tn)) {
+      admin.disableTable(tn);
+      admin.deleteTable(tn);
+    }
+
+    // Create a table
+    HTableDescriptor tableDesc = new HTableDescriptor(tn);
+    tableDesc.addFamily(new HColumnDescriptor("F1"));
+    admin.createTable(tableDesc);
+    assertEquals(0, getNumSpaceQuotas());
+
+    // Set a quota
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, 1024L, SpaceViolationPolicy.NO_INSERTS);
+    admin.setQuota(settings);
+    assertEquals(1, getNumSpaceQuotas());
+
+    // Delete the table and observe the quota being automatically deleted as well
+    admin.disableTable(tn);
+    admin.deleteTable(tn);
+    assertEquals(0, getNumSpaceQuotas());
+  }
+
+  @Test
+  public void testNamespaceQuotaRemoved() throws Exception {
+    final Connection conn = TEST_UTIL.getConnection();
+    final Admin admin = conn.getAdmin();
+    final String ns = testName.getMethodName();
+    // Drop the ns if it somehow exists
+    if (namespaceExists(ns)) {
+      admin.deleteNamespace(ns);
+    }
+
+    // Create the ns
+    NamespaceDescriptor desc = NamespaceDescriptor.create(ns).build();
+    admin.createNamespace(desc);
+    assertEquals(0, getNumSpaceQuotas());
+
+    // Set a quota
+    QuotaSettings settings = QuotaSettingsFactory.limitNamespaceSpace(
+        ns, 1024L, SpaceViolationPolicy.NO_INSERTS);
+    admin.setQuota(settings);
+    assertEquals(1, getNumSpaceQuotas());
+
+    // Delete the table and observe the quota being automatically deleted as well
+    admin.deleteNamespace(ns);
+    assertEquals(0, getNumSpaceQuotas());
+  }
+
+  public boolean namespaceExists(String ns) throws IOException {
+    NamespaceDescriptor[] descs = TEST_UTIL.getAdmin().listNamespaceDescriptors();
+    for (NamespaceDescriptor desc : descs) {
+      if (ns.equals(desc.getName())) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public int getNumSpaceQuotas() throws Exception {
+    QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration());
+    int numSpaceQuotas = 0;
+    for (QuotaSettings quotaSettings : scanner) {
+      if (quotaSettings.getQuotaType() == QuotaType.SPACE) {
+        numSpaceQuotas++;
+      }
+    }
+    return numSpaceQuotas;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2524f8cd/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index f9009f3..f60395b 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -1948,6 +1948,23 @@ and also will disallow all writes then the usage exceeds this limit. Because the
 on 'ns1:t1', this table can grow up to 100TB, but only if 'ns1:t2' and 'ns1:t3' have a usage of zero bytes.
 Practically, it's limit is 100TB less the current usage of 'ns1:t2' and 'ns1:t3'.
 
+[[ops.space.quota.deletion]]
+=== Automatic Space Quota Deletion
+
+By default, if a table or namespace is deleted that has a space quota, the quota itself is
+not also deleted. In some cases, it may be desirable for the space quota to be automatically deleted.
+In these cases, the user may configure the MasterSpaceQuotaObserver to delete any space quota
+automatically in hbase-site.xml.
+
+[source,java]
+----
+
+  <property>
+    <name>hbase.coprocessor.master.classes</name>
+    <value>...,org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver</value>
+  </property>
+----
+
 [[ops.backup]]
 == HBase Backup
 


[06/50] [abbrv] hbase git commit: HBASE-17904 Get runs into NoSuchElementException when using Read Replica, with hbase. ipc.client.specificThreadForWriting to be true and hbase.rpc.client.impl to be org.apache.hadoop.hbase.ipc.RpcClientImpl (Huaxiang Sun

Posted by el...@apache.org.
HBASE-17904 Get runs into NoSuchElementException when using Read Replica, with hbase. ipc.client.specificThreadForWriting
to be true and hbase.rpc.client.impl to be org.apache.hadoop.hbase.ipc.RpcClientImpl (Huaxiang Sun)


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

Branch: refs/heads/HBASE-16961
Commit: 7678855fac011a9c02e5d6a42470c0178482a4ce
Parents: 0cd4cec
Author: Michael Stack <st...@apache.org>
Authored: Sun Apr 16 11:00:57 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Sun Apr 16 11:01:06 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/ipc/BlockingRpcConnection.java |  2 +-
 .../hbase/client/TestReplicaWithCluster.java    | 50 ++++++++++++++++++++
 2 files changed, 51 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7678855f/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
index 15eb10c..1012ad0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
@@ -156,7 +156,7 @@ class BlockingRpcConnection extends RpcConnection implements Runnable {
     }
 
     public void remove(Call call) {
-      callsToWrite.remove();
+      callsToWrite.remove(call);
       // By removing the call from the expected call list, we make the list smaller, but
       // it means as well that we don't know how many calls we cancelled.
       calls.remove(call.id);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7678855f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index becb2eb..2c77541 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Waiter;
+
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -515,7 +516,56 @@ public class TestReplicaWithCluster {
 
       Assert.assertTrue(r.isStale());
     } finally {
+      HTU.getAdmin().disableTable(hdt.getTableName());
+      HTU.deleteTable(hdt.getTableName());
+    }
+  }
+
+  @Test
+  public void testReplicaGetWithRpcClientImpl() throws IOException {
+    HTU.getConfiguration().setBoolean("hbase.ipc.client.specificThreadForWriting", true);
+    HTU.getConfiguration().set("hbase.rpc.client.impl", "org.apache.hadoop.hbase.ipc.RpcClientImpl");
+    // Create table then get the single region for our new table.
+    HTableDescriptor hdt = HTU.createTableDescriptor("testReplicaGetWithRpcClientImpl");
+    hdt.setRegionReplication(NB_SERVERS);
+    hdt.addCoprocessor(SlowMeCopro.class.getName());
+
+    try {
+      Table table = HTU.createTable(hdt, new byte[][] { f }, null);
+
+      Put p = new Put(row);
+      p.addColumn(f, row, row);
+      table.put(p);
 
+      // Flush so it can be picked by the replica refresher thread
+      HTU.flush(table.getName());
+
+      // Sleep for some time until data is picked up by replicas
+      try {
+        Thread.sleep(2 * REFRESH_PERIOD);
+      } catch (InterruptedException e1) {
+        LOG.error(e1);
+      }
+
+      try {
+        // Create the new connection so new config can kick in
+        Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration());
+        Table t = connection.getTable(hdt.getTableName());
+
+        // But if we ask for stale we will get it
+        SlowMeCopro.cdl.set(new CountDownLatch(1));
+        Get g = new Get(row);
+        g.setConsistency(Consistency.TIMELINE);
+        Result r = t.get(g);
+        Assert.assertTrue(r.isStale());
+        SlowMeCopro.cdl.get().countDown();
+      } finally {
+        SlowMeCopro.cdl.get().countDown();
+        SlowMeCopro.sleepTime.set(0);
+      }
+    } finally {
+      HTU.getConfiguration().unset("hbase.ipc.client.specificThreadForWriting");
+      HTU.getConfiguration().unset("hbase.rpc.client.impl");
       HTU.getAdmin().disableTable(hdt.getTableName());
       HTU.deleteTable(hdt.getTableName());
     }


[26/50] [abbrv] hbase git commit: HBASE-16995 Build client Java API and client protobuf messages - addendum fixes line lengths (Josh Elser)

Posted by el...@apache.org.
HBASE-16995 Build client Java API and client protobuf messages - addendum fixes line lengths (Josh Elser)


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

Branch: refs/heads/HBASE-16961
Commit: 988a23eff38e460ec383544930d48147d4771d4c
Parents: eaeef44
Author: tedyu <yu...@gmail.com>
Authored: Mon Nov 21 13:00:27 2016 -0800
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:31 2017 -0400

----------------------------------------------------------------------
 .../hbase/quotas/QuotaSettingsFactory.java      | 20 ++++++++++++--------
 .../hadoop/hbase/quotas/SpaceLimitSettings.java |  8 ++++----
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  9 +++++----
 3 files changed, 21 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/988a23ef/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
index 8512e39..7f1c180 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
@@ -128,7 +128,8 @@ public class QuotaSettingsFactory {
 
   static QuotaSettings fromSpace(TableName table, String namespace, SpaceQuota protoQuota) {
     if ((null == table && null == namespace) || (null != table && null != namespace)) {
-      throw new IllegalArgumentException("Can only construct SpaceLimitSettings for a table or namespace.");
+      throw new IllegalArgumentException(
+          "Can only construct SpaceLimitSettings for a table or namespace.");
     }
     if (null != table) {
       return SpaceLimitSettings.fromSpaceQuota(table, protoQuota);
@@ -300,29 +301,32 @@ public class QuotaSettingsFactory {
    */
 
   /**
-   * Creates a {@link QuotaSettings} object to limit the FileSystem space usage for the given table to the given size in bytes.
-   * When the space usage is exceeded by the table, the provided {@link SpaceViolationPolicy} is enacted on the table.
+   * Creates a {@link QuotaSettings} object to limit the FileSystem space usage for the given table
+   * to the given size in bytes. When the space usage is exceeded by the table, the provided
+   * {@link SpaceViolationPolicy} is enacted on the table.
    *
    * @param tableName The name of the table on which the quota should be applied.
    * @param sizeLimit The limit of a table's size in bytes.
    * @param violationPolicy The action to take when the quota is exceeded.
    * @return An {@link QuotaSettings} object.
    */
-  public static QuotaSettings limitTableSpace(final TableName tableName, long sizeLimit, final SpaceViolationPolicy violationPolicy) {
+  public static QuotaSettings limitTableSpace(
+      final TableName tableName, long sizeLimit, final SpaceViolationPolicy violationPolicy) {
     return new SpaceLimitSettings(tableName, sizeLimit, violationPolicy);
   }
 
   /**
-   * Creates a {@link QuotaSettings} object to limit the FileSystem space usage for the given namespace to the given size in bytes.
-   * When the space usage is exceeded by all tables in the namespace, the provided {@link SpaceViolationPolicy} is enacted on
-   * all tables in the namespace.
+   * Creates a {@link QuotaSettings} object to limit the FileSystem space usage for the given
+   * namespace to the given size in bytes. When the space usage is exceeded by all tables in the
+   * namespace, the provided {@link SpaceViolationPolicy} is enacted on all tables in the namespace.
    *
    * @param namespace The namespace on which the quota should be applied.
    * @param sizeLimit The limit of the namespace's size in bytes.
    * @param violationPolicy The action to take when the the quota is exceeded.
    * @return An {@link QuotaSettings} object.
    */
-  public static QuotaSettings limitNamespaceSpace(final String namespace, long sizeLimit, final SpaceViolationPolicy violationPolicy) {
+  public static QuotaSettings limitNamespaceSpace(
+      final String namespace, long sizeLimit, final SpaceViolationPolicy violationPolicy) {
     return new SpaceLimitSettings(namespace, sizeLimit, violationPolicy);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/988a23ef/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
index dded9b5..e54882e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
@@ -93,8 +93,8 @@ class SpaceLimitSettings extends QuotaSettings {
    * @param proto The protobuf representation.
    * @return A QuotaSettings.
    */
-  static SpaceLimitSettings fromSpaceQuota(final TableName tableName,
-      final QuotaProtos.SpaceQuota proto) {
+  static SpaceLimitSettings fromSpaceQuota(
+      final TableName tableName, final QuotaProtos.SpaceQuota proto) {
     validateProtoArguments(proto);
     return new SpaceLimitSettings(tableName, proto.getSoftLimit(),
         ProtobufUtil.toViolationPolicy(proto.getViolationPolicy()));
@@ -107,8 +107,8 @@ class SpaceLimitSettings extends QuotaSettings {
    * @param proto The protobuf representation.
    * @return A QuotaSettings.
    */
-  static SpaceLimitSettings fromSpaceQuota(final String namespace,
-      final QuotaProtos.SpaceQuota proto) {
+  static SpaceLimitSettings fromSpaceQuota(
+      final String namespace, final QuotaProtos.SpaceQuota proto) {
     validateProtoArguments(proto);
     return new SpaceLimitSettings(namespace, proto.getSoftLimit(),
         ProtobufUtil.toViolationPolicy(proto.getViolationPolicy()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/988a23ef/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 460fb59..b5ea9a0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -2559,7 +2559,8 @@ public final class ProtobufUtil {
    * @param proto The protocol buffer space violation policy.
    * @return The corresponding client SpaceViolationPolicy.
    */
-  public static SpaceViolationPolicy toViolationPolicy(final QuotaProtos.SpaceViolationPolicy proto) {
+  public static SpaceViolationPolicy toViolationPolicy(
+      final QuotaProtos.SpaceViolationPolicy proto) {
     switch (proto) {
       case DISABLE: return SpaceViolationPolicy.DISABLE;
       case NO_WRITES_COMPACTIONS: return SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
@@ -2610,8 +2611,8 @@ public final class ProtobufUtil {
    * @param violationPolicy The policy to apply when the quota is violated.
    * @return The protocol buffer SpaceQuota.
    */
-  public static QuotaProtos.SpaceQuota toProtoSpaceQuota(final long limit,
-      final SpaceViolationPolicy violationPolicy) {
+  public static QuotaProtos.SpaceQuota toProtoSpaceQuota(
+      final long limit, final SpaceViolationPolicy violationPolicy) {
     return QuotaProtos.SpaceQuota.newBuilder()
         .setSoftLimit(limit)
         .setViolationPolicy(toProtoViolationPolicy(violationPolicy))
@@ -3313,4 +3314,4 @@ public final class ProtobufUtil {
     int port = Addressing.parsePort(str);
     return ServerName.valueOf(hostname, port, -1L);
   }
-}
\ No newline at end of file
+}


[35/50] [abbrv] hbase git commit: HBASE-17259 API to remove space quotas on a table/namespace

Posted by el...@apache.org.
HBASE-17259 API to remove space quotas on a table/namespace


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

Branch: refs/heads/HBASE-16961
Commit: 0effca421562066d10d13575f5c12434e2741c63
Parents: 0d76d66
Author: Josh Elser <el...@apache.org>
Authored: Wed Jan 11 12:47:06 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:32 2017 -0400

----------------------------------------------------------------------
 .../hbase/quotas/QuotaSettingsFactory.java      |  22 +++
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |   6 +-
 .../hadoop/hbase/quotas/SpaceLimitSettings.java |  44 ++++-
 .../hbase/quotas/TestQuotaSettingsFactory.java  |  20 +++
 .../shaded/protobuf/generated/QuotaProtos.java  | 157 +++++++++++++++---
 .../src/main/protobuf/Quota.proto               |   1 +
 .../hbase/protobuf/generated/QuotaProtos.java   | 159 ++++++++++++++++---
 hbase-protocol/src/main/protobuf/Quota.proto    |   1 +
 .../hadoop/hbase/quotas/MasterQuotaManager.java |   9 +-
 .../hadoop/hbase/quotas/TestQuotaAdmin.java     |  49 +++++-
 10 files changed, 423 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0effca42/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
index 7f1c180..184277d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
@@ -316,6 +316,17 @@ public class QuotaSettingsFactory {
   }
 
   /**
+   * Creates a {@link QuotaSettings} object to remove the FileSystem space quota for the given
+   * table.
+   *
+   * @param tableName The name of the table to remove the quota for.
+   * @return A {@link QuotaSettings} object.
+   */
+  public static QuotaSettings removeTableSpaceLimit(TableName tableName) {
+    return new SpaceLimitSettings(tableName, true);
+  }
+
+  /**
    * Creates a {@link QuotaSettings} object to limit the FileSystem space usage for the given
    * namespace to the given size in bytes. When the space usage is exceeded by all tables in the
    * namespace, the provided {@link SpaceViolationPolicy} is enacted on all tables in the namespace.
@@ -329,4 +340,15 @@ public class QuotaSettingsFactory {
       final String namespace, long sizeLimit, final SpaceViolationPolicy violationPolicy) {
     return new SpaceLimitSettings(namespace, sizeLimit, violationPolicy);
   }
+
+  /**
+   * Creates a {@link QuotaSettings} object to remove the FileSystem space quota for the given
+	 * namespace.
+   *
+   * @param namespace The namespace to remove the quota on.
+   * @return A {@link QuotaSettings} object.
+   */
+  public static QuotaSettings removeNamespaceSpaceLimit(String namespace) {
+    return new SpaceLimitSettings(namespace, true);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0effca42/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 66535b2..ce4cd04 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
@@ -422,7 +422,11 @@ public class QuotaTableUtil {
     boolean hasSettings = false;
     hasSettings |= quotas.hasThrottle();
     hasSettings |= quotas.hasBypassGlobals();
-    hasSettings |= quotas.hasSpace();
+    // Only when there is a space quota, make sure there's actually both fields provided
+    // Otherwise, it's a noop.
+    if (quotas.hasSpace()) {
+      hasSettings |= (quotas.getSpace().hasSoftLimit() && quotas.getSpace().hasViolationPolicy());
+    }
     return !hasSettings;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0effca42/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
index e54882e..8ff7623 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
@@ -41,7 +41,15 @@ class SpaceLimitSettings extends QuotaSettings {
     if (0L > sizeLimit) {
       throw new IllegalArgumentException("Size limit must be a non-negative value.");
     }
-    proto = buildProtoQuota(sizeLimit, Objects.requireNonNull(violationPolicy));
+    proto = buildProtoAddQuota(sizeLimit, Objects.requireNonNull(violationPolicy));
+  }
+
+  SpaceLimitSettings(TableName tableName, boolean remove) {
+    super(null, Objects.requireNonNull(tableName), null);
+    if (!remove) {
+      throw new IllegalArgumentException("A value of 'false' for removing a quota makes no sense");
+    }
+    proto = buildProtoRemoveQuota();
   }
 
   SpaceLimitSettings(String namespace, long sizeLimit, SpaceViolationPolicy violationPolicy) {
@@ -49,7 +57,15 @@ class SpaceLimitSettings extends QuotaSettings {
     if (0L > sizeLimit) {
       throw new IllegalArgumentException("Size limit must be a non-negative value.");
     }
-    proto = buildProtoQuota(sizeLimit, Objects.requireNonNull(violationPolicy));
+    proto = buildProtoAddQuota(sizeLimit, Objects.requireNonNull(violationPolicy));
+  }
+
+  SpaceLimitSettings(String namespace, boolean remove) {
+    super(null, null, Objects.requireNonNull(namespace));
+    if (!remove) {
+      throw new IllegalArgumentException("A value of 'false' for removing a quota makes no sense");
+    }
+    proto = buildProtoRemoveQuota();
   }
 
   /**
@@ -59,7 +75,8 @@ class SpaceLimitSettings extends QuotaSettings {
    * @param violationPolicy The action to take when the quota is exceeded.
    * @return The protobuf SpaceQuota representation.
    */
-  private SpaceLimitRequest buildProtoQuota(long sizeLimit, SpaceViolationPolicy violationPolicy) {
+  private SpaceLimitRequest buildProtoAddQuota(
+      long sizeLimit, SpaceViolationPolicy violationPolicy) {
     return SpaceLimitRequest.newBuilder().setQuota(
         SpaceQuota.newBuilder()
             .setSoftLimit(sizeLimit)
@@ -69,6 +86,19 @@ class SpaceLimitSettings extends QuotaSettings {
   }
 
   /**
+   * Builds a {@link SpaceQuota} protobuf object to remove a quota.
+   *
+   * @return The protobuf SpaceQuota representation.
+   */
+  private SpaceLimitRequest buildProtoRemoveQuota() {
+    return SpaceLimitRequest.newBuilder().setQuota(
+        SpaceQuota.newBuilder()
+            .setRemove(true)
+            .build())
+        .build();
+  }
+
+  /**
    * Returns a copy of the internal state of <code>this</code>
    */
   SpaceLimitRequest getProto() {
@@ -159,8 +189,12 @@ class SpaceLimitSettings extends QuotaSettings {
     if (null != getNamespace()) {
       sb.append(", NAMESPACE => ").append(getNamespace());
     }
-    sb.append(", LIMIT => ").append(proto.getQuota().getSoftLimit());
-    sb.append(", VIOLATION_POLICY => ").append(proto.getQuota().getViolationPolicy());
+    if (proto.getQuota().getRemove()) {
+      sb.append(", REMOVE => ").append(proto.getQuota().getRemove());
+    } else {
+      sb.append(", LIMIT => ").append(proto.getQuota().getSoftLimit());
+      sb.append(", VIOLATION_POLICY => ").append(proto.getQuota().getViolationPolicy());
+    }
     return sb.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0effca42/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
index e0012a7..edf6926 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
@@ -144,5 +144,25 @@ public class TestQuotaSettingsFactory {
     SpaceQuota quota = protoRequest.getQuota();
     assertEquals(sizeLimit, quota.getSoftLimit());
     assertEquals(violationPolicy, ProtobufUtil.toViolationPolicy(quota.getViolationPolicy()));
+    assertFalse("The remove attribute should be false", quota.getRemove());
   }
+
+  @Test
+  public void testSpaceLimitSettingsForDeletes() {
+    final String ns = "ns1";
+    final TableName tn = TableName.valueOf("tn1");
+    QuotaSettings nsSettings = QuotaSettingsFactory.removeNamespaceSpaceLimit(ns);
+    assertNotNull("QuotaSettings should not be null", nsSettings);
+    assertTrue("Should be an instance of SpaceLimitSettings", nsSettings instanceof SpaceLimitSettings);
+    SpaceLimitRequest nsProto = ((SpaceLimitSettings) nsSettings).getProto();
+    assertTrue("Request should have a SpaceQuota", nsProto.hasQuota());
+    assertTrue("The remove attribute should be true", nsProto.getQuota().getRemove());
+
+    QuotaSettings tableSettings = QuotaSettingsFactory.removeTableSpaceLimit(tn);
+    assertNotNull("QuotaSettings should not be null", tableSettings);
+    assertTrue("Should be an instance of SpaceLimitSettings", tableSettings instanceof SpaceLimitSettings);
+    SpaceLimitRequest tableProto = ((SpaceLimitSettings) tableSettings).getProto();
+    assertTrue("Request should have a SpaceQuota", tableProto.hasQuota());
+    assertTrue("The remove attribute should be true", tableProto.getQuota().getRemove());
+   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0effca42/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
index 117e839..a4c6095 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
@@ -4833,6 +4833,23 @@ public final class QuotaProtos {
      * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 2;</code>
      */
     org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy();
+
+    /**
+     * <pre>
+     * When true, remove the quota.
+     * </pre>
+     *
+     * <code>optional bool remove = 3 [default = false];</code>
+     */
+    boolean hasRemove();
+    /**
+     * <pre>
+     * When true, remove the quota.
+     * </pre>
+     *
+     * <code>optional bool remove = 3 [default = false];</code>
+     */
+    boolean getRemove();
   }
   /**
    * <pre>
@@ -4852,6 +4869,7 @@ public final class QuotaProtos {
     private SpaceQuota() {
       softLimit_ = 0L;
       violationPolicy_ = 1;
+      remove_ = false;
     }
 
     @java.lang.Override
@@ -4898,6 +4916,11 @@ public final class QuotaProtos {
               }
               break;
             }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              remove_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4970,6 +4993,29 @@ public final class QuotaProtos {
       return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE : result;
     }
 
+    public static final int REMOVE_FIELD_NUMBER = 3;
+    private boolean remove_;
+    /**
+     * <pre>
+     * When true, remove the quota.
+     * </pre>
+     *
+     * <code>optional bool remove = 3 [default = false];</code>
+     */
+    public boolean hasRemove() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <pre>
+     * When true, remove the quota.
+     * </pre>
+     *
+     * <code>optional bool remove = 3 [default = false];</code>
+     */
+    public boolean getRemove() {
+      return remove_;
+    }
+
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
@@ -4988,6 +5034,9 @@ public final class QuotaProtos {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeEnum(2, violationPolicy_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(3, remove_);
+      }
       unknownFields.writeTo(output);
     }
 
@@ -5004,6 +5053,10 @@ public final class QuotaProtos {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(2, violationPolicy_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, remove_);
+      }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
       return size;
@@ -5030,6 +5083,11 @@ public final class QuotaProtos {
       if (hasViolationPolicy()) {
         result = result && violationPolicy_ == other.violationPolicy_;
       }
+      result = result && (hasRemove() == other.hasRemove());
+      if (hasRemove()) {
+        result = result && (getRemove()
+            == other.getRemove());
+      }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
     }
@@ -5050,6 +5108,11 @@ public final class QuotaProtos {
         hash = (37 * hash) + VIOLATION_POLICY_FIELD_NUMBER;
         hash = (53 * hash) + violationPolicy_;
       }
+      if (hasRemove()) {
+        hash = (37 * hash) + REMOVE_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
+            getRemove());
+      }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -5176,6 +5239,8 @@ public final class QuotaProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         violationPolicy_ = 1;
         bitField0_ = (bitField0_ & ~0x00000002);
+        remove_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -5208,6 +5273,10 @@ public final class QuotaProtos {
           to_bitField0_ |= 0x00000002;
         }
         result.violationPolicy_ = violationPolicy_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.remove_ = remove_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -5256,6 +5325,9 @@ public final class QuotaProtos {
         if (other.hasViolationPolicy()) {
           setViolationPolicy(other.getViolationPolicy());
         }
+        if (other.hasRemove()) {
+          setRemove(other.getRemove());
+        }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
         return this;
@@ -5383,6 +5455,54 @@ public final class QuotaProtos {
         onChanged();
         return this;
       }
+
+      private boolean remove_ ;
+      /**
+       * <pre>
+       * When true, remove the quota.
+       * </pre>
+       *
+       * <code>optional bool remove = 3 [default = false];</code>
+       */
+      public boolean hasRemove() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <pre>
+       * When true, remove the quota.
+       * </pre>
+       *
+       * <code>optional bool remove = 3 [default = false];</code>
+       */
+      public boolean getRemove() {
+        return remove_;
+      }
+      /**
+       * <pre>
+       * When true, remove the quota.
+       * </pre>
+       *
+       * <code>optional bool remove = 3 [default = false];</code>
+       */
+      public Builder setRemove(boolean value) {
+        bitField0_ |= 0x00000004;
+        remove_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * When true, remove the quota.
+       * </pre>
+       *
+       * <code>optional bool remove = 3 [default = false];</code>
+       */
+      public Builder clearRemove() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        remove_ = false;
+        onChanged();
+        return this;
+      }
       public final Builder setUnknownFields(
           final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
@@ -7426,24 +7546,25 @@ public final class QuotaProtos {
       "edQuota\"r\n\006Quotas\022\035\n\016bypass_globals\030\001 \001(" +
       "\010:\005false\022$\n\010throttle\030\002 \001(\0132\022.hbase.pb.Th" +
       "rottle\022#\n\005space\030\003 \001(\0132\024.hbase.pb.SpaceQu" +
-      "ota\"\014\n\nQuotaUsage\"Z\n\nSpaceQuota\022\022\n\nsoft_" +
+      "ota\"\014\n\nQuotaUsage\"q\n\nSpaceQuota\022\022\n\nsoft_" +
       "limit\030\001 \001(\004\0228\n\020violation_policy\030\002 \001(\0162\036." +
-      "hbase.pb.SpaceViolationPolicy\"8\n\021SpaceLi" +
-      "mitRequest\022#\n\005quota\030\001 \001(\0132\024.hbase.pb.Spa",
-      "ceQuota\"X\n\020SpaceQuotaStatus\022.\n\006policy\030\001 " +
-      "\001(\0162\036.hbase.pb.SpaceViolationPolicy\022\024\n\014i" +
-      "n_violation\030\002 \001(\010\"^\n\022SpaceQuotaSnapshot\022" +
-      "*\n\006status\030\001 \001(\0132\032.hbase.pb.SpaceQuotaSta" +
-      "tus\022\r\n\005usage\030\002 \001(\004\022\r\n\005limit\030\003 \001(\004*&\n\nQuo" +
-      "taScope\022\013\n\007CLUSTER\020\001\022\013\n\007MACHINE\020\002*v\n\014Thr" +
-      "ottleType\022\022\n\016REQUEST_NUMBER\020\001\022\020\n\014REQUEST" +
-      "_SIZE\020\002\022\020\n\014WRITE_NUMBER\020\003\022\016\n\nWRITE_SIZE\020" +
-      "\004\022\017\n\013READ_NUMBER\020\005\022\r\n\tREAD_SIZE\020\006*$\n\tQuo" +
-      "taType\022\014\n\010THROTTLE\020\001\022\t\n\005SPACE\020\002*]\n\024Space",
-      "ViolationPolicy\022\013\n\007DISABLE\020\001\022\031\n\025NO_WRITE" +
-      "S_COMPACTIONS\020\002\022\r\n\tNO_WRITES\020\003\022\016\n\nNO_INS" +
-      "ERTS\020\004BH\n1org.apache.hadoop.hbase.shaded" +
-      ".protobuf.generatedB\013QuotaProtosH\001\210\001\001\240\001\001"
+      "hbase.pb.SpaceViolationPolicy\022\025\n\006remove\030" +
+      "\003 \001(\010:\005false\"8\n\021SpaceLimitRequest\022#\n\005quo",
+      "ta\030\001 \001(\0132\024.hbase.pb.SpaceQuota\"X\n\020SpaceQ" +
+      "uotaStatus\022.\n\006policy\030\001 \001(\0162\036.hbase.pb.Sp" +
+      "aceViolationPolicy\022\024\n\014in_violation\030\002 \001(\010" +
+      "\"^\n\022SpaceQuotaSnapshot\022*\n\006status\030\001 \001(\0132\032" +
+      ".hbase.pb.SpaceQuotaStatus\022\r\n\005usage\030\002 \001(" +
+      "\004\022\r\n\005limit\030\003 \001(\004*&\n\nQuotaScope\022\013\n\007CLUSTE" +
+      "R\020\001\022\013\n\007MACHINE\020\002*v\n\014ThrottleType\022\022\n\016REQU" +
+      "EST_NUMBER\020\001\022\020\n\014REQUEST_SIZE\020\002\022\020\n\014WRITE_" +
+      "NUMBER\020\003\022\016\n\nWRITE_SIZE\020\004\022\017\n\013READ_NUMBER\020" +
+      "\005\022\r\n\tREAD_SIZE\020\006*$\n\tQuotaType\022\014\n\010THROTTL",
+      "E\020\001\022\t\n\005SPACE\020\002*]\n\024SpaceViolationPolicy\022\013" +
+      "\n\007DISABLE\020\001\022\031\n\025NO_WRITES_COMPACTIONS\020\002\022\r" +
+      "\n\tNO_WRITES\020\003\022\016\n\nNO_INSERTS\020\004BH\n1org.apa" +
+      "che.hadoop.hbase.shaded.protobuf.generat" +
+      "edB\013QuotaProtosH\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -7493,7 +7614,7 @@ public final class QuotaProtos {
     internal_static_hbase_pb_SpaceQuota_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SpaceQuota_descriptor,
-        new java.lang.String[] { "SoftLimit", "ViolationPolicy", });
+        new java.lang.String[] { "SoftLimit", "ViolationPolicy", "Remove", });
     internal_static_hbase_pb_SpaceLimitRequest_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/0effca42/hbase-protocol-shaded/src/main/protobuf/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Quota.proto b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
index 5ea1ddb..597b059 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Quota.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
@@ -89,6 +89,7 @@ enum SpaceViolationPolicy {
 message SpaceQuota {
   optional uint64 soft_limit = 1; // The limit of bytes for this quota
   optional SpaceViolationPolicy violation_policy = 2; // The action to take when the quota is violated
+  optional bool remove = 3 [default = false]; // When true, remove the quota.
 }
 
 // The Request to limit space usage (to allow for schema evolution not tied to SpaceQuota).

http://git-wip-us.apache.org/repos/asf/hbase/blob/0effca42/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
index d466e59..3f65424 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
@@ -4658,6 +4658,24 @@ public final class QuotaProtos {
      * </pre>
      */
     org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy();
+
+    // optional bool remove = 3 [default = false];
+    /**
+     * <code>optional bool remove = 3 [default = false];</code>
+     *
+     * <pre>
+     * When true, remove the quota.
+     * </pre>
+     */
+    boolean hasRemove();
+    /**
+     * <code>optional bool remove = 3 [default = false];</code>
+     *
+     * <pre>
+     * When true, remove the quota.
+     * </pre>
+     */
+    boolean getRemove();
   }
   /**
    * Protobuf type {@code hbase.pb.SpaceQuota}
@@ -4730,6 +4748,11 @@ public final class QuotaProtos {
               }
               break;
             }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              remove_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4818,9 +4841,34 @@ public final class QuotaProtos {
       return violationPolicy_;
     }
 
+    // optional bool remove = 3 [default = false];
+    public static final int REMOVE_FIELD_NUMBER = 3;
+    private boolean remove_;
+    /**
+     * <code>optional bool remove = 3 [default = false];</code>
+     *
+     * <pre>
+     * When true, remove the quota.
+     * </pre>
+     */
+    public boolean hasRemove() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional bool remove = 3 [default = false];</code>
+     *
+     * <pre>
+     * When true, remove the quota.
+     * </pre>
+     */
+    public boolean getRemove() {
+      return remove_;
+    }
+
     private void initFields() {
       softLimit_ = 0L;
       violationPolicy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+      remove_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4840,6 +4888,9 @@ public final class QuotaProtos {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeEnum(2, violationPolicy_.getNumber());
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(3, remove_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4857,6 +4908,10 @@ public final class QuotaProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeEnumSize(2, violationPolicy_.getNumber());
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, remove_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4890,6 +4945,11 @@ public final class QuotaProtos {
         result = result &&
             (getViolationPolicy() == other.getViolationPolicy());
       }
+      result = result && (hasRemove() == other.hasRemove());
+      if (hasRemove()) {
+        result = result && (getRemove()
+            == other.getRemove());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -4911,6 +4971,10 @@ public final class QuotaProtos {
         hash = (37 * hash) + VIOLATION_POLICY_FIELD_NUMBER;
         hash = (53 * hash) + hashEnum(getViolationPolicy());
       }
+      if (hasRemove()) {
+        hash = (37 * hash) + REMOVE_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getRemove());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -5028,6 +5092,8 @@ public final class QuotaProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         violationPolicy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
         bitField0_ = (bitField0_ & ~0x00000002);
+        remove_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -5064,6 +5130,10 @@ public final class QuotaProtos {
           to_bitField0_ |= 0x00000002;
         }
         result.violationPolicy_ = violationPolicy_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.remove_ = remove_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -5086,6 +5156,9 @@ public final class QuotaProtos {
         if (other.hasViolationPolicy()) {
           setViolationPolicy(other.getViolationPolicy());
         }
+        if (other.hasRemove()) {
+          setRemove(other.getRemove());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -5214,6 +5287,55 @@ public final class QuotaProtos {
         return this;
       }
 
+      // optional bool remove = 3 [default = false];
+      private boolean remove_ ;
+      /**
+       * <code>optional bool remove = 3 [default = false];</code>
+       *
+       * <pre>
+       * When true, remove the quota.
+       * </pre>
+       */
+      public boolean hasRemove() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional bool remove = 3 [default = false];</code>
+       *
+       * <pre>
+       * When true, remove the quota.
+       * </pre>
+       */
+      public boolean getRemove() {
+        return remove_;
+      }
+      /**
+       * <code>optional bool remove = 3 [default = false];</code>
+       *
+       * <pre>
+       * When true, remove the quota.
+       * </pre>
+       */
+      public Builder setRemove(boolean value) {
+        bitField0_ |= 0x00000004;
+        remove_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool remove = 3 [default = false];</code>
+       *
+       * <pre>
+       * When true, remove the quota.
+       * </pre>
+       */
+      public Builder clearRemove() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        remove_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.SpaceQuota)
     }
 
@@ -7126,24 +7248,25 @@ public final class QuotaProtos {
       "edQuota\"r\n\006Quotas\022\035\n\016bypass_globals\030\001 \001(" +
       "\010:\005false\022$\n\010throttle\030\002 \001(\0132\022.hbase.pb.Th" +
       "rottle\022#\n\005space\030\003 \001(\0132\024.hbase.pb.SpaceQu" +
-      "ota\"\014\n\nQuotaUsage\"Z\n\nSpaceQuota\022\022\n\nsoft_" +
+      "ota\"\014\n\nQuotaUsage\"q\n\nSpaceQuota\022\022\n\nsoft_" +
       "limit\030\001 \001(\004\0228\n\020violation_policy\030\002 \001(\0162\036." +
-      "hbase.pb.SpaceViolationPolicy\"8\n\021SpaceLi" +
-      "mitRequest\022#\n\005quota\030\001 \001(\0132\024.hbase.pb.Spa",
-      "ceQuota\"X\n\020SpaceQuotaStatus\022.\n\006policy\030\001 " +
-      "\001(\0162\036.hbase.pb.SpaceViolationPolicy\022\024\n\014i" +
-      "n_violation\030\002 \001(\010\"^\n\022SpaceQuotaSnapshot\022" +
-      "*\n\006status\030\001 \001(\0132\032.hbase.pb.SpaceQuotaSta" +
-      "tus\022\r\n\005usage\030\002 \001(\004\022\r\n\005limit\030\003 \001(\004*&\n\nQuo" +
-      "taScope\022\013\n\007CLUSTER\020\001\022\013\n\007MACHINE\020\002*v\n\014Thr" +
-      "ottleType\022\022\n\016REQUEST_NUMBER\020\001\022\020\n\014REQUEST" +
-      "_SIZE\020\002\022\020\n\014WRITE_NUMBER\020\003\022\016\n\nWRITE_SIZE\020" +
-      "\004\022\017\n\013READ_NUMBER\020\005\022\r\n\tREAD_SIZE\020\006*$\n\tQuo" +
-      "taType\022\014\n\010THROTTLE\020\001\022\t\n\005SPACE\020\002*]\n\024Space",
-      "ViolationPolicy\022\013\n\007DISABLE\020\001\022\031\n\025NO_WRITE" +
-      "S_COMPACTIONS\020\002\022\r\n\tNO_WRITES\020\003\022\016\n\nNO_INS" +
-      "ERTS\020\004BA\n*org.apache.hadoop.hbase.protob" +
-      "uf.generatedB\013QuotaProtosH\001\210\001\001\240\001\001"
+      "hbase.pb.SpaceViolationPolicy\022\025\n\006remove\030" +
+      "\003 \001(\010:\005false\"8\n\021SpaceLimitRequest\022#\n\005quo",
+      "ta\030\001 \001(\0132\024.hbase.pb.SpaceQuota\"X\n\020SpaceQ" +
+      "uotaStatus\022.\n\006policy\030\001 \001(\0162\036.hbase.pb.Sp" +
+      "aceViolationPolicy\022\024\n\014in_violation\030\002 \001(\010" +
+      "\"^\n\022SpaceQuotaSnapshot\022*\n\006status\030\001 \001(\0132\032" +
+      ".hbase.pb.SpaceQuotaStatus\022\r\n\005usage\030\002 \001(" +
+      "\004\022\r\n\005limit\030\003 \001(\004*&\n\nQuotaScope\022\013\n\007CLUSTE" +
+      "R\020\001\022\013\n\007MACHINE\020\002*v\n\014ThrottleType\022\022\n\016REQU" +
+      "EST_NUMBER\020\001\022\020\n\014REQUEST_SIZE\020\002\022\020\n\014WRITE_" +
+      "NUMBER\020\003\022\016\n\nWRITE_SIZE\020\004\022\017\n\013READ_NUMBER\020" +
+      "\005\022\r\n\tREAD_SIZE\020\006*$\n\tQuotaType\022\014\n\010THROTTL",
+      "E\020\001\022\t\n\005SPACE\020\002*]\n\024SpaceViolationPolicy\022\013" +
+      "\n\007DISABLE\020\001\022\031\n\025NO_WRITES_COMPACTIONS\020\002\022\r" +
+      "\n\tNO_WRITES\020\003\022\016\n\nNO_INSERTS\020\004BA\n*org.apa" +
+      "che.hadoop.hbase.protobuf.generatedB\013Quo" +
+      "taProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -7185,7 +7308,7 @@ public final class QuotaProtos {
           internal_static_hbase_pb_SpaceQuota_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_SpaceQuota_descriptor,
-              new java.lang.String[] { "SoftLimit", "ViolationPolicy", });
+              new java.lang.String[] { "SoftLimit", "ViolationPolicy", "Remove", });
           internal_static_hbase_pb_SpaceLimitRequest_descriptor =
             getDescriptor().getMessageTypes().get(6);
           internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/0effca42/hbase-protocol/src/main/protobuf/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Quota.proto b/hbase-protocol/src/main/protobuf/Quota.proto
index 02d825b..d55918a 100644
--- a/hbase-protocol/src/main/protobuf/Quota.proto
+++ b/hbase-protocol/src/main/protobuf/Quota.proto
@@ -89,6 +89,7 @@ enum SpaceViolationPolicy {
 message SpaceQuota {
   optional uint64 soft_limit = 1; // The limit of bytes for this quota
   optional SpaceViolationPolicy violation_policy = 2; // The action to take when the quota is violated
+  optional bool remove = 3 [default = false]; // When true, remove the quota.
 }
 
 // The Request to limit space usage (to allow for schema evolution not tied to SpaceQuota).

http://git-wip-us.apache.org/repos/asf/hbase/blob/0effca42/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index 206d81d..a5832f9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -454,7 +454,14 @@ public class MasterQuotaManager implements RegionStateListener {
    */
   void applySpaceLimit(final Quotas.Builder quotas, final SpaceLimitRequest req) {
     if (req.hasQuota()) {
-      applySpaceQuota(quotas, req.getQuota());
+      SpaceQuota spaceQuota = req.getQuota();
+      // If we have the remove flag, unset the space quota.
+      if (spaceQuota.getRemove()) {
+        quotas.setSpace(SpaceQuota.getDefaultInstance());
+      } else {
+        // Otherwise, update the new quota
+        applySpaceQuota(quotas, req.getQuota());
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0effca42/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
index be93c2f..52921af 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
@@ -47,6 +47,7 @@ import org.junit.experimental.categories.Category;
 import com.google.common.collect.Iterables;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -246,7 +247,7 @@ public class TestQuotaAdmin {
   }
 
   @Test
-  public void testSetAndGetSpaceQuota() throws Exception {
+  public void testSetGetRemoveSpaceQuota() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
     final TableName tn = TableName.valueOf("table1");
     final long sizeLimit = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
@@ -274,10 +275,32 @@ public class TestQuotaAdmin {
     } finally {
       scanner.close();
     }
+
+    // Now, remove the quota
+    QuotaSettings removeQuota = QuotaSettingsFactory.removeTableSpaceLimit(tn);
+    admin.setQuota(removeQuota);
+
+    // Verify that the record doesn't exist in the table
+    try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      ResultScanner rs = quotaTable.getScanner(new Scan());
+      try {
+        assertNull("Did not expect to find a quota entry", rs.next());
+      } finally {
+        rs.close();
+      }
+    }
+
+    // Verify that we can also not fetch it via the API
+    scanner = QuotaRetriever.open(admin.getConfiguration());
+    try {
+      assertNull("Did not expect to find a quota entry", scanner.next());
+    } finally {
+      scanner.close();
+    }
   }
 
   @Test
-  public void testSetAndModifyQuota() throws Exception {
+  public void testSetModifyRemoveQuota() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
     final TableName tn = TableName.valueOf("table1");
     final long originalSizeLimit = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
@@ -334,6 +357,28 @@ public class TestQuotaAdmin {
     } finally {
       quotaScanner.close();
     }
+
+    // Now, remove the quota
+    QuotaSettings removeQuota = QuotaSettingsFactory.removeTableSpaceLimit(tn);
+    admin.setQuota(removeQuota);
+
+    // Verify that the record doesn't exist in the table
+    try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      ResultScanner scanner = quotaTable.getScanner(new Scan());
+      try {
+        assertNull("Did not expect to find a quota entry", scanner.next());
+      } finally {
+        scanner.close();
+      }
+    }
+
+    // Verify that we can also not fetch it via the API
+    quotaScanner = QuotaRetriever.open(admin.getConfiguration());
+    try {
+      assertNull("Did not expect to find a quota entry", quotaScanner.next());
+    } finally {
+      quotaScanner.close();
+    }
   }
 
   private void assertNumResults(int expected, final QuotaFilter filter) throws Exception {


[24/50] [abbrv] hbase git commit: HBASE-16995 Build client Java API and client protobuf messages (Josh Elser)

Posted by el...@apache.org.
HBASE-16995 Build client Java API and client protobuf messages (Josh Elser)


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

Branch: refs/heads/HBASE-16961
Commit: 990062a93a257a42adc84b7b8448d788517c9baa
Parents: ecdfb82
Author: tedyu <yu...@gmail.com>
Authored: Thu Nov 17 10:19:52 2016 -0800
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:35:31 2017 -0400

----------------------------------------------------------------------
 .../hbase/quotas/QuotaSettingsFactory.java      |   47 +
 .../apache/hadoop/hbase/quotas/QuotaType.java   |    1 +
 .../hadoop/hbase/quotas/SpaceLimitSettings.java |  166 ++
 .../hbase/quotas/SpaceViolationPolicy.java      |   44 +
 .../hbase/shaded/protobuf/ProtobufUtil.java     |   51 +
 .../hbase/quotas/TestQuotaSettingsFactory.java  |  148 ++
 .../hbase/quotas/TestSpaceLimitSettings.java    |  119 ++
 .../shaded/protobuf/generated/MasterProtos.java |  584 ++++--
 .../shaded/protobuf/generated/QuotaProtos.java  | 1739 +++++++++++++++++-
 .../src/main/protobuf/Master.proto              |    2 +
 .../src/main/protobuf/Quota.proto               |   21 +
 .../hbase/protobuf/generated/QuotaProtos.java   | 1682 ++++++++++++++++-
 hbase-protocol/src/main/protobuf/Quota.proto    |   21 +
 13 files changed, 4291 insertions(+), 334 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
index 3622a32..8512e39 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRe
 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;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 
 @InterfaceAudience.Public
 public class QuotaSettingsFactory {
@@ -89,6 +90,9 @@ public class QuotaSettingsFactory {
     if (quotas.getBypassGlobals() == true) {
       settings.add(new QuotaGlobalsSettingsBypass(userName, tableName, namespace, true));
     }
+    if (quotas.hasSpace()) {
+      settings.add(fromSpace(tableName, namespace, quotas.getSpace()));
+    }
     return settings;
   }
 
@@ -122,6 +126,18 @@ public class QuotaSettingsFactory {
     return settings;
   }
 
+  static QuotaSettings fromSpace(TableName table, String namespace, SpaceQuota protoQuota) {
+    if ((null == table && null == namespace) || (null != table && null != namespace)) {
+      throw new IllegalArgumentException("Can only construct SpaceLimitSettings for a table or namespace.");
+    }
+    if (null != table) {
+      return SpaceLimitSettings.fromSpaceQuota(table, protoQuota);
+    } else {
+      // namespace must be non-null
+      return SpaceLimitSettings.fromSpaceQuota(namespace, protoQuota);
+    }
+  }
+
   /* ==========================================================================
    *  RPC Throttle
    */
@@ -278,4 +294,35 @@ public class QuotaSettingsFactory {
   public static QuotaSettings bypassGlobals(final String userName, final boolean bypassGlobals) {
     return new QuotaGlobalsSettingsBypass(userName, null, null, bypassGlobals);
   }
+
+  /* ==========================================================================
+   *  FileSystem Space Settings
+   */
+
+  /**
+   * Creates a {@link QuotaSettings} object to limit the FileSystem space usage for the given table to the given size in bytes.
+   * When the space usage is exceeded by the table, the provided {@link SpaceViolationPolicy} is enacted on the table.
+   *
+   * @param tableName The name of the table on which the quota should be applied.
+   * @param sizeLimit The limit of a table's size in bytes.
+   * @param violationPolicy The action to take when the quota is exceeded.
+   * @return An {@link QuotaSettings} object.
+   */
+  public static QuotaSettings limitTableSpace(final TableName tableName, long sizeLimit, final SpaceViolationPolicy violationPolicy) {
+    return new SpaceLimitSettings(tableName, sizeLimit, violationPolicy);
+  }
+
+  /**
+   * Creates a {@link QuotaSettings} object to limit the FileSystem space usage for the given namespace to the given size in bytes.
+   * When the space usage is exceeded by all tables in the namespace, the provided {@link SpaceViolationPolicy} is enacted on
+   * all tables in the namespace.
+   *
+   * @param namespace The namespace on which the quota should be applied.
+   * @param sizeLimit The limit of the namespace's size in bytes.
+   * @param violationPolicy The action to take when the the quota is exceeded.
+   * @return An {@link QuotaSettings} object.
+   */
+  public static QuotaSettings limitNamespaceSpace(final String namespace, long sizeLimit, final SpaceViolationPolicy violationPolicy) {
+    return new SpaceLimitSettings(namespace, sizeLimit, violationPolicy);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaType.java
index 1ec649f..f291667 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaType.java
@@ -26,4 +26,5 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 public enum QuotaType {
   THROTTLE,
   GLOBAL_BYPASS,
+  SPACE,
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
new file mode 100644
index 0000000..dded9b5
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import java.util.Objects;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+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.MasterProtos.SetQuotaRequest.Builder;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+
+/**
+ * A {@link QuotaSettings} implementation for implementing filesystem-use quotas.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+class SpaceLimitSettings extends QuotaSettings {
+
+  private final SpaceLimitRequest proto;
+
+  SpaceLimitSettings(TableName tableName, long sizeLimit, SpaceViolationPolicy violationPolicy) {
+    super(null, Objects.requireNonNull(tableName), null);
+    if (0L > sizeLimit) {
+      throw new IllegalArgumentException("Size limit must be a non-negative value.");
+    }
+    proto = buildProtoQuota(sizeLimit, Objects.requireNonNull(violationPolicy));
+  }
+
+  SpaceLimitSettings(String namespace, long sizeLimit, SpaceViolationPolicy violationPolicy) {
+    super(null, null, Objects.requireNonNull(namespace));
+    if (0L > sizeLimit) {
+      throw new IllegalArgumentException("Size limit must be a non-negative value.");
+    }
+    proto = buildProtoQuota(sizeLimit, Objects.requireNonNull(violationPolicy));
+  }
+
+  /**
+   * Builds a {@link SpaceQuota} protobuf object given the arguments.
+   *
+   * @param sizeLimit The size limit of the quota.
+   * @param violationPolicy The action to take when the quota is exceeded.
+   * @return The protobuf SpaceQuota representation.
+   */
+  private SpaceLimitRequest buildProtoQuota(long sizeLimit, SpaceViolationPolicy violationPolicy) {
+    return SpaceLimitRequest.newBuilder().setQuota(
+        SpaceQuota.newBuilder()
+            .setSoftLimit(sizeLimit)
+            .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(violationPolicy))
+            .build())
+        .build();
+  }
+
+  /**
+   * Returns a copy of the internal state of <code>this</code>
+   */
+  SpaceLimitRequest getProto() {
+    return proto.toBuilder().build();
+  }
+
+  @Override
+  public QuotaType getQuotaType() {
+    return QuotaType.SPACE;
+  }
+
+  @Override
+  protected void setupSetQuotaRequest(Builder builder) {
+    // TableName/Namespace are serialized in QuotaSettings
+    builder.setSpaceLimit(proto);
+  }
+
+  /**
+   * Constructs a {@link SpaceLimitSettings} from the provided protobuf message and tablename.
+   *
+   * @param tableName The target tablename for the limit.
+   * @param proto The protobuf representation.
+   * @return A QuotaSettings.
+   */
+  static SpaceLimitSettings fromSpaceQuota(final TableName tableName,
+      final QuotaProtos.SpaceQuota proto) {
+    validateProtoArguments(proto);
+    return new SpaceLimitSettings(tableName, proto.getSoftLimit(),
+        ProtobufUtil.toViolationPolicy(proto.getViolationPolicy()));
+  }
+
+  /**
+   * Constructs a {@link SpaceLimitSettings} from the provided protobuf message and namespace.
+   *
+   * @param namespace The target namespace for the limit.
+   * @param proto The protobuf representation.
+   * @return A QuotaSettings.
+   */
+  static SpaceLimitSettings fromSpaceQuota(final String namespace,
+      final QuotaProtos.SpaceQuota proto) {
+    validateProtoArguments(proto);
+    return new SpaceLimitSettings(namespace, proto.getSoftLimit(),
+        ProtobufUtil.toViolationPolicy(proto.getViolationPolicy()));
+  }
+
+  /**
+   * Validates that the provided protobuf SpaceQuota has the necessary information to construct
+   * a {@link SpaceLimitSettings}.
+   *
+   * @param proto The protobuf message to validate.
+   */
+  static void validateProtoArguments(final QuotaProtos.SpaceQuota proto) {
+    if (!Objects.requireNonNull(proto).hasSoftLimit()) {
+      throw new IllegalArgumentException("Cannot handle SpaceQuota without a soft limit");
+    }
+    if (!proto.hasViolationPolicy()) {
+      throw new IllegalArgumentException("Cannot handle SpaceQuota without a violation policy");
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(getTableName(), getNamespace(), proto);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o == this) {
+      return true;
+    }
+    if (!(o instanceof SpaceLimitSettings)) {
+      return false;
+    }
+    // o is non-null and an instance of SpaceLimitSettings
+    SpaceLimitSettings other = (SpaceLimitSettings) o;
+    return Objects.equals(getTableName(), other.getTableName()) &&
+        Objects.equals(getNamespace(), other.getNamespace()) &&
+        Objects.equals(proto, other.proto);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("TYPE => SPACE");
+    if (null != getTableName()) {
+      sb.append(", TABLE => ").append(getTableName());
+    }
+    if (null != getNamespace()) {
+      sb.append(", NAMESPACE => ").append(getNamespace());
+    }
+    sb.append(", LIMIT => ").append(proto.getQuota().getSoftLimit());
+    sb.append(", VIOLATION_POLICY => ").append(proto.getQuota().getViolationPolicy());
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.java
new file mode 100644
index 0000000..c63acb0
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Enumeration that represents the action HBase will take when a space quota is violated.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum SpaceViolationPolicy {
+  /**
+   * Disables the table(s).
+   */
+  DISABLE,
+  /**
+   * Disallows any mutations or compactions on the table(s).
+   */
+  NO_WRITES_COMPACTIONS,
+  /**
+   * Disallows any mutations (but allows compactions) on the table(s).
+   */
+  NO_WRITES,
+  /**
+   * Disallows any updates (but allows deletes and compactions) on the table(s).
+   */
+  NO_INSERTS,
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index e969ded..460fb59 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -85,6 +85,7 @@ import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
 import org.apache.hadoop.hbase.quotas.QuotaScope;
 import org.apache.hadoop.hbase.quotas.QuotaType;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.quotas.ThrottleType;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
@@ -2533,6 +2534,7 @@ public final class ProtobufUtil {
   public static QuotaType toQuotaScope(final QuotaProtos.QuotaType proto) {
     switch (proto) {
       case THROTTLE: return QuotaType.THROTTLE;
+      case SPACE: return QuotaType.SPACE;
     }
     throw new RuntimeException("Invalid QuotaType " + proto);
   }
@@ -2546,11 +2548,45 @@ public final class ProtobufUtil {
   public static QuotaProtos.QuotaType toProtoQuotaScope(final QuotaType type) {
     switch (type) {
       case THROTTLE: return QuotaProtos.QuotaType.THROTTLE;
+      case SPACE: return QuotaProtos.QuotaType.SPACE;
     }
     throw new RuntimeException("Invalid QuotaType " + type);
   }
 
   /**
+   * Converts a protocol buffer SpaceViolationPolicy to a client SpaceViolationPolicy.
+   *
+   * @param proto The protocol buffer space violation policy.
+   * @return The corresponding client SpaceViolationPolicy.
+   */
+  public static SpaceViolationPolicy toViolationPolicy(final QuotaProtos.SpaceViolationPolicy proto) {
+    switch (proto) {
+      case DISABLE: return SpaceViolationPolicy.DISABLE;
+      case NO_WRITES_COMPACTIONS: return SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
+      case NO_WRITES: return SpaceViolationPolicy.NO_WRITES;
+      case NO_INSERTS: return SpaceViolationPolicy.NO_INSERTS;
+    }
+    throw new RuntimeException("Invalid SpaceViolationPolicy " + proto);
+  }
+
+  /**
+   * Converts a client SpaceViolationPolicy to a protocol buffer SpaceViolationPolicy.
+   *
+   * @param policy The client SpaceViolationPolicy object.
+   * @return The corresponding protocol buffer SpaceViolationPolicy.
+   */
+  public static QuotaProtos.SpaceViolationPolicy toProtoViolationPolicy(
+      final SpaceViolationPolicy policy) {
+    switch (policy) {
+      case DISABLE: return QuotaProtos.SpaceViolationPolicy.DISABLE;
+      case NO_WRITES_COMPACTIONS: return QuotaProtos.SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
+      case NO_WRITES: return QuotaProtos.SpaceViolationPolicy.NO_WRITES;
+      case NO_INSERTS: return QuotaProtos.SpaceViolationPolicy.NO_INSERTS;
+    }
+    throw new RuntimeException("Invalid SpaceViolationPolicy " + policy);
+  }
+
+  /**
    * Build a protocol buffer TimedQuota
    *
    * @param limit the allowed number of request/data per timeUnit
@@ -2568,6 +2604,21 @@ public final class ProtobufUtil {
   }
 
   /**
+   * Builds a protocol buffer SpaceQuota.
+   *
+   * @param limit The maximum space usage for the quota in bytes.
+   * @param violationPolicy The policy to apply when the quota is violated.
+   * @return The protocol buffer SpaceQuota.
+   */
+  public static QuotaProtos.SpaceQuota toProtoSpaceQuota(final long limit,
+      final SpaceViolationPolicy violationPolicy) {
+    return QuotaProtos.SpaceQuota.newBuilder()
+        .setSoftLimit(limit)
+        .setViolationPolicy(toProtoViolationPolicy(violationPolicy))
+        .build();
+  }
+
+  /**
    * Generates a marker for the WAL so that we propagate the notion of a bulk region load
    * throughout the WAL.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
new file mode 100644
index 0000000..17015d6
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
@@ -0,0 +1,148 @@
+/*
+ * 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 static org.junit.Assert.fail;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link QuotaSettingsFactory}.
+ */
+@Category(SmallTests.class)
+public class TestQuotaSettingsFactory {
+  
+  @Test
+  public void testAllQuotasAddedToList() {
+    final SpaceQuota spaceQuota = SpaceQuota.newBuilder()
+        .setSoftLimit(1024L * 1024L * 1024L * 50L) // 50G
+        .setViolationPolicy(QuotaProtos.SpaceViolationPolicy.DISABLE) // Disable the table
+        .build();
+    final long readLimit = 1000;
+    final long writeLimit = 500;
+    final Throttle throttle = Throttle.newBuilder()
+        // 1000 read reqs/min
+        .setReadNum(TimedQuota.newBuilder().setSoftLimit(readLimit).setTimeUnit(HBaseProtos.TimeUnit.MINUTES).build())
+        // 500 write reqs/min
+        .setWriteNum(TimedQuota.newBuilder().setSoftLimit(writeLimit).setTimeUnit(HBaseProtos.TimeUnit.MINUTES).build())
+        .build();
+    final Quotas quotas = Quotas.newBuilder()
+        .setSpace(spaceQuota) // Set the FS quotas
+        .setThrottle(throttle) // Set some RPC limits
+        .build();
+    final TableName tn = TableName.valueOf("my_table");
+    List<QuotaSettings> settings = QuotaSettingsFactory.fromTableQuotas(tn, quotas);
+    assertEquals(3, settings.size());
+    boolean seenRead = false;
+    boolean seenWrite = false;
+    boolean seenSpace = false;
+    for (QuotaSettings setting : settings) {
+      if (setting instanceof ThrottleSettings) {
+        ThrottleSettings throttleSettings = (ThrottleSettings) setting;
+        switch (throttleSettings.getThrottleType()) {
+          case READ_NUMBER:
+            assertFalse("Should not have multiple read quotas", seenRead);
+            assertEquals(readLimit, throttleSettings.getSoftLimit());
+            assertEquals(TimeUnit.MINUTES, throttleSettings.getTimeUnit());
+            assertEquals(tn, throttleSettings.getTableName());
+            assertNull("Username should be null", throttleSettings.getUserName());
+            assertNull("Namespace should be null", throttleSettings.getNamespace());
+            seenRead = true;
+            break;
+          case WRITE_NUMBER:
+            assertFalse("Should not have multiple write quotas", seenWrite);
+            assertEquals(writeLimit, throttleSettings.getSoftLimit());
+            assertEquals(TimeUnit.MINUTES, throttleSettings.getTimeUnit());
+            assertEquals(tn, throttleSettings.getTableName());
+            assertNull("Username should be null", throttleSettings.getUserName());
+            assertNull("Namespace should be null", throttleSettings.getNamespace());
+            seenWrite = true;
+            break;
+          default:
+            fail("Unexpected throttle type: " + throttleSettings.getThrottleType());
+        }
+      } else if (setting instanceof SpaceLimitSettings) {
+        assertFalse("Should not have multiple space quotas", seenSpace);
+        SpaceLimitSettings spaceLimit = (SpaceLimitSettings) setting;
+        assertEquals(tn, spaceLimit.getTableName());
+        assertNull("Username should be null", spaceLimit.getUserName());
+        assertNull("Namespace should be null", spaceLimit.getNamespace());
+        assertTrue("SpaceLimitSettings should have a SpaceQuota", spaceLimit.getProto().hasQuota());
+        assertEquals(spaceQuota, spaceLimit.getProto().getQuota());
+        seenSpace = true;
+      } else {
+        fail("Unexpected QuotaSettings implementation: " + setting.getClass());
+      }
+    }
+    assertTrue("Should have seen a read quota", seenRead);
+    assertTrue("Should have seen a write quota", seenWrite);
+    assertTrue("Should have seen a space quota", seenSpace);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testNeitherTableNorNamespace() {
+    final SpaceQuota spaceQuota = SpaceQuota.newBuilder()
+        .setSoftLimit(1L)
+        .setViolationPolicy(QuotaProtos.SpaceViolationPolicy.DISABLE)
+        .build();
+    QuotaSettingsFactory.fromSpace(null, null, spaceQuota);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testBothTableAndNamespace() {
+    final SpaceQuota spaceQuota = SpaceQuota.newBuilder()
+        .setSoftLimit(1L)
+        .setViolationPolicy(QuotaProtos.SpaceViolationPolicy.DISABLE)
+        .build();
+    QuotaSettingsFactory.fromSpace(TableName.valueOf("foo"), "bar", spaceQuota);
+  }
+
+  @Test
+  public void testSpaceLimitSettings() {
+    final TableName tableName = TableName.valueOf("foo");
+    final long sizeLimit = 1024L * 1024L * 1024L * 75; // 75GB
+    final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tableName, sizeLimit, violationPolicy);
+    assertNotNull("QuotaSettings should not be null", settings);
+    assertTrue("Should be an instance of SpaceLimitSettings", settings instanceof SpaceLimitSettings);
+    SpaceLimitSettings spaceLimitSettings = (SpaceLimitSettings) settings;
+    SpaceLimitRequest protoRequest = spaceLimitSettings.getProto();
+    assertTrue("Request should have a SpaceQuota", protoRequest.hasQuota());
+    SpaceQuota quota = protoRequest.getQuota();
+    assertEquals(sizeLimit, quota.getSoftLimit());
+    assertEquals(violationPolicy, ProtobufUtil.toViolationPolicy(quota.getViolationPolicy()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java
new file mode 100644
index 0000000..77a00da
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link SpaceLimitSettings}.
+ */
+@Category({SmallTests.class})
+public class TestSpaceLimitSettings {
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidTableQuotaSizeLimit() {
+    new SpaceLimitSettings(TableName.valueOf("foo"), -1, SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullTableName() {
+    TableName tn = null;
+    new SpaceLimitSettings(tn, 1, SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullTableViolationPolicy() {
+    new SpaceLimitSettings(TableName.valueOf("foo"), 1, null);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidNamespaceQuotaSizeLimit() {
+    new SpaceLimitSettings("foo_ns", -1, SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullNamespace() {
+    String ns = null;
+    new SpaceLimitSettings(ns, 1, SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullNamespaceViolationPolicy() {
+    new SpaceLimitSettings("foo_ns", 1, null);
+  }
+
+  @Test
+  public void testTableQuota() {
+    final TableName tableName = TableName.valueOf("foo");
+    final long sizeLimit = 1024 * 1024;
+    final SpaceViolationPolicy policy = SpaceViolationPolicy.NO_WRITES;
+    SpaceLimitSettings settings = new SpaceLimitSettings(tableName, sizeLimit, policy);
+    SetQuotaRequest proto = QuotaSettings.buildSetQuotaRequestProto(settings);
+
+    assertFalse("User should be missing", proto.hasUserName());
+    assertFalse("Namespace should be missing", proto.hasNamespace());
+    assertEquals(ProtobufUtil.toProtoTableName(tableName), proto.getTableName());
+    SpaceLimitRequest spaceLimitReq = proto.getSpaceLimit();
+    assertNotNull("SpaceLimitRequest was null", spaceLimitReq);
+    SpaceQuota spaceQuota = spaceLimitReq.getQuota();
+    assertNotNull("SpaceQuota was null", spaceQuota);
+    assertEquals(sizeLimit, spaceQuota.getSoftLimit());
+    assertEquals(ProtobufUtil.toProtoViolationPolicy(policy), spaceQuota.getViolationPolicy());
+
+    assertEquals(QuotaType.SPACE, settings.getQuotaType());
+
+    SpaceLimitSettings copy = new SpaceLimitSettings(tableName, sizeLimit, policy);
+    assertEquals(settings, copy);
+    assertEquals(settings.hashCode(), copy.hashCode());
+  }
+
+  @Test
+  public void testNamespaceQuota() {
+    final String namespace = "foo_ns";
+    final long sizeLimit = 1024 * 1024;
+    final SpaceViolationPolicy policy = SpaceViolationPolicy.NO_WRITES;
+    SpaceLimitSettings settings = new SpaceLimitSettings(namespace, sizeLimit, policy);
+    SetQuotaRequest proto = QuotaSettings.buildSetQuotaRequestProto(settings);
+
+    assertFalse("User should be missing", proto.hasUserName());
+    assertFalse("TableName should be missing", proto.hasTableName());
+    assertEquals(namespace, proto.getNamespace());
+    SpaceLimitRequest spaceLimitReq = proto.getSpaceLimit();
+    assertNotNull("SpaceLimitRequest was null", spaceLimitReq);
+    SpaceQuota spaceQuota = spaceLimitReq.getQuota();
+    assertNotNull("SpaceQuota was null", spaceQuota);
+    assertEquals(sizeLimit, spaceQuota.getSoftLimit());
+    assertEquals(ProtobufUtil.toProtoViolationPolicy(policy), spaceQuota.getViolationPolicy());
+
+    assertEquals(QuotaType.SPACE, settings.getQuotaType());
+
+    SpaceLimitSettings copy = new SpaceLimitSettings(namespace, sizeLimit, policy);
+    assertEquals(settings, copy);
+    assertEquals(settings.hashCode(), copy.hashCode());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/990062a9/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index 8ff19b2..0c3248c 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -62233,6 +62233,19 @@ public final class MasterProtos {
      * <code>optional .hbase.pb.ThrottleRequest throttle = 7;</code>
      */
     org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequestOrBuilder getThrottleOrBuilder();
+
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    boolean hasSpaceLimit();
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest getSpaceLimit();
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder getSpaceLimitOrBuilder();
   }
   /**
    * Protobuf type {@code hbase.pb.SetQuotaRequest}
@@ -62335,6 +62348,19 @@ public final class MasterProtos {
               bitField0_ |= 0x00000040;
               break;
             }
+            case 66: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000080) == 0x00000080)) {
+                subBuilder = spaceLimit_.toBuilder();
+              }
+              spaceLimit_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(spaceLimit_);
+                spaceLimit_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000080;
+              break;
+            }
           }
         }
       } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@@ -62558,6 +62584,27 @@ public final class MasterProtos {
       return throttle_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest.getDefaultInstance() : throttle_;
     }
 
+    public static final int SPACE_LIMIT_FIELD_NUMBER = 8;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest spaceLimit_;
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    public boolean hasSpaceLimit() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest getSpaceLimit() {
+      return spaceLimit_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance() : spaceLimit_;
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder getSpaceLimitOrBuilder() {
+      return spaceLimit_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance() : spaceLimit_;
+    }
+
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
@@ -62603,6 +62650,9 @@ public final class MasterProtos {
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
         output.writeMessage(7, getThrottle());
       }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeMessage(8, getSpaceLimit());
+      }
       unknownFields.writeTo(output);
     }
 
@@ -62636,6 +62686,10 @@ public final class MasterProtos {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(7, getThrottle());
       }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(8, getSpaceLimit());
+      }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
       return size;
@@ -62688,6 +62742,11 @@ public final class MasterProtos {
         result = result && getThrottle()
             .equals(other.getThrottle());
       }
+      result = result && (hasSpaceLimit() == other.hasSpaceLimit());
+      if (hasSpaceLimit()) {
+        result = result && getSpaceLimit()
+            .equals(other.getSpaceLimit());
+      }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
     }
@@ -62729,6 +62788,10 @@ public final class MasterProtos {
         hash = (37 * hash) + THROTTLE_FIELD_NUMBER;
         hash = (53 * hash) + getThrottle().hashCode();
       }
+      if (hasSpaceLimit()) {
+        hash = (37 * hash) + SPACE_LIMIT_FIELD_NUMBER;
+        hash = (53 * hash) + getSpaceLimit().hashCode();
+      }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -62845,6 +62908,7 @@ public final class MasterProtos {
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
           getThrottleFieldBuilder();
+          getSpaceLimitFieldBuilder();
         }
       }
       public Builder clear() {
@@ -62871,6 +62935,12 @@ public final class MasterProtos {
           throttleBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000040);
+        if (spaceLimitBuilder_ == null) {
+          spaceLimit_ = null;
+        } else {
+          spaceLimitBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000080);
         return this;
       }
 
@@ -62931,6 +63001,14 @@ public final class MasterProtos {
         } else {
           result.throttle_ = throttleBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        if (spaceLimitBuilder_ == null) {
+          result.spaceLimit_ = spaceLimit_;
+        } else {
+          result.spaceLimit_ = spaceLimitBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -63000,6 +63078,9 @@ public final class MasterProtos {
         if (other.hasThrottle()) {
           mergeThrottle(other.getThrottle());
         }
+        if (other.hasSpaceLimit()) {
+          mergeSpaceLimit(other.getSpaceLimit());
+        }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
         return this;
@@ -63565,6 +63646,124 @@ public final class MasterProtos {
         }
         return throttleBuilder_;
       }
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest spaceLimit_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder> spaceLimitBuilder_;
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public boolean hasSpaceLimit() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest getSpaceLimit() {
+        if (spaceLimitBuilder_ == null) {
+          return spaceLimit_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance() : spaceLimit_;
+        } else {
+          return spaceLimitBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public Builder setSpaceLimit(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest value) {
+        if (spaceLimitBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          spaceLimit_ = value;
+          onChanged();
+        } else {
+          spaceLimitBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000080;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public Builder setSpaceLimit(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder builderForValue) {
+        if (spaceLimitBuilder_ == null) {
+          spaceLimit_ = builderForValue.build();
+          onChanged();
+        } else {
+          spaceLimitBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000080;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public Builder mergeSpaceLimit(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest value) {
+        if (spaceLimitBuilder_ == null) {
+          if (((bitField0_ & 0x00000080) == 0x00000080) &&
+              spaceLimit_ != null &&
+              spaceLimit_ != org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance()) {
+            spaceLimit_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.newBuilder(spaceLimit_).mergeFrom(value).buildPartial();
+          } else {
+            spaceLimit_ = value;
+          }
+          onChanged();
+        } else {
+          spaceLimitBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000080;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public Builder clearSpaceLimit() {
+        if (spaceLimitBuilder_ == null) {
+          spaceLimit_ = null;
+          onChanged();
+        } else {
+          spaceLimitBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000080);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder getSpaceLimitBuilder() {
+        bitField0_ |= 0x00000080;
+        onChanged();
+        return getSpaceLimitFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder getSpaceLimitOrBuilder() {
+        if (spaceLimitBuilder_ != null) {
+          return spaceLimitBuilder_.getMessageOrBuilder();
+        } else {
+          return spaceLimit_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance() : spaceLimit_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder> 
+          getSpaceLimitFieldBuilder() {
+        if (spaceLimitBuilder_ == null) {
+          spaceLimitBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder>(
+                  getSpaceLimit(),
+                  getParentForChildren(),
+                  isClean());
+          spaceLimit_ = null;
+        }
+        return spaceLimitBuilder_;
+      }
       public final Builder setUnknownFields(
           final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
@@ -76641,202 +76840,203 @@ public final class MasterProtos {
       " \001(\010:\004true\"6\n\026AbortProcedureResponse\022\034\n\024" +
       "is_procedure_aborted\030\001 \002(\010\"\027\n\025ListProced" +
       "uresRequest\"@\n\026ListProceduresResponse\022&\n" +
-      "\tprocedure\030\001 \003(\0132\023.hbase.pb.Procedure\"\315\001" +
+      "\tprocedure\030\001 \003(\0132\023.hbase.pb.Procedure\"\377\001" +
       "\n\017SetQuotaRequest\022\021\n\tuser_name\030\001 \001(\t\022\022\n\n" +
       "user_group\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\'\n\nt" +
       "able_name\030\004 \001(\0132\023.hbase.pb.TableName\022\022\n\n" +
       "remove_all\030\005 \001(\010\022\026\n\016bypass_globals\030\006 \001(\010" +
       "\022+\n\010throttle\030\007 \001(\0132\031.hbase.pb.ThrottleRe" +
-      "quest\"\022\n\020SetQuotaResponse\"J\n\037MajorCompac",
-      "tionTimestampRequest\022\'\n\ntable_name\030\001 \002(\013" +
-      "2\023.hbase.pb.TableName\"U\n(MajorCompaction" +
-      "TimestampForRegionRequest\022)\n\006region\030\001 \002(" +
-      "\0132\031.hbase.pb.RegionSpecifier\"@\n MajorCom" +
-      "pactionTimestampResponse\022\034\n\024compaction_t" +
-      "imestamp\030\001 \002(\003\"\035\n\033SecurityCapabilitiesRe" +
-      "quest\"\354\001\n\034SecurityCapabilitiesResponse\022G" +
-      "\n\014capabilities\030\001 \003(\01621.hbase.pb.Security" +
-      "CapabilitiesResponse.Capability\"\202\001\n\nCapa" +
-      "bility\022\031\n\025SIMPLE_AUTHENTICATION\020\000\022\031\n\025SEC",
-      "URE_AUTHENTICATION\020\001\022\021\n\rAUTHORIZATION\020\002\022" +
-      "\026\n\022CELL_AUTHORIZATION\020\003\022\023\n\017CELL_VISIBILI" +
-      "TY\020\004\"\"\n ListDrainingRegionServersRequest" +
-      "\"N\n!ListDrainingRegionServersResponse\022)\n" +
-      "\013server_name\030\001 \003(\0132\024.hbase.pb.ServerName" +
-      "\"F\n\031DrainRegionServersRequest\022)\n\013server_" +
-      "name\030\001 \003(\0132\024.hbase.pb.ServerName\"\034\n\032Drai" +
-      "nRegionServersResponse\"P\n#RemoveDrainFro" +
-      "mRegionServersRequest\022)\n\013server_name\030\001 \003" +
-      "(\0132\024.hbase.pb.ServerName\"&\n$RemoveDrainF",
-      "romRegionServersResponse*(\n\020MasterSwitch" +
-      "Type\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\3013\n\rMasterSer" +
-      "vice\022e\n\024GetSchemaAlterStatus\022%.hbase.pb." +
-      "GetSchemaAlterStatusRequest\032&.hbase.pb.G" +
-      "etSchemaAlterStatusResponse\022b\n\023GetTableD" +
-      "escriptors\022$.hbase.pb.GetTableDescriptor" +
-      "sRequest\032%.hbase.pb.GetTableDescriptorsR" +
-      "esponse\022P\n\rGetTableNames\022\036.hbase.pb.GetT" +
-      "ableNamesRequest\032\037.hbase.pb.GetTableName" +
-      "sResponse\022Y\n\020GetClusterStatus\022!.hbase.pb",
-      ".GetClusterStatusRequest\032\".hbase.pb.GetC" +
-      "lusterStatusResponse\022V\n\017IsMasterRunning\022" +
-      " .hbase.pb.IsMasterRunningRequest\032!.hbas" +
-      "e.pb.IsMasterRunningResponse\022D\n\tAddColum" +
-      "n\022\032.hbase.pb.AddColumnRequest\032\033.hbase.pb" +
-      ".AddColumnResponse\022M\n\014DeleteColumn\022\035.hba" +
-      "se.pb.DeleteColumnRequest\032\036.hbase.pb.Del" +
-      "eteColumnResponse\022M\n\014ModifyColumn\022\035.hbas" +
-      "e.pb.ModifyColumnRequest\032\036.hbase.pb.Modi" +
-      "fyColumnResponse\022G\n\nMoveRegion\022\033.hbase.p",
-      "b.MoveRegionRequest\032\034.hbase.pb.MoveRegio" +
-      "nResponse\022\\\n\021MergeTableRegions\022\".hbase.p" +
-      "b.MergeTableRegionsRequest\032#.hbase.pb.Me" +
-      "rgeTableRegionsResponse\022M\n\014AssignRegion\022" +
-      "\035.hbase.pb.AssignRegionRequest\032\036.hbase.p" +
-      "b.AssignRegionResponse\022S\n\016UnassignRegion" +
-      "\022\037.hbase.pb.UnassignRegionRequest\032 .hbas" +
-      "e.pb.UnassignRegionResponse\022P\n\rOfflineRe" +
-      "gion\022\036.hbase.pb.OfflineRegionRequest\032\037.h" +
-      "base.pb.OfflineRegionResponse\022J\n\013DeleteT",
-      "able\022\034.hbase.pb.DeleteTableRequest\032\035.hba" +
-      "se.pb.DeleteTableResponse\022P\n\rtruncateTab" +
-      "le\022\036.hbase.pb.TruncateTableRequest\032\037.hba" +
-      "se.pb.TruncateTableResponse\022J\n\013EnableTab" +
-      "le\022\034.hbase.pb.EnableTableRequest\032\035.hbase" +
-      ".pb.EnableTableResponse\022M\n\014DisableTable\022" +
-      "\035.hbase.pb.DisableTableRequest\032\036.hbase.p" +
-      "b.DisableTableResponse\022J\n\013ModifyTable\022\034." +
-      "hbase.pb.ModifyTableRequest\032\035.hbase.pb.M" +
-      "odifyTableResponse\022J\n\013CreateTable\022\034.hbas",
-      "e.pb.CreateTableRequest\032\035.hbase.pb.Creat" +
-      "eTableResponse\022A\n\010Shutdown\022\031.hbase.pb.Sh" +
-      "utdownRequest\032\032.hbase.pb.ShutdownRespons" +
-      "e\022G\n\nStopMaster\022\033.hbase.pb.StopMasterReq" +
-      "uest\032\034.hbase.pb.StopMasterResponse\022h\n\031Is" +
-      "MasterInMaintenanceMode\022$.hbase.pb.IsInM" +
-      "aintenanceModeRequest\032%.hbase.pb.IsInMai" +
-      "ntenanceModeResponse\022>\n\007Balance\022\030.hbase." +
-      "pb.BalanceRequest\032\031.hbase.pb.BalanceResp" +
-      "onse\022_\n\022SetBalancerRunning\022#.hbase.pb.Se",
-      "tBalancerRunningRequest\032$.hbase.pb.SetBa" +
-      "lancerRunningResponse\022\\\n\021IsBalancerEnabl" +
-      "ed\022\".hbase.pb.IsBalancerEnabledRequest\032#" +
-      ".hbase.pb.IsBalancerEnabledResponse\022k\n\026S" +
-      "etSplitOrMergeEnabled\022\'.hbase.pb.SetSpli" +
-      "tOrMergeEnabledRequest\032(.hbase.pb.SetSpl" +
-      "itOrMergeEnabledResponse\022h\n\025IsSplitOrMer" +
-      "geEnabled\022&.hbase.pb.IsSplitOrMergeEnabl" +
-      "edRequest\032\'.hbase.pb.IsSplitOrMergeEnabl" +
-      "edResponse\022D\n\tNormalize\022\032.hbase.pb.Norma",
-      "lizeRequest\032\033.hbase.pb.NormalizeResponse" +
-      "\022e\n\024SetNormalizerRunning\022%.hbase.pb.SetN" +
-      "ormalizerRunningRequest\032&.hbase.pb.SetNo" +
-      "rmalizerRunningResponse\022b\n\023IsNormalizerE" +
-      "nabled\022$.hbase.pb.IsNormalizerEnabledReq" +
-      "uest\032%.hbase.pb.IsNormalizerEnabledRespo" +
-      "nse\022S\n\016RunCatalogScan\022\037.hbase.pb.RunCata" +
-      "logScanRequest\032 .hbase.pb.RunCatalogScan" +
-      "Response\022e\n\024EnableCatalogJanitor\022%.hbase" +
-      ".pb.EnableCatalogJanitorRequest\032&.hbase.",
-      "pb.EnableCatalogJanitorResponse\022n\n\027IsCat" +
-      "alogJanitorEnabled\022(.hbase.pb.IsCatalogJ" +
-      "anitorEnabledRequest\032).hbase.pb.IsCatalo" +
-      "gJanitorEnabledResponse\022V\n\017RunCleanerCho" +
-      "re\022 .hbase.pb.RunCleanerChoreRequest\032!.h" +
-      "base.pb.RunCleanerChoreResponse\022k\n\026SetCl" +
-      "eanerChoreRunning\022\'.hbase.pb.SetCleanerC" +
-      "horeRunningRequest\032(.hbase.pb.SetCleaner" +
-      "ChoreRunningResponse\022h\n\025IsCleanerChoreEn" +
-      "abled\022&.hbase.pb.IsCleanerChoreEnabledRe",
-      "quest\032\'.hbase.pb.IsCleanerChoreEnabledRe" +
-      "sponse\022^\n\021ExecMasterService\022#.hbase.pb.C" +
-      "oprocessorServiceRequest\032$.hbase.pb.Copr" +
-      "ocessorServiceResponse\022A\n\010Snapshot\022\031.hba" +
-      "se.pb.SnapshotRequest\032\032.hbase.pb.Snapsho" +
-      "tResponse\022h\n\025GetCompletedSnapshots\022&.hba" +
-      "se.pb.GetCompletedSnapshotsRequest\032\'.hba" +
-      "se.pb.GetCompletedSnapshotsResponse\022S\n\016D" +
-      "eleteSnapshot\022\037.hbase.pb.DeleteSnapshotR" +
-      "equest\032 .hbase.pb.DeleteSnapshotResponse",
-      "\022S\n\016IsSnapshotDone\022\037.hbase.pb.IsSnapshot" +
-      "DoneRequest\032 .hbase.pb.IsSnapshotDoneRes" +
-      "ponse\022V\n\017RestoreSnapshot\022 .hbase.pb.Rest" +
-      "oreSnapshotRequest\032!.hbase.pb.RestoreSna" +
-      "pshotResponse\022P\n\rExecProcedure\022\036.hbase.p" +
-      "b.ExecProcedureRequest\032\037.hbase.pb.ExecPr" +
-      "ocedureResponse\022W\n\024ExecProcedureWithRet\022" +
-      "\036.hbase.pb.ExecProcedureRequest\032\037.hbase." +
-      "pb.ExecProcedureResponse\022V\n\017IsProcedureD" +
-      "one\022 .hbase.pb.IsProcedureDoneRequest\032!.",
-      "hbase.pb.IsProcedureDoneResponse\022V\n\017Modi" +
-      "fyNamespace\022 .hbase.pb.ModifyNamespaceRe" +
-      "quest\032!.hbase.pb.ModifyNamespaceResponse" +
-      "\022V\n\017CreateNamespace\022 .hbase.pb.CreateNam" +
-      "espaceRequest\032!.hbase.pb.CreateNamespace" +
-      "Response\022V\n\017DeleteNamespace\022 .hbase.pb.D" +
-      "eleteNamespaceRequest\032!.hbase.pb.DeleteN" +
-      "amespaceResponse\022k\n\026GetNamespaceDescript" +
-      "or\022\'.hbase.pb.GetNamespaceDescriptorRequ" +
-      "est\032(.hbase.pb.GetNamespaceDescriptorRes",
-      "ponse\022q\n\030ListNamespaceDescriptors\022).hbas" +
-      "e.pb.ListNamespaceDescriptorsRequest\032*.h" +
-      "base.pb.ListNamespaceDescriptorsResponse" +
-      "\022\206\001\n\037ListTableDescriptorsByNamespace\0220.h" +
-      "base.pb.ListTableDescriptorsByNamespaceR" +
-      "equest\0321.hbase.pb.ListTableDescriptorsBy" +
-      "NamespaceResponse\022t\n\031ListTableNamesByNam" +
-      "espace\022*.hbase.pb.ListTableNamesByNamesp" +
-      "aceRequest\032+.hbase.pb.ListTableNamesByNa" +
-      "mespaceResponse\022P\n\rGetTableState\022\036.hbase",
-      ".pb.GetTableStateRequest\032\037.hbase.pb.GetT" +
-      "ableStateResponse\022A\n\010SetQuota\022\031.hbase.pb" +
-      ".SetQuotaRequest\032\032.hbase.pb.SetQuotaResp" +
-      "onse\022x\n\037getLastMajorCompactionTimestamp\022" +
-      ").hbase.pb.MajorCompactionTimestampReque" +
-      "st\032*.hbase.pb.MajorCompactionTimestampRe" +
-      "sponse\022\212\001\n(getLastMajorCompactionTimesta" +
-      "mpForRegion\0222.hbase.pb.MajorCompactionTi" +
-      "mestampForRegionRequest\032*.hbase.pb.Major" +
-      "CompactionTimestampResponse\022_\n\022getProced",
-      "ureResult\022#.hbase.pb.GetProcedureResultR" +
-      "equest\032$.hbase.pb.GetProcedureResultResp" +
-      "onse\022h\n\027getSecurityCapabilities\022%.hbase." +
-      "pb.SecurityCapabilitiesRequest\032&.hbase.p" +
-      "b.SecurityCapabilitiesResponse\022S\n\016AbortP" +
-      "rocedure\022\037.hbase.pb.AbortProcedureReques" +
-      "t\032 .hbase.pb.AbortProcedureResponse\022S\n\016L" +
-      "istProcedures\022\037.hbase.pb.ListProceduresR" +
-      "equest\032 .hbase.pb.ListProceduresResponse" +
-      "\022_\n\022AddReplicationPeer\022#.hbase.pb.AddRep",
-      "licationPeerRequest\032$.hbase.pb.AddReplic" +
-      "ationPeerResponse\022h\n\025RemoveReplicationPe" +
-      "er\022&.hbase.pb.RemoveReplicationPeerReque" +
-      "st\032\'.hbase.pb.RemoveReplicationPeerRespo" +
-      "nse\022h\n\025EnableReplicationPeer\022&.hbase.pb." +
-      "EnableReplicationPeerRequest\032\'.hbase.pb." +
-      "EnableReplicationPeerResponse\022k\n\026Disable" +
-      "ReplicationPeer\022\'.hbase.pb.DisableReplic" +
-      "ationPeerRequest\032(.hbase.pb.DisableRepli" +
-      "cationPeerResponse\022q\n\030GetReplicationPeer",
-      "Config\022).hbase.pb.GetReplicationPeerConf" +
-      "igRequest\032*.hbase.pb.GetReplicationPeerC" +
-      "onfigResponse\022z\n\033UpdateReplicationPeerCo" +
-      "nfig\022,.hbase.pb.UpdateReplicationPeerCon" +
-      "figRequest\032-.hbase.pb.UpdateReplicationP" +
-      "eerConfigResponse\022e\n\024ListReplicationPeer" +
-      "s\022%.hbase.pb.ListReplicationPeersRequest" +
-      "\032&.hbase.pb.ListReplicationPeersResponse" +
-      "\022t\n\031listDrainingRegionServers\022*.hbase.pb" +
-      ".ListDrainingRegionServersRequest\032+.hbas",
-      "e.pb.ListDrainingRegionServersResponse\022_" +
-      "\n\022drainRegionServers\022#.hbase.pb.DrainReg" +
-      "ionServersRequest\032$.hbase.pb.DrainRegion" +
-      "ServersResponse\022}\n\034removeDrainFromRegion" +
-      "Servers\022-.hbase.pb.RemoveDrainFromRegion" +
-      "ServersRequest\032..hbase.pb.RemoveDrainFro" +
-      "mRegionServersResponseBI\n1org.apache.had" +
-      "oop.hbase.shaded.protobuf.generatedB\014Mas" +
-      "terProtosH\001\210\001\001\240\001\001"
+      "quest\0220\n\013space_limit\030\010 \001(\0132\033.hbase.pb.Sp",
+      "aceLimitRequest\"\022\n\020SetQuotaResponse\"J\n\037M" +
+      "ajorCompactionTimestampRequest\022\'\n\ntable_" +
+      "name\030\001 \002(\0132\023.hbase.pb.TableName\"U\n(Major" +
+      "CompactionTimestampForRegionRequest\022)\n\006r" +
+      "egion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\"@" +
+      "\n MajorCompactionTimestampResponse\022\034\n\024co" +
+      "mpaction_timestamp\030\001 \002(\003\"\035\n\033SecurityCapa" +
+      "bilitiesRequest\"\354\001\n\034SecurityCapabilities" +
+      "Response\022G\n\014capabilities\030\001 \003(\01621.hbase.p" +
+      "b.SecurityCapabilitiesResponse.Capabilit",
+      "y\"\202\001\n\nCapability\022\031\n\025SIMPLE_AUTHENTICATIO" +
+      "N\020\000\022\031\n\025SECURE_AUTHENTICATION\020\001\022\021\n\rAUTHOR" +
+      "IZATION\020\002\022\026\n\022CELL_AUTHORIZATION\020\003\022\023\n\017CEL" +
+      "L_VISIBILITY\020\004\"\"\n ListDrainingRegionServ" +
+      "ersRequest\"N\n!ListDrainingRegionServersR" +
+      "esponse\022)\n\013server_name\030\001 \003(\0132\024.hbase.pb." +
+      "ServerName\"F\n\031DrainRegionServersRequest\022" +
+      ")\n\013server_name\030\001 \003(\0132\024.hbase.pb.ServerNa" +
+      "me\"\034\n\032DrainRegionServersResponse\"P\n#Remo" +
+      "veDrainFromRegionServersRequest\022)\n\013serve",
+      "r_name\030\001 \003(\0132\024.hbase.pb.ServerName\"&\n$Re" +
+      "moveDrainFromRegionServersResponse*(\n\020Ma" +
+      "sterSwitchType\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\3013\n" +
+      "\rMasterService\022e\n\024GetSchemaAlterStatus\022%" +
+      ".hbase.pb.GetSchemaAlterStatusRequest\032&." +
+      "hbase.pb.GetSchemaAlterStatusResponse\022b\n" +
+      "\023GetTableDescriptors\022$.hbase.pb.GetTable" +
+      "DescriptorsRequest\032%.hbase.pb.GetTableDe" +
+      "scriptorsResponse\022P\n\rGetTableNames\022\036.hba" +
+      "se.pb.GetTableNamesRequest\032\037.hbase.pb.Ge",
+      "tTableNamesResponse\022Y\n\020GetClusterStatus\022" +
+      "!.hbase.pb.GetClusterStatusRequest\032\".hba" +
+      "se.pb.GetClusterStatusResponse\022V\n\017IsMast" +
+      "erRunning\022 .hbase.pb.IsMasterRunningRequ" +
+      "est\032!.hbase.pb.IsMasterRunningResponse\022D" +
+      "\n\tAddColumn\022\032.hbase.pb.AddColumnRequest\032" +
+      "\033.hbase.pb.AddColumnResponse\022M\n\014DeleteCo" +
+      "lumn\022\035.hbase.pb.DeleteColumnRequest\032\036.hb" +
+      "ase.pb.DeleteColumnResponse\022M\n\014ModifyCol" +
+      "umn\022\035.hbase.pb.ModifyColumnRequest\032\036.hba",
+      "se.pb.ModifyColumnResponse\022G\n\nMoveRegion" +
+      "\022\033.hbase.pb.MoveRegionRequest\032\034.hbase.pb" +
+      ".MoveRegionResponse\022\\\n\021MergeTableRegions" +
+      "\022\".hbase.pb.MergeTableRegionsRequest\032#.h" +
+      "base.pb.MergeTableRegionsResponse\022M\n\014Ass" +
+      "ignRegion\022\035.hbase.pb.AssignRegionRequest" +
+      "\032\036.hbase.pb.AssignRegionResponse\022S\n\016Unas" +
+      "signRegion\022\037.hbase.pb.UnassignRegionRequ" +
+      "est\032 .hbase.pb.UnassignRegionResponse\022P\n" +
+      "\rOfflineRegion\022\036.hbase.pb.OfflineRegionR",
+      "equest\032\037.hbase.pb.OfflineRegionResponse\022" +
+      "J\n\013DeleteTable\022\034.hbase.pb.DeleteTableReq" +
+      "uest\032\035.hbase.pb.DeleteTableResponse\022P\n\rt" +
+      "runcateTable\022\036.hbase.pb.TruncateTableReq" +
+      "uest\032\037.hbase.pb.TruncateTableResponse\022J\n" +
+      "\013EnableTable\022\034.hbase.pb.EnableTableReque" +
+      "st\032\035.hbase.pb.EnableTableResponse\022M\n\014Dis" +
+      "ableTable\022\035.hbase.pb.DisableTableRequest" +
+      "\032\036.hbase.pb.DisableTableResponse\022J\n\013Modi" +
+      "fyTable\022\034.hbase.pb.ModifyTableRequest\032\035.",
+      "hbase.pb.ModifyTableResponse\022J\n\013CreateTa" +
+      "ble\022\034.hbase.pb.CreateTableRequest\032\035.hbas" +
+      "e.pb.CreateTableResponse\022A\n\010Shutdown\022\031.h" +
+      "base.pb.ShutdownRequest\032\032.hbase.pb.Shutd" +
+      "ownResponse\022G\n\nStopMaster\022\033.hbase.pb.Sto" +
+      "pMasterRequest\032\034.hbase.pb.StopMasterResp" +
+      "onse\022h\n\031IsMasterInMaintenanceMode\022$.hbas" +
+      "e.pb.IsInMaintenanceModeRequest\032%.hbase." +
+      "pb.IsInMaintenanceModeResponse\022>\n\007Balanc" +
+      "e\022\030.hbase.pb.BalanceRequest\032\031.hbase.pb.B",
+      "alanceResponse\022_\n\022SetBalancerRunning\022#.h" +
+      "base.pb.SetBalancerRunningRequest\032$.hbas" +
+      "e.pb.SetBalancerRunningResponse\022\\\n\021IsBal" +
+      "ancerEnabled\022\".hbase.pb.IsBalancerEnable" +
+      "dRequest\032#.hbase.pb.IsBalancerEnabledRes" +
+      "ponse\022k\n\026SetSplitOrMergeEnabled\022\'.hbase." +
+      "pb.SetSplitOrMergeEnabledRequest\032(.hbase" +
+      ".pb.SetSplitOrMergeEnabledResponse\022h\n\025Is" +
+      "SplitOrMergeEnabled\022&.hbase.pb.IsSplitOr" +
+      "MergeEnabledRequest\032\'.hbase.pb.IsSplitOr",
+      "MergeEnabledResponse\022D\n\tNormalize\022\032.hbas" +
+      "e.pb.NormalizeRequest\032\033.hbase.pb.Normali" +
+      "zeResponse\022e\n\024SetNormalizerRunning\022%.hba" +
+      "se.pb.SetNormalizerRunningRequest\032&.hbas" +
+      "e.pb.SetNormalizerRunningResponse\022b\n\023IsN" +
+      "ormalizerEnabled\022$.hbase.pb.IsNormalizer" +
+      "EnabledRequest\032%.hbase.pb.IsNormalizerEn" +
+      "abledResponse\022S\n\016RunCatalogScan\022\037.hbase." +
+      "pb.RunCatalogScanRequest\032 .hbase.pb.RunC" +
+      "atalogScanResponse\022e\n\024EnableCatalogJanit",
+      "or\022%.hbase.pb.EnableCatalogJanitorReques" +
+      "t\032&.hbase.pb.EnableCatalogJanitorRespons" +
+      "e\022n\n\027IsCatalogJanitorEnabled\022(.hbase.pb." +
+      "IsCatalogJanitorEnabledRequest\032).hbase.p" +
+      "b.IsCatalogJanitorEnabledResponse\022V\n\017Run" +
+      "CleanerChore\022 .hbase.pb.RunCleanerChoreR" +
+      "equest\032!.hbase.pb.RunCleanerChoreRespons" +
+      "e\022k\n\026SetCleanerChoreRunning\022\'.hbase.pb.S" +
+      "etCleanerChoreRunningRequest\032(.hbase.pb." +
+      "SetCleanerChoreRunningResponse\022h\n\025IsClea",
+      "nerChoreEnabled\022&.hbase.pb.IsCleanerChor" +
+      "eEnabledRequest\032\'.hbase.pb.IsCleanerChor" +
+      "eEnabledResponse\022^\n\021ExecMasterService\022#." +
+      "hbase.pb.CoprocessorServiceRequest\032$.hba" +
+      "se.pb.CoprocessorServiceResponse\022A\n\010Snap" +
+      "shot\022\031.hbase.pb.SnapshotRequest\032\032.hbase." +
+      "pb.SnapshotResponse\022h\n\025GetCompletedSnaps" +
+      "hots\022&.hbase.pb.GetCompletedSnapshotsReq" +
+      "uest\032\'.hbase.pb.GetCompletedSnapshotsRes" +
+      "ponse\022S\n\016DeleteSnapshot\022\037.hbase.pb.Delet",
+      "eSnapshotRequest\032 .hbase.pb.DeleteSnapsh" +
+      "otResponse\022S\n\016IsSnapshotDone\022\037.hbase.pb." +
+      "IsSnapshotDoneRequest\032 .hbase.pb.IsSnaps" +
+      "hotDoneResponse\022V\n\017RestoreSnapshot\022 .hba" +
+      "se.pb.RestoreSnapshotRequest\032!.hbase.pb." +
+      "RestoreSnapshotResponse\022P\n\rExecProcedure" +
+      "\022\036.hbase.pb.ExecProcedureRequest\032\037.hbase" +
+      ".pb.ExecProcedureResponse\022W\n\024ExecProcedu" +
+      "reWithRet\022\036.hbase.pb.ExecProcedureReques" +
+      "t\032\037.hbase.pb.ExecProcedureResponse\022V\n\017Is",
+      "ProcedureDone\022 .hbase.pb.IsProcedureDone" +
+      "Request\032!.hbase.pb.IsProcedureDoneRespon" +
+      "se\022V\n\017ModifyNamespace\022 .hbase.pb.ModifyN" +
+      "amespaceRequest\032!.hbase.pb.ModifyNamespa" +
+      "ceResponse\022V\n\017CreateNamespace\022 .hbase.pb" +
+      ".CreateNamespaceRequest\032!.hbase.pb.Creat" +
+      "eNamespaceResponse\022V\n\017DeleteNamespace\022 ." +
+      "hbase.pb.DeleteNamespaceRequest\032!.hbase." +
+      "pb.DeleteNamespaceResponse\022k\n\026GetNamespa" +
+      "ceDescriptor\022\'.hbase.pb.GetNamespaceDesc",
+      "riptorRequest\032(.hbase.pb.GetNamespaceDes" +
+      "criptorResponse\022q\n\030ListNamespaceDescript" +
+      "ors\022).hbase.pb.ListNamespaceDescriptorsR" +
+      "equest\032*.hbase.pb.ListNamespaceDescripto" +
+      "rsResponse\022\206\001\n\037ListTableDescriptorsByNam" +
+      "espace\0220.hbase.pb.ListTableDescriptorsBy" +
+      "NamespaceRequest\0321.hbase.pb.ListTableDes" +
+      "criptorsByNamespaceResponse\022t\n\031ListTable" +
+      "NamesByNamespace\022*.hbase.pb.ListTableNam" +
+      "esByNamespaceRequest\032+.hbase.pb.ListTabl",
+      "eNamesByNamespaceResponse\022P\n\rGetTableSta" +
+      "te\022\036.hbase.pb.GetTableStateRequest\032\037.hba" +
+      "se.pb.GetTableStateResponse\022A\n\010SetQuota\022" +
+      "\031.hbase.pb.SetQuotaRequest\032\032.hbase.pb.Se" +
+      "tQuotaResponse\022x\n\037getLastMajorCompaction" +
+      "Timestamp\022).hbase.pb.MajorCompactionTime" +
+      "stampRequest\032*.hbase.pb.MajorCompactionT" +
+      "imestampResponse\022\212\001\n(getLastMajorCompact" +
+      "ionTimestampForRegion\0222.hbase.pb.MajorCo" +
+      "mpactionTimestampForRegionRequest\032*.hbas",
+      "e.pb.MajorCompactionTimestampResponse\022_\n" +
+      "\022getProcedureResult\022#.hbase.pb.GetProced" +
+      "ureResultRequest\032$.hbase.pb.GetProcedure" +
+      "ResultResponse\022h\n\027getSecurityCapabilitie" +
+      "s\022%.hbase.pb.SecurityCapabilitiesRequest" +
+      "\032&.hbase.pb.SecurityCapabilitiesResponse" +
+      "\022S\n\016AbortProcedure\022\037.hbase.pb.AbortProce" +
+      "dureRequest\032 .hbase.pb.AbortProcedureRes" +
+      "ponse\022S\n\016ListProcedures\022\037.hbase.pb.ListP" +
+      "roceduresRequest\032 .hbase.pb.ListProcedur",
+      "esResponse\022_\n\022AddReplicationPeer\022#.hbase" +
+      ".pb.AddReplicationPeerRequest\032$.hbase.pb" +
+      ".AddReplicationPeerResponse\022h\n\025RemoveRep" +
+      "licationPeer\022&.hbase.pb.RemoveReplicatio" +
+      "nPeerRequest\032\'.hbase.pb.RemoveReplicatio" +
+      "nPeerResponse\022h\n\025EnableReplicationPeer\022&" +
+      ".hbase.pb.EnableReplicationPeerRequest\032\'" +
+      ".hbase.pb.EnableReplicationPeerResponse\022" +
+      "k\n\026DisableReplicationPeer\022\'.hbase.pb.Dis" +
+      "ableReplicationPeerRequest\032(.hbase.pb.Di",
+      "sableReplicationPeerResponse\022q\n\030GetRepli" +
+      "cationPeerConfig\022).hbase.pb.GetReplicati" +
+      "onPeerConfigRequest\032*.hbase.pb.GetReplic" +
+      "ationPeerConfigResponse\022z\n\033UpdateReplica" +
+      "tionPeerConfig\022,.hbase.pb.UpdateReplicat" +
+      "ionPeerConfigRequest\032-.hbase.pb.UpdateRe" +
+      "plicationPeerConfigResponse\022e\n\024ListRepli" +
+      "cationPeers\022%.hbase.pb.ListReplicationPe" +
+      "ersRequest\032&.hbase.pb.ListReplicationPee" +
+      "rsResponse\022t\n\031listDrainingRegionServers\022",
+      "*.hbase.pb.ListDrainingRegionServersRequ" +
+      "est\032+.hbase.pb.ListDrainingRegionServers" +
+      "Response\022_\n\022drainRegionServers\022#.hbase.p" +
+      "b.DrainRegionServersRequest\032$.hbase.pb.D" +
+      "rainRegionServersResponse\022}\n\034removeDrain" +
+      "FromRegionServers\022-.hbase.pb.RemoveDrain" +
+      "FromRegionServersRequest\032..hbase.pb.Remo" +
+      "veDrainFromRegionServersResponseBI\n1org." +
+      "apache.hadoop.hbase.shaded.protobuf.gene" +
+      "ratedB\014MasterProtosH\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -77522,7 +77722,7 @@ public final class MasterProtos {
     internal_static_hbase_pb_SetQuotaRequest_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SetQuotaRequest_descriptor,
-        new java.lang.String[] { "UserName", "UserGroup", "Namespace", "TableName", "RemoveAll", "BypassGlobals", "Throttle", });
+        new java.lang.String[] { "UserName", "UserGroup", "Namespace", "TableName", "RemoveAll", "BypassGlobals", "Throttle", "SpaceLimit", });
     internal_static_hbase_pb_SetQuotaResponse_descriptor =
       getDescriptor().getMessageTypes().get(111);
     internal_static_hbase_pb_SetQuotaResponse_fieldAccessorTable = new


[44/50] [abbrv] hbase git commit: HBASE-17516 Correctly handle case where table and NS quotas both apply

Posted by el...@apache.org.
HBASE-17516 Correctly handle case where table and NS quotas both apply

The logic surrounding when a table and namespace quota both apply
to a table was incorrect, leading to a case where a table quota
violation which should have fired did not because of the less-strict
namespace quota.


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

Branch: refs/heads/HBASE-16961
Commit: 48332eebfd7fdd0a2065db448e82e82e1548cfc4
Parents: 095fabf
Author: Josh Elser <el...@apache.org>
Authored: Wed Feb 22 18:32:55 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:44:00 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hbase/quotas/QuotaObserverChore.java | 10 ++-
 .../TestQuotaObserverChoreWithMiniCluster.java  | 66 ++++++++++++--------
 .../hbase/quotas/TestQuotaStatusRPCs.java       | 21 ++++++-
 .../hadoop/hbase/quotas/TestSpaceQuotas.java    | 32 +++++++++-
 4 files changed, 97 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/48332eeb/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 973ac8c..b9f4592 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
@@ -287,7 +287,8 @@ public class QuotaObserverChore extends ScheduledChore {
       // We want to have a policy of "NONE", moving out of violation
       if (!targetStatus.isInViolation()) {
         for (TableName tableInNS : tablesByNamespace.get(namespace)) {
-          if (!tableSnapshotStore.getCurrentState(tableInNS).getQuotaStatus().isInViolation()) {
+          // If there is a quota on this table in violation
+          if (tableSnapshotStore.getCurrentState(tableInNS).getQuotaStatus().isInViolation()) {
             // Table-level quota violation policy is being applied here.
             if (LOG.isTraceEnabled()) {
               LOG.trace("Not activating Namespace violation policy because a Table violation"
@@ -298,16 +299,21 @@ public class QuotaObserverChore extends ScheduledChore {
             this.snapshotNotifier.transitionTable(tableInNS, targetSnapshot);
           }
         }
+      // We want to move into violation at the NS level
       } else {
         // Moving tables in the namespace into violation or to a different violation policy
         for (TableName tableInNS : tablesByNamespace.get(namespace)) {
-          if (tableSnapshotStore.getCurrentState(tableInNS).getQuotaStatus().isInViolation()) {
+          final SpaceQuotaSnapshot tableQuotaSnapshot =
+                tableSnapshotStore.getCurrentState(tableInNS);
+          final boolean hasTableQuota = QuotaSnapshotStore.NO_QUOTA != tableQuotaSnapshot;
+          if (hasTableQuota && tableQuotaSnapshot.getQuotaStatus().isInViolation()) {
             // Table-level quota violation policy is being applied here.
             if (LOG.isTraceEnabled()) {
               LOG.trace("Not activating Namespace violation policy because a Table violation"
                   + " policy is already in effect for " + tableInNS);
             }
           } else {
+            // No table quota present or a table quota present that is not in violation
             LOG.info(tableInNS + " moving into violation of namespace space quota with policy " + targetStatus.getPolicy());
             this.snapshotNotifier.transitionTable(tableInNS, targetSnapshot);
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48332eeb/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 943c898..63198a8 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
@@ -193,40 +193,42 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
     helper.writeData(tn1, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn1);
-    Map<TableName,SpaceQuotaSnapshot> violatedQuotas = snapshotNotifier.copySnapshots();
+    Map<TableName,SpaceQuotaSnapshot> snapshots = snapshotNotifier.copySnapshots();
     for (int i = 0; i < 5; i++) {
       // Check a few times to make sure we don't prematurely move to violation
-      assertEquals("Should not see any quota violations after writing 2MB of data", 0, violatedQuotas.size());
+      assertEquals(
+          "Should not see any quota violations after writing 2MB of data", 0,
+          numSnapshotsInViolation(snapshots));
       try {
         Thread.sleep(DEFAULT_WAIT_MILLIS);
       } catch (InterruptedException e) {
         LOG.debug("Interrupted while sleeping." , e);
       }
-      violatedQuotas = snapshotNotifier.copySnapshots();
+      snapshots = snapshotNotifier.copySnapshots();
     }
 
     helper.writeData(tn2, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn2);
-    violatedQuotas = snapshotNotifier.copySnapshots();
+    snapshots = snapshotNotifier.copySnapshots();
     for (int i = 0; i < 5; i++) {
       // Check a few times to make sure we don't prematurely move to violation
       assertEquals("Should not see any quota violations after writing 4MB of data", 0,
-          violatedQuotas.size());
+          numSnapshotsInViolation(snapshots));
       try {
         Thread.sleep(DEFAULT_WAIT_MILLIS);
       } catch (InterruptedException e) {
         LOG.debug("Interrupted while sleeping." , e);
       }
-      violatedQuotas = snapshotNotifier.copySnapshots();
+      snapshots = snapshotNotifier.copySnapshots();
     }
 
     // Writing the final 2MB of data will push the namespace over the 5MB limit (6MB in total)
     // and should push all three tables in the namespace into violation.
     helper.writeData(tn3, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn3);
-    violatedQuotas = snapshotNotifier.copySnapshots();
-    while (violatedQuotas.size() < 3) {
-      LOG.debug("Saw fewer violations than desired (expected 3): " + violatedQuotas
+    snapshots = snapshotNotifier.copySnapshots();
+    while (numSnapshotsInViolation(snapshots) < 3) {
+      LOG.debug("Saw fewer violations than desired (expected 3): " + snapshots
           + ". Current reports: " + master.getMasterQuotaManager().snapshotRegionSizes());
       try {
         Thread.sleep(DEFAULT_WAIT_MILLIS);
@@ -234,19 +236,19 @@ public class TestQuotaObserverChoreWithMiniCluster {
         LOG.debug("Interrupted while sleeping.", e);
         Thread.currentThread().interrupt();
       }
-      violatedQuotas = snapshotNotifier.copySnapshots();
+      snapshots = snapshotNotifier.copySnapshots();
     }
 
-    SpaceQuotaSnapshot snapshot1 = violatedQuotas.remove(tn1);
+    SpaceQuotaSnapshot snapshot1 = snapshots.remove(tn1);
     assertNotNull("tn1 should be in violation", snapshot1);
     assertEquals(violationPolicy, snapshot1.getQuotaStatus().getPolicy());
-    SpaceQuotaSnapshot snapshot2 = violatedQuotas.remove(tn2);
+    SpaceQuotaSnapshot snapshot2 = snapshots.remove(tn2);
     assertNotNull("tn2 should be in violation", snapshot2);
     assertEquals(violationPolicy, snapshot2.getQuotaStatus().getPolicy());
-    SpaceQuotaSnapshot snapshot3 = violatedQuotas.remove(tn3);
+    SpaceQuotaSnapshot snapshot3 = snapshots.remove(tn3);
     assertNotNull("tn3 should be in violation", snapshot3);
     assertEquals(violationPolicy, snapshot3.getQuotaStatus().getPolicy());
-    assertTrue("Unexpected additional quota violations: " + violatedQuotas, violatedQuotas.isEmpty());
+    assertTrue("Unexpected additional quota violations: " + snapshots, snapshots.isEmpty());
   }
 
   @Test
@@ -272,24 +274,24 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
     helper.writeData(tn1, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn1);
-    Map<TableName,SpaceQuotaSnapshot> violatedQuotas = snapshotNotifier.copySnapshots();
+    Map<TableName,SpaceQuotaSnapshot> snapshots = snapshotNotifier.copySnapshots();
     for (int i = 0; i < 5; i++) {
       // Check a few times to make sure we don't prematurely move to violation
-      assertEquals("Should not see any quota violations after writing 2MB of data", 0,
-          violatedQuotas.size());
+      assertEquals("Should not see any quota violations after writing 2MB of data: " + snapshots, 0,
+          numSnapshotsInViolation(snapshots));
       try {
         Thread.sleep(DEFAULT_WAIT_MILLIS);
       } catch (InterruptedException e) {
         LOG.debug("Interrupted while sleeping." , e);
       }
-      violatedQuotas = snapshotNotifier.copySnapshots();
+      snapshots = snapshotNotifier.copySnapshots();
     }
 
     helper.writeData(tn2, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
     admin.flush(tn2);
-    violatedQuotas = snapshotNotifier.copySnapshots();
-    while (violatedQuotas.size() < 2) {
-      LOG.debug("Saw fewer violations than desired (expected 2): " + violatedQuotas
+    snapshots = snapshotNotifier.copySnapshots();
+    while (numSnapshotsInViolation(snapshots) < 2) {
+      LOG.debug("Saw fewer violations than desired (expected 2): " + snapshots
           + ". Current reports: " + master.getMasterQuotaManager().snapshotRegionSizes());
       try {
         Thread.sleep(DEFAULT_WAIT_MILLIS);
@@ -297,13 +299,13 @@ public class TestQuotaObserverChoreWithMiniCluster {
         LOG.debug("Interrupted while sleeping.", e);
         Thread.currentThread().interrupt();
       }
-      violatedQuotas = snapshotNotifier.copySnapshots();
+      snapshots = snapshotNotifier.copySnapshots();
     }
 
-    SpaceQuotaSnapshot actualPolicyTN1 = violatedQuotas.get(tn1);
+    SpaceQuotaSnapshot actualPolicyTN1 = snapshots.get(tn1);
     assertNotNull("Expected to see violation policy for tn1", actualPolicyTN1);
     assertEquals(namespaceViolationPolicy, actualPolicyTN1.getQuotaStatus().getPolicy());
-    SpaceQuotaSnapshot actualPolicyTN2 = violatedQuotas.get(tn2);
+    SpaceQuotaSnapshot actualPolicyTN2 = snapshots.get(tn2);
     assertNotNull("Expected to see violation policy for tn2", actualPolicyTN2);
     assertEquals(namespaceViolationPolicy, actualPolicyTN2.getQuotaStatus().getPolicy());
 
@@ -316,8 +318,8 @@ public class TestQuotaObserverChoreWithMiniCluster {
 
     // Keep checking for the table quota policy to override the namespace quota
     while (true) {
-      violatedQuotas = snapshotNotifier.copySnapshots();
-      SpaceQuotaSnapshot actualTableSnapshot = violatedQuotas.get(tn1);
+      snapshots = snapshotNotifier.copySnapshots();
+      SpaceQuotaSnapshot actualTableSnapshot = snapshots.get(tn1);
       assertNotNull("Violation policy should never be null", actualTableSnapshot);
       if (tableViolationPolicy != actualTableSnapshot.getQuotaStatus().getPolicy()) {
         LOG.debug("Saw unexpected table violation policy, waiting and re-checking.");
@@ -334,7 +336,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
     }
 
     // This should not change with the introduction of the table quota for tn1
-    actualPolicyTN2 = violatedQuotas.get(tn2);
+    actualPolicyTN2 = snapshots.get(tn2);
     assertNotNull("Expected to see violation policy for tn2", actualPolicyTN2);
     assertEquals(namespaceViolationPolicy, actualPolicyTN2.getQuotaStatus().getPolicy());
   }
@@ -440,6 +442,16 @@ public class TestQuotaObserverChoreWithMiniCluster {
     assertNull(chore.getTableSnapshotStore().getSpaceQuota(tableWithoutQuota));
   }
 
+  private int numSnapshotsInViolation(Map<TableName,SpaceQuotaSnapshot> snapshots) {
+    int sum = 0;
+    for (SpaceQuotaSnapshot snapshot : snapshots.values()) {
+      if (snapshot.getQuotaStatus().isInViolation()) {
+        sum++;
+      }
+    }
+    return sum;
+  }
+
   private void sleepWithInterrupt(long millis) {
     try {
       Thread.sleep(millis);

http://git-wip-us.apache.org/repos/asf/hbase/blob/48332eeb/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 d42f3d2..5a00aaf 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
@@ -22,8 +22,11 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -47,6 +50,7 @@ import org.junit.rules.TestName;
  */
 @Category({MediumTests.class})
 public class TestQuotaStatusRPCs {
+  private static final Log LOG = LogFactory.getLog(TestQuotaStatusRPCs.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final AtomicLong COUNTER = new AtomicLong(0);
 
@@ -92,7 +96,10 @@ public class TestQuotaStatusRPCs {
     Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        return numRegions == countRegionsForTable(tn, quotaManager.snapshotRegionSizes());
+        Map<HRegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
+        LOG.trace("Region sizes=" + regionSizes);
+        return numRegions == countRegionsForTable(tn, regionSizes) &&
+            tableSize <= getTableSize(tn, regionSizes);
       }
     });
 
@@ -189,4 +196,16 @@ public class TestQuotaStatusRPCs {
     }
     return size;
   }
+
+  private int getTableSize(TableName tn, Map<HRegionInfo,Long> regionSizes) {
+    int tableSize = 0;
+    for (Entry<HRegionInfo,Long> entry : regionSizes.entrySet()) {
+      HRegionInfo regionInfo = entry.getKey();
+      long regionSize = entry.getValue();
+      if (tn.equals(regionInfo.getTable())) {
+        tableSize += regionSize;
+      }
+    }
+    return tableSize;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48332eeb/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 ea7dcf1..ffe0ce2 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
@@ -340,6 +340,31 @@ public class TestSpaceQuotas {
     }
   }
 
+  @Test(timeout=120000)
+  public void testTableQuotaOverridesNamespaceQuota() throws Exception {
+    final SpaceViolationPolicy policy = SpaceViolationPolicy.NO_INSERTS;
+    final TableName tn = helper.createTableWithRegions(10);
+
+    // 2MB limit on the table, 1GB limit on the namespace
+    final long tableLimit = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    final long namespaceLimit = 1024L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+    TEST_UTIL.getAdmin().setQuota(QuotaSettingsFactory.limitTableSpace(tn, tableLimit, policy));
+    TEST_UTIL.getAdmin().setQuota(QuotaSettingsFactory.limitNamespaceSpace(
+        tn.getNamespaceAsString(), namespaceLimit, policy));
+
+    // Write more data than should be allowed and flush it to disk
+    helper.writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
+
+    // This should be sufficient time for the chores to run and see the change.
+    Thread.sleep(5000);
+
+    // The write should be rejected because the table quota takes priority over the namespace
+    Put p = new Put(Bytes.toBytes("to_reject"));
+    p.addColumn(
+        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    verifyViolation(policy, tn, p);
+  }
+
   private Map<HRegionInfo,Long> getReportedSizesForTable(TableName tn) {
     HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
     MasterQuotaManager quotaManager = master.getMasterQuotaManager();
@@ -371,7 +396,12 @@ public class TestSpaceQuotas {
   private TableName writeUntilViolationAndVerifyViolation(
       SpaceViolationPolicy policyToViolate, Mutation m) throws Exception {
     final TableName tn = writeUntilViolation(policyToViolate);
+    verifyViolation(policyToViolate, tn, m);
+		return tn;
+  }
 
+  private void verifyViolation(
+			SpaceViolationPolicy policyToViolate, TableName tn, Mutation m) throws Exception {
     // But let's try a few times to get the exception before failing
     boolean sawError = false;
     for (int i = 0; i < NUM_RETRIES && !sawError; i++) {
@@ -411,8 +441,6 @@ public class TestSpaceQuotas {
     }
     assertTrue(
         "Expected to see an exception writing data to a table exceeding its quota", sawError);
-
-    return tn;
   }
 
   private ClientServiceCallable<Void> generateFileToLoad(


[49/50] [abbrv] hbase git commit: HBASE-17002 JMX metrics and some UI additions for space quotas

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/96c6b8fa/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
index d56def5..4577bcf 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
@@ -13024,6 +13024,3031 @@ public final class QuotaProtos {
 
   }
 
+  public interface GetQuotaStatesRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.GetQuotaStatesRequest)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+  }
+  /**
+   * Protobuf type {@code hbase.pb.GetQuotaStatesRequest}
+   */
+  public  static final class GetQuotaStatesRequest extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.GetQuotaStatesRequest)
+      GetQuotaStatesRequestOrBuilder {
+    // Use GetQuotaStatesRequest.newBuilder() to construct.
+    private GetQuotaStatesRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private GetQuotaStatesRequest() {
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GetQuotaStatesRequest(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesRequest_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest.Builder.class);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest) obj;
+
+      boolean result = true;
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GetQuotaStatesRequest}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.GetQuotaStatesRequest)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequestOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesRequest_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest(this);
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest.getDefaultInstance()) return this;
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.GetQuotaStatesRequest)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.GetQuotaStatesRequest)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<GetQuotaStatesRequest>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<GetQuotaStatesRequest>() {
+      public GetQuotaStatesRequest parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new GetQuotaStatesRequest(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<GetQuotaStatesRequest> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<GetQuotaStatesRequest> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface GetQuotaStatesResponseOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.GetQuotaStatesResponse)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot> 
+        getTableSnapshotsList();
+    /**
+     * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot getTableSnapshots(int index);
+    /**
+     * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
+     */
+    int getTableSnapshotsCount();
+    /**
+     * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder> 
+        getTableSnapshotsOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder getTableSnapshotsOrBuilder(
+        int index);
+
+    /**
+     * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot> 
+        getNsSnapshotsList();
+    /**
+     * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot getNsSnapshots(int index);
+    /**
+     * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
+     */
+    int getNsSnapshotsCount();
+    /**
+     * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder> 
+        getNsSnapshotsOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder getNsSnapshotsOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.GetQuotaStatesResponse}
+   */
+  public  static final class GetQuotaStatesResponse extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.GetQuotaStatesResponse)
+      GetQuotaStatesResponseOrBuilder {
+    // Use GetQuotaStatesResponse.newBuilder() to construct.
+    private GetQuotaStatesResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private GetQuotaStatesResponse() {
+      tableSnapshots_ = java.util.Collections.emptyList();
+      nsSnapshots_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GetQuotaStatesResponse(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                tableSnapshots_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              tableSnapshots_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.PARSER, extensionRegistry));
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                nsSnapshots_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              nsSnapshots_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          tableSnapshots_ = java.util.Collections.unmodifiableList(tableSnapshots_);
+        }
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          nsSnapshots_ = java.util.Collections.unmodifiableList(nsSnapshots_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.Builder.class);
+    }
+
+    public interface TableQuotaSnapshotOrBuilder extends
+        // @@protoc_insertion_point(interface_extends:hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      boolean hasTableName();
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName();
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      boolean hasSnapshot();
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot getSnapshot();
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder getSnapshotOrBuilder();
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot}
+     */
+    public  static final class TableQuotaSnapshot extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+        // @@protoc_insertion_point(message_implements:hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot)
+        TableQuotaSnapshotOrBuilder {
+      // Use TableQuotaSnapshot.newBuilder() to construct.
+      private TableQuotaSnapshot(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+        super(builder);
+      }
+      private TableQuotaSnapshot() {
+      }
+
+      @java.lang.Override
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+      getUnknownFields() {
+        return this.unknownFields;
+      }
+      private TableQuotaSnapshot(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        this();
+        int mutable_bitField0_ = 0;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+                if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                  subBuilder = tableName_.toBuilder();
+                }
+                tableName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+                if (subBuilder != null) {
+                  subBuilder.mergeFrom(tableName_);
+                  tableName_ = subBuilder.buildPartial();
+                }
+                bitField0_ |= 0x00000001;
+                break;
+              }
+              case 18: {
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder subBuilder = null;
+                if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                  subBuilder = snapshot_.toBuilder();
+                }
+                snapshot_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.PARSER, extensionRegistry);
+                if (subBuilder != null) {
+                  subBuilder.mergeFrom(snapshot_);
+                  snapshot_ = subBuilder.buildPartial();
+                }
+                bitField0_ |= 0x00000002;
+                break;
+              }
+            }
+          }
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+              e).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_TableQuotaSnapshot_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_TableQuotaSnapshot_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.Builder.class);
+      }
+
+      private int bitField0_;
+      public static final int TABLE_NAME_FIELD_NUMBER = 1;
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_;
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+        return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+      }
+
+      public static final int SNAPSHOT_FIELD_NUMBER = 2;
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot snapshot_;
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      public boolean hasSnapshot() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot getSnapshot() {
+        return snapshot_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.getDefaultInstance() : snapshot_;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder getSnapshotOrBuilder() {
+        return snapshot_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.getDefaultInstance() : snapshot_;
+      }
+
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized == 1) return true;
+        if (isInitialized == 0) return false;
+
+        if (hasTableName()) {
+          if (!getTableName().isInitialized()) {
+            memoizedIsInitialized = 0;
+            return false;
+          }
+        }
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          output.writeMessage(1, getTableName());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeMessage(2, getSnapshot());
+        }
+        unknownFields.writeTo(output);
+      }
+
+      public int getSerializedSize() {
+        int size = memoizedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+            .computeMessageSize(1, getTableName());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+            .computeMessageSize(2, getSnapshot());
+        }
+        size += unknownFields.getSerializedSize();
+        memoizedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      public boolean equals(final java.lang.Object obj) {
+        if (obj == this) {
+         return true;
+        }
+        if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot)) {
+          return super.equals(obj);
+        }
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot other = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot) obj;
+
+        boolean result = true;
+        result = result && (hasTableName() == other.hasTableName());
+        if (hasTableName()) {
+          result = result && getTableName()
+              .equals(other.getTableName());
+        }
+        result = result && (hasSnapshot() == other.hasSnapshot());
+        if (hasSnapshot()) {
+          result = result && getSnapshot()
+              .equals(other.getSnapshot());
+        }
+        result = result && unknownFields.equals(other.unknownFields);
+        return result;
+      }
+
+      @java.lang.Override
+      public int hashCode() {
+        if (memoizedHashCode != 0) {
+          return memoizedHashCode;
+        }
+        int hash = 41;
+        hash = (19 * hash) + getDescriptor().hashCode();
+        if (hasTableName()) {
+          hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+          hash = (53 * hash) + getTableName().hashCode();
+        }
+        if (hasSnapshot()) {
+          hash = (37 * hash) + SNAPSHOT_FIELD_NUMBER;
+          hash = (53 * hash) + getSnapshot().hashCode();
+        }
+        hash = (29 * hash) + unknownFields.hashCode();
+        memoizedHashCode = hash;
+        return hash;
+      }
+
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot parseFrom(byte[] data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot parseFrom(
+          byte[] data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot parseFrom(
+          java.io.InputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseDelimitedWithIOException(PARSER, input);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot parseDelimitedFrom(
+          java.io.InputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input, extensionRegistry);
+      }
+
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder() {
+        return DEFAULT_INSTANCE.toBuilder();
+      }
+      public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot prototype) {
+        return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() {
+        return this == DEFAULT_INSTANCE
+            ? new Builder() : new Builder().mergeFrom(this);
+      }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+        return builder;
+      }
+      /**
+       * Protobuf type {@code hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot}
+       */
+      public static final class Builder extends
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+          // @@protoc_insertion_point(builder_implements:hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot)
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder {
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_TableQuotaSnapshot_descriptor;
+        }
+
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_TableQuotaSnapshot_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.Builder.class);
+        }
+
+        // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          super(parent);
+          maybeForceBuilderInitialization();
+        }
+        private void maybeForceBuilderInitialization() {
+          if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                  .alwaysUseFieldBuilders) {
+            getTableNameFieldBuilder();
+            getSnapshotFieldBuilder();
+          }
+        }
+        public Builder clear() {
+          super.clear();
+          if (tableNameBuilder_ == null) {
+            tableName_ = null;
+          } else {
+            tableNameBuilder_.clear();
+          }
+          bitField0_ = (bitField0_ & ~0x00000001);
+          if (snapshotBuilder_ == null) {
+            snapshot_ = null;
+          } else {
+            snapshotBuilder_.clear();
+          }
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_TableQuotaSnapshot_descriptor;
+        }
+
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot getDefaultInstanceForType() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.getDefaultInstance();
+        }
+
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot build() {
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot buildPartial() {
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot result = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          if (tableNameBuilder_ == null) {
+            result.tableName_ = tableName_;
+          } else {
+            result.tableName_ = tableNameBuilder_.build();
+          }
+          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          if (snapshotBuilder_ == null) {
+            result.snapshot_ = snapshot_;
+          } else {
+            result.snapshot_ = snapshotBuilder_.build();
+          }
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder clone() {
+          return (Builder) super.clone();
+        }
+        public Builder setField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+            Object value) {
+          return (Builder) super.setField(field, value);
+        }
+        public Builder clearField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+          return (Builder) super.clearField(field);
+        }
+        public Builder clearOneof(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          return (Builder) super.clearOneof(oneof);
+        }
+        public Builder setRepeatedField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+            int index, Object value) {
+          return (Builder) super.setRepeatedField(field, index, value);
+        }
+        public Builder addRepeatedField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+            Object value) {
+          return (Builder) super.addRepeatedField(field, value);
+        }
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+          if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot) {
+            return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot other) {
+          if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.getDefaultInstance()) return this;
+          if (other.hasTableName()) {
+            mergeTableName(other.getTableName());
+          }
+          if (other.hasSnapshot()) {
+            mergeSnapshot(other.getSnapshot());
+          }
+          this.mergeUnknownFields(other.unknownFields);
+          onChanged();
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          if (hasTableName()) {
+            if (!getTableName().isInitialized()) {
+              return false;
+            }
+          }
+          return true;
+        }
+
+        public Builder mergeFrom(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot parsedMessage = null;
+          try {
+            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+          } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+            parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot) e.getUnfinishedMessage();
+            throw e.unwrapIOException();
+          } finally {
+            if (parsedMessage != null) {
+              mergeFrom(parsedMessage);
+            }
+          }
+          return this;
+        }
+        private int bitField0_;
+
+        private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+            org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public boolean hasTableName() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+          if (tableNameBuilder_ == null) {
+            return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+          } else {
+            return tableNameBuilder_.getMessage();
+          }
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public Builder setTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+          if (tableNameBuilder_ == null) {
+            if (value == null) {
+              throw new NullPointerException();
+            }
+            tableName_ = value;
+            onChanged();
+          } else {
+            tableNameBuilder_.setMessage(value);
+          }
+          bitField0_ |= 0x00000001;
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public Builder setTableName(
+            org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+          if (tableNameBuilder_ == null) {
+            tableName_ = builderForValue.build();
+            onChanged();
+          } else {
+            tableNameBuilder_.setMessage(builderForValue.build());
+          }
+          bitField0_ |= 0x00000001;
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public Builder mergeTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+          if (tableNameBuilder_ == null) {
+            if (((bitField0_ & 0x00000001) == 0x00000001) &&
+                tableName_ != null &&
+                tableName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+              tableName_ =
+                org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+            } else {
+              tableName_ = value;
+            }
+            onChanged();
+          } else {
+            tableNameBuilder_.mergeFrom(value);
+          }
+          bitField0_ |= 0x00000001;
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public Builder clearTableName() {
+          if (tableNameBuilder_ == null) {
+            tableName_ = null;
+            onChanged();
+          } else {
+            tableNameBuilder_.clear();
+          }
+          bitField0_ = (bitField0_ & ~0x00000001);
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+          bitField0_ |= 0x00000001;
+          onChanged();
+          return getTableNameFieldBuilder().getBuilder();
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+          if (tableNameBuilder_ != null) {
+            return tableNameBuilder_.getMessageOrBuilder();
+          } else {
+            return tableName_ == null ?
+                org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+          }
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+            org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+            getTableNameFieldBuilder() {
+          if (tableNameBuilder_ == null) {
+            tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+                org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                    getTableName(),
+                    getParentForChildren(),
+                    isClean());
+            tableName_ = null;
+          }
+          return tableNameBuilder_;
+        }
+
+        private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot snapshot_ = null;
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder> snapshotBuilder_;
+        /**
+         * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+         */
+        public boolean hasSnapshot() {
+          return ((bitField0_ & 0x00000002) == 0x00000002);
+        }
+        /**
+         * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+         */
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot getSnapshot() {
+          if (snapshotBuilder_ == null) {
+            return snapshot_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.getDefaultInstance() : snapshot_;
+          } else {
+            return snapshotBuilder_.getMessage();
+          }
+        }
+        /**
+         * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+         */
+        public Builder setSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot value) {
+          if (snapshotBuilder_ == null) {
+            if (value == null) {
+              throw new NullPointerException();
+            }
+            snapshot_ = value;
+            onChanged();
+          } else {
+            snapshotBuilder_.setMessage(value);
+          }
+          bitField0_ |= 0x00000002;
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+         */
+        public Builder setSnapshot(
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder builderForValue) {
+          if (snapshotBuilder_ == null) {
+            snapshot_ = builderForValue.build();
+            onChanged();
+          } else {
+            snapshotBuilder_.setMessage(builderForValue.build());
+          }
+          bitField0_ |= 0x00000002;
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+         */
+        public Builder mergeSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot value) {
+          if (snapshotBuilder_ == null) {
+            if (((bitField0_ & 0x00000002) == 0x00000002) &&
+                snapshot_ != null &&
+                snapshot_ != org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.getDefaultInstance()) {
+              snapshot_ =
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.newBuilder(snapshot_).mergeFrom(value).buildPartial();
+            } else {
+              snapshot_ = value;
+            }
+            onChanged();
+          } else {
+            snapshotBuilder_.mergeFrom(value);
+          }
+          bitField0_ |= 0x00000002;
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+         */
+        public Builder clearSnapshot() {
+          if (snapshotBuilder_ == null) {
+            snapshot_ = null;
+            onChanged();
+          } else {
+            snapshotBuilder_.clear();
+          }
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+         */
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder getSnapshotBuilder() {
+          bitField0_ |= 0x00000002;
+          onChanged();
+          return getSnapshotFieldBuilder().getBuilder();
+        }
+        /**
+         * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+         */
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder getSnapshotOrBuilder() {
+          if (snapshotBuilder_ != null) {
+            return snapshotBuilder_.getMessageOrBuilder();
+          } else {
+            return snapshot_ == null ?
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.getDefaultInstance() : snapshot_;
+          }
+        }
+        /**
+         * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+         */
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder> 
+            getSnapshotFieldBuilder() {
+          if (snapshotBuilder_ == null) {
+            snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder>(
+                    getSnapshot(),
+                    getParentForChildren(),
+                    isClean());
+            snapshot_ = null;
+          }
+          return snapshotBuilder_;
+        }
+        public final Builder setUnknownFields(
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+          return super.setUnknownFields(unknownFields);
+        }
+
+        public final Builder mergeUnknownFields(
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+          return super.mergeUnknownFields(unknownFields);
+        }
+
+
+        // @@protoc_insertion_point(builder_scope:hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot)
+      }
+
+      // @@protoc_insertion_point(class_scope:hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot)
+      private static final org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot DEFAULT_INSTANCE;
+      static {
+        DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot();
+      }
+
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot getDefaultInstance() {
+        return DEFAULT_INSTANCE;
+      }
+
+      @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableQuotaSnapshot>
+          PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<TableQuotaSnapshot>() {
+        public TableQuotaSnapshot parsePartialFrom(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+            return new TableQuotaSnapshot(input, extensionRegistry);
+        }
+      };
+
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableQuotaSnapshot> parser() {
+        return PARSER;
+      }
+
+      @java.lang.Override
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableQuotaSnapshot> getParserForType() {
+        return PARSER;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot getDefaultInstanceForType() {
+        return DEFAULT_INSTANCE;
+      }
+
+    }
+
+    public interface NamespaceQuotaSnapshotOrBuilder extends
+        // @@protoc_insertion_point(interface_extends:hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+      /**
+       * <code>optional string namespace = 1;</code>
+       */
+      boolean hasNamespace();
+      /**
+       * <code>optional string namespace = 1;</code>
+       */
+      java.lang.String getNamespace();
+      /**
+       * <code>optional string namespace = 1;</code>
+       */
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+          getNamespaceBytes();
+
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      boolean hasSnapshot();
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot getSnapshot();
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder getSnapshotOrBuilder();
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot}
+     */
+    public  static final class NamespaceQuotaSnapshot extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+        // @@protoc_insertion_point(message_implements:hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot)
+        NamespaceQuotaSnapshotOrBuilder {
+      // Use NamespaceQuotaSnapshot.newBuilder() to construct.
+      private NamespaceQuotaSnapshot(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+        super(builder);
+      }
+      private NamespaceQuotaSnapshot() {
+        namespace_ = "";
+      }
+
+      @java.lang.Override
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+      getUnknownFields() {
+        return this.unknownFields;
+      }
+      private NamespaceQuotaSnapshot(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        this();
+        int mutable_bitField0_ = 0;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
+                bitField0_ |= 0x00000001;
+                namespace_ = bs;
+                break;
+              }
+              case 18: {
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder subBuilder = null;
+                if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                  subBuilder = snapshot_.toBuilder();
+                }
+                snapshot_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.PARSER, extensionRegistry);
+                if (subBuilder != null) {
+                  subBuilder.mergeFrom(snapshot_);
+                  snapshot_ = subBuilder.buildPartial();
+                }
+                bitField0_ |= 0x00000002;
+                break;
+              }
+            }
+          }
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+              e).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_NamespaceQuotaSnapshot_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_NamespaceQuotaSnapshot_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.Builder.class);
+      }
+
+      private int bitField0_;
+      public static final int NAMESPACE_FIELD_NUMBER = 1;
+      private volatile java.lang.Object namespace_;
+      /**
+       * <code>optional string namespace = 1;</code>
+       */
+      public boolean hasNamespace() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional string namespace = 1;</code>
+       */
+      public java.lang.String getNamespace() {
+        java.lang.Object ref = namespace_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            namespace_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>optional string namespace = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+          getNamespaceBytes() {
+        java.lang.Object ref = namespace_;
+        if (ref instanceof java.lang.String) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          namespace_ = b;
+          return b;
+        } else {
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      public static final int SNAPSHOT_FIELD_NUMBER = 2;
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot snapshot_;
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      public boolean hasSnapshot() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot getSnapshot() {
+        return snapshot_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.getDefaultInstance() : snapshot_;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder getSnapshotOrBuilder() {
+        return snapshot_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.getDefaultInstance() : snapshot_;
+      }
+
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized == 1) return true;
+        if (isInitialized == 0) return false;
+
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespace_);
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeMessage(2, getSnapshot());
+        }
+        unknownFields.writeTo(output);
+      }
+
+      public int getSerializedSize() {
+        int size = memoizedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespace_);
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+            .computeMessageSize(2, getSnapshot());
+        }
+        size += unknownFields.getSerializedSize();
+        memoizedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      public boolean equals(final java.lang.Object obj) {
+        if (obj == this) {
+         return true;
+        }
+        if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot)) {
+          return super.equals(obj);
+        }
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot other = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot) obj;
+
+        boolean result = true;
+        result = result && (hasNamespace() == other.hasNamespace());
+        if (hasNamespace()) {
+          result = result && getNamespace()
+              .equals(other.getNamespace());
+        }
+        result = result && (hasSnapshot() == other.hasSnapshot());
+        if (hasSnapshot()) {
+          result = result && getSnapshot()
+              .equals(other.getSnapshot());
+        }
+        result = result && unknownFields.equals(other.unknownFields);
+        return result;
+      }
+
+      @java.lang.Override
+      public int hashCode() {
+        if (memoizedHashCode != 0) {
+          return memoizedHashCode;
+        }
+        int hash = 41;
+        hash = (19 * hash) + getDescriptor().hashCode();
+        if (hasNamespace()) {
+          hash = (37 * hash) + NAMESPACE_FIELD_NUMBER;
+          hash = (53 * hash) + getNamespace().hashCode();
+        }
+        if (hasSnapshot()) {
+          hash = (37 * hash) + SNAPSHOT_FIELD_NUMBER;
+          hash = (53 * hash) + getSnapshot().hashCode();
+        }
+        hash = (29 * hash) + unknownFields.hashCode();
+        memoizedHashCode = hash;
+        return hash;
+      }
+
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot parseFrom(byte[] data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot parseFrom(
+          byte[] data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot parseFrom(
+          java.io.InputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseDelimitedWithIOException(PARSER, input);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot parseDelimitedFrom(
+          java.io.InputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input, extensionRegistry);
+      }
+
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder() {
+        return DEFAULT_INSTANCE.toBuilder();
+      }
+      public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot prototype) {
+        return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() {
+        return this == DEFAULT_INSTANCE
+            ? new Builder() : new Builder().mergeFrom(this);
+      }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+        return builder;
+      }
+      /**
+       * Protobuf type {@code hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot}
+       */
+      public static final class Builder extends
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+          // @@protoc_insertion_point(builder_implements:hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot)
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder {
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_NamespaceQuotaSnapshot_descriptor;
+        }
+
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_NamespaceQuotaSnapshot_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.Builder.class);
+        }
+
+        // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          super(parent);
+          maybeForceBuilderInitialization();
+        }
+        private void maybeForceBuilderInitialization() {
+          if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                  .alwaysUseFieldBuilders) {
+            getSnapshotFieldBuilder();
+          }
+        }
+        public Builder clear() {
+          super.clear();
+          namespace_ = "";
+          bitField0_ = (bitField0_ & ~0x00000001);
+          if (snapshotBuilder_ == null) {
+            snapshot_ = null;
+          } else {
+            snapshotBuilder_.clear();
+          }
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetQuotaStatesResponse_NamespaceQuotaSnapshot_descriptor;
+        }
+
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot getDefaultInstanceForType() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.getDefaultInstance();
+        }
+
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot build() {
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot buildPartial() {
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot result = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          result.namespace_ = namespace_;
+          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          if (snapshotBuilder_ == null) {
+            result.snapshot_ = snapshot_;
+          } else {
+            result.snapshot_ = snapshotBuilder_.build();
+          }
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder clone() {
+          return (Builder) super.clone();
+        }
+        public Builder setField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+            Object value) {
+          return (Builder) super.setField(field, value);
+        }
+        public Builder clearField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+          return (Builder) super.clearField(field);
+        }
+        public Builder clearOneof(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          return (Builder) super.clearOneof(oneof);
+        }
+        public Builder setRepeatedField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+            int index, Object value) {
+          return (Builder) super.setRepeatedField(field, index, value);
+        }
+        public Builder addRepeatedField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+            Object value) {
+          return (Builder) super.addRepeatedField(field, value);
+        }
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+          if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot) {
+            return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot other) {
+          if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.getDefaultInstance()) return this;
+          if (other.hasNamespace()) {
+            bitField0_ |= 0x00000001;
+            namespace_ = other.namespace_;
+            onChanged();
+          }
+          if (other.hasSnapshot()) {
+            mergeSnapshot(other.getSnapshot());
+          }
+          this.mergeUnknownFields(other.unknownFields);
+          onChanged();
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          return true;
+        }
+
+        public Builder mergeFrom(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot parsedMessage = null;
+          try {
+        

<TRUNCATED>

[07/50] [abbrv] hbase git commit: HBASE-17866: Implement async setQuota/getQuota methods

Posted by el...@apache.org.
HBASE-17866: Implement async setQuota/getQuota methods

Signed-off-by: Guanghao Zhang <zg...@apache.org>


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

Branch: refs/heads/HBASE-16961
Commit: 8db9760363890d4d0bfaba25ae6797d45aaf7fec
Parents: 7678855
Author: huzheng <op...@gmail.com>
Authored: Fri Apr 14 14:51:38 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Mon Apr 17 09:49:30 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  16 ++
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |  47 +++++
 .../hadoop/hbase/quotas/QuotaRetriever.java     |  32 +--
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |  32 +++
 .../hbase/client/TestAsyncQuotaAdminApi.java    | 207 +++++++++++++++++++
 5 files changed, 306 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8db97603/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index ab791c2..270f28f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.regex.Pattern;
 
@@ -27,6 +28,8 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.quotas.QuotaFilter;
+import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.util.Pair;
 
 /**
@@ -465,4 +468,17 @@ public interface AsyncAdmin {
    *          startcode. Here is an example: <code> host187.example.com,60020,1289493121758</code>
    */
   CompletableFuture<Void> move(final byte[] regionName, final byte[] destServerName);
+
+  /**
+   * Apply the new quota settings.
+   * @param quota the quota settings
+   */
+  CompletableFuture<Void> setQuota(final QuotaSettings quota);
+
+  /**
+   * List the quotas based on the filter.
+   * @param filter the quota settings filter
+   * @return the QuotaSetting list, which wrapped by a CompletableFuture.
+   */
+  CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8db97603/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index e42ee57..180cd19 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -56,6 +56,9 @@ import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterReques
 import org.apache.hadoop.hbase.client.Scan.ReadType;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.quotas.QuotaFilter;
+import org.apache.hadoop.hbase.quotas.QuotaSettings;
+import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -112,6 +115,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineReg
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
@@ -1149,6 +1154,48 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
     return future;
   }
 
+  @Override
+  public CompletableFuture<Void> setQuota(QuotaSettings quota){
+    return this.<Void> newMasterCaller()
+        .action((controller, stub) -> this.<SetQuotaRequest, SetQuotaResponse, Void> call(
+          controller, stub, QuotaSettings.buildSetQuotaRequestProto(quota),
+          (s, c, req, done) -> s.setQuota(c, req, done), (resp) -> null))
+        .call();
+  }
+
+  @Override
+  public CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter) {
+    CompletableFuture<List<QuotaSettings>> future = new CompletableFuture<>();
+    Scan scan = QuotaTableUtil.makeScan(filter);
+    this.connection.getRawTableBuilder(QuotaTableUtil.QUOTA_TABLE_NAME).build().scan(scan,
+      new RawScanResultConsumer() {
+        List<QuotaSettings> settings = new ArrayList<>();
+
+        @Override
+        public void onNext(Result[] results, ScanController controller) {
+          for (Result result : results) {
+            try {
+              QuotaTableUtil.parseResultToCollection(result, settings);
+            } catch (IOException e) {
+              controller.terminate();
+              future.completeExceptionally(e);
+            }
+          }
+        }
+
+        @Override
+        public void onError(Throwable error) {
+          future.completeExceptionally(error);
+        }
+
+        @Override
+        public void onComplete() {
+          future.complete(settings);
+        }
+      });
+    return future;
+  }
+
   private byte[][] getSplitKeys(byte[] startKey, byte[] endKey, int numRegions) {
     if (numRegions < 3) {
       throw new IllegalArgumentException("Must create at least three regions");

http://git-wip-us.apache.org/repos/asf/hbase/blob/8db97603/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
index cba6a24..0f7baa5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
@@ -86,34 +86,10 @@ public class QuotaRetriever implements Closeable, Iterable<QuotaSettings> {
   public QuotaSettings next() throws IOException {
     if (cache.isEmpty()) {
       Result result = scanner.next();
-      if (result == null) return null;
-
-      QuotaTableUtil.parseResult(result, new QuotaTableUtil.QuotasVisitor() {
-        @Override
-        public void visitUserQuotas(String userName, Quotas quotas) {
-          cache.addAll(QuotaSettingsFactory.fromUserQuotas(userName, quotas));
-        }
-
-        @Override
-        public void visitUserQuotas(String userName, TableName table, Quotas quotas) {
-          cache.addAll(QuotaSettingsFactory.fromUserQuotas(userName, table, quotas));
-        }
-
-        @Override
-        public void visitUserQuotas(String userName, String namespace, Quotas quotas) {
-          cache.addAll(QuotaSettingsFactory.fromUserQuotas(userName, namespace, quotas));
-        }
-
-        @Override
-        public void visitTableQuotas(TableName tableName, Quotas quotas) {
-          cache.addAll(QuotaSettingsFactory.fromTableQuotas(tableName, quotas));
-        }
-
-        @Override
-        public void visitNamespaceQuotas(String namespace, Quotas quotas) {
-          cache.addAll(QuotaSettingsFactory.fromNamespaceQuotas(namespace, quotas));
-        }
-      });
+      if (result == null) {
+        return null;
+      }
+      QuotaTableUtil.parseResultToCollection(result, cache);
     }
     return cache.poll();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8db97603/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 116dd0c..c44090f 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
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.quotas;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.regex.Pattern;
@@ -236,6 +237,37 @@ public class QuotaTableUtil {
     }
   }
 
+  public static void parseResultToCollection(final Result result,
+      Collection<QuotaSettings> quotaSettings) throws IOException {
+
+    QuotaTableUtil.parseResult(result, new QuotaTableUtil.QuotasVisitor() {
+      @Override
+      public void visitUserQuotas(String userName, Quotas quotas) {
+        quotaSettings.addAll(QuotaSettingsFactory.fromUserQuotas(userName, quotas));
+      }
+
+      @Override
+      public void visitUserQuotas(String userName, TableName table, Quotas quotas) {
+        quotaSettings.addAll(QuotaSettingsFactory.fromUserQuotas(userName, table, quotas));
+      }
+
+      @Override
+      public void visitUserQuotas(String userName, String namespace, Quotas quotas) {
+        quotaSettings.addAll(QuotaSettingsFactory.fromUserQuotas(userName, namespace, quotas));
+      }
+
+      @Override
+      public void visitTableQuotas(TableName tableName, Quotas quotas) {
+        quotaSettings.addAll(QuotaSettingsFactory.fromTableQuotas(tableName, quotas));
+      }
+
+      @Override
+      public void visitNamespaceQuotas(String namespace, Quotas quotas) {
+        quotaSettings.addAll(QuotaSettingsFactory.fromNamespaceQuotas(namespace, quotas));
+      }
+    });
+  }
+
   public static void parseNamespaceResult(final Result result,
       final NamespaceQuotasVisitor visitor) throws IOException {
     String namespace = getNamespaceFromRowKey(result.getRow());

http://git-wip-us.apache.org/repos/asf/hbase/blob/8db97603/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java
new file mode 100644
index 0000000..ac9bc16
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java
@@ -0,0 +1,207 @@
+/**
+ * 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.client;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.quotas.QuotaCache;
+import org.apache.hadoop.hbase.quotas.QuotaFilter;
+import org.apache.hadoop.hbase.quotas.QuotaSettings;
+import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory;
+import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
+import org.apache.hadoop.hbase.quotas.QuotaUtil;
+import org.apache.hadoop.hbase.quotas.ThrottleType;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+@Category({ ClientTests.class, MediumTests.class })
+public class TestAsyncQuotaAdminApi {
+  private static final Log LOG = LogFactory.getLog(TestAsyncQuotaAdminApi.class);
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static AsyncConnection ASYNC_CONN;
+  protected AsyncAdmin admin;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    TEST_UTIL.getConfiguration().setInt(QuotaCache.REFRESH_CONF_KEY, 2000);
+    TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
+    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
+    TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
+    TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
+    TEST_UTIL.startMiniCluster(1);
+    TEST_UTIL.waitTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME);
+    ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    IOUtils.closeQuietly(ASYNC_CONN);
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    this.admin = ASYNC_CONN.getAdmin();
+  }
+
+  @Test
+  public void testThrottleType() throws Exception {
+    String userName = User.getCurrent().getShortName();
+
+    admin.setQuota(
+      QuotaSettingsFactory.throttleUser(userName, ThrottleType.READ_NUMBER, 6, TimeUnit.MINUTES))
+        .get();
+    admin.setQuota(
+      QuotaSettingsFactory.throttleUser(userName, ThrottleType.WRITE_NUMBER, 12, TimeUnit.MINUTES))
+        .get();
+    admin.setQuota(QuotaSettingsFactory.bypassGlobals(userName, true)).get();
+
+    int countThrottle = 0;
+    int countGlobalBypass = 0;
+    for (QuotaSettings settings : admin.getQuota(null).get()) {
+      switch (settings.getQuotaType()) {
+      case THROTTLE:
+        countThrottle++;
+        break;
+      case GLOBAL_BYPASS:
+        countGlobalBypass++;
+        break;
+      default:
+        fail("unexpected settings type: " + settings.getQuotaType());
+      }
+    }
+    assertEquals(2, countThrottle);
+    assertEquals(1, countGlobalBypass);
+
+    admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName)).get();
+    assertNumResults(1, null);
+    admin.setQuota(QuotaSettingsFactory.bypassGlobals(userName, false)).get();
+    assertNumResults(0, null);
+  }
+
+  @Test
+  public void testQuotaRetrieverFilter() throws Exception {
+    TableName[] tables = new TableName[] { TableName.valueOf("T0"), TableName.valueOf("T01"),
+        TableName.valueOf("NS0:T2"), };
+    String[] namespaces = new String[] { "NS0", "NS01", "NS2" };
+    String[] users = new String[] { "User0", "User01", "User2" };
+
+    for (String user : users) {
+      admin.setQuota(
+        QuotaSettingsFactory.throttleUser(user, ThrottleType.REQUEST_NUMBER, 1, TimeUnit.MINUTES))
+          .get();
+
+      for (TableName table : tables) {
+        admin.setQuota(QuotaSettingsFactory.throttleUser(user, table, ThrottleType.REQUEST_NUMBER,
+          2, TimeUnit.MINUTES)).get();
+      }
+
+      for (String ns : namespaces) {
+        admin.setQuota(QuotaSettingsFactory.throttleUser(user, ns, ThrottleType.REQUEST_NUMBER, 3,
+          TimeUnit.MINUTES)).get();
+      }
+    }
+    assertNumResults(21, null);
+
+    for (TableName table : tables) {
+      admin.setQuota(
+        QuotaSettingsFactory.throttleTable(table, ThrottleType.REQUEST_NUMBER, 4, TimeUnit.MINUTES))
+          .get();
+    }
+    assertNumResults(24, null);
+
+    for (String ns : namespaces) {
+      admin.setQuota(QuotaSettingsFactory.throttleNamespace(ns, ThrottleType.REQUEST_NUMBER, 5,
+        TimeUnit.MINUTES)).get();
+    }
+    assertNumResults(27, null);
+
+    assertNumResults(7, new QuotaFilter().setUserFilter("User0"));
+    assertNumResults(0, new QuotaFilter().setUserFilter("User"));
+    assertNumResults(21, new QuotaFilter().setUserFilter("User.*"));
+    assertNumResults(3, new QuotaFilter().setUserFilter("User.*").setTableFilter("T0"));
+    assertNumResults(3, new QuotaFilter().setUserFilter("User.*").setTableFilter("NS.*"));
+    assertNumResults(0, new QuotaFilter().setUserFilter("User.*").setTableFilter("T"));
+    assertNumResults(6, new QuotaFilter().setUserFilter("User.*").setTableFilter("T.*"));
+    assertNumResults(3, new QuotaFilter().setUserFilter("User.*").setNamespaceFilter("NS0"));
+    assertNumResults(0, new QuotaFilter().setUserFilter("User.*").setNamespaceFilter("NS"));
+    assertNumResults(9, new QuotaFilter().setUserFilter("User.*").setNamespaceFilter("NS.*"));
+    assertNumResults(6,
+      new QuotaFilter().setUserFilter("User.*").setTableFilter("T0").setNamespaceFilter("NS0"));
+    assertNumResults(1, new QuotaFilter().setTableFilter("T0"));
+    assertNumResults(0, new QuotaFilter().setTableFilter("T"));
+    assertNumResults(2, new QuotaFilter().setTableFilter("T.*"));
+    assertNumResults(3, new QuotaFilter().setTableFilter(".*T.*"));
+    assertNumResults(1, new QuotaFilter().setNamespaceFilter("NS0"));
+    assertNumResults(0, new QuotaFilter().setNamespaceFilter("NS"));
+    assertNumResults(3, new QuotaFilter().setNamespaceFilter("NS.*"));
+
+    for (String user : users) {
+      admin.setQuota(QuotaSettingsFactory.unthrottleUser(user)).get();
+      for (TableName table : tables) {
+        admin.setQuota(QuotaSettingsFactory.unthrottleUser(user, table)).get();
+      }
+      for (String ns : namespaces) {
+        admin.setQuota(QuotaSettingsFactory.unthrottleUser(user, ns)).get();
+      }
+    }
+    assertNumResults(6, null);
+
+    for (TableName table : tables) {
+      admin.setQuota(QuotaSettingsFactory.unthrottleTable(table)).get();
+    }
+    assertNumResults(3, null);
+
+    for (String ns : namespaces) {
+      admin.setQuota(QuotaSettingsFactory.unthrottleNamespace(ns)).get();
+    }
+    assertNumResults(0, null);
+  }
+
+  private void assertNumResults(int expected, final QuotaFilter filter) throws Exception {
+    assertEquals(expected, countResults(filter));
+  }
+
+  private int countResults(final QuotaFilter filter) throws Exception {
+    int count = 0;
+    for (QuotaSettings settings : admin.getQuota(filter).get()) {
+      LOG.debug(settings);
+      count++;
+    }
+    return count;
+  }
+}


[15/50] [abbrv] hbase git commit: Revert "HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)"

Posted by el...@apache.org.
Revert "HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)"

This reverts commit c2c2178b2eebe4439eadec6b37fae2566944c16b.


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

Branch: refs/heads/HBASE-16961
Commit: ecdfb82326035ad8221940919bbeb3fe16ec2658
Parents: c2c2178
Author: Ramkrishna <ra...@intel.com>
Authored: Tue Apr 18 00:00:12 2017 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Tue Apr 18 00:00:12 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/CellUtil.java  |  24 ++
 .../org/apache/hadoop/hbase/ExtendedCell.java   |  10 -
 .../org/apache/hadoop/hbase/master/HMaster.java |   2 -
 .../hbase/regionserver/ByteBufferChunkCell.java |  48 ---
 .../apache/hadoop/hbase/regionserver/Chunk.java |  60 +--
 .../hadoop/hbase/regionserver/ChunkCreator.java | 404 -------------------
 .../hbase/regionserver/HRegionServer.java       |  14 +-
 .../hbase/regionserver/MemStoreChunkPool.java   | 265 ++++++++++++
 .../hadoop/hbase/regionserver/MemStoreLAB.java  |   4 +-
 .../hbase/regionserver/MemStoreLABImpl.java     | 171 ++++----
 .../regionserver/NoTagByteBufferChunkCell.java  |  48 ---
 .../hadoop/hbase/regionserver/OffheapChunk.java |  31 +-
 .../hadoop/hbase/regionserver/OnheapChunk.java  |  32 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |   3 -
 .../coprocessor/TestCoprocessorInterface.java   |   4 -
 .../TestRegionObserverScannerOpenHook.java      |   3 -
 .../coprocessor/TestRegionObserverStacking.java |   3 -
 .../io/hfile/TestScannerFromBucketCache.java    |   3 -
 .../hadoop/hbase/master/TestCatalogJanitor.java |   7 -
 .../hadoop/hbase/regionserver/TestBulkLoad.java |   2 +-
 .../hbase/regionserver/TestCellFlatSet.java     |   2 +-
 .../regionserver/TestCompactingMemStore.java    |  37 +-
 .../TestCompactingToCellArrayMapMemStore.java   |  16 +-
 .../TestCompactionArchiveConcurrentClose.java   |   1 -
 .../TestCompactionArchiveIOException.java       |   1 -
 .../regionserver/TestCompactionPolicy.java      |   1 -
 .../hbase/regionserver/TestDefaultMemStore.java |  14 +-
 .../regionserver/TestFailedAppendAndSync.java   |   1 -
 .../hbase/regionserver/TestHMobStore.java       |   2 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |   2 -
 .../regionserver/TestHRegionReplayEvents.java   |   2 +-
 .../regionserver/TestMemStoreChunkPool.java     |  48 +--
 .../hbase/regionserver/TestMemStoreLAB.java     |  27 +-
 .../TestMemstoreLABWithoutPool.java             | 168 --------
 .../hbase/regionserver/TestRecoveredEdits.java  |   1 -
 .../hbase/regionserver/TestRegionIncrement.java |   1 -
 .../hadoop/hbase/regionserver/TestStore.java    |   1 -
 .../TestStoreFileRefresherChore.java            |   1 -
 .../hbase/regionserver/TestWALLockup.java       |   1 -
 .../TestWALMonotonicallyIncreasingSeqId.java    |   1 -
 .../hbase/regionserver/wal/TestDurability.java  |   3 -
 41 files changed, 479 insertions(+), 990 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 56de21b..e1bc969 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -3135,4 +3135,28 @@ public final class CellUtil {
       return Type.DeleteFamily.getCode();
     }
   }
+
+  /**
+   * Clone the passed cell by copying its data into the passed buf.
+   */
+  public static Cell copyCellTo(Cell cell, ByteBuffer buf, int offset, int len) {
+    int tagsLen = cell.getTagsLength();
+    if (cell instanceof ExtendedCell) {
+      ((ExtendedCell) cell).write(buf, offset);
+    } else {
+      // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the
+      // other case also. The data fragments within Cell is copied into buf as in KeyValue
+      // serialization format only.
+      KeyValueUtil.appendTo(cell, buf, offset, true);
+    }
+    if (tagsLen == 0) {
+      // When tagsLen is 0, make a NoTagsByteBufferKeyValue version. This is an optimized class
+      // which directly return tagsLen as 0. So we avoid parsing many length components in
+      // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
+      // call getTagsLength().
+      return new NoTagsByteBufferKeyValue(buf, offset, len, cell.getSequenceId());
+    } else {
+      return new ByteBufferKeyValue(buf, offset, len, cell.getSequenceId());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
index 10f20ca..517873f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.io.HeapSize;
 public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestamp, HeapSize,
     Cloneable {
 
-  public static int CELL_NOT_BASED_ON_CHUNK = -1;
   /**
    * Write this cell to an OutputStream in a {@link KeyValue} format.
    * <br> KeyValue format <br>
@@ -74,13 +73,4 @@ public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestam
    * @return The deep cloned cell
    */
   Cell deepClone();
-
-  /**
-   * Extracts the id of the backing bytebuffer of this cell if it was obtained from fixed sized
-   * chunks as in case of MemstoreLAB
-   * @return the chunk id if the cell is backed by fixed sized Chunks, else return -1
-   */
-  default int getChunkId() {
-    return CELL_NOT_BASED_ON_CHUNK;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/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 f9670e1..bb9f282 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
@@ -748,8 +748,6 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     this.masterActiveTime = System.currentTimeMillis();
     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
-    // Initialize the chunkCreator
-    initializeMemStoreChunkCreator();
     this.fileSystemManager = new MasterFileSystem(this);
     this.walManager = new MasterWalManager(this);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
deleted file mode 100644
index a8f1000..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.ByteBufferKeyValue;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
-
-/**
- * ByteBuffer based cell which has the chunkid at the 0th offset
- * @see MemStoreLAB
- */
-//TODO : When moving this cell to CellChunkMap we will have the following things
-// to be serialized
-// chunkId (Integer) + offset (Integer) + length (Integer) + seqId (Long) = 20 bytes
-@InterfaceAudience.Private
-public class ByteBufferChunkCell extends ByteBufferKeyValue {
-  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
-    super(buf, offset, length);
-  }
-
-  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
-    super(buf, offset, length, seqId);
-  }
-
-  @Override
-  public int getChunkId() {
-    // The chunkId is embedded at the 0th offset of the bytebuffer
-    return ByteBufferUtils.toInt(buf, 0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
index fc4aa0b..2cbf0a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
@@ -21,10 +21,8 @@ import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 
 /**
  * A chunk of memory out of which allocations are sliced.
@@ -48,41 +46,13 @@ public abstract class Chunk {
   /** Size of chunk in bytes */
   protected final int size;
 
-  // The unique id associated with the chunk.
-  private final int id;
-
-  // indicates if the chunk is formed by ChunkCreator#MemstorePool
-  private final boolean fromPool;
-
-  /**
-   * Create an uninitialized chunk. Note that memory is not allocated yet, so
-   * this is cheap.
-   * @param size in bytes
-   * @param id the chunk id
-   */
-  public Chunk(int size, int id) {
-    this(size, id, false);
-  }
-
   /**
-   * Create an uninitialized chunk. Note that memory is not allocated yet, so
-   * this is cheap.
+   * Create an uninitialized chunk. Note that memory is not allocated yet, so this is cheap.
+   *
    * @param size in bytes
-   * @param id the chunk id
-   * @param fromPool if the chunk is formed by pool
    */
-  public Chunk(int size, int id, boolean fromPool) {
+  Chunk(int size) {
     this.size = size;
-    this.id = id;
-    this.fromPool = fromPool;
-  }
-
-  int getId() {
-    return this.id;
-  }
-
-  boolean isFromPool() {
-    return this.fromPool;
   }
 
   /**
@@ -90,24 +60,7 @@ public abstract class Chunk {
    * constructed the chunk. It is thread-safe against other threads calling alloc(), who will block
    * until the allocation is complete.
    */
-  public void init() {
-    assert nextFreeOffset.get() == UNINITIALIZED;
-    try {
-      allocateDataBuffer();
-    } catch (OutOfMemoryError e) {
-      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
-      assert failInit; // should be true.
-      throw e;
-    }
-    // Mark that it's ready for use
-    // Move 8 bytes since the first 8 bytes are having the chunkid in it
-    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, Bytes.SIZEOF_LONG);
-    // We should always succeed the above CAS since only one thread
-    // calls init()!
-    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
-  }
-
-  abstract void allocateDataBuffer();
+  public abstract void init();
 
   /**
    * Reset the offset to UNINITIALIZED before before reusing an old chunk
@@ -121,8 +74,7 @@ public abstract class Chunk {
 
   /**
    * Try to allocate <code>size</code> bytes from the chunk.
-   * If a chunk is tried to get allocated before init() call, the thread doing the allocation
-   * will be in busy-wait state as it will keep looping till the nextFreeOffset is set.
+   *
    * @return the offset of the successful allocation, or -1 to indicate not-enough-space
    */
   public int alloc(int size) {
@@ -144,7 +96,7 @@ public abstract class Chunk {
       if (oldOffset + size > data.capacity()) {
         return -1; // alloc doesn't fit
       }
-      // TODO : If seqID is to be written add 8 bytes here for nextFreeOFfset
+
       // Try to atomically claim this chunk
       if (nextFreeOffset.compareAndSet(oldOffset, oldOffset + size)) {
         // we got the alloc

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
deleted file mode 100644
index 073fb25..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
+++ /dev/null
@@ -1,404 +0,0 @@
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.lang.ref.SoftReference;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-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.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * Does the management of memstoreLAB chunk creations. A monotonically incrementing id is associated
- * with every chunk
- */
-@InterfaceAudience.Private
-public class ChunkCreator {
-  private final Log LOG = LogFactory.getLog(ChunkCreator.class);
-  // monotonically increasing chunkid
-  private AtomicInteger chunkID = new AtomicInteger(1);
-  // maps the chunk against the monotonically increasing chunk id. We need to preserve the
-  // natural ordering of the key
-  // CellChunkMap creation should convert the soft ref to hard reference
-  private Map<Integer, SoftReference<Chunk>> chunkIdMap =
-      new ConcurrentHashMap<Integer, SoftReference<Chunk>>();
-  private final int chunkSize;
-  private final boolean offheap;
-  @VisibleForTesting
-  static ChunkCreator INSTANCE;
-  @VisibleForTesting
-  static boolean chunkPoolDisabled = false;
-  private MemStoreChunkPool pool;
-
-  @VisibleForTesting
-  ChunkCreator(int chunkSize, boolean offheap, long globalMemStoreSize, float poolSizePercentage,
-      float initialCountPercentage, HeapMemoryManager heapMemoryManager) {
-    this.chunkSize = chunkSize;
-    this.offheap = offheap;
-    this.pool = initializePool(globalMemStoreSize, poolSizePercentage, initialCountPercentage);
-    if (heapMemoryManager != null && this.pool != null) {
-      // Register with Heap Memory manager
-      heapMemoryManager.registerTuneObserver(this.pool);
-    }
-  }
-
-  /**
-   * Initializes the instance of MSLABChunkCreator
-   * @param chunkSize the chunkSize
-   * @param offheap indicates if the chunk is to be created offheap or not
-   * @param globalMemStoreSize  the global memstore size
-   * @param poolSizePercentage pool size percentage
-   * @param initialCountPercentage the initial count of the chunk pool if any
-   * @param heapMemoryManager the heapmemory manager
-   * @return singleton MSLABChunkCreator
-   */
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
-      justification = "Method is called by single thread at the starting of RS")
-  @VisibleForTesting
-  public static ChunkCreator initialize(int chunkSize, boolean offheap, long globalMemStoreSize,
-      float poolSizePercentage, float initialCountPercentage, HeapMemoryManager heapMemoryManager) {
-    if (INSTANCE != null) return INSTANCE;
-    INSTANCE = new ChunkCreator(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
-        initialCountPercentage, heapMemoryManager);
-    return INSTANCE;
-  }
-
-  static ChunkCreator getInstance() {
-    return INSTANCE;
-  }
-
-  /**
-   * Creates and inits a chunk.
-   * @return the chunk that was initialized
-   */
-  Chunk getChunk() {
-    Chunk chunk = null;
-    if (pool != null) {
-      //  the pool creates the chunk internally. The chunk#init() call happens here
-      chunk = this.pool.getChunk();
-      // the pool has run out of maxCount
-      if (chunk == null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("The chunk pool is full. Reached maxCount= " + this.pool.getMaxCount()
-              + ". Creating chunk onheap.");
-        }
-      }
-    }
-    if (chunk == null) {
-      chunk = createChunk();
-    }
-    // put this chunk into the chunkIdMap
-    this.chunkIdMap.put(chunk.getId(), new SoftReference<>(chunk));
-    // now we need to actually do the expensive memory allocation step in case of a new chunk,
-    // else only the offset is set to the beginning of the chunk to accept allocations
-    chunk.init();
-    return chunk;
-  }
-
-  private Chunk createChunk() {
-    return createChunk(false);
-  }
-
-  /**
-   * Creates the chunk either onheap or offheap
-   * @param pool indicates if the chunks have to be created which will be used by the Pool
-   * @return the chunk
-   */
-  private Chunk createChunk(boolean pool) {
-    int id = chunkID.getAndIncrement();
-    assert id > 0;
-    // do not create offheap chunk on demand
-    if (pool && this.offheap) {
-      return new OffheapChunk(chunkSize, id, pool);
-    } else {
-      return new OnheapChunk(chunkSize, id, pool);
-    }
-  }
-
-  @VisibleForTesting
-  // TODO : To be used by CellChunkMap
-  Chunk getChunk(int id) {
-    SoftReference<Chunk> ref = chunkIdMap.get(id);
-    if (ref != null) {
-      return ref.get();
-    }
-    return null;
-  }
-
-  int getChunkSize() {
-    return this.chunkSize;
-  }
-
-  boolean isOffheap() {
-    return this.offheap;
-  }
-
-  private void removeChunks(Set<Integer> chunkIDs) {
-    this.chunkIdMap.keySet().removeAll(chunkIDs);
-  }
-
-  Chunk removeChunk(int chunkId) {
-    SoftReference<Chunk> ref = this.chunkIdMap.remove(chunkId);
-    if (ref != null) {
-      return ref.get();
-    }
-    return null;
-  }
-
-  @VisibleForTesting
-  int size() {
-    return this.chunkIdMap.size();
-  }
-
-  @VisibleForTesting
-  void clearChunkIds() {
-    this.chunkIdMap.clear();
-  }
-
-  /**
-   * A pool of {@link Chunk} instances.
-   *
-   * MemStoreChunkPool caches a number of retired chunks for reusing, it could
-   * decrease allocating bytes when writing, thereby optimizing the garbage
-   * collection on JVM.
-   */
-  private  class MemStoreChunkPool implements HeapMemoryTuneObserver {
-    private int maxCount;
-
-    // A queue of reclaimed chunks
-    private final BlockingQueue<Chunk> reclaimedChunks;
-    private final float poolSizePercentage;
-
-    /** Statistics thread schedule pool */
-    private final ScheduledExecutorService scheduleThreadPool;
-    /** Statistics thread */
-    private static final int statThreadPeriod = 60 * 5;
-    private final AtomicLong chunkCount = new AtomicLong();
-    private final AtomicLong reusedChunkCount = new AtomicLong();
-
-    MemStoreChunkPool(int maxCount, int initialCount, float poolSizePercentage) {
-      this.maxCount = maxCount;
-      this.poolSizePercentage = poolSizePercentage;
-      this.reclaimedChunks = new LinkedBlockingQueue<>();
-      for (int i = 0; i < initialCount; i++) {
-        Chunk chunk = createChunk(true);
-        chunk.init();
-        reclaimedChunks.add(chunk);
-      }
-      chunkCount.set(initialCount);
-      final String n = Thread.currentThread().getName();
-      scheduleThreadPool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
-          .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
-      this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(), statThreadPeriod,
-          statThreadPeriod, TimeUnit.SECONDS);
-    }
-
-    /**
-     * Poll a chunk from the pool, reset it if not null, else create a new chunk to return if we have
-     * not yet created max allowed chunks count. When we have already created max allowed chunks and
-     * no free chunks as of now, return null. It is the responsibility of the caller to make a chunk
-     * then.
-     * Note: Chunks returned by this pool must be put back to the pool after its use.
-     * @return a chunk
-     * @see #putbackChunks(Set)
-     */
-    Chunk getChunk() {
-      Chunk chunk = reclaimedChunks.poll();
-      if (chunk != null) {
-        chunk.reset();
-        reusedChunkCount.incrementAndGet();
-      } else {
-        // Make a chunk iff we have not yet created the maxCount chunks
-        while (true) {
-          long created = this.chunkCount.get();
-          if (created < this.maxCount) {
-            if (this.chunkCount.compareAndSet(created, created + 1)) {
-              chunk = createChunk(true);
-              break;
-            }
-          } else {
-            break;
-          }
-        }
-      }
-      return chunk;
-    }
-
-    /**
-     * Add the chunks to the pool, when the pool achieves the max size, it will skip the remaining
-     * chunks
-     * @param chunks
-     */
-    private void putbackChunks(Set<Integer> chunks) {
-      int toAdd = Math.min(chunks.size(), this.maxCount - reclaimedChunks.size());
-      Iterator<Integer> iterator = chunks.iterator();
-      while (iterator.hasNext()) {
-        Integer chunkId = iterator.next();
-        // remove the chunks every time though they are from the pool or not
-        Chunk chunk = ChunkCreator.this.removeChunk(chunkId);
-        if (chunk != null) {
-          if (chunk.isFromPool() && toAdd > 0) {
-            reclaimedChunks.add(chunk);
-          }
-          toAdd--;
-        }
-      }
-    }
-
-    private class StatisticsThread extends Thread {
-      StatisticsThread() {
-        super("MemStoreChunkPool.StatisticsThread");
-        setDaemon(true);
-      }
-
-      @Override
-      public void run() {
-        logStats();
-      }
-
-      private void logStats() {
-        if (!LOG.isDebugEnabled()) return;
-        long created = chunkCount.get();
-        long reused = reusedChunkCount.get();
-        long total = created + reused;
-        LOG.debug("Stats: current pool size=" + reclaimedChunks.size()
-            + ",created chunk count=" + created
-            + ",reused chunk count=" + reused
-            + ",reuseRatio=" + (total == 0 ? "0" : StringUtils.formatPercent(
-                (float) reused / (float) total, 2)));
-      }
-    }
-
-    private int getMaxCount() {
-      return this.maxCount;
-    }
-
-    @Override
-    public void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize) {
-      // don't do any tuning in case of offheap memstore
-      if (isOffheap()) {
-        LOG.warn("Not tuning the chunk pool as it is offheap");
-        return;
-      }
-      int newMaxCount =
-          (int) (newMemstoreSize * poolSizePercentage / getChunkSize());
-      if (newMaxCount != this.maxCount) {
-        // We need an adjustment in the chunks numbers
-        if (newMaxCount > this.maxCount) {
-          // Max chunks getting increased. Just change the variable. Later calls to getChunk() would
-          // create and add them to Q
-          LOG.info("Max count for chunks increased from " + this.maxCount + " to " + newMaxCount);
-          this.maxCount = newMaxCount;
-        } else {
-          // Max chunks getting decreased. We may need to clear off some of the pooled chunks now
-          // itself. If the extra chunks are serving already, do not pool those when we get them back
-          LOG.info("Max count for chunks decreased from " + this.maxCount + " to " + newMaxCount);
-          this.maxCount = newMaxCount;
-          if (this.reclaimedChunks.size() > newMaxCount) {
-            synchronized (this) {
-              while (this.reclaimedChunks.size() > newMaxCount) {
-                this.reclaimedChunks.poll();
-              }
-            }
-          }
-        }
-      }
-    }
-  }
-
-  @VisibleForTesting
-  static void clearDisableFlag() {
-    chunkPoolDisabled = false;
-  }
-
-  private MemStoreChunkPool initializePool(long globalMemStoreSize, float poolSizePercentage,
-      float initialCountPercentage) {
-    if (poolSizePercentage <= 0) {
-      LOG.info("PoolSizePercentage is less than 0. So not using pool");
-      return null;
-    }
-    if (chunkPoolDisabled) {
-      return null;
-    }
-    if (poolSizePercentage > 1.0) {
-      throw new IllegalArgumentException(
-          MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
-    }
-    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / getChunkSize());
-    if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
-      throw new IllegalArgumentException(
-          MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
-    }
-    int initialCount = (int) (initialCountPercentage * maxCount);
-    LOG.info("Allocating MemStoreChunkPool with chunk size "
-        + StringUtils.byteDesc(getChunkSize()) + ", max count " + maxCount
-        + ", initial count " + initialCount);
-    return new MemStoreChunkPool(maxCount, initialCount, poolSizePercentage);
-  }
-
-  @VisibleForTesting
-  int getMaxCount() {
-    if (pool != null) {
-      return pool.getMaxCount();
-    }
-    return 0;
-  }
-
-  @VisibleForTesting
-  int getPoolSize() {
-    if (pool != null) {
-      return pool.reclaimedChunks.size();
-    }
-    return 0;
-  }
-
-  /*
-   * Only used in testing
-   */
-  @VisibleForTesting
-  void clearChunksInPool() {
-    if (pool != null) {
-      pool.reclaimedChunks.clear();
-    }
-  }
-
-  synchronized void putbackChunks(Set<Integer> chunks) {
-    if (pool != null) {
-      pool.putbackChunks(chunks);
-    } else {
-      this.removeChunks(chunks);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 41eb0a3..b3b5113 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1472,7 +1472,7 @@ public class HRegionServer extends HasThread implements
       startServiceThreads();
       startHeapMemoryManager();
       // Call it after starting HeapMemoryManager.
-      initializeMemStoreChunkCreator();
+      initializeMemStoreChunkPool();
       LOG.info("Serving as " + this.serverName +
         ", RpcServer on " + rpcServices.isa +
         ", sessionid=0x" +
@@ -1492,7 +1492,7 @@ public class HRegionServer extends HasThread implements
     }
   }
 
-  protected void initializeMemStoreChunkCreator() {
+  private void initializeMemStoreChunkPool() {
     if (MemStoreLAB.isEnabled(conf)) {
       // MSLAB is enabled. So initialize MemStoreChunkPool
       // By this time, the MemstoreFlusher is already initialized. We can get the global limits from
@@ -1506,10 +1506,12 @@ public class HRegionServer extends HasThread implements
       float initialCountPercentage = conf.getFloat(MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY,
           MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT);
       int chunkSize = conf.getInt(MemStoreLAB.CHUNK_SIZE_KEY, MemStoreLAB.CHUNK_SIZE_DEFAULT);
-      // init the chunkCreator
-      ChunkCreator chunkCreator =
-          ChunkCreator.initialize(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
-            initialCountPercentage, this.hMemManager);
+      MemStoreChunkPool pool = MemStoreChunkPool.initialize(globalMemStoreSize, poolSizePercentage,
+          initialCountPercentage, chunkSize, offheap);
+      if (pool != null && this.hMemManager != null) {
+        // Register with Heap Memory manager
+        this.hMemManager.registerTuneObserver(pool);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
new file mode 100644
index 0000000..b7ac212
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
@@ -0,0 +1,265 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.regionserver;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * A pool of {@link Chunk} instances.
+ * 
+ * MemStoreChunkPool caches a number of retired chunks for reusing, it could
+ * decrease allocating bytes when writing, thereby optimizing the garbage
+ * collection on JVM.
+ * 
+ * The pool instance is globally unique and could be obtained through
+ * {@link MemStoreChunkPool#initialize(long, float, float, int, boolean)}
+ * 
+ * {@link MemStoreChunkPool#getChunk()} is called when MemStoreLAB allocating
+ * bytes, and {@link MemStoreChunkPool#putbackChunks(BlockingQueue)} is called
+ * when MemStore clearing snapshot for flush
+ */
+@SuppressWarnings("javadoc")
+@InterfaceAudience.Private
+public class MemStoreChunkPool implements HeapMemoryTuneObserver {
+  private static final Log LOG = LogFactory.getLog(MemStoreChunkPool.class);
+
+  // Static reference to the MemStoreChunkPool
+  static MemStoreChunkPool GLOBAL_INSTANCE;
+  /** Boolean whether we have disabled the memstore chunk pool entirely. */
+  static boolean chunkPoolDisabled = false;
+
+  private int maxCount;
+
+  // A queue of reclaimed chunks
+  private final BlockingQueue<Chunk> reclaimedChunks;
+  private final int chunkSize;
+  private final float poolSizePercentage;
+
+  /** Statistics thread schedule pool */
+  private final ScheduledExecutorService scheduleThreadPool;
+  /** Statistics thread */
+  private static final int statThreadPeriod = 60 * 5;
+  private final AtomicLong chunkCount = new AtomicLong();
+  private final AtomicLong reusedChunkCount = new AtomicLong();
+  private final boolean offheap;
+
+  MemStoreChunkPool(int chunkSize, int maxCount, int initialCount, float poolSizePercentage,
+      boolean offheap) {
+    this.maxCount = maxCount;
+    this.chunkSize = chunkSize;
+    this.poolSizePercentage = poolSizePercentage;
+    this.offheap = offheap;
+    this.reclaimedChunks = new LinkedBlockingQueue<>();
+    for (int i = 0; i < initialCount; i++) {
+      Chunk chunk = this.offheap ? new OffheapChunk(chunkSize) : new OnheapChunk(chunkSize);
+      chunk.init();
+      reclaimedChunks.add(chunk);
+    }
+    chunkCount.set(initialCount);
+    final String n = Thread.currentThread().getName();
+    scheduleThreadPool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
+        .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
+    this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(), statThreadPeriod,
+        statThreadPeriod, TimeUnit.SECONDS);
+  }
+
+  /**
+   * Poll a chunk from the pool, reset it if not null, else create a new chunk to return if we have
+   * not yet created max allowed chunks count. When we have already created max allowed chunks and
+   * no free chunks as of now, return null. It is the responsibility of the caller to make a chunk
+   * then.
+   * Note: Chunks returned by this pool must be put back to the pool after its use.
+   * @return a chunk
+   * @see #putbackChunk(Chunk)
+   * @see #putbackChunks(BlockingQueue)
+   */
+  Chunk getChunk() {
+    Chunk chunk = reclaimedChunks.poll();
+    if (chunk != null) {
+      chunk.reset();
+      reusedChunkCount.incrementAndGet();
+    } else {
+      // Make a chunk iff we have not yet created the maxCount chunks
+      while (true) {
+        long created = this.chunkCount.get();
+        if (created < this.maxCount) {
+          chunk = this.offheap ? new OffheapChunk(this.chunkSize) : new OnheapChunk(this.chunkSize);
+          if (this.chunkCount.compareAndSet(created, created + 1)) {
+            break;
+          }
+        } else {
+          break;
+        }
+      }
+    }
+    return chunk;
+  }
+
+  /**
+   * Add the chunks to the pool, when the pool achieves the max size, it will
+   * skip the remaining chunks
+   * @param chunks
+   */
+  synchronized void putbackChunks(BlockingQueue<Chunk> chunks) {
+    int toAdd = Math.min(chunks.size(), this.maxCount - reclaimedChunks.size());
+    Chunk chunk = null;
+    while ((chunk = chunks.poll()) != null && toAdd > 0) {
+      reclaimedChunks.add(chunk);
+      toAdd--;
+    }
+  }
+
+  /**
+   * Add the chunk to the pool, if the pool has achieved the max size, it will
+   * skip it
+   * @param chunk
+   */
+  synchronized void putbackChunk(Chunk chunk) {
+    if (reclaimedChunks.size() < this.maxCount) {
+      reclaimedChunks.add(chunk);
+    }
+  }
+
+  int getPoolSize() {
+    return this.reclaimedChunks.size();
+  }
+
+  /*
+   * Only used in testing
+   */
+  void clearChunks() {
+    this.reclaimedChunks.clear();
+  }
+
+  private class StatisticsThread extends Thread {
+    StatisticsThread() {
+      super("MemStoreChunkPool.StatisticsThread");
+      setDaemon(true);
+    }
+
+    @Override
+    public void run() {
+      logStats();
+    }
+
+    private void logStats() {
+      if (!LOG.isDebugEnabled()) return;
+      long created = chunkCount.get();
+      long reused = reusedChunkCount.get();
+      long total = created + reused;
+      LOG.debug("Stats: current pool size=" + reclaimedChunks.size()
+          + ",created chunk count=" + created
+          + ",reused chunk count=" + reused
+          + ",reuseRatio=" + (total == 0 ? "0" : StringUtils.formatPercent(
+              (float) reused / (float) total, 2)));
+    }
+  }
+
+  /**
+   * @return the global MemStoreChunkPool instance
+   */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
+      justification = "Method is called by single thread at the starting of RS")
+  static MemStoreChunkPool initialize(long globalMemStoreSize, float poolSizePercentage,
+      float initialCountPercentage, int chunkSize, boolean offheap) {
+    if (GLOBAL_INSTANCE != null) return GLOBAL_INSTANCE;
+    if (chunkPoolDisabled) return null;
+
+    if (poolSizePercentage <= 0) {
+      chunkPoolDisabled = true;
+      return null;
+    }
+    if (poolSizePercentage > 1.0) {
+      throw new IllegalArgumentException(
+          MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
+    }
+    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / chunkSize);
+    if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
+      throw new IllegalArgumentException(
+          MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
+    }
+    int initialCount = (int) (initialCountPercentage * maxCount);
+    LOG.info("Allocating MemStoreChunkPool with chunk size " + StringUtils.byteDesc(chunkSize)
+        + ", max count " + maxCount + ", initial count " + initialCount);
+    GLOBAL_INSTANCE = new MemStoreChunkPool(chunkSize, maxCount, initialCount, poolSizePercentage,
+        offheap);
+    return GLOBAL_INSTANCE;
+  }
+
+  /**
+   * @return The singleton instance of this pool.
+   */
+  static MemStoreChunkPool getPool() {
+    return GLOBAL_INSTANCE;
+  }
+
+  int getMaxCount() {
+    return this.maxCount;
+  }
+
+  @VisibleForTesting
+  static void clearDisableFlag() {
+    chunkPoolDisabled = false;
+  }
+
+  @Override
+  public void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize) {
+    // don't do any tuning in case of offheap memstore
+    if (this.offheap) {
+      LOG.warn("Not tuning the chunk pool as it is offheap");
+      return;
+    }
+    int newMaxCount = (int) (newMemstoreSize * poolSizePercentage / chunkSize);
+    if (newMaxCount != this.maxCount) {
+      // We need an adjustment in the chunks numbers
+      if (newMaxCount > this.maxCount) {
+        // Max chunks getting increased. Just change the variable. Later calls to getChunk() would
+        // create and add them to Q
+        LOG.info("Max count for chunks increased from " + this.maxCount + " to " + newMaxCount);
+        this.maxCount = newMaxCount;
+      } else {
+        // Max chunks getting decreased. We may need to clear off some of the pooled chunks now
+        // itself. If the extra chunks are serving already, do not pool those when we get them back
+        LOG.info("Max count for chunks decreased from " + this.maxCount + " to " + newMaxCount);
+        this.maxCount = newMaxCount;
+        if (this.reclaimedChunks.size() > newMaxCount) {
+          synchronized (this) {
+            while (this.reclaimedChunks.size() > newMaxCount) {
+              this.reclaimedChunks.poll();
+            }
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
index 72e937c..f6d1607 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
  * <p>
  * The MemStoreLAB is basically a bump-the-pointer allocator that allocates big (2MB) chunks from
  * and then doles it out to threads that request slices into the array. These chunks can get pooled
- * as well. See {@link ChunkCreator}.
+ * as well. See {@link MemStoreChunkPool}.
  * <p>
  * The purpose of this is to combat heap fragmentation in the regionserver. By ensuring that all
  * Cells in a given memstore refer only to large chunks of contiguous memory, we ensure that
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
  * {@link #copyCellInto(Cell)} gets called. This allocates enough size in the chunk to hold this
  * cell's data and copies into this area and then recreate a Cell over this copied data.
  * <p>
- * @see ChunkCreator
+ * @see MemStoreChunkPool
  */
 @InterfaceAudience.Private
 public interface MemStoreLAB {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
index 4fba82d..4e87135 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
@@ -18,26 +18,23 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.nio.ByteBuffer;
-import java.util.Set;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantLock;
 
 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.ExtendedCell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+
 /**
  * A memstore-local allocation buffer.
  * <p>
@@ -58,8 +55,8 @@ import com.google.common.base.Preconditions;
  * would provide a performance improvement - probably would speed up the
  * Bytes.toLong/Bytes.toInt calls in KeyValue, but some of those are cached
  * anyway.
- * The chunks created by this MemStoreLAB can get pooled at {@link ChunkCreator}.
- * When the Chunk comes from pool, it can be either an on heap or an off heap backed chunk. The chunks,
+ * The chunks created by this MemStoreLAB can get pooled at {@link MemStoreChunkPool}.
+ * When the Chunk comes pool, it can be either an on heap or an off heap backed chunk. The chunks,
  * which this MemStoreLAB creates on its own (when no chunk available from pool), those will be
  * always on heap backed.
  */
@@ -69,15 +66,14 @@ public class MemStoreLABImpl implements MemStoreLAB {
   static final Log LOG = LogFactory.getLog(MemStoreLABImpl.class);
 
   private AtomicReference<Chunk> curChunk = new AtomicReference<>();
-  // Lock to manage multiple handlers requesting for a chunk
-  private ReentrantLock lock = new ReentrantLock();
-
-  // A set of chunks contained by this memstore LAB
+  // A queue of chunks from pool contained by this memstore LAB
+  // TODO: in the future, it would be better to have List implementation instead of Queue,
+  // as FIFO order is not so important here
   @VisibleForTesting
-  Set<Integer> chunks = new ConcurrentSkipListSet<Integer>();
+  BlockingQueue<Chunk> pooledChunkQueue = null;
   private final int chunkSize;
   private final int maxAlloc;
-  private final ChunkCreator chunkCreator;
+  private final MemStoreChunkPool chunkPool;
 
   // This flag is for closing this instance, its set when clearing snapshot of
   // memstore
@@ -96,12 +92,20 @@ public class MemStoreLABImpl implements MemStoreLAB {
   public MemStoreLABImpl(Configuration conf) {
     chunkSize = conf.getInt(CHUNK_SIZE_KEY, CHUNK_SIZE_DEFAULT);
     maxAlloc = conf.getInt(MAX_ALLOC_KEY, MAX_ALLOC_DEFAULT);
-    this.chunkCreator = ChunkCreator.getInstance();
+    this.chunkPool = MemStoreChunkPool.getPool();
+    // currently chunkQueue is only used for chunkPool
+    if (this.chunkPool != null) {
+      // set queue length to chunk pool max count to avoid keeping reference of
+      // too many non-reclaimable chunks
+      pooledChunkQueue = new LinkedBlockingQueue<>(chunkPool.getMaxCount());
+    }
+
     // if we don't exclude allocations >CHUNK_SIZE, we'd infiniteloop on one!
     Preconditions.checkArgument(maxAlloc <= chunkSize,
         MAX_ALLOC_KEY + " must be less than " + CHUNK_SIZE_KEY);
   }
 
+
   @Override
   public Cell copyCellInto(Cell cell) {
     int size = KeyValueUtil.length(cell);
@@ -114,52 +118,19 @@ public class MemStoreLABImpl implements MemStoreLAB {
     Chunk c = null;
     int allocOffset = 0;
     while (true) {
-      // Try to get the chunk
       c = getOrMakeChunk();
-      // we may get null because the some other thread succeeded in getting the lock
-      // and so the current thread has to try again to make its chunk or grab the chunk
-      // that the other thread created
       // Try to allocate from this chunk
-      if (c != null) {
-        allocOffset = c.alloc(size);
-        if (allocOffset != -1) {
-          // We succeeded - this is the common case - small alloc
-          // from a big buffer
-          break;
-        }
-        // not enough space!
-        // try to retire this chunk
-        tryRetireChunk(c);
+      allocOffset = c.alloc(size);
+      if (allocOffset != -1) {
+        // We succeeded - this is the common case - small alloc
+        // from a big buffer
+        break;
       }
+      // not enough space!
+      // try to retire this chunk
+      tryRetireChunk(c);
     }
-    return copyToChunkCell(cell, c.getData(), allocOffset, size);
-  }
-
-  /**
-   * Clone the passed cell by copying its data into the passed buf and create a cell with a chunkid
-   * out of it
-   */
-  private Cell copyToChunkCell(Cell cell, ByteBuffer buf, int offset, int len) {
-    int tagsLen = cell.getTagsLength();
-    if (cell instanceof ExtendedCell) {
-      ((ExtendedCell) cell).write(buf, offset);
-    } else {
-      // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the
-      // other case also. The data fragments within Cell is copied into buf as in KeyValue
-      // serialization format only.
-      KeyValueUtil.appendTo(cell, buf, offset, true);
-    }
-    // TODO : write the seqid here. For writing seqId we should create a new cell type so
-    // that seqId is not used as the state
-    if (tagsLen == 0) {
-      // When tagsLen is 0, make a NoTagsByteBufferKeyValue version. This is an optimized class
-      // which directly return tagsLen as 0. So we avoid parsing many length components in
-      // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
-      // call getTagsLength().
-      return new NoTagByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
-    } else {
-      return new ByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
-    }
+    return CellUtil.copyCellTo(cell, c.getData(), allocOffset, size);
   }
 
   /**
@@ -171,9 +142,9 @@ public class MemStoreLABImpl implements MemStoreLAB {
     this.closed = true;
     // We could put back the chunks to pool for reusing only when there is no
     // opening scanner which will read their data
-    int count  = openScannerCount.get();
-    if(count == 0) {
-      recycleChunks();
+    if (chunkPool != null && openScannerCount.get() == 0
+        && reclaimed.compareAndSet(false, true)) {
+      chunkPool.putbackChunks(this.pooledChunkQueue);
     }
   }
 
@@ -191,14 +162,9 @@ public class MemStoreLABImpl implements MemStoreLAB {
   @Override
   public void decScannerCount() {
     int count = this.openScannerCount.decrementAndGet();
-    if (this.closed && count == 0) {
-      recycleChunks();
-    }
-  }
-
-  private void recycleChunks() {
-    if (reclaimed.compareAndSet(false, true)) {
-      chunkCreator.putbackChunks(chunks);
+    if (this.closed && chunkPool != null && count == 0
+        && reclaimed.compareAndSet(false, true)) {
+      chunkPool.putbackChunks(this.pooledChunkQueue);
     }
   }
 
@@ -224,33 +190,45 @@ public class MemStoreLABImpl implements MemStoreLAB {
    * allocate a new one from the JVM.
    */
   private Chunk getOrMakeChunk() {
-    // Try to get the chunk
-    Chunk c = curChunk.get();
-    if (c != null) {
-      return c;
-    }
-    // No current chunk, so we want to allocate one. We race
-    // against other allocators to CAS in an uninitialized chunk
-    // (which is cheap to allocate)
-    if (lock.tryLock()) {
-      try {
-        // once again check inside the lock
-        c = curChunk.get();
-        if (c != null) {
-          return c;
-        }
-        c = this.chunkCreator.getChunk();
-        if (c != null) {
-          // set the curChunk. No need of CAS as only one thread will be here
-          curChunk.set(c);
-          chunks.add(c.getId());
-          return c;
+    while (true) {
+      // Try to get the chunk
+      Chunk c = curChunk.get();
+      if (c != null) {
+        return c;
+      }
+
+      // No current chunk, so we want to allocate one. We race
+      // against other allocators to CAS in an uninitialized chunk
+      // (which is cheap to allocate)
+      if (chunkPool != null) {
+        c = chunkPool.getChunk();
+      }
+      boolean pooledChunk = false;
+      if (c != null) {
+        // This is chunk from pool
+        pooledChunk = true;
+      } else {
+        c = new OnheapChunk(chunkSize);// When chunk is not from pool, always make it as on heap.
+      }
+      if (curChunk.compareAndSet(null, c)) {
+        // we won race - now we need to actually do the expensive
+        // allocation step
+        c.init();
+        if (pooledChunk) {
+          if (!this.closed && !this.pooledChunkQueue.offer(c)) {
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Chunk queue is full, won't reuse this new chunk. Current queue size: "
+                  + pooledChunkQueue.size());
+            }
+          }
         }
-      } finally {
-        lock.unlock();
+        return c;
+      } else if (pooledChunk) {
+        chunkPool.putbackChunk(c);
       }
+      // someone else won race - that's fine, we'll try to grab theirs
+      // in the next iteration of the loop.
     }
-    return null;
   }
 
   @VisibleForTesting
@@ -258,15 +236,8 @@ public class MemStoreLABImpl implements MemStoreLAB {
     return this.curChunk.get();
   }
 
-  @VisibleForTesting
+
   BlockingQueue<Chunk> getPooledChunks() {
-    BlockingQueue<Chunk> pooledChunks = new LinkedBlockingQueue<>();
-    for (Integer id : this.chunks) {
-      Chunk chunk = chunkCreator.getChunk(id);
-      if (chunk != null && chunk.isFromPool()) {
-        pooledChunks.add(chunk);
-      }
-    }
-    return pooledChunks;
+    return this.pooledChunkQueue;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
deleted file mode 100644
index a8ba50c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
-
-
-/**
- * ByteBuffer based cell which has the chunkid at the 0th offset and with no tags
- * @see MemStoreLAB
- */
-@InterfaceAudience.Private
-public class NoTagByteBufferChunkCell extends NoTagsByteBufferKeyValue {
-
-  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
-    super(buf, offset, length);
-  }
-
-  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
-    super(buf, offset, length, seqId);
-  }
-
-  @Override
-  public int getChunkId() {
-    // The chunkId is embedded at the 0th offset of the bytebuffer
-    return ByteBufferUtils.toInt(buf, 0);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
index e244a33..ed98cfa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
@@ -21,27 +21,34 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
+import com.google.common.base.Preconditions;
+
 /**
  * An off heap chunk implementation.
  */
 @InterfaceAudience.Private
 public class OffheapChunk extends Chunk {
 
-  OffheapChunk(int size, int id) {
-    // better if this is always created fromPool. This should not be called
-    super(size, id);
-  }
-
-  OffheapChunk(int size, int id, boolean fromPool) {
-    super(size, id, fromPool);
-    assert fromPool == true;
+  OffheapChunk(int size) {
+    super(size);
   }
 
   @Override
-  void allocateDataBuffer() {
-    if (data == null) {
-      data = ByteBuffer.allocateDirect(this.size);
-      data.putLong(0, this.getId());
+  public void init() {
+    assert nextFreeOffset.get() == UNINITIALIZED;
+    try {
+      if (data == null) {
+        data = ByteBuffer.allocateDirect(this.size);
+      }
+    } catch (OutOfMemoryError e) {
+      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
+      assert failInit; // should be true.
+      throw e;
     }
+    // Mark that it's ready for use
+    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
+    // We should always succeed the above CAS since only one thread
+    // calls init()!
+    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
index da34e24..bd33cb5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
@@ -21,25 +21,33 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
+import com.google.common.base.Preconditions;
+
 /**
  * An on heap chunk implementation.
  */
 @InterfaceAudience.Private
 public class OnheapChunk extends Chunk {
 
-  OnheapChunk(int size, int id) {
-    super(size, id);
-  }
-
-  OnheapChunk(int size, int id, boolean fromPool) {
-    super(size, id, fromPool);
+  OnheapChunk(int size) {
+    super(size);
   }
 
-  @Override
-  void allocateDataBuffer() {
-    if (data == null) {
-      data = ByteBuffer.allocate(this.size);
-      data.putLong(0, this.getId());
+  public void init() {
+    assert nextFreeOffset.get() == UNINITIALIZED;
+    try {
+      if (data == null) {
+        data = ByteBuffer.allocate(this.size);
+      }
+    } catch (OutOfMemoryError e) {
+      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
+      assert failInit; // should be true.
+      throw e;
     }
+    // Mark that it's ready for use
+    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
+    // We should always succeed the above CAS since only one thread
+    // calls init()!
+    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 6563122..82c2eab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -96,8 +96,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -2428,7 +2426,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
       final Configuration conf, final HTableDescriptor htd, boolean initialize)
       throws IOException {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     WAL wal = createWal(conf, rootDir, info);
     return HRegion.createHRegion(info, rootDir, conf, htd, wal, initialize);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
index 8d8b6df..422c54b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
@@ -27,7 +27,6 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -50,10 +49,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
@@ -400,7 +397,6 @@ public class TestCoprocessorInterface {
     for(byte [] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(tableName, null, null, false);
     Path path = new Path(DIR + callingMethod);
     Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
index b99087d..80d0e3a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
@@ -47,12 +47,10 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -154,7 +152,6 @@ public class TestRegionObserverScannerOpenHook {
     for (byte[] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     Path path = new Path(DIR + callingMethod);
     WAL wal = HBaseTestingUtility.createWal(conf, path, info);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
index 15d449d..2e44dee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
@@ -34,9 +34,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -102,7 +100,6 @@ public class TestRegionObserverStacking extends TestCase {
     for(byte [] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     Path path = new Path(DIR + callingMethod);
     HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
index fae7247..f1775d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
@@ -40,10 +40,8 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -344,7 +342,6 @@ public class TestScannerFromBucketCache {
   private static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
       String callingMethod, Configuration conf, HBaseTestingUtility test_util, boolean isReadOnly,
       byte[]... families) throws IOException {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     Path logDir = test_util.getDataTestDirOnTestFS(callingMethod + ".log");
     HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
     final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 32bce26..cc73d9d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -65,8 +65,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActi
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
 import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -75,7 +73,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.Triple;
-import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -91,10 +88,6 @@ public class TestCatalogJanitor {
   @Rule
   public TestName name = new TestName();
 
-  @BeforeClass
-  public static void setup() throws Exception {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
-  }
   /**
    * Mock MasterServices for tests below.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
index 096c5ef..418aadf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
@@ -241,7 +241,7 @@ public class TestBulkLoad {
     for (byte[] family : families) {
       hTableDescriptor.addFamily(new HColumnDescriptor(family));
     }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+
     // TODO We need a way to do this without creating files
     return HRegion.createHRegion(hRegionInfo,
         new Path(testFolder.newFolder().toURI()),

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
index 09877b0..3b4d068 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
@@ -73,7 +73,7 @@ public class TestCellFlatSet extends TestCase {
     descCbOnHeap = new CellArrayMap(CellComparator.COMPARATOR,descCells,0,NUM_OF_CELLS,true);
     CONF.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
     CONF.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
-    ChunkCreator.chunkPoolDisabled = false;
+    MemStoreChunkPool.chunkPoolDisabled = false;
   }
 
   /* Create and test CellSet based on CellArrayMap */

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
index 9e90f3e..a888c45 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -51,7 +50,7 @@ import static org.junit.Assert.assertTrue;
 public class TestCompactingMemStore extends TestDefaultMemStore {
 
   private static final Log LOG = LogFactory.getLog(TestCompactingMemStore.class);
-  protected static ChunkCreator chunkCreator;
+  protected static MemStoreChunkPool chunkPool;
   protected HRegion region;
   protected RegionServicesForStores regionServicesForStores;
   protected HStore store;
@@ -66,7 +65,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
 
   @After
   public void tearDown() throws Exception {
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
   }
 
   @Override
@@ -85,21 +84,15 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000);
     HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf);
     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("foobar"));
-    htd.addFamily(hcd);
-    HRegionInfo info =
-        new HRegionInfo(TableName.valueOf("foobar"), null, null, false);
-    WAL wal = hbaseUtility.createWal(conf, hbaseUtility.getDataTestDir(), info);
-    this.region = HRegion.createHRegion(info, hbaseUtility.getDataTestDir(), conf, htd, wal, true);
-    //this.region = hbaseUtility.createTestRegion("foobar", hcd);
+    this.region = hbaseUtility.createTestRegion("foobar", hcd);
     this.regionServicesForStores = region.getRegionServicesForStores();
     this.store = new HStore(region, hcd, conf);
 
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    chunkCreator = ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
-      globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
-    assertTrue(chunkCreator != null);
+    chunkPool = MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f,
+        MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
+    assertTrue(chunkPool != null);
   }
 
   /**
@@ -397,7 +390,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkCreator.getPoolSize();
+    int chunkCount = chunkPool.getPoolSize();
     assertTrue(chunkCount > 0);
 
   }
@@ -441,16 +434,16 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkCreator.getPoolSize() == 0);
+    assertTrue(chunkPool.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
 
     // clear chunks
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
 
     // Creating another snapshot
 
@@ -471,7 +464,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
   }
 
   @Test
@@ -523,16 +516,16 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     memstore.add(new KeyValue(row, fam, qf1, 3, val), null);
     assertEquals(3, memstore.getActive().getCellsCount());
 
-    assertTrue(chunkCreator.getPoolSize() == 0);
+    assertTrue(chunkPool.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
 
     // clear chunks
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
 
     // Creating another snapshot
 
@@ -560,7 +553,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
   }
 
   //////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
index 66e107a..5a48455 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
@@ -44,13 +44,17 @@ import java.util.List;
 public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore {
 
   private static final Log LOG = LogFactory.getLog(TestCompactingToCellArrayMapMemStore.class);
+  //private static MemStoreChunkPool chunkPool;
+  //private HRegion region;
+  //private RegionServicesForStores regionServicesForStores;
+  //private HStore store;
 
   //////////////////////////////////////////////////////////////////////////////
   // Helpers
   //////////////////////////////////////////////////////////////////////////////
 
   @Override public void tearDown() throws Exception {
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
   }
 
   @Override public void setUp() throws Exception {
@@ -404,16 +408,16 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkCreator.getPoolSize() == 0);
+    assertTrue(chunkPool.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
 
     // clear chunks
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
 
     // Creating another snapshot
 
@@ -434,7 +438,7 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
   }
 
   @Test
@@ -468,7 +472,7 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkCreator.getPoolSize();
+    int chunkCount = chunkPool.getPoolSize();
     assertTrue(chunkCount > 0);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
index e320368..8e85730 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
@@ -164,7 +164,6 @@ public class TestCompactionArchiveConcurrentClose {
 
     HRegionFileSystem fs = new WaitingHRegionFileSystem(conf, tableDir.getFileSystem(conf),
         tableDir, info);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + info.getEncodedName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
index e7fcf18..89b2368 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
@@ -174,7 +174,6 @@ public class TestCompactionArchiveIOException {
   private HRegion initHRegion(HTableDescriptor htd, HRegionInfo info)
       throws IOException {
     Configuration conf = testUtil.getConfiguration();
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
     Path regionDir = new Path(tableDir, info.getEncodedName());
     Path storeDir = new Path(regionDir, htd.getColumnFamilies()[0].getNameAsString());

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
index bff5bec..7154511 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
@@ -104,7 +104,6 @@ public class TestCompactionPolicy {
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
 
     hlog = new FSHLog(fs, basedir, logName, conf);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     region = HRegion.createHRegion(info, basedir, conf, htd, hlog);
     region.close();
     Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index 41b304b..7434eb1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.wal.WALFactory;
-import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -85,7 +84,6 @@ public class TestDefaultMemStore {
   protected static final byte[] FAMILY = Bytes.toBytes("column");
   protected MultiVersionConcurrencyControl mvcc;
   protected AtomicLong startSeqNum = new AtomicLong(0);
-  protected ChunkCreator chunkCreator;
 
   private String getName() {
     return this.name.getMethodName();
@@ -94,17 +92,9 @@ public class TestDefaultMemStore {
   @Before
   public void setUp() throws Exception {
     internalSetUp();
-    // no pool
-    this.chunkCreator =
-        ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     this.memstore = new DefaultMemStore();
   }
 
-  @AfterClass
-  public static void tearDownClass() throws Exception {
-    ChunkCreator.getInstance().clearChunkIds();
-  }
-
   protected void internalSetUp() throws Exception {
     this.mvcc = new MultiVersionConcurrencyControl();
   }
@@ -139,9 +129,7 @@ public class TestDefaultMemStore {
       assertEquals(Segment.getCellLength(kv), sizeChangeForSecondCell.getDataSize());
       // make sure chunk size increased even when writing the same cell, if using MSLAB
       if (msLab instanceof MemStoreLABImpl) {
-        // since we add the chunkID at the 0th offset of the chunk and the
-        // chunkid is a long we need to account for those 8 bytes
-        assertEquals(2 * Segment.getCellLength(kv) + Bytes.SIZEOF_LONG,
+        assertEquals(2 * Segment.getCellLength(kv),
           ((MemStoreLABImpl) msLab).getCurrentChunk().getNextFreeOffset());
       }
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
index 24e850d..73fb9cf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
@@ -266,7 +266,6 @@ public class TestFailedAppendAndSync {
    */
   public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
   throws IOException {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.SYNC_WAL,
       wal, COLUMN_FAMILY_BYTES);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index 0f24a24..b416c7d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -153,7 +153,7 @@ public class TestHMobStore {
 
     htd.addFamily(hcd);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);
     final WALFactory wals = new WALFactory(walConf, null, methodName);


[20/50] [abbrv] hbase git commit: HBASE-17000 Implement computation of online region sizes and report to the Master

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dea6764/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
index d7d4db0..e90c934 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
@@ -10164,6 +10164,1912 @@ public final class RegionServerStatusProtos {
 
   }
 
+  public interface RegionSpaceUseOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.RegionSpaceUse)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <pre>
+     * A region identifier
+     * </pre>
+     *
+     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     */
+    boolean hasRegion();
+    /**
+     * <pre>
+     * A region identifier
+     * </pre>
+     *
+     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegion();
+    /**
+     * <pre>
+     * A region identifier
+     * </pre>
+     *
+     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionOrBuilder();
+
+    /**
+     * <pre>
+     * The size in bytes of the region
+     * </pre>
+     *
+     * <code>optional uint64 size = 2;</code>
+     */
+    boolean hasSize();
+    /**
+     * <pre>
+     * The size in bytes of the region
+     * </pre>
+     *
+     * <code>optional uint64 size = 2;</code>
+     */
+    long getSize();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.RegionSpaceUse}
+   */
+  public  static final class RegionSpaceUse extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.RegionSpaceUse)
+      RegionSpaceUseOrBuilder {
+    // Use RegionSpaceUse.newBuilder() to construct.
+    private RegionSpaceUse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private RegionSpaceUse() {
+      size_ = 0L;
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private RegionSpaceUse(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = region_.toBuilder();
+              }
+              region_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(region_);
+                region_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              size_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUse_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int REGION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo region_;
+    /**
+     * <pre>
+     * A region identifier
+     * </pre>
+     *
+     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     */
+    public boolean hasRegion() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <pre>
+     * A region identifier
+     * </pre>
+     *
+     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegion() {
+      return region_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : region_;
+    }
+    /**
+     * <pre>
+     * A region identifier
+     * </pre>
+     *
+     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionOrBuilder() {
+      return region_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : region_;
+    }
+
+    public static final int SIZE_FIELD_NUMBER = 2;
+    private long size_;
+    /**
+     * <pre>
+     * The size in bytes of the region
+     * </pre>
+     *
+     * <code>optional uint64 size = 2;</code>
+     */
+    public boolean hasSize() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <pre>
+     * The size in bytes of the region
+     * </pre>
+     *
+     * <code>optional uint64 size = 2;</code>
+     */
+    public long getSize() {
+      return size_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (hasRegion()) {
+        if (!getRegion().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, getRegion());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, size_);
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getRegion());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, size_);
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse) obj;
+
+      boolean result = true;
+      result = result && (hasRegion() == other.hasRegion());
+      if (hasRegion()) {
+        result = result && getRegion()
+            .equals(other.getRegion());
+      }
+      result = result && (hasSize() == other.hasSize());
+      if (hasSize()) {
+        result = result && (getSize()
+            == other.getSize());
+      }
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasRegion()) {
+        hash = (37 * hash) + REGION_FIELD_NUMBER;
+        hash = (53 * hash) + getRegion().hashCode();
+      }
+      if (hasSize()) {
+        hash = (37 * hash) + SIZE_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+            getSize());
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.RegionSpaceUse}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.RegionSpaceUse)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUse_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getRegionFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (regionBuilder_ == null) {
+          region_ = null;
+        } else {
+          regionBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        size_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUse_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (regionBuilder_ == null) {
+          result.region_ = region_;
+        } else {
+          result.region_ = regionBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.size_ = size_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.getDefaultInstance()) return this;
+        if (other.hasRegion()) {
+          mergeRegion(other.getRegion());
+        }
+        if (other.hasSize()) {
+          setSize(other.getSize());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (hasRegion()) {
+          if (!getRegion().isInitialized()) {
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo region_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionBuilder_;
+      /**
+       * <pre>
+       * A region identifier
+       * </pre>
+       *
+       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       */
+      public boolean hasRegion() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <pre>
+       * A region identifier
+       * </pre>
+       *
+       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegion() {
+        if (regionBuilder_ == null) {
+          return region_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : region_;
+        } else {
+          return regionBuilder_.getMessage();
+        }
+      }
+      /**
+       * <pre>
+       * A region identifier
+       * </pre>
+       *
+       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       */
+      public Builder setRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          region_ = value;
+          onChanged();
+        } else {
+          regionBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <pre>
+       * A region identifier
+       * </pre>
+       *
+       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       */
+      public Builder setRegion(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionBuilder_ == null) {
+          region_ = builderForValue.build();
+          onChanged();
+        } else {
+          regionBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <pre>
+       * A region identifier
+       * </pre>
+       *
+       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       */
+      public Builder mergeRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              region_ != null &&
+              region_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance()) {
+            region_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.newBuilder(region_).mergeFrom(value).buildPartial();
+          } else {
+            region_ = value;
+          }
+          onChanged();
+        } else {
+          regionBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <pre>
+       * A region identifier
+       * </pre>
+       *
+       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       */
+      public Builder clearRegion() {
+        if (regionBuilder_ == null) {
+          region_ = null;
+          onChanged();
+        } else {
+          regionBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <pre>
+       * A region identifier
+       * </pre>
+       *
+       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegionFieldBuilder().getBuilder();
+      }
+      /**
+       * <pre>
+       * A region identifier
+       * </pre>
+       *
+       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionOrBuilder() {
+        if (regionBuilder_ != null) {
+          return regionBuilder_.getMessageOrBuilder();
+        } else {
+          return region_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : region_;
+        }
+      }
+      /**
+       * <pre>
+       * A region identifier
+       * </pre>
+       *
+       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+          getRegionFieldBuilder() {
+        if (regionBuilder_ == null) {
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+                  getRegion(),
+                  getParentForChildren(),
+                  isClean());
+          region_ = null;
+        }
+        return regionBuilder_;
+      }
+
+      private long size_ ;
+      /**
+       * <pre>
+       * The size in bytes of the region
+       * </pre>
+       *
+       * <code>optional uint64 size = 2;</code>
+       */
+      public boolean hasSize() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <pre>
+       * The size in bytes of the region
+       * </pre>
+       *
+       * <code>optional uint64 size = 2;</code>
+       */
+      public long getSize() {
+        return size_;
+      }
+      /**
+       * <pre>
+       * The size in bytes of the region
+       * </pre>
+       *
+       * <code>optional uint64 size = 2;</code>
+       */
+      public Builder setSize(long value) {
+        bitField0_ |= 0x00000002;
+        size_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * The size in bytes of the region
+       * </pre>
+       *
+       * <code>optional uint64 size = 2;</code>
+       */
+      public Builder clearSize() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        size_ = 0L;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.RegionSpaceUse)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.RegionSpaceUse)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSpaceUse>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<RegionSpaceUse>() {
+      public RegionSpaceUse parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new RegionSpaceUse(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSpaceUse> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSpaceUse> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface RegionSpaceUseReportRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.RegionSpaceUseReportRequest)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse>
+        getSpaceUseList();
+    /**
+     * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse getSpaceUse(int index);
+    /**
+     * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+     */
+    int getSpaceUseCount();
+    /**
+     * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
+        getSpaceUseOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder getSpaceUseOrBuilder(
+        int index);
+  }
+  /**
+   * <pre>
+   **
+   * Reports filesystem usage for regions.
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.RegionSpaceUseReportRequest}
+   */
+  public  static final class RegionSpaceUseReportRequest extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.RegionSpaceUseReportRequest)
+      RegionSpaceUseReportRequestOrBuilder {
+    // Use RegionSpaceUseReportRequest.newBuilder() to construct.
+    private RegionSpaceUseReportRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private RegionSpaceUseReportRequest() {
+      spaceUse_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private RegionSpaceUseReportRequest(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                spaceUse_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              spaceUse_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          spaceUse_ = java.util.Collections.unmodifiableList(spaceUse_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUseReportRequest_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUseReportRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest.Builder.class);
+    }
+
+    public static final int SPACE_USE_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse> spaceUse_;
+    /**
+     * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse> getSpaceUseList() {
+      return spaceUse_;
+    }
+    /**
+     * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
+        getSpaceUseOrBuilderList() {
+      return spaceUse_;
+    }
+    /**
+     * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+     */
+    public int getSpaceUseCount() {
+      return spaceUse_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse getSpaceUse(int index) {
+      return spaceUse_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder getSpaceUseOrBuilder(
+        int index) {
+      return spaceUse_.get(index);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      for (int i = 0; i < getSpaceUseCount(); i++) {
+        if (!getSpaceUse(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      for (int i = 0; i < spaceUse_.size(); i++) {
+        output.writeMessage(1, spaceUse_.get(i));
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < spaceUse_.size(); i++) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, spaceUse_.get(i));
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest) obj;
+
+      boolean result = true;
+      result = result && getSpaceUseList()
+          .equals(other.getSpaceUseList());
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (getSpaceUseCount() > 0) {
+        hash = (37 * hash) + SPACE_USE_FIELD_NUMBER;
+        hash = (53 * hash) + getSpaceUseList().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     **
+     * Reports filesystem usage for regions.
+     * </pre>
+     *
+     * Protobuf type {@code hbase.pb.RegionSpaceUseReportRequest}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.RegionSpaceUseReportRequest)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequestOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUseReportRequest_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUseReportRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getSpaceUseFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (spaceUseBuilder_ == null) {
+          spaceUse_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          spaceUseBuilder_.clear();
+        }
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUseReportRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest(this);
+        int from_bitField0_ = bitField0_;
+        if (spaceUseBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            spaceUse_ = java.util.Collections.unmodifiableList(spaceUse_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.spaceUse_ = spaceUse_;
+        } else {
+          result.spaceUse_ = spaceUseBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest.getDefaultInstance()) return this;
+        if (spaceUseBuilder_ == null) {
+          if (!other.spaceUse_.isEmpty()) {
+            if (spaceUse_.isEmpty()) {
+              spaceUse_ = other.spaceUse_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensureSpaceUseIsMutable();
+              spaceUse_.addAll(other.spaceUse_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.spaceUse_.isEmpty()) {
+            if (spaceUseBuilder_.isEmpty()) {
+              spaceUseBuilder_.dispose();
+              spaceUseBuilder_ = null;
+              spaceUse_ = other.spaceUse_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              spaceUseBuilder_ =
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getSpaceUseFieldBuilder() : null;
+            } else {
+              spaceUseBuilder_.addAllMessages(other.spaceUse_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        for (int i = 0; i < getSpaceUseCount(); i++) {
+          if (!getSpaceUse(i).isInitialized()) {
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse> spaceUse_ =
+        java.util.Collections.emptyList();
+      private void ensureSpaceUseIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          spaceUse_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse>(spaceUse_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder> spaceUseBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse> getSpaceUseList() {
+        if (spaceUseBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(spaceUse_);
+        } else {
+          return spaceUseBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public int getSpaceUseCount() {
+        if (spaceUseBuilder_ == null) {
+          return spaceUse_.size();
+        } else {
+          return spaceUseBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse getSpaceUse(int index) {
+        if (spaceUseBuilder_ == null) {
+          return spaceUse_.get(index);
+        } else {
+          return spaceUseBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public Builder setSpaceUse(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse value) {
+        if (spaceUseBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureSpaceUseIsMutable();
+          spaceUse_.set(index, value);
+          onChanged();
+        } else {
+          spaceUseBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public Builder setSpaceUse(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder builderForValue) {
+        if (spaceUseBuilder_ == null) {
+          ensureSpaceUseIsMutable();
+          spaceUse_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          spaceUseBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public Builder addSpaceUse(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse value) {
+        if (spaceUseBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureSpaceUseIsMutable();
+          spaceUse_.add(value);
+          onChanged();
+        } else {
+          spaceUseBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public Builder addSpaceUse(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse value) {
+        if (spaceUseBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureSpaceUseIsMutable();
+          spaceUse_.add(index, value);
+          onChanged();
+        } else {
+          spaceUseBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public Builder addSpaceUse(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder builderForValue) {
+        if (spaceUseBuilder_ == null) {
+          ensureSpaceUseIsMutable();
+          spaceUse_.add(builderForValue.build());
+          onChanged();
+        } else {
+          spaceUseBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public Builder addSpaceUse(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder builderForValue) {
+        if (spaceUseBuilder_ == null) {
+          ensureSpaceUseIsMutable();
+          spaceUse_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          spaceUseBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public Builder addAllSpaceUse(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse> values) {
+        if (spaceUseBuilder_ == null) {
+          ensureSpaceUseIsMutable();
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
+              values, spaceUse_);
+          onChanged();
+        } else {
+          spaceUseBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public Builder clearSpaceUse() {
+        if (spaceUseBuilder_ == null) {
+          spaceUse_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          spaceUseBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public Builder removeSpaceUse(int index) {
+        if (spaceUseBuilder_ == null) {
+          ensureSpaceUseIsMutable();
+          spaceUse_.remove(index);
+          onChanged();
+        } else {
+          spaceUseBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder getSpaceUseBuilder(
+          int index) {
+        return getSpaceUseFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder getSpaceUseOrBuilder(
+          int index) {
+        if (spaceUseBuilder_ == null) {
+          return spaceUse_.get(index);  } else {
+          return spaceUseBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
+           getSpaceUseOrBuilderList() {
+        if (spaceUseBuilder_ != null) {
+          return spaceUseBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(spaceUse_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder addSpaceUseBuilder() {
+        return getSpaceUseFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder addSpaceUseBuilder(
+          int index) {
+        return getSpaceUseFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder>
+           getSpaceUseBuilderList() {
+        return getSpaceUseFieldBuilder().getBuilderList();
+      }
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
+          getSpaceUseFieldBuilder() {
+        if (spaceUseBuilder_ == null) {
+          spaceUseBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>(
+                  spaceUse_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          spaceUse_ = null;
+        }
+        return spaceUseBuilder_;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.RegionSpaceUseReportRequest)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.RegionSpaceUseReportRequest)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSpaceUseReportRequest>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<RegionSpaceUseReportRequest>() {
+      public RegionSpaceUseReportRequest parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new RegionSpaceUseReportRequest(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSpaceUseReportRequest> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSpaceUseReportRequest> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface RegionSpaceUseReportResponseOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.RegionSpaceUseReportResponse)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+  }
+  /**
+   * Protobuf type {@code hbase.pb.RegionSpaceUseReportResponse}
+   */
+  public  static final class RegionSpaceUseReportResponse extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.RegionSpaceUseReportResponse)
+      RegionSpaceUseReportResponseOrBuilder {
+    // Use RegionSpaceUseReportResponse.newBuilder() to construct.
+    private RegionSpaceUseReportResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private RegionSpaceUseReportResponse() {
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private RegionSpaceUseReportResponse(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUseReportResponse_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUseReportResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse.Builder.class);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse) obj;
+
+      boolean result = true;
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.RegionSpaceUseReportResponse}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.RegionSpaceUseReportResponse)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponseOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUseReportResponse_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUseReportResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionSpaceUseReportResponse_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse(this);
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse.getDefaultInstance()) return this;
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.RegionSpaceUseReportResponse)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.RegionSpaceUseReportResponse)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSpaceUseReportResponse>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<RegionSpaceUseReportResponse>() {
+      public RegionSpaceUseReportResponse parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new RegionSpaceUseReportResponse(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSpaceUseReportResponse> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSpaceUseReportResponse> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
   /**
    * Protobuf service {@code hbase.pb.RegionServerStatusService}
    */
@@ -10265,6 +12171,19 @@ public final class RegionServerStatusProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse> done);
 
+      /**
+       * <pre>
+       **
+       * Reports Region filesystem space use
+       * </pre>
+       *
+       * <code>rpc ReportRegionSpaceUse(.hbase.pb.RegionSpaceUseReportRequest) returns (.hbase.pb.RegionSpaceUseReportResponse);</code>
+       */
+      public abstract void reportRegionSpaceUse(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse> done);
+
     }
 
     public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Service newReflectiveService(
@@ -10326,6 +12245,14 @@ public final class RegionServerStatusProtos {
           impl.getProcedureResult(controller, request, done);
         }
 
+        @java.lang.Override
+        public  void reportRegionSpaceUse(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse> done) {
+          impl.reportRegionSpaceUse(controller, request, done);
+        }
+
       };
     }
 
@@ -10362,6 +12289,8 @@ public final class RegionServerStatusProtos {
               return impl.splitRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest)request);
             case 6:
               return impl.getProcedureResult(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest)request);
+            case 7:
+              return impl.reportRegionSpaceUse(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -10390

<TRUNCATED>

[05/50] [abbrv] hbase git commit: Revert "HBASE-17906 When a huge amount of data writing to hbase through thrift2, there will be a deadlock error. (Albert Lee)" Mistaken commit.

Posted by el...@apache.org.
Revert "HBASE-17906 When a huge amount of data writing to hbase through thrift2, there will be a deadlock error. (Albert Lee)"
Mistaken commit.

This reverts commit 9dd5cda01747ffb91ac084792fa4a8670859e810.


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

Branch: refs/heads/HBASE-16961
Commit: 0cd4cec5d24b5e7194a903e4d900f5558ed8b9a7
Parents: c846145
Author: Michael Stack <st...@apache.org>
Authored: Fri Apr 14 12:07:40 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Fri Apr 14 12:07:40 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java   | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0cd4cec5/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
index 8f56b10..560ae64 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
@@ -432,6 +432,9 @@ public class ThriftServer extends Configured implements Tool {
       throw new RuntimeException("Could not parse the value provided for the port option", e);
     }
 
+    // Thrift's implementation uses '0' as a placeholder for 'use the default.'
+    int backlog = conf.getInt(BACKLOG_CONF_KEY, 0);
+
     // Local hostname and user name,
     // used only if QOP is configured.
     String host = null;


[43/50] [abbrv] hbase git commit: HBASE-17428 Implement informational RPCs for space quotas

Posted by el...@apache.org.
HBASE-17428 Implement informational RPCs for space quotas

Create some RPCs that can expose the in-memory state that the
RegionServers and Master hold to drive the space quota "state machine".
Then, create some hbase shell commands to interact with those.


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

Branch: refs/heads/HBASE-16961
Commit: 095fabf16d91ce0e5c042b657a9f07a548d49a49
Parents: efd6edc
Author: Josh Elser <el...@apache.org>
Authored: Tue Feb 21 15:36:39 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 17 15:44:00 2017 -0400

----------------------------------------------------------------------
 .../hbase/client/ConnectionImplementation.java  |    9 +
 .../hadoop/hbase/client/QuotaStatusCalls.java   |  125 +
 .../client/ShortCircuitMasterConnection.java    |    7 +
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |   77 +
 .../hbase/shaded/protobuf/RequestConverter.java |   33 +
 .../shaded/protobuf/generated/AdminProtos.java  |  394 +-
 .../shaded/protobuf/generated/MasterProtos.java |   92 +-
 .../shaded/protobuf/generated/QuotaProtos.java  | 5986 +++++++++++++++++-
 .../generated/RegionServerStatusProtos.java     |   28 +-
 .../src/main/protobuf/Admin.proto               |    9 +
 .../src/main/protobuf/Master.proto              |    4 +
 .../src/main/protobuf/Quota.proto               |   35 +
 .../hbase/protobuf/generated/QuotaProtos.java   |    6 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   40 +
 .../hbase/quotas/ActivePolicyEnforcement.java   |    8 +
 .../hbase/regionserver/RSRpcServices.java       |   57 +
 .../hadoop/hbase/master/MockRegionServer.java   |   18 +
 .../hbase/quotas/TestQuotaStatusRPCs.java       |  192 +
 hbase-shell/src/main/ruby/hbase/quotas.rb       |   16 +
 hbase-shell/src/main/ruby/shell.rb              |    3 +
 .../ruby/shell/commands/list_quota_snapshots.rb |   59 +
 .../shell/commands/list_quota_table_sizes.rb    |   47 +
 .../shell/commands/list_quota_violations.rb     |   48 +
 hbase-shell/src/test/ruby/hbase/quotas_test.rb  |   24 -
 .../test/ruby/hbase/quotas_test_no_cluster.rb   |   69 +
 25 files changed, 7066 insertions(+), 320 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 99feb14..3f27e1c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -94,6 +94,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
@@ -1731,6 +1733,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
           ListReplicationPeersRequest request) throws ServiceException {
         return stub.listReplicationPeers(controller, request);
       }
+
+      @Override
+      public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(
+          RpcController controller, GetSpaceQuotaRegionSizesRequest request)
+          throws ServiceException {
+        return stub.getSpaceQuotaRegionSizes(controller, request);
+      }
     };
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
new file mode 100644
index 0000000..f0f385d
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
@@ -0,0 +1,125 @@
+/*
+ * 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.client;
+
+import java.io.IOException;
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
+
+/**
+ * Client class to wrap RPCs to HBase servers for space quota status information.
+ */
+@InterfaceAudience.Private
+public class QuotaStatusCalls {
+
+  /**
+   * {@link #getMasterRegionSizes(Connection, RpcControllerFactory, RpcRetryingCallerFactory, int)}
+   */
+  public static GetSpaceQuotaRegionSizesResponse getMasterRegionSizes(
+      ClusterConnection clusterConn, int timeout) throws IOException {
+    RpcControllerFactory rpcController = clusterConn.getRpcControllerFactory();
+    RpcRetryingCallerFactory rpcCaller = clusterConn.getRpcRetryingCallerFactory();
+    return getMasterRegionSizes(clusterConn, rpcController, rpcCaller, timeout);
+  }
+
+  /**
+   * Executes an RPC to the HBase master to fetch its view on the Region sizes.
+   */
+  public static GetSpaceQuotaRegionSizesResponse getMasterRegionSizes(
+      Connection conn, RpcControllerFactory factory, RpcRetryingCallerFactory rpcCaller,
+      int timeout) throws IOException {
+    MasterCallable<GetSpaceQuotaRegionSizesResponse> callable =
+        new MasterCallable<GetSpaceQuotaRegionSizesResponse>(conn, factory) {
+      @Override
+      protected GetSpaceQuotaRegionSizesResponse rpcCall() throws Exception {
+        return master.getSpaceQuotaRegionSizes(
+            getRpcController(), RequestConverter.buildGetSpaceQuotaRegionSizesRequest());
+      }
+    };
+    RpcRetryingCaller<GetSpaceQuotaRegionSizesResponse> caller = rpcCaller.newCaller();
+    try {
+      return caller.callWithoutRetries(callable, timeout);
+    } finally {
+      callable.close();
+    }
+  }
+
+  /**
+   * {@link #getRegionServerQuotaSnapshot(ClusterConnection, RpcControllerFactory, int, ServerName)}
+   */
+  public static GetSpaceQuotaSnapshotsResponse getRegionServerQuotaSnapshot(
+      ClusterConnection clusterConn, int timeout, ServerName sn) throws IOException {
+    RpcControllerFactory rpcController = clusterConn.getRpcControllerFactory();
+    return getRegionServerQuotaSnapshot(clusterConn, rpcController, timeout, sn);
+  }
+
+  /**
+   * Executes an RPC to the RegionServer identified by the {@code ServerName} to fetch its view
+   * on space quotas.
+   */
+  public static GetSpaceQuotaSnapshotsResponse getRegionServerQuotaSnapshot(
+      ClusterConnection conn, RpcControllerFactory factory,
+      int timeout, ServerName sn) throws IOException {
+    final AdminService.BlockingInterface admin = conn.getAdmin(sn);
+    Callable<GetSpaceQuotaSnapshotsResponse> callable =
+        new Callable<GetSpaceQuotaSnapshotsResponse>() {
+      @Override
+      public GetSpaceQuotaSnapshotsResponse call() throws Exception {
+        return admin.getSpaceQuotaSnapshots(
+            factory.newController(), RequestConverter.buildGetSpaceQuotaSnapshotsRequest());
+      }
+    };
+    return ProtobufUtil.call(callable);
+  }
+
+  /**
+   * {@link #getRegionServerSpaceQuotaEnforcements(ClusterConnection, RpcControllerFactory, int, ServerName)}
+   */
+  public static GetSpaceQuotaEnforcementsResponse getRegionServerSpaceQuotaEnforcements(
+      ClusterConnection clusterConn, int timeout, ServerName sn) throws IOException {
+    RpcControllerFactory rpcController = clusterConn.getRpcControllerFactory();
+    return getRegionServerSpaceQuotaEnforcements(clusterConn, rpcController, timeout, sn);
+  }
+
+  /**
+   * Executes an RPC to the RegionServer identified by the {@code ServerName} to fetch its view on
+   * enforced space quotas.
+   */
+  public static GetSpaceQuotaEnforcementsResponse getRegionServerSpaceQuotaEnforcements(
+      ClusterConnection conn, RpcControllerFactory factory,
+      int timeout, ServerName sn) throws IOException {
+    final AdminService.BlockingInterface admin = conn.getAdmin(sn);
+    Callable<GetSpaceQuotaEnforcementsResponse> callable =
+        new Callable<GetSpaceQuotaEnforcementsResponse>() {
+      @Override
+      public GetSpaceQuotaEnforcementsResponse call() throws Exception {
+        return admin.getSpaceQuotaEnforcements(
+            factory.newController(), RequestConverter.buildGetSpaceQuotaEnforcementsRequest());
+      }
+    };
+    return ProtobufUtil.call(callable);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
index 72b2a15..30ae79c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
@@ -23,6 +23,8 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.*;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
@@ -478,4 +480,9 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
     return stub.listReplicationPeers(controller, request);
   }
 
+  @Override
+  public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(RpcController controller,
+      GetSpaceQuotaRegionSizesRequest request) throws ServiceException {
+    return stub.getSpaceQuotaRegionSizes(controller, request);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/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 ce4cd04..560f40c 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
@@ -22,6 +22,7 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -31,12 +32,15 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.QuotaStatusCalls;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
@@ -46,12 +50,19 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.filter.RegexStringComparator;
 import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -395,6 +406,72 @@ public class QuotaTableUtil {
     return p;
   }
 
+
+  /* =========================================================================
+   *  Space quota status RPC helpers
+   */
+  /**
+   * Fetches the table sizes on the filesystem as tracked by the HBase Master.
+   */
+  public static Map<TableName,Long> getMasterReportedTableSizes(
+      Connection conn) throws IOException {
+    if (!(conn instanceof ClusterConnection)) {
+      throw new IllegalArgumentException("Expected a ClusterConnection");
+    }
+    ClusterConnection clusterConn = (ClusterConnection) conn;
+    GetSpaceQuotaRegionSizesResponse response = QuotaStatusCalls.getMasterRegionSizes(
+        clusterConn, 0);
+    Map<TableName,Long> tableSizes = new HashMap<>();
+    for (RegionSizes sizes : response.getSizesList()) {
+      TableName tn = ProtobufUtil.toTableName(sizes.getTableName());
+      tableSizes.put(tn, sizes.getSize());
+    }
+    return tableSizes;
+  }
+
+  /**
+   * Fetches the observed {@link SpaceQuotaSnapshot}s observed by a RegionServer.
+   */
+  public static Map<TableName,SpaceQuotaSnapshot> getRegionServerQuotaSnapshots(
+      Connection conn, ServerName regionServer) throws IOException {
+    if (!(conn instanceof ClusterConnection)) {
+      throw new IllegalArgumentException("Expected a ClusterConnection");
+    }
+    ClusterConnection clusterConn = (ClusterConnection) conn;
+    GetSpaceQuotaSnapshotsResponse response = QuotaStatusCalls.getRegionServerQuotaSnapshot(
+        clusterConn, 0, regionServer);
+    Map<TableName,SpaceQuotaSnapshot> snapshots = new HashMap<>();
+    for (TableQuotaSnapshot snapshot : response.getSnapshotsList()) {
+      snapshots.put(
+          ProtobufUtil.toTableName(snapshot.getTableName()),
+          SpaceQuotaSnapshot.toSpaceQuotaSnapshot(snapshot.getSnapshot()));
+    }
+    return snapshots;
+  }
+
+  /**
+   * Fetches the active {@link SpaceViolationPolicy}'s that are being enforced on the
+   * given RegionServer.
+   */
+  public static Map<TableName,SpaceViolationPolicy> getRegionServerQuotaViolations(
+      Connection conn, ServerName regionServer) throws IOException {
+    if (!(conn instanceof ClusterConnection)) {
+      throw new IllegalArgumentException("Expected a ClusterConnection");
+    }
+    ClusterConnection clusterConn = (ClusterConnection) conn;
+    RpcControllerFactory rpcController = clusterConn.getRpcControllerFactory();
+    GetSpaceQuotaEnforcementsResponse response =
+        QuotaStatusCalls.getRegionServerSpaceQuotaEnforcements(
+            clusterConn, rpcController, 0, regionServer);
+    Map<TableName,SpaceViolationPolicy> policies = new HashMap<>();
+    for (TableViolationPolicy policy : response.getViolationPoliciesList()) {
+      policies.put(
+          ProtobufUtil.toTableName(policy.getTableName()),
+          ProtobufUtil.toViolationPolicy(policy.getViolationPolicy()));
+    }
+    return policies;
+  }
+
   /* =========================================================================
    *  Quotas protobuf helpers
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index a513d66..b37be68 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -116,6 +116,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormali
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
@@ -1695,4 +1698,34 @@ public final class RequestConverter {
     builder.setNamespaceName(name);
     return builder.build();
   }
+
+  private static final GetSpaceQuotaRegionSizesRequest GET_SPACE_QUOTA_REGION_SIZES_REQUEST =
+      GetSpaceQuotaRegionSizesRequest.newBuilder().build();
+
+  /**
+   * Returns a {@link GetSpaceQuotaRegionSizesRequest} object.
+   */
+  public static GetSpaceQuotaRegionSizesRequest buildGetSpaceQuotaRegionSizesRequest() {
+    return GET_SPACE_QUOTA_REGION_SIZES_REQUEST;
+  }
+
+  private static final GetSpaceQuotaSnapshotsRequest GET_SPACE_QUOTA_SNAPSHOTS_REQUEST =
+      GetSpaceQuotaSnapshotsRequest.newBuilder().build();
+
+  /**
+   * Returns a {@link GetSpaceQuotaSnapshotsRequest} object.
+   */
+  public static GetSpaceQuotaSnapshotsRequest buildGetSpaceQuotaSnapshotsRequest() {
+    return GET_SPACE_QUOTA_SNAPSHOTS_REQUEST;
+  }
+
+  private static final GetSpaceQuotaEnforcementsRequest GET_SPACE_QUOTA_ENFORCEMENTS_REQUEST =
+      GetSpaceQuotaEnforcementsRequest.newBuilder().build();
+
+  /**
+   * Returns a {@link GetSpaceQuotaEnforcementsRequest} object.
+   */
+  public static GetSpaceQuotaEnforcementsRequest buildGetSpaceQuotaEnforcementsRequest() {
+    return GET_SPACE_QUOTA_ENFORCEMENTS_REQUEST;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
index 703de38..8e52626 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
@@ -24902,6 +24902,30 @@ public final class AdminProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse> done);
 
+      /**
+       * <pre>
+       ** Fetches the RegionServer's view of space quotas 
+       * </pre>
+       *
+       * <code>rpc GetSpaceQuotaSnapshots(.hbase.pb.GetSpaceQuotaSnapshotsRequest) returns (.hbase.pb.GetSpaceQuotaSnapshotsResponse);</code>
+       */
+      public abstract void getSpaceQuotaSnapshots(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse> done);
+
+      /**
+       * <pre>
+       ** Fetches the RegionServer's space quota active enforcements 
+       * </pre>
+       *
+       * <code>rpc GetSpaceQuotaEnforcements(.hbase.pb.GetSpaceQuotaEnforcementsRequest) returns (.hbase.pb.GetSpaceQuotaEnforcementsResponse);</code>
+       */
+      public abstract void getSpaceQuotaEnforcements(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse> done);
+
     }
 
     public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Service newReflectiveService(
@@ -25051,6 +25075,22 @@ public final class AdminProtos {
           impl.getRegionLoad(controller, request, done);
         }
 
+        @java.lang.Override
+        public  void getSpaceQuotaSnapshots(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse> done) {
+          impl.getSpaceQuotaSnapshots(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getSpaceQuotaEnforcements(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse> done) {
+          impl.getSpaceQuotaEnforcements(controller, request, done);
+        }
+
       };
     }
 
@@ -25109,6 +25149,10 @@ public final class AdminProtos {
               return impl.updateConfiguration(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest)request);
             case 17:
               return impl.getRegionLoad(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest)request);
+            case 18:
+              return impl.getSpaceQuotaSnapshots(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest)request);
+            case 19:
+              return impl.getSpaceQuotaEnforcements(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -25159,6 +25203,10 @@ public final class AdminProtos {
               return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest.getDefaultInstance();
             case 17:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest.getDefaultInstance();
+            case 18:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest.getDefaultInstance();
+            case 19:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -25209,6 +25257,10 @@ public final class AdminProtos {
               return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance();
             case 17:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse.getDefaultInstance();
+            case 18:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.getDefaultInstance();
+            case 19:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -25361,6 +25413,30 @@ public final class AdminProtos {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse> done);
 
+    /**
+     * <pre>
+     ** Fetches the RegionServer's view of space quotas 
+     * </pre>
+     *
+     * <code>rpc GetSpaceQuotaSnapshots(.hbase.pb.GetSpaceQuotaSnapshotsRequest) returns (.hbase.pb.GetSpaceQuotaSnapshotsResponse);</code>
+     */
+    public abstract void getSpaceQuotaSnapshots(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest request,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse> done);
+
+    /**
+     * <pre>
+     ** Fetches the RegionServer's space quota active enforcements 
+     * </pre>
+     *
+     * <code>rpc GetSpaceQuotaEnforcements(.hbase.pb.GetSpaceQuotaEnforcementsRequest) returns (.hbase.pb.GetSpaceQuotaEnforcementsResponse);</code>
+     */
+    public abstract void getSpaceQuotaEnforcements(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest request,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse> done);
+
     public static final
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
@@ -25473,6 +25549,16 @@ public final class AdminProtos {
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse>specializeCallback(
               done));
           return;
+        case 18:
+          this.getSpaceQuotaSnapshots(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest)request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse>specializeCallback(
+              done));
+          return;
+        case 19:
+          this.getSpaceQuotaEnforcements(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest)request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse>specializeCallback(
+              done));
+          return;
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -25523,6 +25609,10 @@ public final class AdminProtos {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest.getDefaultInstance();
         case 17:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest.getDefaultInstance();
+        case 18:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest.getDefaultInstance();
+        case 19:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -25573,6 +25663,10 @@ public final class AdminProtos {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance();
         case 17:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse.getDefaultInstance();
+        case 18:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.getDefaultInstance();
+        case 19:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -25863,6 +25957,36 @@ public final class AdminProtos {
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse.getDefaultInstance()));
       }
+
+      public  void getSpaceQuotaSnapshots(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(18),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.getDefaultInstance(),
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.class,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.getDefaultInstance()));
+      }
+
+      public  void getSpaceQuotaEnforcements(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(19),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.getDefaultInstance(),
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.class,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.getDefaultInstance()));
+      }
     }
 
     public static BlockingInterface newBlockingStub(
@@ -25960,6 +26084,16 @@ public final class AdminProtos {
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse getSpaceQuotaEnforcements(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
@@ -26184,6 +26318,30 @@ public final class AdminProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse.getDefaultInstance());
       }
 
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(18),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.getDefaultInstance());
+      }
+
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse getSpaceQuotaEnforcements(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(19),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.getDefaultInstance());
+      }
+
     }
 
     // @@protoc_insertion_point(class_scope:hbase.pb.AdminService)
@@ -26389,120 +26547,126 @@ public final class AdminProtos {
   static {
     java.lang.String[] descriptorData = {
       "\n\013Admin.proto\022\010hbase.pb\032\023ClusterStatus.p" +
-      "roto\032\013HBase.proto\032\tWAL.proto\"[\n\024GetRegio" +
-      "nInfoRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb." +
-      "RegionSpecifier\022\030\n\020compaction_state\030\002 \001(" +
-      "\010\"\353\001\n\025GetRegionInfoResponse\022)\n\013region_in" +
-      "fo\030\001 \002(\0132\024.hbase.pb.RegionInfo\022I\n\020compac" +
-      "tion_state\030\002 \001(\0162/.hbase.pb.GetRegionInf" +
-      "oResponse.CompactionState\022\024\n\014isRecoverin" +
-      "g\030\003 \001(\010\"F\n\017CompactionState\022\010\n\004NONE\020\000\022\t\n\005" +
-      "MINOR\020\001\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR_AND_MINOR\020\003\"",
-      "P\n\023GetStoreFileRequest\022)\n\006region\030\001 \002(\0132\031" +
-      ".hbase.pb.RegionSpecifier\022\016\n\006family\030\002 \003(" +
-      "\014\"*\n\024GetStoreFileResponse\022\022\n\nstore_file\030" +
-      "\001 \003(\t\"\030\n\026GetOnlineRegionRequest\"D\n\027GetOn" +
-      "lineRegionResponse\022)\n\013region_info\030\001 \003(\0132" +
-      "\024.hbase.pb.RegionInfo\"\263\002\n\021OpenRegionRequ" +
-      "est\022=\n\topen_info\030\001 \003(\0132*.hbase.pb.OpenRe" +
-      "gionRequest.RegionOpenInfo\022\027\n\017serverStar" +
-      "tCode\030\002 \001(\004\022\032\n\022master_system_time\030\005 \001(\004\032" +
-      "\251\001\n\016RegionOpenInfo\022$\n\006region\030\001 \002(\0132\024.hba",
-      "se.pb.RegionInfo\022\037\n\027version_of_offline_n" +
-      "ode\030\002 \001(\r\022+\n\rfavored_nodes\030\003 \003(\0132\024.hbase" +
-      ".pb.ServerName\022#\n\033openForDistributedLogR" +
-      "eplay\030\004 \001(\010\"\246\001\n\022OpenRegionResponse\022F\n\rop" +
-      "ening_state\030\001 \003(\0162/.hbase.pb.OpenRegionR" +
-      "esponse.RegionOpeningState\"H\n\022RegionOpen" +
-      "ingState\022\n\n\006OPENED\020\000\022\022\n\016ALREADY_OPENED\020\001" +
-      "\022\022\n\016FAILED_OPENING\020\002\"?\n\023WarmupRegionRequ" +
-      "est\022(\n\nregionInfo\030\001 \002(\0132\024.hbase.pb.Regio" +
-      "nInfo\"\026\n\024WarmupRegionResponse\"\313\001\n\022CloseR",
-      "egionRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb." +
-      "RegionSpecifier\022\037\n\027version_of_closing_no" +
-      "de\030\002 \001(\r\022\036\n\020transition_in_ZK\030\003 \001(\010:\004true" +
-      "\0220\n\022destination_server\030\004 \001(\0132\024.hbase.pb." +
-      "ServerName\022\027\n\017serverStartCode\030\005 \001(\004\"%\n\023C" +
-      "loseRegionResponse\022\016\n\006closed\030\001 \002(\010\"N\n!Cl" +
-      "oseRegionForSplitOrMergeRequest\022)\n\006regio" +
-      "n\030\001 \003(\0132\031.hbase.pb.RegionSpecifier\"4\n\"Cl" +
-      "oseRegionForSplitOrMergeResponse\022\016\n\006clos" +
-      "ed\030\001 \002(\010\"y\n\022FlushRegionRequest\022)\n\006region",
-      "\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022\030\n\020if_" +
-      "older_than_ts\030\002 \001(\004\022\036\n\026write_flush_wal_m" +
-      "arker\030\003 \001(\010\"_\n\023FlushRegionResponse\022\027\n\017la" +
-      "st_flush_time\030\001 \002(\004\022\017\n\007flushed\030\002 \001(\010\022\036\n\026" +
-      "wrote_flush_wal_marker\030\003 \001(\010\"T\n\022SplitReg" +
-      "ionRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Re" +
-      "gionSpecifier\022\023\n\013split_point\030\002 \001(\014\"\025\n\023Sp" +
-      "litRegionResponse\"`\n\024CompactRegionReques" +
-      "t\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpeci" +
-      "fier\022\r\n\005major\030\002 \001(\010\022\016\n\006family\030\003 \001(\014\"\027\n\025C",
-      "ompactRegionResponse\"\315\001\n\031UpdateFavoredNo" +
-      "desRequest\022I\n\013update_info\030\001 \003(\01324.hbase." +
-      "pb.UpdateFavoredNodesRequest.RegionUpdat" +
-      "eInfo\032e\n\020RegionUpdateInfo\022$\n\006region\030\001 \002(" +
-      "\0132\024.hbase.pb.RegionInfo\022+\n\rfavored_nodes" +
-      "\030\002 \003(\0132\024.hbase.pb.ServerName\".\n\032UpdateFa" +
-      "voredNodesResponse\022\020\n\010response\030\001 \001(\r\"a\n\010" +
-      "WALEntry\022\035\n\003key\030\001 \002(\0132\020.hbase.pb.WALKey\022" +
-      "\027\n\017key_value_bytes\030\002 \003(\014\022\035\n\025associated_c" +
-      "ell_count\030\003 \001(\005\"\242\001\n\030ReplicateWALEntryReq",
-      "uest\022!\n\005entry\030\001 \003(\0132\022.hbase.pb.WALEntry\022" +
-      "\034\n\024replicationClusterId\030\002 \001(\t\022\"\n\032sourceB" +
-      "aseNamespaceDirPath\030\003 \001(\t\022!\n\031sourceHFile" +
-      "ArchiveDirPath\030\004 \001(\t\"\033\n\031ReplicateWALEntr" +
-      "yResponse\"\026\n\024RollWALWriterRequest\"0\n\025Rol" +
-      "lWALWriterResponse\022\027\n\017region_to_flush\030\001 " +
-      "\003(\014\"#\n\021StopServerRequest\022\016\n\006reason\030\001 \002(\t" +
-      "\"\024\n\022StopServerResponse\"\026\n\024GetServerInfoR" +
-      "equest\"K\n\nServerInfo\022)\n\013server_name\030\001 \002(" +
-      "\0132\024.hbase.pb.ServerName\022\022\n\nwebui_port\030\002 ",
-      "\001(\r\"B\n\025GetServerInfoResponse\022)\n\013server_i" +
-      "nfo\030\001 \002(\0132\024.hbase.pb.ServerInfo\"\034\n\032Updat" +
-      "eConfigurationRequest\"\035\n\033UpdateConfigura" +
-      "tionResponse\"?\n\024GetRegionLoadRequest\022\'\n\n" +
-      "table_name\030\001 \001(\0132\023.hbase.pb.TableName\"C\n" +
-      "\025GetRegionLoadResponse\022*\n\014region_loads\030\001" +
-      " \003(\0132\024.hbase.pb.RegionLoad2\203\014\n\014AdminServ" +
-      "ice\022P\n\rGetRegionInfo\022\036.hbase.pb.GetRegio" +
-      "nInfoRequest\032\037.hbase.pb.GetRegionInfoRes" +
-      "ponse\022M\n\014GetStoreFile\022\035.hbase.pb.GetStor",
-      "eFileRequest\032\036.hbase.pb.GetStoreFileResp" +
-      "onse\022V\n\017GetOnlineRegion\022 .hbase.pb.GetOn" +
-      "lineRegionRequest\032!.hbase.pb.GetOnlineRe" +
-      "gionResponse\022G\n\nOpenRegion\022\033.hbase.pb.Op" +
-      "enRegionRequest\032\034.hbase.pb.OpenRegionRes" +
-      "ponse\022M\n\014WarmupRegion\022\035.hbase.pb.WarmupR" +
-      "egionRequest\032\036.hbase.pb.WarmupRegionResp" +
-      "onse\022J\n\013CloseRegion\022\034.hbase.pb.CloseRegi" +
-      "onRequest\032\035.hbase.pb.CloseRegionResponse" +
-      "\022w\n\032CloseRegionForSplitOrMerge\022+.hbase.p",
-      "b.CloseRegionForSplitOrMergeRequest\032,.hb" +
-      "ase.pb.CloseRegionForSplitOrMergeRespons" +
-      "e\022J\n\013FlushRegion\022\034.hbase.pb.FlushRegionR" +
-      "equest\032\035.hbase.pb.FlushRegionResponse\022J\n" +
-      "\013SplitRegion\022\034.hbase.pb.SplitRegionReque" +
-      "st\032\035.hbase.pb.SplitRegionResponse\022P\n\rCom" +
-      "pactRegion\022\036.hbase.pb.CompactRegionReque" +
-      "st\032\037.hbase.pb.CompactRegionResponse\022\\\n\021R" +
-      "eplicateWALEntry\022\".hbase.pb.ReplicateWAL" +
-      "EntryRequest\032#.hbase.pb.ReplicateWALEntr",
-      "yResponse\022Q\n\006Replay\022\".hbase.pb.Replicate" +
-      "WALEntryRequest\032#.hbase.pb.ReplicateWALE" +
-      "ntryResponse\022P\n\rRollWALWriter\022\036.hbase.pb" +
-      ".RollWALWriterRequest\032\037.hbase.pb.RollWAL" +
-      "WriterResponse\022P\n\rGetServerInfo\022\036.hbase." +
-      "pb.GetServerInfoRequest\032\037.hbase.pb.GetSe" +
-      "rverInfoResponse\022G\n\nStopServer\022\033.hbase.p" +
-      "b.StopServerRequest\032\034.hbase.pb.StopServe" +
-      "rResponse\022_\n\022UpdateFavoredNodes\022#.hbase." +
-      "pb.UpdateFavoredNodesRequest\032$.hbase.pb.",
-      "UpdateFavoredNodesResponse\022b\n\023UpdateConf" +
-      "iguration\022$.hbase.pb.UpdateConfiguration" +
-      "Request\032%.hbase.pb.UpdateConfigurationRe" +
-      "sponse\022P\n\rGetRegionLoad\022\036.hbase.pb.GetRe" +
-      "gionLoadRequest\032\037.hbase.pb.GetRegionLoad" +
+      "roto\032\013HBase.proto\032\tWAL.proto\032\013Quota.prot" +
+      "o\"[\n\024GetRegionInfoRequest\022)\n\006region\030\001 \002(" +
+      "\0132\031.hbase.pb.RegionSpecifier\022\030\n\020compacti" +
+      "on_state\030\002 \001(\010\"\353\001\n\025GetRegionInfoResponse" +
+      "\022)\n\013region_info\030\001 \002(\0132\024.hbase.pb.RegionI" +
+      "nfo\022I\n\020compaction_state\030\002 \001(\0162/.hbase.pb" +
+      ".GetRegionInfoResponse.CompactionState\022\024" +
+      "\n\014isRecovering\030\003 \001(\010\"F\n\017CompactionState\022" +
+      "\010\n\004NONE\020\000\022\t\n\005MINOR\020\001\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR",
+      "_AND_MINOR\020\003\"P\n\023GetStoreFileRequest\022)\n\006r" +
+      "egion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022\016" +
+      "\n\006family\030\002 \003(\014\"*\n\024GetStoreFileResponse\022\022" +
+      "\n\nstore_file\030\001 \003(\t\"\030\n\026GetOnlineRegionReq" +
+      "uest\"D\n\027GetOnlineRegionResponse\022)\n\013regio" +
+      "n_info\030\001 \003(\0132\024.hbase.pb.RegionInfo\"\263\002\n\021O" +
+      "penRegionRequest\022=\n\topen_info\030\001 \003(\0132*.hb" +
+      "ase.pb.OpenRegionRequest.RegionOpenInfo\022" +
+      "\027\n\017serverStartCode\030\002 \001(\004\022\032\n\022master_syste" +
+      "m_time\030\005 \001(\004\032\251\001\n\016RegionOpenInfo\022$\n\006regio",
+      "n\030\001 \002(\0132\024.hbase.pb.RegionInfo\022\037\n\027version" +
+      "_of_offline_node\030\002 \001(\r\022+\n\rfavored_nodes\030" +
+      "\003 \003(\0132\024.hbase.pb.ServerName\022#\n\033openForDi" +
+      "stributedLogReplay\030\004 \001(\010\"\246\001\n\022OpenRegionR" +
+      "esponse\022F\n\ropening_state\030\001 \003(\0162/.hbase.p" +
+      "b.OpenRegionResponse.RegionOpeningState\"" +
+      "H\n\022RegionOpeningState\022\n\n\006OPENED\020\000\022\022\n\016ALR" +
+      "EADY_OPENED\020\001\022\022\n\016FAILED_OPENING\020\002\"?\n\023War" +
+      "mupRegionRequest\022(\n\nregionInfo\030\001 \002(\0132\024.h" +
+      "base.pb.RegionInfo\"\026\n\024WarmupRegionRespon",
+      "se\"\313\001\n\022CloseRegionRequest\022)\n\006region\030\001 \002(" +
+      "\0132\031.hbase.pb.RegionSpecifier\022\037\n\027version_" +
+      "of_closing_node\030\002 \001(\r\022\036\n\020transition_in_Z" +
+      "K\030\003 \001(\010:\004true\0220\n\022destination_server\030\004 \001(" +
+      "\0132\024.hbase.pb.ServerName\022\027\n\017serverStartCo" +
+      "de\030\005 \001(\004\"%\n\023CloseRegionResponse\022\016\n\006close" +
+      "d\030\001 \002(\010\"N\n!CloseRegionForSplitOrMergeReq" +
+      "uest\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSp" +
+      "ecifier\"4\n\"CloseRegionForSplitOrMergeRes" +
+      "ponse\022\016\n\006closed\030\001 \002(\010\"y\n\022FlushRegionRequ",
+      "est\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpe" +
+      "cifier\022\030\n\020if_older_than_ts\030\002 \001(\004\022\036\n\026writ" +
+      "e_flush_wal_marker\030\003 \001(\010\"_\n\023FlushRegionR" +
+      "esponse\022\027\n\017last_flush_time\030\001 \002(\004\022\017\n\007flus" +
+      "hed\030\002 \001(\010\022\036\n\026wrote_flush_wal_marker\030\003 \001(" +
+      "\010\"T\n\022SplitRegionRequest\022)\n\006region\030\001 \002(\0132" +
+      "\031.hbase.pb.RegionSpecifier\022\023\n\013split_poin" +
+      "t\030\002 \001(\014\"\025\n\023SplitRegionResponse\"`\n\024Compac" +
+      "tRegionRequest\022)\n\006region\030\001 \002(\0132\031.hbase.p" +
+      "b.RegionSpecifier\022\r\n\005major\030\002 \001(\010\022\016\n\006fami",
+      "ly\030\003 \001(\014\"\027\n\025CompactRegionResponse\"\315\001\n\031Up" +
+      "dateFavoredNodesRequest\022I\n\013update_info\030\001" +
+      " \003(\01324.hbase.pb.UpdateFavoredNodesReques" +
+      "t.RegionUpdateInfo\032e\n\020RegionUpdateInfo\022$" +
+      "\n\006region\030\001 \002(\0132\024.hbase.pb.RegionInfo\022+\n\r" +
+      "favored_nodes\030\002 \003(\0132\024.hbase.pb.ServerNam" +
+      "e\".\n\032UpdateFavoredNodesResponse\022\020\n\010respo" +
+      "nse\030\001 \001(\r\"a\n\010WALEntry\022\035\n\003key\030\001 \002(\0132\020.hba" +
+      "se.pb.WALKey\022\027\n\017key_value_bytes\030\002 \003(\014\022\035\n" +
+      "\025associated_cell_count\030\003 \001(\005\"\242\001\n\030Replica",
+      "teWALEntryRequest\022!\n\005entry\030\001 \003(\0132\022.hbase" +
+      ".pb.WALEntry\022\034\n\024replicationClusterId\030\002 \001" +
+      "(\t\022\"\n\032sourceBaseNamespaceDirPath\030\003 \001(\t\022!" +
+      "\n\031sourceHFileArchiveDirPath\030\004 \001(\t\"\033\n\031Rep" +
+      "licateWALEntryResponse\"\026\n\024RollWALWriterR" +
+      "equest\"0\n\025RollWALWriterResponse\022\027\n\017regio" +
+      "n_to_flush\030\001 \003(\014\"#\n\021StopServerRequest\022\016\n" +
+      "\006reason\030\001 \002(\t\"\024\n\022StopServerResponse\"\026\n\024G" +
+      "etServerInfoRequest\"K\n\nServerInfo\022)\n\013ser" +
+      "ver_name\030\001 \002(\0132\024.hbase.pb.ServerName\022\022\n\n",
+      "webui_port\030\002 \001(\r\"B\n\025GetServerInfoRespons" +
+      "e\022)\n\013server_info\030\001 \002(\0132\024.hbase.pb.Server" +
+      "Info\"\034\n\032UpdateConfigurationRequest\"\035\n\033Up" +
+      "dateConfigurationResponse\"?\n\024GetRegionLo" +
+      "adRequest\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb" +
+      ".TableName\"C\n\025GetRegionLoadResponse\022*\n\014r" +
+      "egion_loads\030\001 \003(\0132\024.hbase.pb.RegionLoad2" +
+      "\346\r\n\014AdminService\022P\n\rGetRegionInfo\022\036.hbas" +
+      "e.pb.GetRegionInfoRequest\032\037.hbase.pb.Get" +
+      "RegionInfoResponse\022M\n\014GetStoreFile\022\035.hba",
+      "se.pb.GetStoreFileRequest\032\036.hbase.pb.Get" +
+      "StoreFileResponse\022V\n\017GetOnlineRegion\022 .h" +
+      "base.pb.GetOnlineRegionRequest\032!.hbase.p" +
+      "b.GetOnlineRegionResponse\022G\n\nOpenRegion\022" +
+      "\033.hbase.pb.OpenRegionRequest\032\034.hbase.pb." +
+      "OpenRegionResponse\022M\n\014WarmupRegion\022\035.hba" +
+      "se.pb.WarmupRegionRequest\032\036.hbase.pb.War" +
+      "mupRegionResponse\022J\n\013CloseRegion\022\034.hbase" +
+      ".pb.CloseRegionRequest\032\035.hbase.pb.CloseR" +
+      "egionResponse\022w\n\032CloseRegionForSplitOrMe",
+      "rge\022+.hbase.pb.CloseRegionForSplitOrMerg" +
+      "eRequest\032,.hbase.pb.CloseRegionForSplitO" +
+      "rMergeResponse\022J\n\013FlushRegion\022\034.hbase.pb" +
+      ".FlushRegionRequest\032\035.hbase.pb.FlushRegi" +
+      "onResponse\022J\n\013SplitRegion\022\034.hbase.pb.Spl" +
+      "itRegionRequest\032\035.hbase.pb.SplitRegionRe" +
+      "sponse\022P\n\rCompactRegion\022\036.hbase.pb.Compa" +
+      "ctRegionRequest\032\037.hbase.pb.CompactRegion" +
+      "Response\022\\\n\021ReplicateWALEntry\022\".hbase.pb" +
+      ".ReplicateWALEntryRequest\032#.hbase.pb.Rep",
+      "licateWALEntryResponse\022Q\n\006Replay\022\".hbase" +
+      ".pb.ReplicateWALEntryRequest\032#.hbase.pb." +
+      "ReplicateWALEntryResponse\022P\n\rRollWALWrit" +
+      "er\022\036.hbase.pb.RollWALWriterRequest\032\037.hba" +
+      "se.pb.RollWALWriterResponse\022P\n\rGetServer" +
+      "Info\022\036.hbase.pb.GetServerInfoRequest\032\037.h" +
+      "base.pb.GetServerInfoResponse\022G\n\nStopSer" +
+      "ver\022\033.hbase.pb.StopServerRequest\032\034.hbase" +
+      ".pb.StopServerResponse\022_\n\022UpdateFavoredN" +
+      "odes\022#.hbase.pb.UpdateFavoredNodesReques",
+      "t\032$.hbase.pb.UpdateFavoredNodesResponse\022" +
+      "b\n\023UpdateConfiguration\022$.hbase.pb.Update" +
+      "ConfigurationRequest\032%.hbase.pb.UpdateCo" +
+      "nfigurationResponse\022P\n\rGetRegionLoad\022\036.h" +
+      "base.pb.GetRegionLoadRequest\032\037.hbase.pb." +
+      "GetRegionLoadResponse\022k\n\026GetSpaceQuotaSn" +
+      "apshots\022\'.hbase.pb.GetSpaceQuotaSnapshot" +
+      "sRequest\032(.hbase.pb.GetSpaceQuotaSnapsho" +
+      "tsResponse\022t\n\031GetSpaceQuotaEnforcements\022" +
+      "*.hbase.pb.GetSpaceQuotaEnforcementsRequ",
+      "est\032+.hbase.pb.GetSpaceQuotaEnforcements" +
       "ResponseBH\n1org.apache.hadoop.hbase.shad" +
       "ed.protobuf.generatedB\013AdminProtosH\001\210\001\001\240" +
       "\001\001"
@@ -26521,6 +26685,7 @@ public final class AdminProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.getDescriptor(),
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_GetRegionInfoRequest_descriptor =
       getDescriptor().getMessageTypes().get(0);
@@ -26753,6 +26918,7 @@ public final class AdminProtos {
     org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.getDescriptor();
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
     org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.getDescriptor();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.getDescriptor();
   }
 
   // @@protoc_insertion_point(outer_class_scope)

http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index bbc6d1d..b81599b 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -63752,7 +63752,7 @@ public final class MasterProtos {
        * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder>
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder> 
           getSpaceLimitFieldBuilder() {
         if (spaceLimitBuilder_ == null) {
           spaceLimitBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -71159,6 +71159,18 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse> done);
 
+      /**
+       * <pre>
+       ** Fetches the Master's view of space quotas 
+       * </pre>
+       *
+       * <code>rpc GetSpaceQuotaRegionSizes(.hbase.pb.GetSpaceQuotaRegionSizesRequest) returns (.hbase.pb.GetSpaceQuotaRegionSizesResponse);</code>
+       */
+      public abstract void getSpaceQuotaRegionSizes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse> done);
+
     }
 
     public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Service newReflectiveService(
@@ -71724,6 +71736,14 @@ public final class MasterProtos {
           impl.removeDrainFromRegionServers(controller, request, done);
         }
 
+        @java.lang.Override
+        public  void getSpaceQuotaRegionSizes(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse> done) {
+          impl.getSpaceQuotaRegionSizes(controller, request, done);
+        }
+
       };
     }
 
@@ -71886,6 +71906,8 @@ public final class MasterProtos {
               return impl.drainRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest)request);
             case 69:
               return impl.removeDrainFromRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest)request);
+            case 70:
+              return impl.getSpaceQuotaRegionSizes(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -72040,6 +72062,8 @@ public final class MasterProtos {
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest.getDefaultInstance();
             case 69:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest.getDefaultInstance();
+            case 70:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -72194,6 +72218,8 @@ public final class MasterProtos {
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance();
             case 69:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance();
+            case 70:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -73073,6 +73099,18 @@ public final class MasterProtos {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest request,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse> done);
 
+    /**
+     * <pre>
+     ** Fetches the Master's view of space quotas 
+     * </pre>
+     *
+     * <code>rpc GetSpaceQuotaRegionSizes(.hbase.pb.GetSpaceQuotaRegionSizesRequest) returns (.hbase.pb.GetSpaceQuotaRegionSizesResponse);</code>
+     */
+    public abstract void getSpaceQuotaRegionSizes(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest request,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse> done);
+
     public static final
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
@@ -73445,6 +73483,11 @@ public final class MasterProtos {
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse>specializeCallback(
               done));
           return;
+        case 70:
+          this.getSpaceQuotaRegionSizes(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest)request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse>specializeCallback(
+              done));
+          return;
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -73599,6 +73642,8 @@ public final class MasterProtos {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest.getDefaultInstance();
         case 69:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest.getDefaultInstance();
+        case 70:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -73753,6 +73798,8 @@ public final class MasterProtos {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance();
         case 69:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance();
+        case 70:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -74823,6 +74870,21 @@ public final class MasterProtos {
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance()));
       }
+
+      public  void getSpaceQuotaRegionSizes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(70),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.getDefaultInstance(),
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.class,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.getDefaultInstance()));
+      }
     }
 
     public static BlockingInterface newBlockingStub(
@@ -75180,6 +75242,11 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
@@ -76028,6 +76095,18 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance());
       }
 
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(70),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.getDefaultInstance());
+      }
+
     }
 
     // @@protoc_insertion_point(class_scope:hbase.pb.MasterService)
@@ -76869,7 +76948,7 @@ public final class MasterProtos {
       "veDrainFromRegionServersRequest\022)\n\013serve",
       "r_name\030\001 \003(\0132\024.hbase.pb.ServerName\"&\n$Re" +
       "moveDrainFromRegionServersResponse*(\n\020Ma" +
-      "sterSwitchType\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\3013\n" +
+      "sterSwitchType\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\2644\n" +
       "\rMasterService\022e\n\024GetSchemaAlterStatus\022%" +
       ".hbase.pb.GetSchemaAlterStatusRequest\032&." +
       "hbase.pb.GetSchemaAlterStatusResponse\022b\n" +
@@ -77034,9 +77113,12 @@ public final class MasterProtos {
       "rainRegionServersResponse\022}\n\034removeDrain" +
       "FromRegionServers\022-.hbase.pb.RemoveDrain" +
       "FromRegionServersRequest\032..hbase.pb.Remo" +
-      "veDrainFromRegionServersResponseBI\n1org." +
-      "apache.hadoop.hbase.shaded.protobuf.gene" +
-      "ratedB\014MasterProtosH\001\210\001\001\240\001\001"
+      "veDrainFromRegionServersResponse\022q\n\030GetS" +
+      "paceQuotaRegionSizes\022).hbase.pb.GetSpace" +
+      "QuotaRegionSizesRequest\032*.hbase.pb.GetSp",
+      "aceQuotaRegionSizesResponseBI\n1org.apach" +
+      "e.hadoop.hbase.shaded.protobuf.generated" +
+      "B\014MasterProtosH\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {


[42/50] [abbrv] hbase git commit: HBASE-17428 Implement informational RPCs for space quotas

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/095fabf1/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
index a4c6095..d56def5 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
@@ -4362,7 +4362,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
           getSpaceFieldBuilder() {
         if (spaceBuilder_ == null) {
           spaceBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -6077,7 +6077,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
           getQuotaFieldBuilder() {
         if (quotaBuilder_ == null) {
           quotaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -6351,7 +6351,7 @@ public final class QuotaProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPolicy()) {
         hash = (37 * hash) + POLICY_FIELD_NUMBER;
         hash = (53 * hash) + policy_;
@@ -6978,7 +6978,7 @@ public final class QuotaProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasStatus()) {
         hash = (37 * hash) + STATUS_FIELD_NUMBER;
         hash = (53 * hash) + getStatus().hashCode();
@@ -7351,7 +7351,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder>
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder> 
           getStatusFieldBuilder() {
         if (statusBuilder_ == null) {
           statusBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -7476,163 +7476,5829 @@ public final class QuotaProtos {
 
   }
 
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_TimedQuota_descriptor;
-  private static final
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_TimedQuota_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_Throttle_descriptor;
-  private static final
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_Throttle_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_ThrottleRequest_descriptor;
-  private static final
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_Quotas_descriptor;
-  private static final
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_Quotas_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_QuotaUsage_descriptor;
-  private static final
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_QuotaUsage_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SpaceQuota_descriptor;
-  private static final
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_SpaceQuota_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SpaceLimitRequest_descriptor;
-  private static final
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SpaceQuotaStatus_descriptor;
-  private static final
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor;
-  private static final
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable;
-
-  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
+  public interface GetSpaceQuotaRegionSizesRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.GetSpaceQuotaRegionSizesRequest)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
-  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n\013Quota.proto\022\010hbase.pb\032\013HBase.proto\"\204\001\n" +
-      "\nTimedQuota\022%\n\ttime_unit\030\001 \002(\0162\022.hbase.p" +
-      "b.TimeUnit\022\022\n\nsoft_limit\030\002 \001(\004\022\r\n\005share\030" +
-      "\003 \001(\002\022,\n\005scope\030\004 \001(\0162\024.hbase.pb.QuotaSco" +
-      "pe:\007MACHINE\"\375\001\n\010Throttle\022%\n\007req_num\030\001 \001(" +
-      "\0132\024.hbase.pb.TimedQuota\022&\n\010req_size\030\002 \001(" +
-      "\0132\024.hbase.pb.TimedQuota\022\'\n\twrite_num\030\003 \001" +
-      "(\0132\024.hbase.pb.TimedQuota\022(\n\nwrite_size\030\004" +
-      " \001(\0132\024.hbase.pb.TimedQuota\022&\n\010read_num\030\005" +
-      " \001(\0132\024.hbase.pb.TimedQuota\022\'\n\tread_size\030",
-      "\006 \001(\0132\024.hbase.pb.TimedQuota\"b\n\017ThrottleR" +
-      "equest\022$\n\004type\030\001 \001(\0162\026.hbase.pb.Throttle" +
-      "Type\022)\n\013timed_quota\030\002 \001(\0132\024.hbase.pb.Tim" +
-      "edQuota\"r\n\006Quotas\022\035\n\016bypass_globals\030\001 \001(" +
-      "\010:\005false\022$\n\010throttle\030\002 \001(\0132\022.hbase.pb.Th" +
-      "rottle\022#\n\005space\030\003 \001(\0132\024.hbase.pb.SpaceQu" +
-      "ota\"\014\n\nQuotaUsage\"q\n\nSpaceQuota\022\022\n\nsoft_" +
-      "limit\030\001 \001(\004\0228\n\020violation_policy\030\002 \001(\0162\036." +
-      "hbase.pb.SpaceViolationPolicy\022\025\n\006remove\030" +
-      "\003 \001(\010:\005false\"8\n\021SpaceLimitRequest\022#\n\005quo",
-      "ta\030\001 \001(\0132\024.hbase.pb.SpaceQuota\"X\n\020SpaceQ" +
-      "uotaStatus\022.\n\006policy\030\001 \001(\0162\036.hbase.pb.Sp" +
-      "aceViolationPolicy\022\024\n\014in_violation\030\002 \001(\010" +
-      "\"^\n\022SpaceQuotaSnapshot\022*\n\006status\030\001 \001(\0132\032" +
-      ".hbase.pb.SpaceQuotaStatus\022\r\n\005usage\030\002 \001(" +
-      "\004\022\r\n\005limit\030\003 \001(\004*&\n\nQuotaScope\022\013\n\007CLUSTE" +
-      "R\020\001\022\013\n\007MACHINE\020\002*v\n\014ThrottleType\022\022\n\016REQU" +
-      "EST_NUMBER\020\001\022\020\n\014REQUEST_SIZE\020\002\022\020\n\014WRITE_" +
-      "NUMBER\020\003\022\016\n\nWRITE_SIZE\020\004\022\017\n\013READ_NUMBER\020" +
-      "\005\022\r\n\tREAD_SIZE\020\006*$\n\tQuotaType\022\014\n\010THROTTL",
-      "E\020\001\022\t\n\005SPACE\020\002*]\n\024SpaceViolationPolicy\022\013" +
-      "\n\007DISABLE\020\001\022\031\n\025NO_WRITES_COMPACTIONS\020\002\022\r" +
-      "\n\tNO_WRITES\020\003\022\016\n\nNO_INSERTS\020\004BH\n1org.apa" +
-      "che.hadoop.hbase.shaded.protobuf.generat" +
-      "edB\013QuotaProtosH\001\210\001\001\240\001\001"
-    };
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
-              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
-            descriptor = root;
-            return null;
+  /**
+   * Protobuf type {@code hbase.pb.GetSpaceQuotaRegionSizesRequest}
+   */
+  public  static final class GetSpaceQuotaRegionSizesRequest extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.GetSpaceQuotaRegionSizesRequest)
+      GetSpaceQuotaRegionSizesRequestOrBuilder {
+    // Use GetSpaceQuotaRegionSizesRequest.newBuilder() to construct.
+    private GetSpaceQuotaRegionSizesRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private GetSpaceQuotaRegionSizesRequest() {
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GetSpaceQuotaRegionSizesRequest(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
           }
-        };
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
-      .internalBuildGeneratedFileFrom(descriptorData,
-        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
-        }, assigner);
-    internal_static_hbase_pb_TimedQuota_descriptor =
-      getDescriptor().getMessageTypes().get(0);
-    internal_static_hbase_pb_TimedQuota_fieldAccessorTable = new
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_hbase_pb_TimedQuota_descriptor,
-        new java.lang.String[] { "TimeUnit", "SoftLimit", "Share", "Scope", });
-    internal_static_hbase_pb_Throttle_descriptor =
-      getDescriptor().getMessageTypes().get(1);
-    internal_static_hbase_pb_Throttle_fieldAccessorTable = new
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_hbase_pb_Throttle_descriptor,
-        new java.lang.String[] { "ReqNum", "ReqSize", "WriteNum", "WriteSize", "ReadNum", "ReadSize", });
-    internal_static_hbase_pb_ThrottleRequest_descriptor =
-      getDescriptor().getMessageTypes().get(2);
-    internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable = new
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_hbase_pb_ThrottleRequest_descriptor,
-        new java.lang.String[] { "Type", "TimedQuota", });
-    internal_static_hbase_pb_Quotas_descriptor =
-      getDescriptor().getMessageTypes().get(3);
-    internal_static_hbase_pb_Quotas_fieldAccessorTable = new
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_hbase_pb_Quotas_descriptor,
-        new java.lang.String[] { "BypassGlobals", "Throttle", "Space", });
-    internal_static_hbase_pb_QuotaUsage_descriptor =
-      getDescriptor().getMessageTypes().get(4);
-    internal_static_hbase_pb_QuotaUsage_fieldAccessorTable = new
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_hbase_pb_QuotaUsage_descriptor,
-        new java.lang.String[] { });
-    internal_static_hbase_pb_SpaceQuota_descriptor =
-      getDescriptor().getMessageTypes().get(5);
-    internal_static_hbase_pb_SpaceQuota_fieldAccessorTable = new
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_hbase_pb_SpaceQuota_descriptor,
-        new java.lang.String[] { "SoftLimit", "ViolationPolicy", "Remove", });
-    internal_static_hbase_pb_SpaceLimitRequest_descriptor =
-      getDescriptor().getMessageTypes().get(6);
-    internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable = new
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_hbase_pb_SpaceLimitRequest_descriptor,
-        new java.lang.String[] { "Quota", });
-    internal_static_hbase_pb_SpaceQuotaStatus_descriptor =
-      getDescriptor().getMessageTypes().get(7);
-    internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable = new
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_hbase_pb_SpaceQuotaStatus_descriptor,
-        new java.lang.String[] { "Policy", "InViolation", });
-    internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor =
-      getDescriptor().getMessageTypes().get(8);
-    internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable = new
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor,
-        new java.lang.String[] { "Status", "Usage", "Limit", });
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesRequest_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest.Builder.class);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest) obj;
+
+      boolean result = true;
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GetSpaceQuotaRegionSizesRequest}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.GetSpaceQuotaRegionSizesRequest)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequestOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesRequest_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest(this);
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest.getDefaultInstance()) return this;
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.GetSpaceQuotaRegionSizesRequest)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.GetSpaceQuotaRegionSizesRequest)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<GetSpaceQuotaRegionSizesRequest>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<GetSpaceQuotaRegionSizesRequest>() {
+      public GetSpaceQuotaRegionSizesRequest parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new GetSpaceQuotaRegionSizesRequest(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<GetSpaceQuotaRegionSizesRequest> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<GetSpaceQuotaRegionSizesRequest> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface GetSpaceQuotaRegionSizesResponseOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.GetSpaceQuotaRegionSizesResponse)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes> 
+        getSizesList();
+    /**
+     * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes getSizes(int index);
+    /**
+     * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
+     */
+    int getSizesCount();
+    /**
+     * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder> 
+        getSizesOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder getSizesOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.GetSpaceQuotaRegionSizesResponse}
+   */
+  public  static final class GetSpaceQuotaRegionSizesResponse extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.GetSpaceQuotaRegionSizesResponse)
+      GetSpaceQuotaRegionSizesResponseOrBuilder {
+    // Use GetSpaceQuotaRegionSizesResponse.newBuilder() to construct.
+    private GetSpaceQuotaRegionSizesResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private GetSpaceQuotaRegionSizesResponse() {
+      sizes_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GetSpaceQuotaRegionSizesResponse(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                sizes_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              sizes_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          sizes_ = java.util.Collections.unmodifiableList(sizes_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.Builder.class);
+    }
+
+    public interface RegionSizesOrBuilder extends
+        // @@protoc_insertion_point(interface_extends:hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      boolean hasTableName();
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName();
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+      /**
+       * <code>optional uint64 size = 2;</code>
+       */
+      boolean hasSize();
+      /**
+       * <code>optional uint64 size = 2;</code>
+       */
+      long getSize();
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes}
+     */
+    public  static final class RegionSizes extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+        // @@protoc_insertion_point(message_implements:hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes)
+        RegionSizesOrBuilder {
+      // Use RegionSizes.newBuilder() to construct.
+      private RegionSizes(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+        super(builder);
+      }
+      private RegionSizes() {
+        size_ = 0L;
+      }
+
+      @java.lang.Override
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+      getUnknownFields() {
+        return this.unknownFields;
+      }
+      private RegionSizes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        this();
+        int mutable_bitField0_ = 0;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+                if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                  subBuilder = tableName_.toBuilder();
+                }
+                tableName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+                if (subBuilder != null) {
+                  subBuilder.mergeFrom(tableName_);
+                  tableName_ = subBuilder.buildPartial();
+                }
+                bitField0_ |= 0x00000001;
+                break;
+              }
+              case 16: {
+                bitField0_ |= 0x00000002;
+                size_ = input.readUInt64();
+                break;
+              }
+            }
+          }
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+              e).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_RegionSizes_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_RegionSizes_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.Builder.class);
+      }
+
+      private int bitField0_;
+      public static final int TABLE_NAME_FIELD_NUMBER = 1;
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_;
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+        return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+      }
+
+      public static final int SIZE_FIELD_NUMBER = 2;
+      private long size_;
+      /**
+       * <code>optional uint64 size = 2;</code>
+       */
+      public boolean hasSize() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional uint64 size = 2;</code>
+       */
+      public long getSize() {
+        return size_;
+      }
+
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized == 1) return true;
+        if (isInitialized == 0) return false;
+
+        if (hasTableName()) {
+          if (!getTableName().isInitialized()) {
+            memoizedIsInitialized = 0;
+            return false;
+          }
+        }
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          output.writeMessage(1, getTableName());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeUInt64(2, size_);
+        }
+        unknownFields.writeTo(output);
+      }
+
+      public int getSerializedSize() {
+        int size = memoizedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+            .computeMessageSize(1, getTableName());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+            .computeUInt64Size(2, size_);
+        }
+        size += unknownFields.getSerializedSize();
+        memoizedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      public boolean equals(final java.lang.Object obj) {
+        if (obj == this) {
+         return true;
+        }
+        if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes)) {
+          return super.equals(obj);
+        }
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes other = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes) obj;
+
+        boolean result = true;
+        result = result && (hasTableName() == other.hasTableName());
+        if (hasTableName()) {
+          result = result && getTableName()
+              .equals(other.getTableName());
+        }
+        result = result && (hasSize() == other.hasSize());
+        if (hasSize()) {
+          result = result && (getSize()
+              == other.getSize());
+        }
+        result = result && unknownFields.equals(other.unknownFields);
+        return result;
+      }
+
+      @java.lang.Override
+      public int hashCode() {
+        if (memoizedHashCode != 0) {
+          return memoizedHashCode;
+        }
+        int hash = 41;
+        hash = (19 * hash) + getDescriptor().hashCode();
+        if (hasTableName()) {
+          hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+          hash = (53 * hash) + getTableName().hashCode();
+        }
+        if (hasSize()) {
+          hash = (37 * hash) + SIZE_FIELD_NUMBER;
+          hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+              getSize());
+        }
+        hash = (29 * hash) + unknownFields.hashCode();
+        memoizedHashCode = hash;
+        return hash;
+      }
+
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes parseFrom(byte[] data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes parseFrom(
+          byte[] data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes parseFrom(
+          java.io.InputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseDelimitedWithIOException(PARSER, input);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes parseDelimitedFrom(
+          java.io.InputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input);
+      }
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes parseFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+            .parseWithIOException(PARSER, input, extensionRegistry);
+      }
+
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder() {
+        return DEFAULT_INSTANCE.toBuilder();
+      }
+      public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes prototype) {
+        return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() {
+        return this == DEFAULT_INSTANCE
+            ? new Builder() : new Builder().mergeFrom(this);
+      }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+        return builder;
+      }
+      /**
+       * Protobuf type {@code hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes}
+       */
+      public static final class Builder extends
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+          // @@protoc_insertion_point(builder_implements:hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes)
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder {
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_RegionSizes_descriptor;
+        }
+
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_RegionSizes_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.Builder.class);
+        }
+
+        // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          super(parent);
+          maybeForceBuilderInitialization();
+        }
+        private void maybeForceBuilderInitialization() {
+          if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                  .alwaysUseFieldBuilders) {
+            getTableNameFieldBuilder();
+          }
+        }
+        public Builder clear() {
+          super.clear();
+          if (tableNameBuilder_ == null) {
+            tableName_ = null;
+          } else {
+            tableNameBuilder_.clear();
+          }
+          bitField0_ = (bitField0_ & ~0x00000001);
+          size_ = 0L;
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_RegionSizes_descriptor;
+        }
+
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes getDefaultInstanceForType() {
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.getDefaultInstance();
+        }
+
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes build() {
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes buildPartial() {
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes result = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          if (tableNameBuilder_ == null) {
+            result.tableName_ = tableName_;
+          } else {
+            result.tableName_ = tableNameBuilder_.build();
+          }
+          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          result.size_ = size_;
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder clone() {
+          return (Builder) super.clone();
+        }
+        public Builder setField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+            Object value) {
+          return (Builder) super.setField(field, value);
+        }
+        public Builder clearField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+          return (Builder) super.clearField(field);
+        }
+        public Builder clearOneof(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          return (Builder) super.clearOneof(oneof);
+        }
+        public Builder setRepeatedField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+            int index, Object value) {
+          return (Builder) super.setRepeatedField(field, index, value);
+        }
+        public Builder addRepeatedField(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+            Object value) {
+          return (Builder) super.addRepeatedField(field, value);
+        }
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+          if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes) {
+            return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes other) {
+          if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.getDefaultInstance()) return this;
+          if (other.hasTableName()) {
+            mergeTableName(other.getTableName());
+          }
+          if (other.hasSize()) {
+            setSize(other.getSize());
+          }
+          this.mergeUnknownFields(other.unknownFields);
+          onChanged();
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          if (hasTableName()) {
+            if (!getTableName().isInitialized()) {
+              return false;
+            }
+          }
+          return true;
+        }
+
+        public Builder mergeFrom(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes parsedMessage = null;
+          try {
+            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+          } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+            parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes) e.getUnfinishedMessage();
+            throw e.unwrapIOException();
+          } finally {
+            if (parsedMessage != null) {
+              mergeFrom(parsedMessage);
+            }
+          }
+          return this;
+        }
+        private int bitField0_;
+
+        private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+            org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public boolean hasTableName() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+          if (tableNameBuilder_ == null) {
+            return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+          } else {
+            return tableNameBuilder_.getMessage();
+          }
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public Builder setTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+          if (tableNameBuilder_ == null) {
+            if (value == null) {
+              throw new NullPointerException();
+            }
+            tableName_ = value;
+            onChanged();
+          } else {
+            tableNameBuilder_.setMessage(value);
+          }
+          bitField0_ |= 0x00000001;
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public Builder setTableName(
+            org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+          if (tableNameBuilder_ == null) {
+            tableName_ = builderForValue.build();
+            onChanged();
+          } else {
+            tableNameBuilder_.setMessage(builderForValue.build());
+          }
+          bitField0_ |= 0x00000001;
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public Builder mergeTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+          if (tableNameBuilder_ == null) {
+            if (((bitField0_ & 0x00000001) == 0x00000001) &&
+                tableName_ != null &&
+                tableName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+              tableName_ =
+                org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+            } else {
+              tableName_ = value;
+            }
+            onChanged();
+          } else {
+            tableNameBuilder_.mergeFrom(value);
+          }
+          bitField0_ |= 0x00000001;
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public Builder clearTableName() {
+          if (tableNameBuilder_ == null) {
+            tableName_ = null;
+            onChanged();
+          } else {
+            tableNameBuilder_.clear();
+          }
+          bitField0_ = (bitField0_ & ~0x00000001);
+          return this;
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+          bitField0_ |= 0x00000001;
+          onChanged();
+          return getTableNameFieldBuilder().getBuilder();
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+          if (tableNameBuilder_ != null) {
+            return tableNameBuilder_.getMessageOrBuilder();
+          } else {
+            return tableName_ == null ?
+                org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+          }
+        }
+        /**
+         * <code>optional .hbase.pb.TableName table_name = 1;</code>
+         */
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+            org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+            getTableNameFieldBuilder() {
+          if (tableNameBuilder_ == null) {
+            tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+                org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                    getTableName(),
+                    getParentForChildren(),
+                    isClean());
+            tableName_ = null;
+          }
+          return tableNameBuilder_;
+        }
+
+        private long size_ ;
+        /**
+         * <code>optional uint64 size = 2;</code>
+         */
+        public boolean hasSize() {
+          return ((bitField0_ & 0x00000002) == 0x00000002);
+        }
+        /**
+         * <code>optional uint64 size = 2;</code>
+         */
+        public long getSize() {
+          return size_;
+        }
+        /**
+         * <code>optional uint64 size = 2;</code>
+         */
+        public Builder setSize(long value) {
+          bitField0_ |= 0x00000002;
+          size_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>optional uint64 size = 2;</code>
+         */
+        public Builder clearSize() {
+          bitField0_ = (bitField0_ & ~0x00000002);
+          size_ = 0L;
+          onChanged();
+          return this;
+        }
+        public final Builder setUnknownFields(
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+          return super.setUnknownFields(unknownFields);
+        }
+
+        public final Builder mergeUnknownFields(
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+          return super.mergeUnknownFields(unknownFields);
+        }
+
+
+        // @@protoc_insertion_point(builder_scope:hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes)
+      }
+
+      // @@protoc_insertion_point(class_scope:hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes)
+      private static final org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes DEFAULT_INSTANCE;
+      static {
+        DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes();
+      }
+
+      public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes getDefaultInstance() {
+        return DEFAULT_INSTANCE;
+      }
+
+      @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSizes>
+          PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<RegionSizes>() {
+        public RegionSizes parsePartialFrom(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+            return new RegionSizes(input, extensionRegistry);
+        }
+      };
+
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSizes> parser() {
+        return PARSER;
+      }
+
+      @java.lang.Override
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<RegionSizes> getParserForType() {
+        return PARSER;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes getDefaultInstanceForType() {
+        return DEFAULT_INSTANCE;
+      }
+
+    }
+
+    public static final int SIZES_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes> sizes_;
+    /**
+     * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes> getSizesList() {
+      return sizes_;
+    }
+    /**
+     * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder> 
+        getSizesOrBuilderList() {
+      return sizes_;
+    }
+    /**
+     * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
+     */
+    public int getSizesCount() {
+      return sizes_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes getSizes(int index) {
+      return sizes_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder getSizesOrBuilder(
+        int index) {
+      return sizes_.get(index);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      for (int i = 0; i < getSizesCount(); i++) {
+        if (!getSizes(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      for (int i = 0; i < sizes_.size(); i++) {
+        output.writeMessage(1, sizes_.get(i));
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < sizes_.size(); i++) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, sizes_.get(i));
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse) obj;
+
+      boolean result = true;
+      result = result && getSizesList()
+          .equals(other.getSizesList());
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      if (getSizesCount() > 0) {
+        hash = (37 * hash) + SIZES_FIELD_NUMBER;
+        hash = (53 * hash) + getSizesList().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GetSpaceQuotaRegionSizesResponse}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.GetSpaceQuotaRegionSizesResponse)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponseOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getSizesFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (sizesBuilder_ == null) {
+          sizes_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          sizesBuilder_.clear();
+        }
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse(this);
+        int from_bitField0_ = bitField0_;
+        if (sizesBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            sizes_ = java.util.Collections.unmodifiableList(sizes_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.sizes_ = sizes_;
+        } else {
+          result.sizes_ = sizesBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.getDefaultInstance()) return this;
+        if (sizesBuilder_ == null) {
+          if (!other.sizes_.isEmpty()) {
+            if (sizes_.isEmpty()) {
+              sizes_ = other.sizes_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensureSizesIsMutable();
+              sizes_.addAll(other.sizes_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.sizes_.isEmpty()) {
+            if (sizesBuilder_.isEmpty()) {
+              sizesBuilder_.dispose();
+              sizesBuilder_ = null;
+              sizes_ = other.sizes_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              sizesBuilder_ = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getSizesFieldBuilder() : null;
+            } else {
+              sizesBuilder_.addAllMessages(other.sizes_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        for (int i = 0; i < getSizesCount(); i++) {
+          if (!getSizes(i).isInitialized()) {
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes> sizes_ =
+        java.util.Collections.emptyList();
+      private void ensureSizesIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          sizes_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes>(sizes_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder> sizesBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes> getSizesList() {
+        if (sizesBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(sizes_);
+        } else {
+          return sizesBuilder_.getMessageList()

<TRUNCATED>