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/28 17:37:17 UTC

[1/4] hbase git commit: HBASE-17955 Various reviewboard improvements to space quota work

Repository: hbase
Updated Branches:
  refs/heads/HBASE-16961 cb08814a4 -> 70bcf3fe6


http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 a313fa1..c558b26 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
@@ -17,6 +17,7 @@
 package org.apache.hadoop.hbase.quotas;
 
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Objects;
 
@@ -28,7 +29,12 @@ 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.
+ * being enforced. This class is immutable, expect for {@code locallyCachedPolicies}.
+ *
+ * The {@code locallyCachedPolicies} are mutable given the current {@code activePolicies}
+ * and {@code snapshots}. It is expected that when a new instance of this class is
+ * instantiated, we also want to invalidate those previously cached policies (as they
+ * may now be invalidate if we received new quota usage information).
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -36,12 +42,23 @@ public class ActivePolicyEnforcement {
   private final Map<TableName,SpaceViolationPolicyEnforcement> activePolicies;
   private final Map<TableName,SpaceQuotaSnapshot> snapshots;
   private final RegionServerServices rss;
+  private final SpaceViolationPolicyEnforcementFactory factory;
+  private final Map<TableName,SpaceViolationPolicyEnforcement> locallyCachedPolicies;
 
   public ActivePolicyEnforcement(Map<TableName,SpaceViolationPolicyEnforcement> activePolicies,
       Map<TableName,SpaceQuotaSnapshot> snapshots, RegionServerServices rss) {
+    this(activePolicies, snapshots, rss, SpaceViolationPolicyEnforcementFactory.getInstance());
+  }
+
+  public ActivePolicyEnforcement(Map<TableName,SpaceViolationPolicyEnforcement> activePolicies,
+      Map<TableName,SpaceQuotaSnapshot> snapshots, RegionServerServices rss,
+      SpaceViolationPolicyEnforcementFactory factory) {
     this.activePolicies = activePolicies;
     this.snapshots = snapshots;
     this.rss = rss;
+    this.factory = factory;
+    // Mutable!
+    this.locallyCachedPolicies = new HashMap<>();
   }
 
   /**
@@ -65,16 +82,25 @@ public class ActivePolicyEnforcement {
    */
   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();
+    if (policy == null) {
+      synchronized (locallyCachedPolicies) {
+        // When we don't have an policy enforcement for the table, there could be one of two cases:
+        //  1) The table has no quota defined
+        //  2) The table is not in violation of its quota
+        // In both of these cases, we want to make sure that access remains fast and we minimize
+        // object creation. We can accomplish this by locally caching policies instead of creating
+        // a new instance of the policy each time.
+        policy = locallyCachedPolicies.get(tableName);
+        // We have already created/cached the enforcement, use it again. `activePolicies` and
+        // `snapshots` are immutable, thus this policy is valid for the lifetime of `this`.
+        if (policy != null) {
+          return policy;
         }
-        // Create the default policy and cache it
-        return SpaceViolationPolicyEnforcementFactory.getInstance().createWithoutViolation(
-            rss, tableName, snapshot);
+        // Create a PolicyEnforcement for this table and snapshot. The snapshot may be null
+        // which is OK.
+        policy = factory.createWithoutViolation(rss, tableName, snapshots.get(tableName));
+        // Cache the policy we created
+        locallyCachedPolicies.put(tableName, policy);
       }
     }
     return policy;
@@ -87,6 +113,14 @@ public class ActivePolicyEnforcement {
     return Collections.unmodifiableMap(activePolicies);
   }
 
+  /**
+   * Returns an unmodifiable version of the policy enforcements that were cached because they are
+   * not in violation of their quota.
+   */
+  Map<TableName,SpaceViolationPolicyEnforcement> getLocallyCachedPolicies() {
+    return Collections.unmodifiableMap(locallyCachedPolicies);
+  }
+
   @Override
   public String toString() {
     return getClass().getSimpleName() + ": " + activePolicies;

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 efc17ff..418a163 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
@@ -81,7 +81,7 @@ public class FileSystemUtilizationChore extends ScheduledChore {
     Iterator<Region> oldRegionsToProcess = getLeftoverRegions();
     final Iterator<Region> iterator;
     final boolean processingLeftovers;
-    if (null == oldRegionsToProcess) {
+    if (oldRegionsToProcess == null) {
       iterator = onlineRegions.iterator();
       processingLeftovers = false;
     } else {
@@ -179,6 +179,8 @@ public class FileSystemUtilizationChore extends ScheduledChore {
    * Reports the computed region sizes to the currently active Master.
    *
    * @param onlineRegionSizes The computed region sizes to report.
+   * @return {@code false} if FileSystemUtilizationChore should pause reporting to master,
+   *    {@code true} otherwise.
    */
   boolean reportRegionSizesToMaster(Map<HRegionInfo,Long> onlineRegionSizes) {
     return this.rs.reportRegionSizesForQuotas(onlineRegionSizes);

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 0622dba..1fb8cf4 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
@@ -580,19 +580,19 @@ public class MasterQuotaManager implements RegionStateListener {
 
   @VisibleForTesting
   void initializeRegionSizes() {
-    assert null == regionSizes;
+    assert regionSizes == null;
     this.regionSizes = new ConcurrentHashMap<>();
   }
 
   public void addRegionSize(HRegionInfo hri, long size, long time) {
-    if (null == regionSizes) {
+    if (regionSizes == null) {
       return;
     }
     regionSizes.put(hri, new SizeSnapshotWithTimestamp(size, time));
   }
 
   public Map<HRegionInfo, Long> snapshotRegionSizes() {
-    if (null == regionSizes) {
+    if (regionSizes == null) {
       return EMPTY_MAP;
     }
 
@@ -604,7 +604,7 @@ public class MasterQuotaManager implements RegionStateListener {
   }
 
   int pruneEntriesOlderThan(long timeToPruneBefore) {
-    if (null == regionSizes) {
+    if (regionSizes == null) {
       return 0;
     }
     int numEntriesRemoved = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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
index a3abf32..299ba39 100644
--- 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
@@ -57,7 +57,7 @@ public class MasterSpaceQuotaObserver implements MasterObserver {
     final MasterServices master = ctx.getEnvironment().getMasterServices();
     final Connection conn = master.getConnection();
     Quotas quotas = QuotaUtil.getTableQuota(master.getConnection(), tableName);
-    if (null != quotas && quotas.hasSpace()) {
+    if (quotas != null && quotas.hasSpace()) {
       QuotaSettings settings = QuotaSettingsFactory.removeTableSpaceLimit(tableName);
       try (Admin admin = conn.getAdmin()) {
         admin.setQuota(settings);
@@ -75,7 +75,7 @@ public class MasterSpaceQuotaObserver implements MasterObserver {
     final MasterServices master = ctx.getEnvironment().getMasterServices();
     final Connection conn = master.getConnection();
     Quotas quotas = QuotaUtil.getNamespaceQuota(master.getConnection(), namespace);
-    if (null != quotas && quotas.hasSpace()) {
+    if (quotas != null && 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/70bcf3fe/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
index 75550f3..f93d33d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
@@ -57,7 +57,7 @@ public class NamespaceQuotaSnapshotStore implements QuotaSnapshotStore<String> {
   @Override
   public SpaceQuota getSpaceQuota(String namespace) throws IOException {
     Quotas quotas = getQuotaForNamespace(namespace);
-    if (null != quotas && quotas.hasSpace()) {
+    if (quotas != null && quotas.hasSpace()) {
       return quotas.getSpace();
     }
     return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 254f2a1..4404b27 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
@@ -136,7 +136,7 @@ public class QuotaObserverChore extends ScheduledChore {
       }
       long start = System.nanoTime();
       _chore();
-      if (null != metrics) {
+      if (metrics != null) {
         metrics.incrementQuotaObserverTime((System.nanoTime() - start) / 1_000_000);
       }
     } catch (IOException e) {
@@ -152,7 +152,7 @@ public class QuotaObserverChore extends ScheduledChore {
       LOG.trace("Found following tables with quotas: " + tablesWithQuotas);
     }
 
-    if (null != metrics) {
+    if (metrics != null) {
       // Set the number of namespaces and tables with quotas defined
       metrics.setNumSpaceQuotas(tablesWithQuotas.getTableQuotaTables().size()
           + tablesWithQuotas.getNamespacesWithQuotas().size());
@@ -170,7 +170,7 @@ 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) {
+    if (metrics != null) {
       metrics.setNumRegionSizeReports(reportedRegionSpaceUse.size());
     }
 
@@ -216,12 +216,12 @@ public class QuotaObserverChore extends ScheduledChore {
 
   void initializeSnapshotStores(Map<HRegionInfo,Long> regionSizes) {
     Map<HRegionInfo,Long> immutableRegionSpaceUse = Collections.unmodifiableMap(regionSizes);
-    if (null == tableSnapshotStore) {
+    if (tableSnapshotStore == null) {
       tableSnapshotStore = new TableQuotaSnapshotStore(conn, this, immutableRegionSpaceUse);
     } else {
       tableSnapshotStore.setRegionUsage(immutableRegionSpaceUse);
     }
-    if (null == namespaceSnapshotStore) {
+    if (namespaceSnapshotStore == null) {
       namespaceSnapshotStore = new NamespaceQuotaSnapshotStore(
           conn, this, immutableRegionSpaceUse);
     } else {
@@ -239,7 +239,7 @@ public class QuotaObserverChore extends ScheduledChore {
     long numTablesInViolation = 0L;
     for (TableName table : tablesWithTableQuotas) {
       final SpaceQuota spaceQuota = tableSnapshotStore.getSpaceQuota(table);
-      if (null == spaceQuota) {
+      if (spaceQuota == null) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Unexpectedly did not find a space quota for " + table
               + ", maybe it was recently deleted.");
@@ -259,7 +259,7 @@ public class QuotaObserverChore extends ScheduledChore {
       }
     }
     // Report the number of tables in violation
-    if (null != metrics) {
+    if (metrics != null) {
       metrics.setNumTableInSpaceQuotaViolation(numTablesInViolation);
     }
   }
@@ -281,7 +281,7 @@ public class QuotaObserverChore extends ScheduledChore {
     for (String namespace : namespacesWithQuotas) {
       // Get the quota definition for the namespace
       final SpaceQuota spaceQuota = namespaceSnapshotStore.getSpaceQuota(namespace);
-      if (null == spaceQuota) {
+      if (spaceQuota == null) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Could not get Namespace space quota for " + namespace
               + ", maybe it was recently deleted.");
@@ -303,7 +303,7 @@ public class QuotaObserverChore extends ScheduledChore {
     }
 
     // Report the number of namespaces in violation
-    if (null != metrics) {
+    if (metrics != null) {
       metrics.setNumNamespacesInSpaceQuotaViolation(numNamespacesInViolation);
     }
   }
@@ -451,9 +451,8 @@ public class QuotaObserverChore extends ScheduledChore {
    */
   TablesWithQuotas fetchAllTablesWithQuotasDefined() throws IOException {
     final Scan scan = QuotaTableUtil.makeScan(null);
-    final QuotaRetriever scanner = new QuotaRetriever();
     final TablesWithQuotas tablesWithQuotas = new TablesWithQuotas(conn, conf);
-    try {
+    try (final QuotaRetriever scanner = new QuotaRetriever()) {
       scanner.init(conn, scan);
       for (QuotaSettings quotaSettings : scanner) {
         // Only one of namespace and tablename should be 'null'
@@ -463,11 +462,10 @@ public class QuotaObserverChore extends ScheduledChore {
           continue;
         }
 
-        if (null != namespace) {
-          assert null == tableName;
+        if (namespace != null) {
+          assert tableName == null;
           // Collect all of the tables in the namespace
-          TableName[] tablesInNS = conn.getAdmin()
-              .listTableNamesByNamespace(namespace);
+          TableName[] tablesInNS = conn.getAdmin().listTableNamesByNamespace(namespace);
           for (TableName tableUnderNs : tablesInNS) {
             if (LOG.isTraceEnabled()) {
               LOG.trace("Adding " + tableUnderNs + " under " +  namespace
@@ -476,7 +474,7 @@ public class QuotaObserverChore extends ScheduledChore {
             tablesWithQuotas.addNamespaceQuotaTable(tableUnderNs);
           }
         } else {
-          assert null != tableName;
+          assert tableName != null;
           if (LOG.isTraceEnabled()) {
             LOG.trace("Adding " + tableName + " as having table quota.");
           }
@@ -485,10 +483,6 @@ public class QuotaObserverChore extends ScheduledChore {
         }
       }
       return tablesWithQuotas;
-    } finally {
-      if (null != scanner) {
-        scanner.close();
-      }
     }
   }
 
@@ -504,7 +498,7 @@ public class QuotaObserverChore extends ScheduledChore {
 
   /**
    * Returns an unmodifiable view over the current {@link SpaceQuotaSnapshot} objects
-   * for each HBase table with a quota.
+   * for each HBase table with a quota defined.
    */
   public Map<TableName,SpaceQuotaSnapshot> getTableQuotaSnapshots() {
     return readOnlyTableQuotaSnapshots;
@@ -512,7 +506,7 @@ public class QuotaObserverChore extends ScheduledChore {
 
   /**
    * Returns an unmodifiable view over the current {@link SpaceQuotaSnapshot} objects
-   * for each HBase namespace with a quota.
+   * for each HBase namespace with a quota defined.
    */
   public Map<String,SpaceQuotaSnapshot> getNamespaceQuotaSnapshots() {
     return readOnlyNamespaceSnapshots;
@@ -522,9 +516,8 @@ public class QuotaObserverChore extends ScheduledChore {
    * Fetches the {@link SpaceQuotaSnapshot} for the given table.
    */
   SpaceQuotaSnapshot getTableQuotaSnapshot(TableName table) {
-    // TODO Can one instance of a Chore be executed concurrently?
     SpaceQuotaSnapshot state = this.tableQuotaSnapshots.get(table);
-    if (null == state) {
+    if (state == null) {
       // No tracked state implies observance.
       return QuotaSnapshotStore.NO_QUOTA;
     }
@@ -539,12 +532,11 @@ public class QuotaObserverChore extends ScheduledChore {
   }
 
   /**
-   * Fetches the {@link SpaceQuotaSnapshot} for the given namespace.
+   * Fetches the {@link SpaceQuotaSnapshot} for the given namespace from this chore.
    */
   SpaceQuotaSnapshot getNamespaceQuotaSnapshot(String namespace) {
-    // TODO Can one instance of a Chore be executed concurrently?
     SpaceQuotaSnapshot state = this.namespaceQuotaSnapshots.get(namespace);
-    if (null == state) {
+    if (state == null) {
       // No tracked state implies observance.
       return QuotaSnapshotStore.NO_QUOTA;
     }
@@ -552,7 +544,7 @@ public class QuotaObserverChore extends ScheduledChore {
   }
 
   /**
-   * Stores the quota state for the given namespace.
+   * Stores the given {@code snapshot} for the given {@code namespace} in this chore.
    */
   void setNamespaceQuotaSnapshot(String namespace, SpaceQuotaSnapshot snapshot) {
     this.namespaceQuotaSnapshots.put(namespace, snapshot);
@@ -562,7 +554,8 @@ public class QuotaObserverChore extends ScheduledChore {
    * Extracts the period for the chore from the configuration.
    *
    * @param conf The configuration object.
-   * @return The configured chore period or the default value.
+   * @return The configured chore period or the default value in the given timeunit.
+   * @see #getTimeUnit(Configuration)
    */
   static int getPeriod(Configuration conf) {
     return conf.getInt(QUOTA_OBSERVER_CHORE_PERIOD_KEY,
@@ -573,7 +566,8 @@ public class QuotaObserverChore extends ScheduledChore {
    * 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.
+   * @return The configured chore initial delay or the default value in the given timeunit.
+   * @see #getTimeUnit(Configuration)
    */
   static long getInitialDelay(Configuration conf) {
     return conf.getLong(QUOTA_OBSERVER_CHORE_DELAY_KEY,
@@ -606,8 +600,8 @@ public class QuotaObserverChore extends ScheduledChore {
   }
 
   /**
-   * 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.
+   * A container which encapsulates the tables that have either a table quota or are contained in a
+   * namespace which have a namespace quota.
    */
   static class TablesWithQuotas {
     private final Set<TableName> tablesWithTableQuotas = new HashSet<>();
@@ -702,7 +696,7 @@ public class QuotaObserverChore extends ScheduledChore {
         }
         final int numRegionsInTable = getNumRegions(table);
         // If the table doesn't exist (no regions), bail out.
-        if (0 == numRegionsInTable) {
+        if (numRegionsInTable == 0) {
           if (LOG.isTraceEnabled()) {
             LOG.trace("Filtering " + table + " because no regions were reported");
           }
@@ -734,7 +728,7 @@ public class QuotaObserverChore extends ScheduledChore {
      */
     int getNumRegions(TableName table) throws IOException {
       List<HRegionInfo> regions = this.conn.getAdmin().getTableRegions(table);
-      if (null == regions) {
+      if (regions == null) {
         return 0;
       }
       return regions.size();

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 1c82808..86bdf08 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
@@ -85,7 +85,7 @@ public class RegionServerSpaceQuotaManager {
   }
 
   public synchronized void stop() {
-    if (null != spaceQuotaRefresher) {
+    if (spaceQuotaRefresher != null) {
       spaceQuotaRefresher.cancel();
       spaceQuotaRefresher = null;
     }
@@ -133,7 +133,7 @@ public class RegionServerSpaceQuotaManager {
     final Map<TableName, SpaceQuotaSnapshot> policies = new HashMap<>();
     for (Entry<TableName, SpaceViolationPolicyEnforcement> entry : enforcements.entrySet()) {
       final SpaceQuotaSnapshot snapshot = entry.getValue().getQuotaSnapshot();
-      if (null != snapshot) {
+      if (snapshot != null) {
         policies.put(entry.getKey(), snapshot);
       }
     }
@@ -158,9 +158,10 @@ public class RegionServerSpaceQuotaManager {
     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.
+    // HBASE-XXXX: 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. This is probably OK, but not sure if
+    // it would become a bottleneck at large clusters/number of tables.
     synchronized (enforcedPolicies) {
       try {
         enforcement.enable();
@@ -181,10 +182,9 @@ public class RegionServerSpaceQuotaManager {
       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) {
+      if (enforcement != null) {
         try {
           enforcement.disable();
         } catch (IOException e) {
@@ -205,7 +205,7 @@ public class RegionServerSpaceQuotaManager {
    */
   public boolean areCompactionsDisabled(TableName tableName) {
     SpaceViolationPolicyEnforcement enforcement = this.enforcedPolicies.get(Objects.requireNonNull(tableName));
-    if (null != enforcement) {
+    if (enforcement != null) {
       return enforcement.areCompactionsDisabled();
     }
     return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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
index 904903f..e4262c3 100644
--- 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
@@ -39,7 +39,7 @@ public class SpaceLimitingException extends QuotaExceededException {
 
     // Hack around ResponseConverter expecting to invoke a single-arg String constructor
     // on this class
-    if (null != msg) {
+    if (msg != null) {
       for (SpaceViolationPolicy definedPolicy : SpaceViolationPolicy.values()) {
         if (msg.indexOf(definedPolicy.name()) != -1) {
           policyName = definedPolicy.name();
@@ -74,7 +74,7 @@ public class SpaceLimitingException extends QuotaExceededException {
     // 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 (originalMessage != null && originalMessage.startsWith(MESSAGE_PREFIX)) {
       // If it starts with the class name, rip off the policy too.
       try {
         int index = originalMessage.indexOf(' ', MESSAGE_PREFIX.length());
@@ -90,6 +90,6 @@ public class SpaceLimitingException extends QuotaExceededException {
 
   @Override
   public String getMessage() {
-    return (null == policyName ? "(unknown policy)" : policyName) + " " + super.getMessage();
+    return (policyName == null ? "(unknown policy)" : policyName) + " " + super.getMessage();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 8587e79..5adb9ca 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
@@ -133,7 +133,7 @@ public class SpaceQuotaRefresherChore extends ScheduledChore {
    * @return true if the snapshot is in violation, false otherwise.
    */
   boolean isInViolation(SpaceQuotaSnapshot snapshot) {
-    if (null == snapshot) {
+    if (snapshot == null) {
       return false;
     }
     return snapshot.getQuotaStatus().isInViolation();

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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
index 6b754b9..4f1551f 100644
--- 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
@@ -20,8 +20,9 @@ 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.DefaultViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.quotas.policies.DisableTableViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.MissingSnapshotViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.quotas.policies.NoInsertsViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.quotas.policies.NoWritesCompactionsViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement;
@@ -79,16 +80,29 @@ public class SpaceViolationPolicyEnforcementFactory {
 
   /**
    * 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.
+   * violation. This is used to have uniform policy checking for tables in and not quotas. This
+   * policy will still verify that new bulk loads do not exceed the configured quota limit.
+   *
+   * @param rss RegionServerServices instance the policy enforcement should use.
+   * @param tableName The target HBase table.
+   * @param snapshot The current quota snapshot for the {@code tableName}, can be null.
    */
   public SpaceViolationPolicyEnforcement createWithoutViolation(
       RegionServerServices rss, TableName tableName, SpaceQuotaSnapshot snapshot) {
+    if (snapshot == null) {
+      // If we have no snapshot, this is equivalent to no quota for this table.
+      // We should do use the (singleton instance) of this policy to do nothing.
+      return MissingSnapshotViolationPolicyEnforcement.getInstance();
+    }
+    // We have a snapshot which means that there is a quota set on this table, but it's not in
+    // violation of that quota. We need to construct a policy for this table.
     SpaceQuotaStatus status = snapshot.getQuotaStatus();
     if (status.isInViolation()) {
       throw new IllegalArgumentException(
           tableName + " is in violation. Logic error. Snapshot=" + snapshot);
     }
-    BulkLoadVerifyingViolationPolicyEnforcement enforcement = new BulkLoadVerifyingViolationPolicyEnforcement();
+    // We have a unique size snapshot to use. Create an instance for this tablename + snapshot.
+    DefaultViolationPolicyEnforcement enforcement = new DefaultViolationPolicyEnforcement();
     enforcement.initialize(rss, tableName, snapshot);
     return enforcement;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 82d3684..1abf347 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
@@ -58,7 +58,7 @@ public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
   @Override
   public SpaceQuota getSpaceQuota(TableName subject) throws IOException {
     Quotas quotas = getQuotaForTable(subject);
-    if (null != quotas && quotas.hasSpace()) {
+    if (quotas != null && quotas.hasSpace()) {
       return quotas.getSpace();
     }
     return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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
index 2d34d45..981dfd1 100644
--- 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
@@ -16,17 +16,11 @@
  */
 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;
@@ -69,7 +63,8 @@ public abstract class AbstractViolationPolicyEnforcement
   }
 
   @Override
-  public void initialize(RegionServerServices rss, TableName tableName, SpaceQuotaSnapshot snapshot) {
+  public void initialize(
+      RegionServerServices rss, TableName tableName, SpaceQuotaSnapshot snapshot) {
     setRegionServerServices(rss);
     setTableName(tableName);
     setQuotaSnapshot(snapshot);
@@ -79,40 +74,4 @@ public abstract class AbstractViolationPolicyEnforcement
   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/70bcf3fe/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
deleted file mode 100644
index e4171ad..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/BulkLoadVerifyingViolationPolicyEnforcement.java
+++ /dev/null
@@ -1,50 +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.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/70bcf3fe/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DefaultViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DefaultViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DefaultViolationPolicyEnforcement.java
new file mode 100644
index 0000000..f0c4b53
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DefaultViolationPolicyEnforcement.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.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.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+
+/**
+ * The default implementation for {@link SpaceViolationPolicyEnforcement}. This is done because all
+ * tables, whether or not they're in violation now, should be checking bulk loads to proactively
+ * catch a swell of files that would push the table into violation.
+ */
+@InterfaceAudience.Private
+public class DefaultViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+
+  @Override
+  public void enable() throws IOException {}
+
+  @Override
+  public void disable() throws IOException {}
+
+  @Override
+  public String getPolicyName() {
+    return "BulkLoadVerifying";
+  }
+
+  @Override
+  public void check(Mutation m) throws SpaceLimitingException {}
+
+  @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/70bcf3fe/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
index 0d6d886..eb37866 100644
--- 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
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
  * counterpart to {@link SpaceViolationPolicy#DISABLE}.
  */
 @InterfaceAudience.Private
-public class DisableTableViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+public class DisableTableViolationPolicyEnforcement extends DefaultViolationPolicyEnforcement {
   private static final Log LOG = LogFactory.getLog(DisableTableViolationPolicyEnforcement.class);
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/MissingSnapshotViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/MissingSnapshotViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/MissingSnapshotViolationPolicyEnforcement.java
new file mode 100644
index 0000000..d0e4b16
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/MissingSnapshotViolationPolicyEnforcement.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas.policies;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+
+/**
+ * A {@link SpaceViolationPolicyEnforcement} which can be treated as a singleton. When a quota is
+ * not defined on a table or we lack quota information, we want to avoid creating a policy, keeping
+ * this path fast.
+ */
+public class MissingSnapshotViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+  private static final MissingSnapshotViolationPolicyEnforcement SINGLETON =
+      new MissingSnapshotViolationPolicyEnforcement();
+
+  private MissingSnapshotViolationPolicyEnforcement() {}
+
+  public static SpaceViolationPolicyEnforcement getInstance() {
+    return SINGLETON;
+  }
+
+  @Override
+  public boolean shouldCheckBulkLoads() {
+    return false;
+  }
+
+  @Override
+  public void checkBulkLoad(FileSystem fs, List<String> paths) {}
+
+  @Override
+  public void enable() throws IOException {}
+
+  @Override
+  public void disable() throws IOException {}
+
+  @Override
+  public void check(Mutation m) throws SpaceLimitingException {}
+
+  @Override
+  public String getPolicyName() {
+    return "NoQuota";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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
index a60cb45..ed17cac 100644
--- 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
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
  * enforcement counterpart to {@link SpaceViolationPolicy#NO_INSERTS}.
  */
 @InterfaceAudience.Private
-public class NoInsertsViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+public class NoInsertsViolationPolicyEnforcement extends DefaultViolationPolicyEnforcement {
 
   @Override
   public void enable() {}

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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
index a04f418..2ceb051 100644
--- 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
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
  * into HBase. The enforcement counterpart to {@link SpaceViolationPolicy#NO_WRITES}.
  */
 @InterfaceAudience.Private
-public class NoWritesViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
+public class NoWritesViolationPolicyEnforcement extends DefaultViolationPolicyEnforcement {
 
   @Override
   public void enable() {}

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 9aa0042..cfd1db1 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
@@ -340,7 +340,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
 
     final RegionServerSpaceQuotaManager spaceQuotaManager =
       this.server.getRegionServerSpaceQuotaManager();
-    if (null != spaceQuotaManager && spaceQuotaManager.areCompactionsDisabled(
+    if (spaceQuotaManager != null && spaceQuotaManager.areCompactionsDisabled(
         r.getTableDesc().getTableName())) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Ignoring compaction request for " + r + " as an active space quota violation "

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 a50c8c1..3ca061a 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
@@ -1324,8 +1324,8 @@ public class HRegionServer extends HasThread implements
    */
   RegionSpaceUse convertRegionSize(HRegionInfo regionInfo, Long sizeInBytes) {
     return RegionSpaceUse.newBuilder()
-        .setRegion(HRegionInfo.convert(Objects.requireNonNull(regionInfo)))
-        .setSize(Objects.requireNonNull(sizeInBytes))
+        .setRegionInfo(HRegionInfo.convert(Objects.requireNonNull(regionInfo)))
+        .setRegionSize(Objects.requireNonNull(sizeInBytes))
         .build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 1e4bab2..5dfccae 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
@@ -2169,8 +2169,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       // 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) {
+        SpaceViolationPolicyEnforcement enforcement = activeSpaceQuotas.getPolicyEnforcement(
+            region);
+        if (enforcement != null) {
           // Bulk loads must still be atomic. We must enact all or none.
           List<String> filePaths = new ArrayList<>(request.getFamilyPathCount());
           for (FamilyPath familyPath : request.getFamilyPathList()) {
@@ -3296,7 +3297,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           regionServer.getRegionServerSpaceQuotaManager();
       final GetSpaceQuotaSnapshotsResponse.Builder builder =
           GetSpaceQuotaSnapshotsResponse.newBuilder();
-      if (null != manager) {
+      if (manager != null) {
         final Map<TableName,SpaceQuotaSnapshot> snapshots = manager.copyQuotaSnapshots();
         for (Entry<TableName,SpaceQuotaSnapshot> snapshot : snapshots.entrySet()) {
           builder.addSnapshots(TableQuotaSnapshot.newBuilder()
@@ -3320,7 +3321,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           regionServer.getRegionServerSpaceQuotaManager();
       final GetSpaceQuotaEnforcementsResponse.Builder builder =
           GetSpaceQuotaEnforcementsResponse.newBuilder();
-      if (null != manager) {
+      if (manager != null) {
         ActivePolicyEnforcement enforcements = manager.getActiveEnforcements();
         for (Entry<TableName,SpaceViolationPolicyEnforcement> enforcement
             : enforcements.getPolicies().entrySet()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 c62d3a6..fd656c6 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -338,15 +338,15 @@ if ( fqtn != null ) {
     TableName tn = TableName.valueOf(fqtn);
     SpaceQuotaSnapshot masterSnapshot = null;
     Quotas quota = QuotaTableUtil.getTableQuota(master.getConnection(), tn);
-    if (null == quota || !quota.hasSpace()) {
+    if (quota == null || !quota.hasSpace()) {
       quota = QuotaTableUtil.getNamespaceQuota(master.getConnection(), tn.getNamespaceAsString());
-      if (null != quota) {
+      if (quota != null) {
         masterSnapshot = QuotaTableUtil.getCurrentSnapshot(master.getConnection(), tn.getNamespaceAsString());
       }
     } else {
       masterSnapshot = QuotaTableUtil.getCurrentSnapshot(master.getConnection(), tn);
     }
-    if (null != quota && quota.hasSpace()) {
+    if (quota != null && quota.hasSpace()) {
       SpaceQuota spaceQuota = quota.getSpace();
 %>
   <tr>
@@ -366,7 +366,7 @@ if ( fqtn != null ) {
           <td><%= spaceQuota.getViolationPolicy() %></td>
         </tr>
 <%
-      if (null != masterSnapshot) {
+      if (masterSnapshot != null) {
 %>
         <tr>
           <td>Usage</td>

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
index 888978d..b7c51a2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
@@ -34,6 +34,7 @@ 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.Waiter.Predicate;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
@@ -43,6 +44,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.rules.TestName;
 
 import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
 
 @InterfaceAudience.Private
@@ -69,6 +71,64 @@ public class SpaceQuotaHelperForTests {
   // Helpers
   //
 
+  /**
+   * Returns the number of quotas defined in the HBase quota table.
+   */
+  long listNumDefinedQuotas(Connection conn) throws IOException {
+    QuotaRetriever scanner = QuotaRetriever.open(conn.getConfiguration());
+    try {
+      return Iterables.size(scanner);
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
+    }
+  }
+
+  /**
+   * Removes all quotas defined in the HBase quota table.
+   */
+  void removeAllQuotas(Connection conn) throws IOException {
+    QuotaRetriever scanner = QuotaRetriever.open(conn.getConfiguration());
+    try {
+      for (QuotaSettings quotaSettings : scanner) {
+        final String namespace = quotaSettings.getNamespace();
+        final TableName tableName = quotaSettings.getTableName();
+        if (namespace != null) {
+          LOG.debug("Deleting quota for namespace: " + namespace);
+          QuotaUtil.deleteNamespaceQuota(conn, namespace);
+        } else {
+          assert tableName != null;
+          LOG.debug("Deleting quota for table: "+ tableName);
+          QuotaUtil.deleteTableQuota(conn, tableName);
+        }
+      }
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
+    }
+  }
+
+  /**
+   * Waits 30seconds for the HBase quota table to exist.
+   */
+  void waitForQuotaTable(Connection conn) throws IOException {
+    waitForQuotaTable(conn, 30_000);
+  }
+
+  /**
+   * Waits {@code timeout} milliseconds for the HBase quota table to exist.
+   */
+  void waitForQuotaTable(Connection conn, long timeout) throws IOException {
+    testUtil.waitFor(timeout, 1000, new Predicate<IOException>() {
+      @Override
+      public boolean evaluate() throws IOException {
+        return conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME);
+      }
+    });
+  }
+
   void writeData(TableName tn, long sizeInBytes) throws IOException {
     final Connection conn = testUtil.getConnection();
     final Table table = conn.getTable(tn);
@@ -213,14 +273,14 @@ public class SpaceQuotaHelperForTests {
     for (Entry<TableName, QuotaSettings> entry : quotas.entries()) {
       SpaceLimitSettings settings = (SpaceLimitSettings) entry.getValue();
       TableName tn = entry.getKey();
-      if (null != settings.getTableName()) {
+      if (settings.getTableName() != null) {
         tablesWithTableQuota.add(tn);
       }
-      if (null != settings.getNamespace()) {
+      if (settings.getNamespace() != null) {
         tablesWithNamespaceQuota.add(tn);
       }
 
-      if (null == settings.getTableName() && null == settings.getNamespace()) {
+      if (settings.getTableName() == null && settings.getNamespace() == null) {
         fail("Unexpected table name with null tableName and namespace: " + tn);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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
index 80363e8..7c1fd17 100644
--- 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
@@ -25,12 +25,16 @@ import static org.mockito.Mockito.mock;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Map.Entry;
 
 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.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.hadoop.hbase.quotas.policies.DefaultViolationPolicyEnforcement;
+import org.apache.hadoop.hbase.quotas.policies.MissingSnapshotViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -40,6 +44,13 @@ import org.junit.experimental.categories.Category;
 @Category(SmallTests.class)
 public class TestActivePolicyEnforcement {
 
+  private RegionServerServices rss;
+
+  @Before
+  public void setup() {
+    rss = mock(RegionServerServices.class);
+  }
+
   @Test
   public void testGetter() {
     final TableName tableName = TableName.valueOf("table");
@@ -57,8 +68,9 @@ public class TestActivePolicyEnforcement {
         TableName.valueOf("nonexistent"));
     assertNotNull(enforcement);
     assertTrue(
-        "Expected an instance of NoopViolationPolicyEnforcement",
-        enforcement instanceof BulkLoadVerifyingViolationPolicyEnforcement);
+        "Expected an instance of MissingSnapshotViolationPolicyEnforcement, but got "
+            + enforcement.getClass(),
+        enforcement instanceof MissingSnapshotViolationPolicyEnforcement);
   }
 
   @Test
@@ -71,4 +83,48 @@ public class TestActivePolicyEnforcement {
         TableName.valueOf("nonexistent"));
     assertFalse("Should not check bulkloads", enforcement.shouldCheckBulkLoads());
   }
+
+  @Test
+  public void testNoQuotaReturnsSingletonPolicyEnforcement() {
+    final ActivePolicyEnforcement ape = new ActivePolicyEnforcement(
+        Collections.emptyMap(), Collections.emptyMap(), rss);
+    final TableName tableName = TableName.valueOf("my_table");
+    SpaceViolationPolicyEnforcement policyEnforcement = ape.getPolicyEnforcement(tableName);
+    // This should be the same exact instance, the singleton
+    assertTrue(policyEnforcement == MissingSnapshotViolationPolicyEnforcement.getInstance());
+    assertEquals(1, ape.getLocallyCachedPolicies().size());
+    Entry<TableName,SpaceViolationPolicyEnforcement> entry =
+        ape.getLocallyCachedPolicies().entrySet().iterator().next();
+    assertTrue(policyEnforcement == entry.getValue());
+  }
+
+  @Test
+  public void testNonViolatingQuotaCachesPolicyEnforcment() {
+    final Map<TableName,SpaceQuotaSnapshot> snapshots = new HashMap<>();
+    final TableName tableName = TableName.valueOf("my_table");
+    snapshots.put(tableName, new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, 1024));
+    final ActivePolicyEnforcement ape = new ActivePolicyEnforcement(
+        Collections.emptyMap(), snapshots, rss);
+    SpaceViolationPolicyEnforcement policyEnforcement = ape.getPolicyEnforcement(tableName);
+    assertTrue(
+        "Found the wrong class: " + policyEnforcement.getClass(),
+        policyEnforcement instanceof DefaultViolationPolicyEnforcement);
+    SpaceViolationPolicyEnforcement copy = ape.getPolicyEnforcement(tableName);
+    assertTrue("Expected the instance to be cached", policyEnforcement == copy);
+    Entry<TableName,SpaceViolationPolicyEnforcement> entry =
+        ape.getLocallyCachedPolicies().entrySet().iterator().next();
+    assertTrue(policyEnforcement == entry.getValue());
+  }
+
+  @Test
+  public void testViolatingQuotaCachesNothing() {
+    final TableName tableName = TableName.valueOf("my_table");
+    SpaceViolationPolicyEnforcement policyEnforcement = mock(SpaceViolationPolicyEnforcement.class);
+    final Map<TableName,SpaceViolationPolicyEnforcement> activePolicies = new HashMap<>();
+    activePolicies.put(tableName, policyEnforcement);
+    final ActivePolicyEnforcement ape = new ActivePolicyEnforcement(
+        activePolicies, Collections.emptyMap(), rss);
+    assertTrue(ape.getPolicyEnforcement(tableName) == policyEnforcement);
+    assertEquals(0, ape.getLocallyCachedPolicies().size());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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
index a1eee4f..ea59d70 100644
--- 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
@@ -19,9 +19,8 @@ package org.apache.hadoop.hbase.quotas;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
+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;
@@ -45,8 +44,8 @@ import org.junit.rules.TestName;
  */
 @Category(MediumTests.class)
 public class TestMasterSpaceQuotaObserver {
-  private static final Log LOG = LogFactory.getLog(TestSpaceQuotas.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static SpaceQuotaHelperForTests helper;
 
   @Rule
   public TestName testName = new TestName();
@@ -66,28 +65,17 @@ public class TestMasterSpaceQuotaObserver {
 
   @Before
   public void removeAllQuotas() throws Exception {
+    if (helper == null) {
+      helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, new AtomicLong());
+    }
     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));
+      helper.waitForQuotaTable(conn);
     } 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.removeAllQuotas(conn);
+      assertEquals(0, helper.listNumDefinedQuotas(conn));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
index d7cdff9..ae315a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
@@ -90,7 +90,7 @@ public class TestQuotaObserverChoreRegionReports {
     final String FAM1 = "f1";
     final HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
     // Wait for the master to finish initialization.
-    while (null == master.getMasterQuotaManager()) {
+    while (master.getMasterQuotaManager() == null) {
       LOG.debug("MasterQuotaManager is null, waiting...");
       Thread.sleep(500);
     }
@@ -170,7 +170,7 @@ public class TestQuotaObserverChoreRegionReports {
       @Override
       public boolean evaluate() throws Exception {
         SpaceQuotaSnapshot snapshot = getSnapshotForTable(conn, tn);
-        if (null == snapshot) {
+        if (snapshot == null) {
           return false;
         }
         return snapshot.getQuotaStatus().isInViolation();
@@ -188,7 +188,7 @@ public class TestQuotaObserverChoreRegionReports {
       @Override
       public boolean evaluate() throws Exception {
         SpaceQuotaSnapshot snapshot = getSnapshotForTable(conn, tn);
-        if (null == snapshot) {
+        if (snapshot == null) {
           return false;
         }
         return !snapshot.getQuotaStatus().isInViolation();

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 63198a8..dde9e71 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
@@ -95,35 +95,24 @@ public class TestQuotaObserverChoreWithMiniCluster {
   @Before
   public void removeAllQuotas() throws Exception {
     final Connection conn = TEST_UTIL.getConnection();
+    if (helper == null) {
+      helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
+    }
     // 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));
+      helper.waitForQuotaTable(conn);
     } 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.removeAllQuotas(conn);
+      assertEquals(0, helper.listNumDefinedQuotas(conn));
     }
 
     master = TEST_UTIL.getMiniHBaseCluster().getMaster();
     snapshotNotifier =
         (SpaceQuotaSnapshotNotifierForTest) master.getSpaceQuotaSnapshotNotifier();
+    assertNotNull(snapshotNotifier);
     snapshotNotifier.clearSnapshots();
     chore = master.getQuotaObserverChore();
-    helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
   }
 
   @Test
@@ -382,7 +371,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
       @Override
       int getNumReportedRegions(TableName table, QuotaSnapshotStore<TableName> tableStore) {
         Integer i = mockReportedRegions.get(table);
-        if (null == i) {
+        if (i == null) {
           return 0;
         }
         return i;
@@ -424,10 +413,10 @@ public class TestQuotaObserverChoreWithMiniCluster {
           qs instanceof SpaceLimitSettings);
 
       SpaceQuota spaceQuota = null;
-      if (null != qs.getTableName()) {
+      if (qs.getTableName() != null) {
         spaceQuota = chore.getTableSnapshotStore().getSpaceQuota(table);
         assertNotNull("Could not find table space quota for " + table, spaceQuota);
-      } else if (null != qs.getNamespace()) {
+      } else if (qs.getNamespace() != null) {
         spaceQuota = chore.getNamespaceSnapshotStore().getSpaceQuota(table.getNamespaceAsString());
         assertNotNull("Could not find namespace space quota for " + table.getNamespaceAsString(), spaceQuota);
       } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 38dbf66..2cd67c9 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
@@ -37,6 +37,7 @@ 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.quotas.policies.MissingSnapshotViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.AfterClass;
@@ -132,7 +133,7 @@ public class TestQuotaStatusRPCs {
       @Override
       public boolean evaluate() throws Exception {
         SpaceQuotaSnapshot snapshot = manager.copyQuotaSnapshots().get(tn);
-        if (null == snapshot) {
+        if (snapshot == null) {
           return false;
         }
         return snapshot.getUsage() >= tableSize;
@@ -177,6 +178,10 @@ public class TestQuotaStatusRPCs {
       public boolean evaluate() throws Exception {
         ActivePolicyEnforcement enforcements = manager.getActiveEnforcements();
         SpaceViolationPolicyEnforcement enforcement = enforcements.getPolicyEnforcement(tn);
+        // Signifies that we're waiting on the quota snapshot to be fetched
+        if (enforcement instanceof MissingSnapshotViolationPolicyEnforcement) {
+          return false;
+        }
         return enforcement.getQuotaSnapshot().getQuotaStatus().isInViolation();
       }
     });
@@ -215,7 +220,7 @@ public class TestQuotaStatusRPCs {
       public boolean evaluate() throws Exception {
         SpaceQuotaSnapshot snapshot = QuotaTableUtil.getCurrentSnapshot(conn, tn);
         LOG.info("Table snapshot after initial ingest: " + snapshot);
-        if (null == snapshot) {
+        if (snapshot == null) {
           return false;
         }
         return snapshot.getLimit() == sizeLimit && snapshot.getUsage() > 0L;
@@ -229,7 +234,7 @@ public class TestQuotaStatusRPCs {
         SpaceQuotaSnapshot snapshot = QuotaTableUtil.getCurrentSnapshot(
             conn, tn.getNamespaceAsString());
         LOG.debug("Namespace snapshot after initial ingest: " + snapshot);
-        if (null == snapshot) {
+        if (snapshot == null) {
           return false;
         }
         nsUsage.set(snapshot.getUsage());
@@ -249,7 +254,7 @@ public class TestQuotaStatusRPCs {
       public boolean evaluate() throws Exception {
         SpaceQuotaSnapshot snapshot = QuotaTableUtil.getCurrentSnapshot(conn, tn);
         LOG.info("Table snapshot after second ingest: " + snapshot);
-        if (null == snapshot) {
+        if (snapshot == null) {
           return false;
         }
         return snapshot.getQuotaStatus().isInViolation();
@@ -262,7 +267,7 @@ public class TestQuotaStatusRPCs {
         SpaceQuotaSnapshot snapshot = QuotaTableUtil.getCurrentSnapshot(
             conn, tn.getNamespaceAsString());
         LOG.debug("Namespace snapshot after second ingest: " + snapshot);
-        if (null == snapshot) {
+        if (snapshot == null) {
           return false;
         }
         return snapshot.getUsage() > nsUsage.get() && !snapshot.getQuotaStatus().isInViolation();

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 38656e8..5f11950 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
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.quotas.policies.DisableTableViolationPolicyEnforc
 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.quotas.policies.DefaultViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Before;
@@ -95,7 +95,7 @@ public class TestRegionServerSpaceQuotaManager {
     expectedPolicies.put(disablePolicy.getTableName(), disableSnapshot);
 
     enforcements.put(
-        TableName.valueOf("no_policy"), new BulkLoadVerifyingViolationPolicyEnforcement());
+        TableName.valueOf("no_policy"), new DefaultViolationPolicyEnforcement());
 
     Map<TableName, SpaceQuotaSnapshot> actualPolicies = quotaManager.getActivePoliciesAsMap();
     assertEquals(expectedPolicies, actualPolicies);

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 ffe0ce2..e21647f 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
@@ -56,7 +56,7 @@ 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.quotas.policies.DefaultViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
@@ -108,29 +108,17 @@ public class TestSpaceQuotas {
   @Before
   public void removeAllQuotas() throws Exception {
     final Connection conn = TEST_UTIL.getConnection();
+    if (helper == null) {
+      helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
+    }
     // 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));
+      helper.waitForQuotaTable(conn);
     } 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.removeAllQuotas(conn);
+      assertEquals(0, helper.listNumDefinedQuotas(conn));
     }
-    helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
   }
 
   @Test
@@ -285,7 +273,7 @@ public class TestSpaceQuotas {
     Map<HRegionInfo,Long> regionSizes = getReportedSizesForTable(tn);
     while (true) {
       SpaceQuotaSnapshot snapshot = snapshots.get(tn);
-      if (null != snapshot && snapshot.getLimit() > 0) {
+      if (snapshot != null && snapshot.getLimit() > 0) {
         break;
       }
       LOG.debug(
@@ -305,7 +293,7 @@ public class TestSpaceQuotas {
     SpaceViolationPolicyEnforcement enforcement = activePolicies.getPolicyEnforcement(tn);
     assertTrue(
         "Expected to find Noop policy, but got " + enforcement.getClass().getSimpleName(),
-        enforcement instanceof BulkLoadVerifyingViolationPolicyEnforcement);
+        enforcement instanceof DefaultViolationPolicyEnforcement);
 
     // Should generate two files, each of which is over 25KB each
     ClientServiceCallable<Void> callable = generateFileToLoad(tn, 2, 500);

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 d190c8c..6626ab5 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
@@ -68,10 +68,10 @@ public class TestTableSpaceQuotaViolationNotifier {
 
     final Put expectedPut = new Put(Bytes.toBytes("t." + tn.getNameAsString()));
     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)
+        .setQuotaStatus(QuotaProtos.SpaceQuotaStatus.newBuilder().setInViolation(true)
+        .setViolationPolicy(QuotaProtos.SpaceViolationPolicy.NO_INSERTS))
+        .setQuotaLimit(512L)
+        .setQuotaUsage(1024L)
         .build();
     expectedPut.addColumn(Bytes.toBytes("u"), Bytes.toBytes("p"), protoQuota.toByteArray());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java
index abe1b9d..bd0bc8c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java
@@ -49,7 +49,7 @@ public class TestBulkLoadCheckingViolationPolicyEnforcement {
     fs = mock(FileSystem.class);
     rss = mock(RegionServerServices.class);
     tableName = TableName.valueOf("foo");
-    policy = new BulkLoadVerifyingViolationPolicyEnforcement();
+    policy = new DefaultViolationPolicyEnforcement();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
index 3244681..7c16d32 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
@@ -63,11 +63,11 @@ public class TestRegionServerRegionSpaceUseReport {
     RegionSpaceUseReportRequest requests = rs.buildRegionSpaceUseReportRequest(sizes);
     assertEquals(sizes.size(), requests.getSpaceUseCount());
     for (RegionSpaceUse spaceUse : requests.getSpaceUseList()) {
-      RegionInfo ri = spaceUse.getRegion();
+      RegionInfo ri = spaceUse.getRegionInfo();
       HRegionInfo hri = HRegionInfo.convert(ri);
       Long expectedSize = sizes.remove(hri);
       assertNotNull("Could not find size for HRI: " + hri, expectedSize);
-      assertEquals(expectedSize.longValue(), spaceUse.getSize());
+      assertEquals(expectedSize.longValue(), spaceUse.getRegionSize());
     }
     assertTrue("Should not have any space use entries left: " + sizes, sizes.isEmpty());
   }


[2/4] hbase git commit: HBASE-17955 Various reviewboard improvements to space quota work

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 c70b736..b886f5c 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
@@ -10173,42 +10173,42 @@ public final class RegionServerStatusProtos {
      * A region identifier
      * </pre>
      *
-     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     * <code>optional .hbase.pb.RegionInfo region_info = 1;</code>
      */
-    boolean hasRegion();
+    boolean hasRegionInfo();
     /**
      * <pre>
      * A region identifier
      * </pre>
      *
-     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     * <code>optional .hbase.pb.RegionInfo region_info = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegion();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo();
     /**
      * <pre>
      * A region identifier
      * </pre>
      *
-     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     * <code>optional .hbase.pb.RegionInfo region_info = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionOrBuilder();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder();
 
     /**
      * <pre>
      * The size in bytes of the region
      * </pre>
      *
-     * <code>optional uint64 size = 2;</code>
+     * <code>optional uint64 region_size = 2;</code>
      */
-    boolean hasSize();
+    boolean hasRegionSize();
     /**
      * <pre>
      * The size in bytes of the region
      * </pre>
      *
-     * <code>optional uint64 size = 2;</code>
+     * <code>optional uint64 region_size = 2;</code>
      */
-    long getSize();
+    long getRegionSize();
   }
   /**
    * Protobuf type {@code hbase.pb.RegionSpaceUse}
@@ -10222,7 +10222,7 @@ public final class RegionServerStatusProtos {
       super(builder);
     }
     private RegionSpaceUse() {
-      size_ = 0L;
+      regionSize_ = 0L;
     }
 
     @java.lang.Override
@@ -10256,19 +10256,19 @@ public final class RegionServerStatusProtos {
             case 10: {
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null;
               if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = region_.toBuilder();
+                subBuilder = regionInfo_.toBuilder();
               }
-              region_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
+              regionInfo_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
               if (subBuilder != null) {
-                subBuilder.mergeFrom(region_);
-                region_ = subBuilder.buildPartial();
+                subBuilder.mergeFrom(regionInfo_);
+                regionInfo_ = subBuilder.buildPartial();
               }
               bitField0_ |= 0x00000001;
               break;
             }
             case 16: {
               bitField0_ |= 0x00000002;
-              size_ = input.readUInt64();
+              regionSize_ = input.readUInt64();
               break;
             }
           }
@@ -10296,16 +10296,16 @@ public final class RegionServerStatusProtos {
     }
 
     private int bitField0_;
-    public static final int REGION_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo region_;
+    public static final int REGION_INFO_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_;
     /**
      * <pre>
      * A region identifier
      * </pre>
      *
-     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     * <code>optional .hbase.pb.RegionInfo region_info = 1;</code>
      */
-    public boolean hasRegion() {
+    public boolean hasRegionInfo() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
@@ -10313,32 +10313,32 @@ public final class RegionServerStatusProtos {
      * A region identifier
      * </pre>
      *
-     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     * <code>optional .hbase.pb.RegionInfo region_info = 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_;
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() {
+      return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
     }
     /**
      * <pre>
      * A region identifier
      * </pre>
      *
-     * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+     * <code>optional .hbase.pb.RegionInfo region_info = 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 org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() {
+      return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
     }
 
-    public static final int SIZE_FIELD_NUMBER = 2;
-    private long size_;
+    public static final int REGION_SIZE_FIELD_NUMBER = 2;
+    private long regionSize_;
     /**
      * <pre>
      * The size in bytes of the region
      * </pre>
      *
-     * <code>optional uint64 size = 2;</code>
+     * <code>optional uint64 region_size = 2;</code>
      */
-    public boolean hasSize() {
+    public boolean hasRegionSize() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
@@ -10346,10 +10346,10 @@ public final class RegionServerStatusProtos {
      * The size in bytes of the region
      * </pre>
      *
-     * <code>optional uint64 size = 2;</code>
+     * <code>optional uint64 region_size = 2;</code>
      */
-    public long getSize() {
-      return size_;
+    public long getRegionSize() {
+      return regionSize_;
     }
 
     private byte memoizedIsInitialized = -1;
@@ -10358,8 +10358,8 @@ public final class RegionServerStatusProtos {
       if (isInitialized == 1) return true;
       if (isInitialized == 0) return false;
 
-      if (hasRegion()) {
-        if (!getRegion().isInitialized()) {
+      if (hasRegionInfo()) {
+        if (!getRegionInfo().isInitialized()) {
           memoizedIsInitialized = 0;
           return false;
         }
@@ -10371,10 +10371,10 @@ public final class RegionServerStatusProtos {
     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());
+        output.writeMessage(1, getRegionInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeUInt64(2, size_);
+        output.writeUInt64(2, regionSize_);
       }
       unknownFields.writeTo(output);
     }
@@ -10386,11 +10386,11 @@ public final class RegionServerStatusProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(1, getRegion());
+          .computeMessageSize(1, getRegionInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(2, size_);
+          .computeUInt64Size(2, regionSize_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -10409,15 +10409,15 @@ public final class RegionServerStatusProtos {
       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 && (hasRegionInfo() == other.hasRegionInfo());
+      if (hasRegionInfo()) {
+        result = result && getRegionInfo()
+            .equals(other.getRegionInfo());
       }
-      result = result && (hasSize() == other.hasSize());
-      if (hasSize()) {
-        result = result && (getSize()
-            == other.getSize());
+      result = result && (hasRegionSize() == other.hasRegionSize());
+      if (hasRegionSize()) {
+        result = result && (getRegionSize()
+            == other.getRegionSize());
       }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
@@ -10430,14 +10430,14 @@ public final class RegionServerStatusProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptor().hashCode();
-      if (hasRegion()) {
-        hash = (37 * hash) + REGION_FIELD_NUMBER;
-        hash = (53 * hash) + getRegion().hashCode();
+      if (hasRegionInfo()) {
+        hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getRegionInfo().hashCode();
       }
-      if (hasSize()) {
-        hash = (37 * hash) + SIZE_FIELD_NUMBER;
+      if (hasRegionSize()) {
+        hash = (37 * hash) + REGION_SIZE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getSize());
+            getRegionSize());
       }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
@@ -10553,18 +10553,18 @@ public final class RegionServerStatusProtos {
       private void maybeForceBuilderInitialization() {
         if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
-          getRegionFieldBuilder();
+          getRegionInfoFieldBuilder();
         }
       }
       public Builder clear() {
         super.clear();
-        if (regionBuilder_ == null) {
-          region_ = null;
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = null;
         } else {
-          regionBuilder_.clear();
+          regionInfoBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        size_ = 0L;
+        regionSize_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
@@ -10593,15 +10593,15 @@ public final class RegionServerStatusProtos {
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        if (regionBuilder_ == null) {
-          result.region_ = region_;
+        if (regionInfoBuilder_ == null) {
+          result.regionInfo_ = regionInfo_;
         } else {
-          result.region_ = regionBuilder_.build();
+          result.regionInfo_ = regionInfoBuilder_.build();
         }
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.size_ = size_;
+        result.regionSize_ = regionSize_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -10644,11 +10644,11 @@ public final class RegionServerStatusProtos {
 
       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.hasRegionInfo()) {
+          mergeRegionInfo(other.getRegionInfo());
         }
-        if (other.hasSize()) {
-          setSize(other.getSize());
+        if (other.hasRegionSize()) {
+          setRegionSize(other.getRegionSize());
         }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
@@ -10656,8 +10656,8 @@ public final class RegionServerStatusProtos {
       }
 
       public final boolean isInitialized() {
-        if (hasRegion()) {
-          if (!getRegion().isInitialized()) {
+        if (hasRegionInfo()) {
+          if (!getRegionInfo().isInitialized()) {
             return false;
           }
         }
@@ -10683,17 +10683,17 @@ public final class RegionServerStatusProtos {
       }
       private int bitField0_;
 
-      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo region_ = null;
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = 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_;
+          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> regionInfoBuilder_;
       /**
        * <pre>
        * A region identifier
        * </pre>
        *
-       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       * <code>optional .hbase.pb.RegionInfo region_info = 1;</code>
        */
-      public boolean hasRegion() {
+      public boolean hasRegionInfo() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
@@ -10701,13 +10701,13 @@ public final class RegionServerStatusProtos {
        * A region identifier
        * </pre>
        *
-       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       * <code>optional .hbase.pb.RegionInfo region_info = 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_;
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
         } else {
-          return regionBuilder_.getMessage();
+          return regionInfoBuilder_.getMessage();
         }
       }
       /**
@@ -10715,17 +10715,17 @@ public final class RegionServerStatusProtos {
        * A region identifier
        * </pre>
        *
-       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       * <code>optional .hbase.pb.RegionInfo region_info = 1;</code>
        */
-      public Builder setRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
-        if (regionBuilder_ == null) {
+      public Builder setRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          region_ = value;
+          regionInfo_ = value;
           onChanged();
         } else {
-          regionBuilder_.setMessage(value);
+          regionInfoBuilder_.setMessage(value);
         }
         bitField0_ |= 0x00000001;
         return this;
@@ -10735,15 +10735,15 @@ public final class RegionServerStatusProtos {
        * A region identifier
        * </pre>
        *
-       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       * <code>optional .hbase.pb.RegionInfo region_info = 1;</code>
        */
-      public Builder setRegion(
+      public Builder setRegionInfo(
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
-        if (regionBuilder_ == null) {
-          region_ = builderForValue.build();
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = builderForValue.build();
           onChanged();
         } else {
-          regionBuilder_.setMessage(builderForValue.build());
+          regionInfoBuilder_.setMessage(builderForValue.build());
         }
         bitField0_ |= 0x00000001;
         return this;
@@ -10753,21 +10753,21 @@ public final class RegionServerStatusProtos {
        * A region identifier
        * </pre>
        *
-       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       * <code>optional .hbase.pb.RegionInfo region_info = 1;</code>
        */
-      public Builder mergeRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
-        if (regionBuilder_ == null) {
+      public Builder mergeRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == 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();
+              regionInfo_ != null &&
+              regionInfo_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance()) {
+            regionInfo_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.newBuilder(regionInfo_).mergeFrom(value).buildPartial();
           } else {
-            region_ = value;
+            regionInfo_ = value;
           }
           onChanged();
         } else {
-          regionBuilder_.mergeFrom(value);
+          regionInfoBuilder_.mergeFrom(value);
         }
         bitField0_ |= 0x00000001;
         return this;
@@ -10777,14 +10777,14 @@ public final class RegionServerStatusProtos {
        * A region identifier
        * </pre>
        *
-       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       * <code>optional .hbase.pb.RegionInfo region_info = 1;</code>
        */
-      public Builder clearRegion() {
-        if (regionBuilder_ == null) {
-          region_ = null;
+      public Builder clearRegionInfo() {
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = null;
           onChanged();
         } else {
-          regionBuilder_.clear();
+          regionInfoBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
@@ -10794,26 +10794,26 @@ public final class RegionServerStatusProtos {
        * A region identifier
        * </pre>
        *
-       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       * <code>optional .hbase.pb.RegionInfo region_info = 1;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionBuilder() {
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder() {
         bitField0_ |= 0x00000001;
         onChanged();
-        return getRegionFieldBuilder().getBuilder();
+        return getRegionInfoFieldBuilder().getBuilder();
       }
       /**
        * <pre>
        * A region identifier
        * </pre>
        *
-       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       * <code>optional .hbase.pb.RegionInfo region_info = 1;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionOrBuilder() {
-        if (regionBuilder_ != null) {
-          return regionBuilder_.getMessageOrBuilder();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() {
+        if (regionInfoBuilder_ != null) {
+          return regionInfoBuilder_.getMessageOrBuilder();
         } else {
-          return region_ == null ?
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : region_;
+          return regionInfo_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
         }
       }
       /**
@@ -10821,31 +10821,31 @@ public final class RegionServerStatusProtos {
        * A region identifier
        * </pre>
        *
-       * <code>optional .hbase.pb.RegionInfo region = 1;</code>
+       * <code>optional .hbase.pb.RegionInfo region_info = 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>
+          getRegionInfoFieldBuilder() {
+        if (regionInfoBuilder_ == null) {
+          regionInfoBuilder_ = 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(),
+                  getRegionInfo(),
                   getParentForChildren(),
                   isClean());
-          region_ = null;
+          regionInfo_ = null;
         }
-        return regionBuilder_;
+        return regionInfoBuilder_;
       }
 
-      private long size_ ;
+      private long regionSize_ ;
       /**
        * <pre>
        * The size in bytes of the region
        * </pre>
        *
-       * <code>optional uint64 size = 2;</code>
+       * <code>optional uint64 region_size = 2;</code>
        */
-      public boolean hasSize() {
+      public boolean hasRegionSize() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
@@ -10853,21 +10853,21 @@ public final class RegionServerStatusProtos {
        * The size in bytes of the region
        * </pre>
        *
-       * <code>optional uint64 size = 2;</code>
+       * <code>optional uint64 region_size = 2;</code>
        */
-      public long getSize() {
-        return size_;
+      public long getRegionSize() {
+        return regionSize_;
       }
       /**
        * <pre>
        * The size in bytes of the region
        * </pre>
        *
-       * <code>optional uint64 size = 2;</code>
+       * <code>optional uint64 region_size = 2;</code>
        */
-      public Builder setSize(long value) {
+      public Builder setRegionSize(long value) {
         bitField0_ |= 0x00000002;
-        size_ = value;
+        regionSize_ = value;
         onChanged();
         return this;
       }
@@ -10876,11 +10876,11 @@ public final class RegionServerStatusProtos {
        * The size in bytes of the region
        * </pre>
        *
-       * <code>optional uint64 size = 2;</code>
+       * <code>optional uint64 region_size = 2;</code>
        */
-      public Builder clearSize() {
+      public Builder clearRegionSize() {
         bitField0_ = (bitField0_ & ~0x00000002);
-        size_ = 0L;
+        regionSize_ = 0L;
         onChanged();
         return this;
       }
@@ -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_;
     }
@@ -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<
@@ -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;
 
@@ -13008,35 +13008,35 @@ public final class RegionServerStatusProtos {
       "est\022)\n\013region_info\030\001 \002(\0132\024.hbase.pb.Regi" +
       "onInfo\022\021\n\tsplit_row\030\002 \002(\014\022\026\n\013nonce_group" +
       "\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"+\n\030SplitTabl" +
-      "eRegionResponse\022\017\n\007proc_id\030\001 \001(\004\"D\n\016Regi" +
-      "onSpaceUse\022$\n\006region\030\001 \001(\0132\024.hbase.pb.Re" +
-      "gionInfo\022\014\n\004size\030\002 \001(\004\"J\n\033RegionSpaceUse" +
-      "ReportRequest\022+\n\tspace_use\030\001 \003(\0132\030.hbase",
-      ".pb.RegionSpaceUse\"\036\n\034RegionSpaceUseRepo" +
-      "rtResponse2\316\006\n\031RegionServerStatusService" +
-      "\022b\n\023RegionServerStartup\022$.hbase.pb.Regio" +
-      "nServerStartupRequest\032%.hbase.pb.RegionS" +
-      "erverStartupResponse\022_\n\022RegionServerRepo" +
-      "rt\022#.hbase.pb.RegionServerReportRequest\032" +
-      "$.hbase.pb.RegionServerReportResponse\022_\n" +
-      "\022ReportRSFatalError\022#.hbase.pb.ReportRSF" +
-      "atalErrorRequest\032$.hbase.pb.ReportRSFata" +
-      "lErrorResponse\022q\n\030GetLastFlushedSequence",
-      "Id\022).hbase.pb.GetLastFlushedSequenceIdRe" +
-      "quest\032*.hbase.pb.GetLastFlushedSequenceI" +
-      "dResponse\022z\n\033ReportRegionStateTransition" +
-      "\022,.hbase.pb.ReportRegionStateTransitionR" +
-      "equest\032-.hbase.pb.ReportRegionStateTrans" +
-      "itionResponse\022T\n\013SplitRegion\022!.hbase.pb." +
-      "SplitTableRegionRequest\032\".hbase.pb.Split" +
-      "TableRegionResponse\022_\n\022getProcedureResul" +
-      "t\022#.hbase.pb.GetProcedureResultRequest\032$" +
-      ".hbase.pb.GetProcedureResultResponse\022e\n\024",
-      "ReportRegionSpaceUse\022%.hbase.pb.RegionSp" +
-      "aceUseReportRequest\032&.hbase.pb.RegionSpa" +
-      "ceUseReportResponseBU\n1org.apache.hadoop" +
-      ".hbase.shaded.protobuf.generatedB\030Region" +
-      "ServerStatusProtosH\001\210\001\001\240\001\001"
+      "eRegionResponse\022\017\n\007proc_id\030\001 \001(\004\"P\n\016Regi" +
+      "onSpaceUse\022)\n\013region_info\030\001 \001(\0132\024.hbase." +
+      "pb.RegionInfo\022\023\n\013region_size\030\002 \001(\004\"J\n\033Re" +
+      "gionSpaceUseReportRequest\022+\n\tspace_use\030\001",
+      " \003(\0132\030.hbase.pb.RegionSpaceUse\"\036\n\034Region" +
+      "SpaceUseReportResponse2\316\006\n\031RegionServerS" +
+      "tatusService\022b\n\023RegionServerStartup\022$.hb" +
+      "ase.pb.RegionServerStartupRequest\032%.hbas" +
+      "e.pb.RegionServerStartupResponse\022_\n\022Regi" +
+      "onServerReport\022#.hbase.pb.RegionServerRe" +
+      "portRequest\032$.hbase.pb.RegionServerRepor" +
+      "tResponse\022_\n\022ReportRSFatalError\022#.hbase." +
+      "pb.ReportRSFatalErrorRequest\032$.hbase.pb." +
+      "ReportRSFatalErrorResponse\022q\n\030GetLastFlu",
+      "shedSequenceId\022).hbase.pb.GetLastFlushed" +
+      "SequenceIdRequest\032*.hbase.pb.GetLastFlus" +
+      "hedSequenceIdResponse\022z\n\033ReportRegionSta" +
+      "teTransition\022,.hbase.pb.ReportRegionStat" +
+      "eTransitionRequest\032-.hbase.pb.ReportRegi" +
+      "onStateTransitionResponse\022T\n\013SplitRegion" +
+      "\022!.hbase.pb.SplitTableRegionRequest\032\".hb" +
+      "ase.pb.SplitTableRegionResponse\022_\n\022getPr" +
+      "ocedureResult\022#.hbase.pb.GetProcedureRes" +
+      "ultRequest\032$.hbase.pb.GetProcedureResult",
+      "Response\022e\n\024ReportRegionSpaceUse\022%.hbase" +
+      ".pb.RegionSpaceUseReportRequest\032&.hbase." +
+      "pb.RegionSpaceUseReportResponseBU\n1org.a" +
+      "pache.hadoop.hbase.shaded.protobuf.gener" +
+      "atedB\030RegionServerStatusProtosH\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() {
@@ -13136,7 +13136,7 @@ public final class RegionServerStatusProtos {
     internal_static_hbase_pb_RegionSpaceUse_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionSpaceUse_descriptor,
-        new java.lang.String[] { "Region", "Size", });
+        new java.lang.String[] { "RegionInfo", "RegionSize", });
     internal_static_hbase_pb_RegionSpaceUseReportRequest_descriptor =
       getDescriptor().getMessageTypes().get(14);
     internal_static_hbase_pb_RegionSpaceUseReportRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 364c58b..0d171b3 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Quota.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
@@ -100,16 +100,16 @@ message SpaceLimitRequest {
 // Represents the state of a quota on a table. Either the quota is not in violation
 // or it is in violation there is a violation policy which should be in effect.
 message SpaceQuotaStatus {
-  optional SpaceViolationPolicy policy = 1;
+  optional SpaceViolationPolicy violation_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;
+  optional SpaceQuotaStatus quota_status = 1;
+  optional uint64 quota_usage = 2;
+  optional uint64 quota_limit = 3;
 }
 
 message GetSpaceQuotaRegionSizesRequest {

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 23ddd43..0b765d6 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -142,8 +142,8 @@ message SplitTableRegionResponse {
 }
 
 message RegionSpaceUse {
-  optional RegionInfo region = 1; // A region identifier
-  optional uint64 size = 2; // The size in bytes of the region
+  optional RegionInfo region_info = 1; // A region identifier
+  optional uint64 region_size = 2; // The size in bytes of the region
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 96a8ab2..fad9f44 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<
@@ -5903,15 +5903,15 @@ public final class QuotaProtos {
   public interface SpaceQuotaStatusOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // optional .hbase.pb.SpaceViolationPolicy policy = 1;
+    // optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;
     /**
-     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
      */
-    boolean hasPolicy();
+    boolean hasViolationPolicy();
     /**
-     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getPolicy();
+    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy();
 
     // optional bool in_violation = 2;
     /**
@@ -5986,7 +5986,7 @@ public final class QuotaProtos {
                 unknownFields.mergeVarintField(1, rawValue);
               } else {
                 bitField0_ |= 0x00000001;
-                policy_ = value;
+                violationPolicy_ = value;
               }
               break;
             }
@@ -6035,20 +6035,20 @@ public final class QuotaProtos {
     }
 
     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_;
+    // optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;
+    public static final int VIOLATION_POLICY_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy violationPolicy_;
     /**
-     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
      */
-    public boolean hasPolicy() {
+    public boolean hasViolationPolicy() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getPolicy() {
-      return policy_;
+    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy() {
+      return violationPolicy_;
     }
 
     // optional bool in_violation = 2;
@@ -6068,7 +6068,7 @@ public final class QuotaProtos {
     }
 
     private void initFields() {
-      policy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+      violationPolicy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
       inViolation_ = false;
     }
     private byte memoizedIsInitialized = -1;
@@ -6084,7 +6084,7 @@ public final class QuotaProtos {
                         throws java.io.IOException {
       getSerializedSize();
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeEnum(1, policy_.getNumber());
+        output.writeEnum(1, violationPolicy_.getNumber());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeBool(2, inViolation_);
@@ -6100,7 +6100,7 @@ public final class QuotaProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeEnumSize(1, policy_.getNumber());
+          .computeEnumSize(1, violationPolicy_.getNumber());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
@@ -6129,10 +6129,10 @@ public final class QuotaProtos {
       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 && (hasViolationPolicy() == other.hasViolationPolicy());
+      if (hasViolationPolicy()) {
         result = result &&
-            (getPolicy() == other.getPolicy());
+            (getViolationPolicy() == other.getViolationPolicy());
       }
       result = result && (hasInViolation() == other.hasInViolation());
       if (hasInViolation()) {
@@ -6152,9 +6152,9 @@ public final class QuotaProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasPolicy()) {
-        hash = (37 * hash) + POLICY_FIELD_NUMBER;
-        hash = (53 * hash) + hashEnum(getPolicy());
+      if (hasViolationPolicy()) {
+        hash = (37 * hash) + VIOLATION_POLICY_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getViolationPolicy());
       }
       if (hasInViolation()) {
         hash = (37 * hash) + IN_VIOLATION_FIELD_NUMBER;
@@ -6274,7 +6274,7 @@ public final class QuotaProtos {
 
       public Builder clear() {
         super.clear();
-        policy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+        violationPolicy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
         bitField0_ = (bitField0_ & ~0x00000001);
         inViolation_ = false;
         bitField0_ = (bitField0_ & ~0x00000002);
@@ -6309,7 +6309,7 @@ public final class QuotaProtos {
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.policy_ = policy_;
+        result.violationPolicy_ = violationPolicy_;
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
@@ -6330,8 +6330,8 @@ public final class QuotaProtos {
 
       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.hasViolationPolicy()) {
+          setViolationPolicy(other.getViolationPolicy());
         }
         if (other.hasInViolation()) {
           setInViolation(other.getInViolation());
@@ -6363,38 +6363,38 @@ public final class QuotaProtos {
       }
       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;
+      // optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy violationPolicy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
       /**
-       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
        */
-      public boolean hasPolicy() {
+      public boolean hasViolationPolicy() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getPolicy() {
-        return policy_;
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy() {
+        return violationPolicy_;
       }
       /**
-       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
        */
-      public Builder setPolicy(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy value) {
+      public Builder setViolationPolicy(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy value) {
         if (value == null) {
           throw new NullPointerException();
         }
         bitField0_ |= 0x00000001;
-        policy_ = value;
+        violationPolicy_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
        */
-      public Builder clearPolicy() {
+      public Builder clearViolationPolicy() {
         bitField0_ = (bitField0_ & ~0x00000001);
-        policy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
+        violationPolicy_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceViolationPolicy.DISABLE;
         onChanged();
         return this;
       }
@@ -6446,39 +6446,39 @@ public final class QuotaProtos {
   public interface SpaceQuotaSnapshotOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // optional .hbase.pb.SpaceQuotaStatus status = 1;
+    // optional .hbase.pb.SpaceQuotaStatus quota_status = 1;
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
      */
-    boolean hasStatus();
+    boolean hasQuotaStatus();
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus getStatus();
+    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus getQuotaStatus();
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getStatusOrBuilder();
+    org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getQuotaStatusOrBuilder();
 
-    // optional uint64 usage = 2;
+    // optional uint64 quota_usage = 2;
     /**
-     * <code>optional uint64 usage = 2;</code>
+     * <code>optional uint64 quota_usage = 2;</code>
      */
-    boolean hasUsage();
+    boolean hasQuotaUsage();
     /**
-     * <code>optional uint64 usage = 2;</code>
+     * <code>optional uint64 quota_usage = 2;</code>
      */
-    long getUsage();
+    long getQuotaUsage();
 
-    // optional uint64 limit = 3;
+    // optional uint64 quota_limit = 3;
     /**
-     * <code>optional uint64 limit = 3;</code>
+     * <code>optional uint64 quota_limit = 3;</code>
      */
-    boolean hasLimit();
+    boolean hasQuotaLimit();
     /**
-     * <code>optional uint64 limit = 3;</code>
+     * <code>optional uint64 quota_limit = 3;</code>
      */
-    long getLimit();
+    long getQuotaLimit();
   }
   /**
    * Protobuf type {@code hbase.pb.SpaceQuotaSnapshot}
@@ -6539,24 +6539,24 @@ public final class QuotaProtos {
             case 10: {
               org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder subBuilder = null;
               if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = status_.toBuilder();
+                subBuilder = quotaStatus_.toBuilder();
               }
-              status_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.PARSER, extensionRegistry);
+              quotaStatus_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.PARSER, extensionRegistry);
               if (subBuilder != null) {
-                subBuilder.mergeFrom(status_);
-                status_ = subBuilder.buildPartial();
+                subBuilder.mergeFrom(quotaStatus_);
+                quotaStatus_ = subBuilder.buildPartial();
               }
               bitField0_ |= 0x00000001;
               break;
             }
             case 16: {
               bitField0_ |= 0x00000002;
-              usage_ = input.readUInt64();
+              quotaUsage_ = input.readUInt64();
               break;
             }
             case 24: {
               bitField0_ |= 0x00000004;
-              limit_ = input.readUInt64();
+              quotaLimit_ = input.readUInt64();
               break;
             }
           }
@@ -6599,64 +6599,64 @@ public final class QuotaProtos {
     }
 
     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_;
+    // optional .hbase.pb.SpaceQuotaStatus quota_status = 1;
+    public static final int QUOTA_STATUS_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus quotaStatus_;
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
      */
-    public boolean hasStatus() {
+    public boolean hasQuotaStatus() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus getStatus() {
-      return status_;
+    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus getQuotaStatus() {
+      return quotaStatus_;
     }
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getStatusOrBuilder() {
-      return status_;
+    public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getQuotaStatusOrBuilder() {
+      return quotaStatus_;
     }
 
-    // optional uint64 usage = 2;
-    public static final int USAGE_FIELD_NUMBER = 2;
-    private long usage_;
+    // optional uint64 quota_usage = 2;
+    public static final int QUOTA_USAGE_FIELD_NUMBER = 2;
+    private long quotaUsage_;
     /**
-     * <code>optional uint64 usage = 2;</code>
+     * <code>optional uint64 quota_usage = 2;</code>
      */
-    public boolean hasUsage() {
+    public boolean hasQuotaUsage() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional uint64 usage = 2;</code>
+     * <code>optional uint64 quota_usage = 2;</code>
      */
-    public long getUsage() {
-      return usage_;
+    public long getQuotaUsage() {
+      return quotaUsage_;
     }
 
-    // optional uint64 limit = 3;
-    public static final int LIMIT_FIELD_NUMBER = 3;
-    private long limit_;
+    // optional uint64 quota_limit = 3;
+    public static final int QUOTA_LIMIT_FIELD_NUMBER = 3;
+    private long quotaLimit_;
     /**
-     * <code>optional uint64 limit = 3;</code>
+     * <code>optional uint64 quota_limit = 3;</code>
      */
-    public boolean hasLimit() {
+    public boolean hasQuotaLimit() {
       return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>optional uint64 limit = 3;</code>
+     * <code>optional uint64 quota_limit = 3;</code>
      */
-    public long getLimit() {
-      return limit_;
+    public long getQuotaLimit() {
+      return quotaLimit_;
     }
 
     private void initFields() {
-      status_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
-      usage_ = 0L;
-      limit_ = 0L;
+      quotaStatus_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
+      quotaUsage_ = 0L;
+      quotaLimit_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -6671,13 +6671,13 @@ public final class QuotaProtos {
                         throws java.io.IOException {
       getSerializedSize();
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeMessage(1, status_);
+        output.writeMessage(1, quotaStatus_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeUInt64(2, usage_);
+        output.writeUInt64(2, quotaUsage_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeUInt64(3, limit_);
+        output.writeUInt64(3, quotaLimit_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -6690,15 +6690,15 @@ public final class QuotaProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(1, status_);
+          .computeMessageSize(1, quotaStatus_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(2, usage_);
+          .computeUInt64Size(2, quotaUsage_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(3, limit_);
+          .computeUInt64Size(3, quotaLimit_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -6723,20 +6723,20 @@ public final class QuotaProtos {
       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 && (hasQuotaStatus() == other.hasQuotaStatus());
+      if (hasQuotaStatus()) {
+        result = result && getQuotaStatus()
+            .equals(other.getQuotaStatus());
       }
-      result = result && (hasUsage() == other.hasUsage());
-      if (hasUsage()) {
-        result = result && (getUsage()
-            == other.getUsage());
+      result = result && (hasQuotaUsage() == other.hasQuotaUsage());
+      if (hasQuotaUsage()) {
+        result = result && (getQuotaUsage()
+            == other.getQuotaUsage());
       }
-      result = result && (hasLimit() == other.hasLimit());
-      if (hasLimit()) {
-        result = result && (getLimit()
-            == other.getLimit());
+      result = result && (hasQuotaLimit() == other.hasQuotaLimit());
+      if (hasQuotaLimit()) {
+        result = result && (getQuotaLimit()
+            == other.getQuotaLimit());
       }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
@@ -6751,17 +6751,17 @@ public final class QuotaProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasStatus()) {
-        hash = (37 * hash) + STATUS_FIELD_NUMBER;
-        hash = (53 * hash) + getStatus().hashCode();
+      if (hasQuotaStatus()) {
+        hash = (37 * hash) + QUOTA_STATUS_FIELD_NUMBER;
+        hash = (53 * hash) + getQuotaStatus().hashCode();
       }
-      if (hasUsage()) {
-        hash = (37 * hash) + USAGE_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getUsage());
+      if (hasQuotaUsage()) {
+        hash = (37 * hash) + QUOTA_USAGE_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getQuotaUsage());
       }
-      if (hasLimit()) {
-        hash = (37 * hash) + LIMIT_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getLimit());
+      if (hasQuotaLimit()) {
+        hash = (37 * hash) + QUOTA_LIMIT_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getQuotaLimit());
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
@@ -6869,7 +6869,7 @@ public final class QuotaProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getStatusFieldBuilder();
+          getQuotaStatusFieldBuilder();
         }
       }
       private static Builder create() {
@@ -6878,15 +6878,15 @@ public final class QuotaProtos {
 
       public Builder clear() {
         super.clear();
-        if (statusBuilder_ == null) {
-          status_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
+        if (quotaStatusBuilder_ == null) {
+          quotaStatus_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
         } else {
-          statusBuilder_.clear();
+          quotaStatusBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        usage_ = 0L;
+        quotaUsage_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
-        limit_ = 0L;
+        quotaLimit_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
@@ -6919,19 +6919,19 @@ public final class QuotaProtos {
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        if (statusBuilder_ == null) {
-          result.status_ = status_;
+        if (quotaStatusBuilder_ == null) {
+          result.quotaStatus_ = quotaStatus_;
         } else {
-          result.status_ = statusBuilder_.build();
+          result.quotaStatus_ = quotaStatusBuilder_.build();
         }
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.usage_ = usage_;
+        result.quotaUsage_ = quotaUsage_;
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000004;
         }
-        result.limit_ = limit_;
+        result.quotaLimit_ = quotaLimit_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -6948,14 +6948,14 @@ public final class QuotaProtos {
 
       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.hasQuotaStatus()) {
+          mergeQuotaStatus(other.getQuotaStatus());
         }
-        if (other.hasUsage()) {
-          setUsage(other.getUsage());
+        if (other.hasQuotaUsage()) {
+          setQuotaUsage(other.getQuotaUsage());
         }
-        if (other.hasLimit()) {
-          setLimit(other.getLimit());
+        if (other.hasQuotaLimit()) {
+          setQuotaLimit(other.getQuotaLimit());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
@@ -6984,185 +6984,185 @@ public final class QuotaProtos {
       }
       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();
+      // optional .hbase.pb.SpaceQuotaStatus quota_status = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus quotaStatus_ = 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_;
+          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> quotaStatusBuilder_;
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public boolean hasStatus() {
+      public boolean hasQuotaStatus() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus getStatus() {
-        if (statusBuilder_ == null) {
-          return status_;
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus getQuotaStatus() {
+        if (quotaStatusBuilder_ == null) {
+          return quotaStatus_;
         } else {
-          return statusBuilder_.getMessage();
+          return quotaStatusBuilder_.getMessage();
         }
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public Builder setStatus(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
-        if (statusBuilder_ == null) {
+      public Builder setQuotaStatus(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
+        if (quotaStatusBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          status_ = value;
+          quotaStatus_ = value;
           onChanged();
         } else {
-          statusBuilder_.setMessage(value);
+          quotaStatusBuilder_.setMessage(value);
         }
         bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public Builder setStatus(
+      public Builder setQuotaStatus(
           org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder builderForValue) {
-        if (statusBuilder_ == null) {
-          status_ = builderForValue.build();
+        if (quotaStatusBuilder_ == null) {
+          quotaStatus_ = builderForValue.build();
           onChanged();
         } else {
-          statusBuilder_.setMessage(builderForValue.build());
+          quotaStatusBuilder_.setMessage(builderForValue.build());
         }
         bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public Builder mergeStatus(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
-        if (statusBuilder_ == null) {
+      public Builder mergeQuotaStatus(org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
+        if (quotaStatusBuilder_ == 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();
+              quotaStatus_ != org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance()) {
+            quotaStatus_ =
+              org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.newBuilder(quotaStatus_).mergeFrom(value).buildPartial();
           } else {
-            status_ = value;
+            quotaStatus_ = value;
           }
           onChanged();
         } else {
-          statusBuilder_.mergeFrom(value);
+          quotaStatusBuilder_.mergeFrom(value);
         }
         bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public Builder clearStatus() {
-        if (statusBuilder_ == null) {
-          status_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
+      public Builder clearQuotaStatus() {
+        if (quotaStatusBuilder_ == null) {
+          quotaStatus_ = org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance();
           onChanged();
         } else {
-          statusBuilder_.clear();
+          quotaStatusBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder getStatusBuilder() {
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder getQuotaStatusBuilder() {
         bitField0_ |= 0x00000001;
         onChanged();
-        return getStatusFieldBuilder().getBuilder();
+        return getQuotaStatusFieldBuilder().getBuilder();
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getStatusOrBuilder() {
-        if (statusBuilder_ != null) {
-          return statusBuilder_.getMessageOrBuilder();
+      public org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getQuotaStatusOrBuilder() {
+        if (quotaStatusBuilder_ != null) {
+          return quotaStatusBuilder_.getMessageOrBuilder();
         } else {
-          return status_;
+          return quotaStatus_;
         }
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_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>
+          getQuotaStatusFieldBuilder() {
+        if (quotaStatusBuilder_ == null) {
+          quotaStatusBuilder_ = 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_,
+                  quotaStatus_,
                   getParentForChildren(),
                   isClean());
-          status_ = null;
+          quotaStatus_ = null;
         }
-        return statusBuilder_;
+        return quotaStatusBuilder_;
       }
 
-      // optional uint64 usage = 2;
-      private long usage_ ;
+      // optional uint64 quota_usage = 2;
+      private long quotaUsage_ ;
       /**
-       * <code>optional uint64 usage = 2;</code>
+       * <code>optional uint64 quota_usage = 2;</code>
        */
-      public boolean hasUsage() {
+      public boolean hasQuotaUsage() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional uint64 usage = 2;</code>
+       * <code>optional uint64 quota_usage = 2;</code>
        */
-      public long getUsage() {
-        return usage_;
+      public long getQuotaUsage() {
+        return quotaUsage_;
       }
       /**
-       * <code>optional uint64 usage = 2;</code>
+       * <code>optional uint64 quota_usage = 2;</code>
        */
-      public Builder setUsage(long value) {
+      public Builder setQuotaUsage(long value) {
         bitField0_ |= 0x00000002;
-        usage_ = value;
+        quotaUsage_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional uint64 usage = 2;</code>
+       * <code>optional uint64 quota_usage = 2;</code>
        */
-      public Builder clearUsage() {
+      public Builder clearQuotaUsage() {
         bitField0_ = (bitField0_ & ~0x00000002);
-        usage_ = 0L;
+        quotaUsage_ = 0L;
         onChanged();
         return this;
       }
 
-      // optional uint64 limit = 3;
-      private long limit_ ;
+      // optional uint64 quota_limit = 3;
+      private long quotaLimit_ ;
       /**
-       * <code>optional uint64 limit = 3;</code>
+       * <code>optional uint64 quota_limit = 3;</code>
        */
-      public boolean hasLimit() {
+      public boolean hasQuotaLimit() {
         return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>optional uint64 limit = 3;</code>
+       * <code>optional uint64 quota_limit = 3;</code>
        */
-      public long getLimit() {
-        return limit_;
+      public long getQuotaLimit() {
+        return quotaLimit_;
       }
       /**
-       * <code>optional uint64 limit = 3;</code>
+       * <code>optional uint64 quota_limit = 3;</code>
        */
-      public Builder setLimit(long value) {
+      public Builder setQuotaLimit(long value) {
         bitField0_ |= 0x00000004;
-        limit_ = value;
+        quotaLimit_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional uint64 limit = 3;</code>
+       * <code>optional uint64 quota_limit = 3;</code>
        */
-      public Builder clearLimit() {
+      public Builder clearQuotaLimit() {
         bitField0_ = (bitField0_ & ~0x00000004);
-        limit_ = 0L;
+        quotaLimit_ = 0L;
         onChanged();
         return this;
       }
@@ -7252,21 +7252,22 @@ public final class QuotaProtos {
       "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\004BA\n*org.apa" +
-      "che.hadoop.hbase.protobuf.generatedB\013Quo" +
-      "taProtosH\001\210\001\001\240\001\001"
+      "ta\030\001 \001(\0132\024.hbase.pb.SpaceQuota\"b\n\020SpaceQ" +
+      "uotaStatus\0228\n\020violation_policy\030\001 \001(\0162\036.h" +
+      "base.pb.SpaceViolationPolicy\022\024\n\014in_viola" +
+      "tion\030\002 \001(\010\"p\n\022SpaceQuotaSnapshot\0220\n\014quot" +
+      "a_status\030\001 \001(\0132\032.hbase.pb.SpaceQuotaStat" +
+      "us\022\023\n\013quota_usage\030\002 \001(\004\022\023\n\013quota_limit\030\003" +
+      " \001(\004*&\n\nQuotaScope\022\013\n\007CLUSTER\020\001\022\013\n\007MACHI" +
+      "NE\020\002*v\n\014ThrottleType\022\022\n\016REQUEST_NUMBER\020\001" +
+      "\022\020\n\014REQUEST_SIZE\020\002\022\020\n\014WRITE_NUMBER\020\003\022\016\n\n" +
+      "WRITE_SIZE\020\004\022\017\n\013READ_NUMBER\020\005\022\r\n\tREAD_SI",
+      "ZE\020\006*$\n\tQuotaType\022\014\n\010THROTTLE\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.apache.hadoop.h" +
+      "base.protobuf.generatedB\013QuotaProtosH\001\210\001" +
+      "\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -7320,13 +7321,13 @@ public final class QuotaProtos {
           internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_SpaceQuotaStatus_descriptor,
-              new java.lang.String[] { "Policy", "InViolation", });
+              new java.lang.String[] { "ViolationPolicy", "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", });
+              new java.lang.String[] { "QuotaStatus", "QuotaUsage", "QuotaLimit", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 d55918a..8e5c17c 100644
--- a/hbase-protocol/src/main/protobuf/Quota.proto
+++ b/hbase-protocol/src/main/protobuf/Quota.proto
@@ -100,14 +100,14 @@ 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.
 message SpaceQuotaStatus {
-  optional SpaceViolationPolicy policy = 1;
+  optional SpaceViolationPolicy violation_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;
+  optional SpaceQuotaStatus quota_status = 1;
+  optional uint64 quota_usage = 2;
+  optional uint64 quota_limit = 3;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 64960ac..a982428 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
@@ -2205,10 +2205,10 @@ public class HMaster extends HRegionServer implements MasterServices {
         // (rather than explicit permissions) we'll do the check here instead of in the
         // coprocessor.
         MasterQuotaManager quotaManager = getMasterQuotaManager();
-        if (null != quotaManager) {
+        if (quotaManager != null) {
           if (quotaManager.isQuotaEnabled()) {
             Quotas quotaForTable = QuotaUtil.getTableQuota(getConnection(), tableName);
-            if (null != quotaForTable && quotaForTable.hasSpace()) {
+            if (quotaForTable != null && quotaForTable.hasSpace()) {
               SpaceViolationPolicy policy = quotaForTable.getSpace().getViolationPolicy();
               if (SpaceViolationPolicy.DISABLE == policy) {
                 throw new AccessDeniedException("Enabling the table '" + tableName

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 faffb3f..7ccc3b1 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
@@ -1951,7 +1951,8 @@ public class MasterRpcServices extends RSRpcServices
       MasterQuotaManager quotaManager = this.master.getMasterQuotaManager();
       final long now = EnvironmentEdgeManager.currentTime();
       for (RegionSpaceUse report : request.getSpaceUseList()) {
-        quotaManager.addRegionSize(HRegionInfo.convert(report.getRegion()), report.getSize(), now);
+        quotaManager.addRegionSize(HRegionInfo.convert(
+            report.getRegionInfo()), report.getRegionSize(), now);
       }
       return RegionSpaceUseReportResponse.newBuilder().build();
     } catch (Exception e) {
@@ -1967,14 +1968,14 @@ public class MasterRpcServices extends RSRpcServices
       MasterQuotaManager quotaManager = this.master.getMasterQuotaManager();
       GetSpaceQuotaRegionSizesResponse.Builder builder =
           GetSpaceQuotaRegionSizesResponse.newBuilder();
-      if (null != quotaManager) {
+      if (quotaManager != null) {
         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) {
+          if (prevSize == null) {
             prevSize = 0L;
           }
           regionSizesByTable.put(tableName, prevSize + entry.getValue());
@@ -2000,7 +2001,7 @@ public class MasterRpcServices extends RSRpcServices
       master.checkInitialized();
       QuotaObserverChore quotaChore = this.master.getQuotaObserverChore();
       GetQuotaStatesResponse.Builder builder = GetQuotaStatesResponse.newBuilder();
-      if (null != quotaChore) {
+      if (quotaChore != null) {
         // The "current" view of all tables with quotas
         Map<TableName, SpaceQuotaSnapshot> tableSnapshots = quotaChore.getTableQuotaSnapshots();
         for (Entry<TableName, SpaceQuotaSnapshot> entry : tableSnapshots.entrySet()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 b5bc3d7..dfa0a87 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
@@ -69,6 +69,8 @@ public class MetricsMaster {
 
   /**
    * Sets the number of space quotas defined.
+   *
+   * @see MetricsMasterQuotaSource#updateNumSpaceQuotas(long)
    */
   public void setNumSpaceQuotas(final long numSpaceQuotas) {
     masterQuotaSource.updateNumSpaceQuotas(numSpaceQuotas);
@@ -76,6 +78,8 @@ public class MetricsMaster {
 
   /**
    * Sets the number of table in violation of a space quota.
+   *
+   * @see MetricsMasterQuotaSource#updateNumTablesInSpaceQuotaViolation(long)
    */
   public void setNumTableInSpaceQuotaViolation(final long numTablesInViolation) {
     masterQuotaSource.updateNumTablesInSpaceQuotaViolation(numTablesInViolation);
@@ -83,13 +87,17 @@ public class MetricsMaster {
 
   /**
    * Sets the number of namespaces in violation of a space quota.
+   *
+   * @see MetricsMasterQuotaSource#updateNumNamespacesInSpaceQuotaViolation(long)
    */
   public void setNumNamespacesInSpaceQuotaViolation(final long numNamespacesInViolation) {
     masterQuotaSource.updateNumNamespacesInSpaceQuotaViolation(numNamespacesInViolation);
   }
 
   /**
-   * Sets the number of region size reports the master has seen.
+   * Sets the number of region size reports the master currently has in memory.
+   *
+   * @see MetricsMasterQuotaSource#updateNumCurrentSpaceQuotaRegionSizeReports(long)
    */
   public void setNumRegionSizeReports(final long numRegionReports) {
     masterQuotaSource.updateNumCurrentSpaceQuotaRegionSizeReports(numRegionReports);
@@ -97,6 +105,9 @@ public class MetricsMaster {
 
   /**
    * Sets the execution time of a period of the QuotaObserverChore.
+   *
+   * @param executionTime The execution time in milliseconds.
+   * @see MetricsMasterQuotaSource#incrementSpaceQuotaObserverChoreTime(long)
    */
   public void incrementQuotaObserverTime(final long executionTime) {
     masterQuotaSource.incrementSpaceQuotaObserverChoreTime(executionTime);

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 cbf7ba5..ed37d19 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
@@ -146,7 +146,7 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
   @Override
   public Map<String,Entry<Long,Long>> getTableSpaceUtilization() {
     QuotaObserverChore quotaChore = master.getQuotaObserverChore();
-    if (null == quotaChore) {
+    if (quotaChore == null) {
       return Collections.emptyMap();
     }
     Map<TableName,SpaceQuotaSnapshot> tableSnapshots = quotaChore.getTableQuotaSnapshots();
@@ -160,7 +160,7 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
   @Override
   public Map<String,Entry<Long,Long>> getNamespaceSpaceUtilization() {
     QuotaObserverChore quotaChore = master.getQuotaObserverChore();
-    if (null == quotaChore) {
+    if (quotaChore == null) {
       return Collections.emptyMap();
     }
     Map<String,SpaceQuotaSnapshot> namespaceSnapshots = quotaChore.getNamespaceQuotaSnapshots();


[4/4] hbase git commit: HBASE-17955 Various reviewboard improvements to space quota work

Posted by el...@apache.org.
HBASE-17955 Various reviewboard improvements to space quota work

Most notable change is to cache SpaceViolationPolicyEnforcement objects
in the write path. When a table has no quota or there is not SpaceQuotaSnapshot
for that table (yet), we want to avoid creating lots of
SpaceViolationPolicyEnforcement instances, caching one instance
instead. This will help reduce GC pressure.


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

Branch: refs/heads/HBASE-16961
Commit: 70bcf3fe6890582e00f9ad0ec7b6b80ebfacf05f
Parents: cb08814
Author: Josh Elser <el...@apache.org>
Authored: Tue Apr 18 16:43:40 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri Apr 28 13:27:19 2017 -0400

----------------------------------------------------------------------
 .../hbase/quotas/QuotaSettingsFactory.java      |  10 +-
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |   7 +-
 .../hadoop/hbase/quotas/SpaceLimitSettings.java |  26 +-
 .../hadoop/hbase/quotas/SpaceQuotaSnapshot.java |  34 +-
 .../hbase/quotas/SpaceViolationPolicy.java      |   5 +-
 .../hbase/master/MetricsMasterQuotaSource.java  |  13 +-
 .../MetricsRegionServerQuotaSource.java         |  10 +-
 .../MetricsMasterQuotaSourceFactoryImpl.java    |   2 +-
 .../master/MetricsMasterQuotaSourceImpl.java    |  10 +-
 .../shaded/protobuf/generated/AdminProtos.java  |   8 +-
 .../shaded/protobuf/generated/MasterProtos.java |  10 +-
 .../shaded/protobuf/generated/QuotaProtos.java  | 637 ++++++++++---------
 .../generated/RegionServerStatusProtos.java     | 340 +++++-----
 .../src/main/protobuf/Quota.proto               |   8 +-
 .../src/main/protobuf/RegionServerStatus.proto  |   4 +-
 .../hbase/protobuf/generated/QuotaProtos.java   | 463 +++++++-------
 hbase-protocol/src/main/protobuf/Quota.proto    |   8 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   4 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   9 +-
 .../hadoop/hbase/master/MetricsMaster.java      |  13 +-
 .../hbase/master/MetricsMasterWrapperImpl.java  |   4 +-
 .../hbase/quotas/ActivePolicyEnforcement.java   |  54 +-
 .../quotas/FileSystemUtilizationChore.java      |   4 +-
 .../hadoop/hbase/quotas/MasterQuotaManager.java |   8 +-
 .../hbase/quotas/MasterSpaceQuotaObserver.java  |   4 +-
 .../quotas/NamespaceQuotaSnapshotStore.java     |   2 +-
 .../hadoop/hbase/quotas/QuotaObserverChore.java |  62 +-
 .../quotas/RegionServerSpaceQuotaManager.java   |  16 +-
 .../hbase/quotas/SpaceLimitingException.java    |   6 +-
 .../hbase/quotas/SpaceQuotaRefresherChore.java  |   2 +-
 .../SpaceViolationPolicyEnforcementFactory.java |  20 +-
 .../hbase/quotas/TableQuotaSnapshotStore.java   |   2 +-
 .../AbstractViolationPolicyEnforcement.java     |  45 +-
 ...LoadVerifyingViolationPolicyEnforcement.java |  50 --
 .../DefaultViolationPolicyEnforcement.java      |  90 +++
 .../DisableTableViolationPolicyEnforcement.java |   2 +-
 ...ssingSnapshotViolationPolicyEnforcement.java |  63 ++
 .../NoInsertsViolationPolicyEnforcement.java    |   2 +-
 .../NoWritesViolationPolicyEnforcement.java     |   2 +-
 .../hbase/regionserver/CompactSplitThread.java  |   2 +-
 .../hbase/regionserver/HRegionServer.java       |   4 +-
 .../hbase/regionserver/RSRpcServices.java       |   9 +-
 .../resources/hbase-webapps/master/table.jsp    |   8 +-
 .../hbase/quotas/SpaceQuotaHelperForTests.java  |  66 +-
 .../quotas/TestActivePolicyEnforcement.java     |  62 +-
 .../quotas/TestMasterSpaceQuotaObserver.java    |  28 +-
 .../TestQuotaObserverChoreRegionReports.java    |   6 +-
 .../TestQuotaObserverChoreWithMiniCluster.java  |  31 +-
 .../hbase/quotas/TestQuotaStatusRPCs.java       |  15 +-
 .../TestRegionServerSpaceQuotaManager.java      |   4 +-
 .../hadoop/hbase/quotas/TestSpaceQuotas.java    |  30 +-
 .../TestTableSpaceQuotaViolationNotifier.java   |   8 +-
 ...kLoadCheckingViolationPolicyEnforcement.java |   2 +-
 .../TestRegionServerRegionSpaceUseReport.java   |   4 +-
 54 files changed, 1289 insertions(+), 1049 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 184277d..a99235f 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
@@ -127,11 +127,11 @@ public class QuotaSettingsFactory {
   }
 
   static QuotaSettings fromSpace(TableName table, String namespace, SpaceQuota protoQuota) {
-    if ((null == table && null == namespace) || (null != table && null != namespace)) {
+    if ((table == null && namespace == null) || (table != null && namespace != null)) {
       throw new IllegalArgumentException(
           "Can only construct SpaceLimitSettings for a table or namespace.");
     }
-    if (null != table) {
+    if (table != null) {
       return SpaceLimitSettings.fromSpaceQuota(table, protoQuota);
     } else {
       // namespace must be non-null
@@ -323,7 +323,7 @@ public class QuotaSettingsFactory {
    * @return A {@link QuotaSettings} object.
    */
   public static QuotaSettings removeTableSpaceLimit(TableName tableName) {
-    return new SpaceLimitSettings(tableName, true);
+    return new SpaceLimitSettings(tableName);
   }
 
   /**
@@ -343,12 +343,12 @@ public class QuotaSettingsFactory {
 
   /**
    * Creates a {@link QuotaSettings} object to remove the FileSystem space quota for the given
-	 * namespace.
+   * 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);
+    return new SpaceLimitSettings(namespace);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 c008702..ec480c4 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
@@ -251,12 +251,12 @@ public class QuotaTableUtil {
   public static void extractQuotaSnapshot(
       Result result, Map<TableName,SpaceQuotaSnapshot> snapshots) {
     byte[] row = Objects.requireNonNull(result).getRow();
-    if (null == row) {
+    if (row == null) {
       throw new IllegalArgumentException("Provided result had a null row");
     }
     final TableName targetTableName = getTableFromRowKey(row);
     Cell c = result.getColumnLatestCell(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY);
-    if (null == c) {
+    if (c == null) {
       throw new IllegalArgumentException("Result did not contain the expected column "
           + QUOTA_POLICY_COLUMN + ", " + result.toString());
     }
@@ -398,7 +398,8 @@ public class QuotaTableUtil {
   }
 
   /**
-   * Creates a {@link Put} to enable the given <code>policy</code> on the <code>table</code>.
+   * Creates a {@link Put} to store the given {@code snapshot} for the given {@code tableName} in
+   * the quota table.
    */
   public static Put createPutSpaceSnapshot(TableName tableName, SpaceQuotaSnapshot snapshot) {
     Put p = new Put(getTableRowKey(tableName));

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 8ff7623..e6d2db8 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
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 
 /**
- * A {@link QuotaSettings} implementation for implementing filesystem-use quotas.
+ * A {@link QuotaSettings} implementation for configuring filesystem-use quotas.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -38,33 +38,33 @@ class SpaceLimitSettings extends QuotaSettings {
 
   SpaceLimitSettings(TableName tableName, long sizeLimit, SpaceViolationPolicy violationPolicy) {
     super(null, Objects.requireNonNull(tableName), null);
-    if (0L > sizeLimit) {
+    if (sizeLimit < 0L) {
       throw new IllegalArgumentException("Size limit must be a non-negative value.");
     }
     proto = buildProtoAddQuota(sizeLimit, Objects.requireNonNull(violationPolicy));
   }
 
-  SpaceLimitSettings(TableName tableName, boolean remove) {
+  /**
+   * Constructs a {@code SpaceLimitSettings} to remove a space quota on the given {@code tableName}.
+   */
+  SpaceLimitSettings(TableName tableName) {
     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) {
     super(null, null, Objects.requireNonNull(namespace));
-    if (0L > sizeLimit) {
+    if (sizeLimit < 0L) {
       throw new IllegalArgumentException("Size limit must be a non-negative value.");
     }
     proto = buildProtoAddQuota(sizeLimit, Objects.requireNonNull(violationPolicy));
   }
 
-  SpaceLimitSettings(String namespace, boolean remove) {
+  /**
+   * Constructs a {@code SpaceLimitSettings} to remove a space quota on the given {@code namespace}.
+   */
+  SpaceLimitSettings(String namespace) {
     super(null, null, Objects.requireNonNull(namespace));
-    if (!remove) {
-      throw new IllegalArgumentException("A value of 'false' for removing a quota makes no sense");
-    }
     proto = buildProtoRemoveQuota();
   }
 
@@ -183,10 +183,10 @@ class SpaceLimitSettings extends QuotaSettings {
   public String toString() {
     StringBuilder sb = new StringBuilder();
     sb.append("TYPE => SPACE");
-    if (null != getTableName()) {
+    if (getTableName() != null) {
       sb.append(", TABLE => ").append(getTableName());
     }
-    if (null != getNamespace()) {
+    if (getNamespace() != null) {
       sb.append(", NAMESPACE => ").append(getNamespace());
     }
     if (proto.getQuota().getRemove()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
index 86d64d3..5c64880 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
@@ -36,7 +36,8 @@ public class SpaceQuotaSnapshot {
 
   /**
    * 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.
+   * If the quota is not in violation, the violation may be null. If the quota is in violation,
+   * there is guaranteed to be a non-null violation policy.
    */
   @InterfaceAudience.Private
   public static class SpaceQuotaStatus {
@@ -44,9 +45,17 @@ public class SpaceQuotaSnapshot {
     final SpaceViolationPolicy policy;
     final boolean inViolation;
 
+    /**
+     * Constructs a {@code SpaceQuotaSnapshot} which is in violation of the provided {@code policy}.
+     *
+     * Use {@link #notInViolation()} to obtain an instance of this class for the cases when the
+     * quota is not in violation.
+     *
+     * @param policy The non-null policy being violated.
+     */
     public SpaceQuotaStatus(SpaceViolationPolicy policy) {
-      this.policy = Objects.requireNonNull(policy);
-      this.inViolation = true;
+      // If the caller is instantiating a status, the policy must be non-null
+      this (Objects.requireNonNull(policy), true);
     }
 
     private SpaceQuotaStatus(SpaceViolationPolicy policy, boolean inViolation) {
@@ -55,16 +64,15 @@ public class SpaceQuotaSnapshot {
     }
 
     /**
-     * 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.
+     * Returns the violation policy, which may be null. It is guaranteed to be non-null if
+     * {@link #isInViolation()} is {@code true}, but may be null otherwise.
      */
     public SpaceViolationPolicy getPolicy() {
       return policy;
     }
 
     /**
-     * <code>true</code> if the quota is being violated, <code>false</code> otherwise.
+     * @return {@code true} if the quota is being violated, {@code false} otherwise.
      */
     public boolean isInViolation() {
       return inViolation;
@@ -104,14 +112,14 @@ public class SpaceQuotaSnapshot {
       QuotaProtos.SpaceQuotaStatus.Builder builder = QuotaProtos.SpaceQuotaStatus.newBuilder();
       builder.setInViolation(status.inViolation);
       if (status.isInViolation()) {
-        builder.setPolicy(ProtobufUtil.toProtoViolationPolicy(status.getPolicy()));
+        builder.setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(status.getPolicy()));
       }
       return builder.build();
     }
 
     public static SpaceQuotaStatus toStatus(QuotaProtos.SpaceQuotaStatus proto) {
       if (proto.getInViolation()) {
-        return new SpaceQuotaStatus(ProtobufUtil.toViolationPolicy(proto.getPolicy()));
+        return new SpaceQuotaStatus(ProtobufUtil.toViolationPolicy(proto.getViolationPolicy()));
       } else {
         return NOT_IN_VIOLATION;
       }
@@ -173,14 +181,14 @@ public class SpaceQuotaSnapshot {
 
   // 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());
+    return new SpaceQuotaSnapshot(SpaceQuotaStatus.toStatus(proto.getQuotaStatus()),
+        proto.getQuotaUsage(), proto.getQuotaLimit());
   }
 
   public static QuotaProtos.SpaceQuotaSnapshot toProtoSnapshot(SpaceQuotaSnapshot snapshot) {
     return QuotaProtos.SpaceQuotaSnapshot.newBuilder()
-        .setStatus(SpaceQuotaStatus.toProto(snapshot.getQuotaStatus()))
-        .setUsage(snapshot.getUsage()).setLimit(snapshot.getLimit()).build();
+        .setQuotaStatus(SpaceQuotaStatus.toProto(snapshot.getQuotaStatus()))
+        .setQuotaUsage(snapshot.getUsage()).setQuotaLimit(snapshot.getLimit()).build();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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
index c63acb0..34d2542 100644
--- 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
@@ -21,6 +21,9 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Enumeration that represents the action HBase will take when a space quota is violated.
+ *
+ * The target for a violation policy is either an HBase table or namespace. In the case of a
+ * namespace, it is treated as a collection of tables (all tables are subject to the same policy).
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
@@ -40,5 +43,5 @@ public enum SpaceViolationPolicy {
   /**
    * Disallows any updates (but allows deletes and compactions) on the table(s).
    */
-  NO_INSERTS,
+  NO_INSERTS;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
index be579be..7625f84 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
@@ -19,7 +19,7 @@ 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.
+ * A collection of exposed metrics for space quotas from the HBase Master.
  */
 public interface MetricsMasterQuotaSource extends BaseSource {
 
@@ -46,30 +46,41 @@ public interface MetricsMasterQuotaSource extends BaseSource {
 
   /**
    * Updates the metric tracking the number of space quotas defined in the system.
+   *
+   * @param numSpaceQuotas The number of space quotas defined
    */
   void updateNumSpaceQuotas(long numSpaceQuotas);
 
   /**
    * Updates the metric tracking the number of tables the master has computed to be in
    * violation of their space quota.
+   *
+   * @param numTablesInViolation The number of tables violating a 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.
+   *
+   * @param numNamespacesInViolation The number of namespaces violating a space quota
    */
   void updateNumNamespacesInSpaceQuotaViolation(long numNamespacesInViolation);
 
   /**
    * Updates the metric tracking the number of region size reports the master is currently
    * retaining in memory.
+   *
+   * @param numCurrentRegionSizeReports The number of region size reports the master is holding in
+   *    memory
    */
   void updateNumCurrentSpaceQuotaRegionSizeReports(long numCurrentRegionSizeReports);
 
   /**
    * Updates the metric tracking the amount of time taken by the {@code QuotaObserverChore}
    * which runs periodically.
+   *
+   * @param time The execution time of the chore in milliseconds
    */
   void incrementSpaceQuotaObserverChoreTime(long time);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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
index 12fa66d..3175aad 100644
--- 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
@@ -19,7 +19,7 @@ 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.
+ * A collection of exposed metrics for space quotas from an HBase RegionServer.
  */
 public interface MetricsRegionServerQuotaSource extends BaseSource {
 
@@ -37,18 +37,24 @@ public interface MetricsRegionServerQuotaSource extends BaseSource {
   /**
    * Updates the metric tracking how many tables this RegionServer has received
    * {@code SpaceQuotaSnapshot}s for.
+   *
+   * @param numSnapshots The number of {@code SpaceQuotaSnapshot}s received from the Master.
    */
   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.
+   *
+   * @param time The execution time of the chore in milliseconds.
    */
   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.
+   * latest information on space quotas from the {@code hbase:quota} table.
+   *
+   * @param time The execution time of the chore in milliseconds.
    */
   void incrementSpaceQuotaRefresherChoreTime(long time);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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
index 96a57c4..7655160 100644
--- 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
@@ -28,7 +28,7 @@ public class MetricsMasterQuotaSourceFactoryImpl implements MetricsMasterQuotaSo
 
   @Override
   public synchronized MetricsMasterQuotaSource create(MetricsMasterWrapper masterWrapper) {
-    if (null == quotaSource) {
+    if (quotaSource == null) {
       quotaSource = new MetricsMasterQuotaSourceImpl(masterWrapper);
     }
     return quotaSource;

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
index 064f7fc..342aa1e 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
@@ -27,6 +27,10 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.lib.Interns;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 
+/**
+ * Implementation of {@link MetricsMasterQuotaSource} which writes the values passed in via the
+ * interface to the metrics backend.
+ */
 @InterfaceAudience.Private
 public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements MetricsMasterQuotaSource {
   private final MetricsMasterWrapper wrapper;
@@ -87,11 +91,11 @@ public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements Metr
   @Override
   public void getMetrics(MetricsCollector metricsCollector, boolean all) {
     MetricsRecordBuilder record = metricsCollector.addRecord(metricsRegistry.info());
-    if (null != wrapper) {
+    if (wrapper != null) {
       // Summarize the tables
       Map<String,Entry<Long,Long>> tableUsages = wrapper.getTableSpaceUtilization();
       String tableSummary = "[]";
-      if (null != tableUsages && !tableUsages.isEmpty()) {
+      if (tableUsages != null && !tableUsages.isEmpty()) {
         tableSummary = generateJsonQuotaSummary(tableUsages.entrySet(), "table");
       }
       record.tag(Interns.info(TABLE_QUOTA_USAGE_NAME, TABLE_QUOTA_USAGE_DESC), tableSummary);
@@ -99,7 +103,7 @@ public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements Metr
       // Summarize the namespaces
       String nsSummary = "[]";
       Map<String,Entry<Long,Long>> namespaceUsages = wrapper.getNamespaceSpaceUtilization();
-      if (null != namespaceUsages && !namespaceUsages.isEmpty()) {
+      if (namespaceUsages != null && !namespaceUsages.isEmpty()) {
         nsSummary = generateJsonQuotaSummary(namespaceUsages.entrySet(), "namespace");
       }
       record.tag(Interns.info(NS_QUOTA_USAGE_NAME, NS_QUOTA_USAGE_DESC), nsSummary);

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 8e52626..9145a63 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
@@ -24904,7 +24904,7 @@ public final class AdminProtos {
 
       /**
        * <pre>
-       ** Fetches the RegionServer's view of space quotas 
+       ** Fetches the RegionServer's view of space quotas
        * </pre>
        *
        * <code>rpc GetSpaceQuotaSnapshots(.hbase.pb.GetSpaceQuotaSnapshotsRequest) returns (.hbase.pb.GetSpaceQuotaSnapshotsResponse);</code>
@@ -24916,7 +24916,7 @@ public final class AdminProtos {
 
       /**
        * <pre>
-       ** Fetches the RegionServer's space quota active enforcements 
+       ** Fetches the RegionServer's space quota active enforcements
        * </pre>
        *
        * <code>rpc GetSpaceQuotaEnforcements(.hbase.pb.GetSpaceQuotaEnforcementsRequest) returns (.hbase.pb.GetSpaceQuotaEnforcementsResponse);</code>
@@ -25415,7 +25415,7 @@ public final class AdminProtos {
 
     /**
      * <pre>
-     ** Fetches the RegionServer's view of space quotas 
+     ** Fetches the RegionServer's view of space quotas
      * </pre>
      *
      * <code>rpc GetSpaceQuotaSnapshots(.hbase.pb.GetSpaceQuotaSnapshotsRequest) returns (.hbase.pb.GetSpaceQuotaSnapshotsResponse);</code>
@@ -25427,7 +25427,7 @@ public final class AdminProtos {
 
     /**
      * <pre>
-     ** Fetches the RegionServer's space quota active enforcements 
+     ** Fetches the RegionServer's space quota active enforcements
      * </pre>
      *
      * <code>rpc GetSpaceQuotaEnforcements(.hbase.pb.GetSpaceQuotaEnforcementsRequest) returns (.hbase.pb.GetSpaceQuotaEnforcementsResponse);</code>

http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 87d51e9..31ac414 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
@@ -64879,7 +64879,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<
@@ -72296,7 +72296,7 @@ public final class MasterProtos {
 
       /**
        * <pre>
-       ** Fetches the Master's view of space utilization 
+       ** Fetches the Master's view of space utilization
        * </pre>
        *
        * <code>rpc GetSpaceQuotaRegionSizes(.hbase.pb.GetSpaceQuotaRegionSizesRequest) returns (.hbase.pb.GetSpaceQuotaRegionSizesResponse);</code>
@@ -72308,7 +72308,7 @@ public final class MasterProtos {
 
       /**
        * <pre>
-       ** Fetches the Master's view of quotas 
+       ** Fetches the Master's view of quotas
        * </pre>
        *
        * <code>rpc GetQuotaStates(.hbase.pb.GetQuotaStatesRequest) returns (.hbase.pb.GetQuotaStatesResponse);</code>
@@ -74284,7 +74284,7 @@ public final class MasterProtos {
 
     /**
      * <pre>
-     ** Fetches the Master's view of space utilization 
+     ** Fetches the Master's view of space utilization
      * </pre>
      *
      * <code>rpc GetSpaceQuotaRegionSizes(.hbase.pb.GetSpaceQuotaRegionSizesRequest) returns (.hbase.pb.GetSpaceQuotaRegionSizesResponse);</code>
@@ -74296,7 +74296,7 @@ public final class MasterProtos {
 
     /**
      * <pre>
-     ** Fetches the Master's view of quotas 
+     ** Fetches the Master's view of quotas
      * </pre>
      *
      * <code>rpc GetQuotaStates(.hbase.pb.GetQuotaStatesRequest) returns (.hbase.pb.GetQuotaStatesResponse);</code>


[3/4] hbase git commit: HBASE-17955 Various reviewboard improvements to space quota work

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/70bcf3fe/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 4577bcf..e8a57e9 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<
@@ -6143,13 +6143,13 @@ public final class QuotaProtos {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
-     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
      */
-    boolean hasPolicy();
+    boolean hasViolationPolicy();
     /**
-     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy getPolicy();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy getViolationPolicy();
 
     /**
      * <code>optional bool in_violation = 2;</code>
@@ -6163,7 +6163,7 @@ public final class QuotaProtos {
   /**
    * <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.
+   * or it is in violation there is a violation policy which should be in effect.
    * </pre>
    *
    * Protobuf type {@code hbase.pb.SpaceQuotaStatus}
@@ -6177,7 +6177,7 @@ public final class QuotaProtos {
       super(builder);
     }
     private SpaceQuotaStatus() {
-      policy_ = 1;
+      violationPolicy_ = 1;
       inViolation_ = false;
     }
 
@@ -6216,7 +6216,7 @@ public final class QuotaProtos {
                 unknownFields.mergeVarintField(1, rawValue);
               } else {
                 bitField0_ |= 0x00000001;
-                policy_ = rawValue;
+                violationPolicy_ = rawValue;
               }
               break;
             }
@@ -6250,19 +6250,19 @@ public final class QuotaProtos {
     }
 
     private int bitField0_;
-    public static final int POLICY_FIELD_NUMBER = 1;
-    private int policy_;
+    public static final int VIOLATION_POLICY_FIELD_NUMBER = 1;
+    private int violationPolicy_;
     /**
-     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
      */
-    public boolean hasPolicy() {
+    public boolean hasViolationPolicy() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+     * <code>optional .hbase.pb.SpaceViolationPolicy violation_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_);
+    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;
     }
 
@@ -6294,7 +6294,7 @@ public final class QuotaProtos {
     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_);
+        output.writeEnum(1, violationPolicy_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeBool(2, inViolation_);
@@ -6309,7 +6309,7 @@ public final class QuotaProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeEnumSize(1, policy_);
+          .computeEnumSize(1, violationPolicy_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
@@ -6332,9 +6332,9 @@ public final class QuotaProtos {
       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 && (hasViolationPolicy() == other.hasViolationPolicy());
+      if (hasViolationPolicy()) {
+        result = result && violationPolicy_ == other.violationPolicy_;
       }
       result = result && (hasInViolation() == other.hasInViolation());
       if (hasInViolation()) {
@@ -6352,9 +6352,9 @@ public final class QuotaProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptor().hashCode();
-      if (hasPolicy()) {
-        hash = (37 * hash) + POLICY_FIELD_NUMBER;
-        hash = (53 * hash) + policy_;
+      if (hasViolationPolicy()) {
+        hash = (37 * hash) + VIOLATION_POLICY_FIELD_NUMBER;
+        hash = (53 * hash) + violationPolicy_;
       }
       if (hasInViolation()) {
         hash = (37 * hash) + IN_VIOLATION_FIELD_NUMBER;
@@ -6446,7 +6446,7 @@ public final class QuotaProtos {
     /**
      * <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.
+     * or it is in violation there is a violation policy which should be in effect.
      * </pre>
      *
      * Protobuf type {@code hbase.pb.SpaceQuotaStatus}
@@ -6484,7 +6484,7 @@ public final class QuotaProtos {
       }
       public Builder clear() {
         super.clear();
-        policy_ = 1;
+        violationPolicy_ = 1;
         bitField0_ = (bitField0_ & ~0x00000001);
         inViolation_ = false;
         bitField0_ = (bitField0_ & ~0x00000002);
@@ -6515,7 +6515,7 @@ public final class QuotaProtos {
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.policy_ = policy_;
+        result.violationPolicy_ = violationPolicy_;
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
@@ -6562,8 +6562,8 @@ public final class QuotaProtos {
 
       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.hasViolationPolicy()) {
+          setViolationPolicy(other.getViolationPolicy());
         }
         if (other.hasInViolation()) {
           setInViolation(other.getInViolation());
@@ -6596,38 +6596,38 @@ public final class QuotaProtos {
       }
       private int bitField0_;
 
-      private int policy_ = 1;
+      private int violationPolicy_ = 1;
       /**
-       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
        */
-      public boolean hasPolicy() {
+      public boolean hasViolationPolicy() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_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_);
+      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;
       }
       /**
-       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
        */
-      public Builder setPolicy(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy value) {
+      public Builder setViolationPolicy(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy value) {
         if (value == null) {
           throw new NullPointerException();
         }
         bitField0_ |= 0x00000001;
-        policy_ = value.getNumber();
+        violationPolicy_ = value.getNumber();
         onChanged();
         return this;
       }
       /**
-       * <code>optional .hbase.pb.SpaceViolationPolicy policy = 1;</code>
+       * <code>optional .hbase.pb.SpaceViolationPolicy violation_policy = 1;</code>
        */
-      public Builder clearPolicy() {
+      public Builder clearViolationPolicy() {
         bitField0_ = (bitField0_ & ~0x00000001);
-        policy_ = 1;
+        violationPolicy_ = 1;
         onChanged();
         return this;
       }
@@ -6717,35 +6717,35 @@ public final class QuotaProtos {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
      */
-    boolean hasStatus();
+    boolean hasQuotaStatus();
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus getStatus();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus getQuotaStatus();
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getStatusOrBuilder();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getQuotaStatusOrBuilder();
 
     /**
-     * <code>optional uint64 usage = 2;</code>
+     * <code>optional uint64 quota_usage = 2;</code>
      */
-    boolean hasUsage();
+    boolean hasQuotaUsage();
     /**
-     * <code>optional uint64 usage = 2;</code>
+     * <code>optional uint64 quota_usage = 2;</code>
      */
-    long getUsage();
+    long getQuotaUsage();
 
     /**
-     * <code>optional uint64 limit = 3;</code>
+     * <code>optional uint64 quota_limit = 3;</code>
      */
-    boolean hasLimit();
+    boolean hasQuotaLimit();
     /**
-     * <code>optional uint64 limit = 3;</code>
+     * <code>optional uint64 quota_limit = 3;</code>
      */
-    long getLimit();
+    long getQuotaLimit();
   }
   /**
    * <pre>
@@ -6764,8 +6764,8 @@ public final class QuotaProtos {
       super(builder);
     }
     private SpaceQuotaSnapshot() {
-      usage_ = 0L;
-      limit_ = 0L;
+      quotaUsage_ = 0L;
+      quotaLimit_ = 0L;
     }
 
     @java.lang.Override
@@ -6799,24 +6799,24 @@ public final class QuotaProtos {
             case 10: {
               org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder subBuilder = null;
               if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = status_.toBuilder();
+                subBuilder = quotaStatus_.toBuilder();
               }
-              status_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.PARSER, extensionRegistry);
+              quotaStatus_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.PARSER, extensionRegistry);
               if (subBuilder != null) {
-                subBuilder.mergeFrom(status_);
-                status_ = subBuilder.buildPartial();
+                subBuilder.mergeFrom(quotaStatus_);
+                quotaStatus_ = subBuilder.buildPartial();
               }
               bitField0_ |= 0x00000001;
               break;
             }
             case 16: {
               bitField0_ |= 0x00000002;
-              usage_ = input.readUInt64();
+              quotaUsage_ = input.readUInt64();
               break;
             }
             case 24: {
               bitField0_ |= 0x00000004;
-              limit_ = input.readUInt64();
+              quotaLimit_ = input.readUInt64();
               break;
             }
           }
@@ -6844,55 +6844,55 @@ public final class QuotaProtos {
     }
 
     private int bitField0_;
-    public static final int STATUS_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus status_;
+    public static final int QUOTA_STATUS_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus quotaStatus_;
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
      */
-    public boolean hasStatus() {
+    public boolean hasQuotaStatus() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_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_;
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus getQuotaStatus() {
+      return quotaStatus_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance() : quotaStatus_;
     }
     /**
-     * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+     * <code>optional .hbase.pb.SpaceQuotaStatus quota_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 org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getQuotaStatusOrBuilder() {
+      return quotaStatus_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance() : quotaStatus_;
     }
 
-    public static final int USAGE_FIELD_NUMBER = 2;
-    private long usage_;
+    public static final int QUOTA_USAGE_FIELD_NUMBER = 2;
+    private long quotaUsage_;
     /**
-     * <code>optional uint64 usage = 2;</code>
+     * <code>optional uint64 quota_usage = 2;</code>
      */
-    public boolean hasUsage() {
+    public boolean hasQuotaUsage() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional uint64 usage = 2;</code>
+     * <code>optional uint64 quota_usage = 2;</code>
      */
-    public long getUsage() {
-      return usage_;
+    public long getQuotaUsage() {
+      return quotaUsage_;
     }
 
-    public static final int LIMIT_FIELD_NUMBER = 3;
-    private long limit_;
+    public static final int QUOTA_LIMIT_FIELD_NUMBER = 3;
+    private long quotaLimit_;
     /**
-     * <code>optional uint64 limit = 3;</code>
+     * <code>optional uint64 quota_limit = 3;</code>
      */
-    public boolean hasLimit() {
+    public boolean hasQuotaLimit() {
       return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>optional uint64 limit = 3;</code>
+     * <code>optional uint64 quota_limit = 3;</code>
      */
-    public long getLimit() {
-      return limit_;
+    public long getQuotaLimit() {
+      return quotaLimit_;
     }
 
     private byte memoizedIsInitialized = -1;
@@ -6908,13 +6908,13 @@ public final class QuotaProtos {
     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());
+        output.writeMessage(1, getQuotaStatus());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeUInt64(2, usage_);
+        output.writeUInt64(2, quotaUsage_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeUInt64(3, limit_);
+        output.writeUInt64(3, quotaLimit_);
       }
       unknownFields.writeTo(output);
     }
@@ -6926,15 +6926,15 @@ public final class QuotaProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(1, getStatus());
+          .computeMessageSize(1, getQuotaStatus());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(2, usage_);
+          .computeUInt64Size(2, quotaUsage_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(3, limit_);
+          .computeUInt64Size(3, quotaLimit_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -6953,20 +6953,20 @@ public final class QuotaProtos {
       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 && (hasQuotaStatus() == other.hasQuotaStatus());
+      if (hasQuotaStatus()) {
+        result = result && getQuotaStatus()
+            .equals(other.getQuotaStatus());
       }
-      result = result && (hasUsage() == other.hasUsage());
-      if (hasUsage()) {
-        result = result && (getUsage()
-            == other.getUsage());
+      result = result && (hasQuotaUsage() == other.hasQuotaUsage());
+      if (hasQuotaUsage()) {
+        result = result && (getQuotaUsage()
+            == other.getQuotaUsage());
       }
-      result = result && (hasLimit() == other.hasLimit());
-      if (hasLimit()) {
-        result = result && (getLimit()
-            == other.getLimit());
+      result = result && (hasQuotaLimit() == other.hasQuotaLimit());
+      if (hasQuotaLimit()) {
+        result = result && (getQuotaLimit()
+            == other.getQuotaLimit());
       }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
@@ -6979,19 +6979,19 @@ public final class QuotaProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptor().hashCode();
-      if (hasStatus()) {
-        hash = (37 * hash) + STATUS_FIELD_NUMBER;
-        hash = (53 * hash) + getStatus().hashCode();
+      if (hasQuotaStatus()) {
+        hash = (37 * hash) + QUOTA_STATUS_FIELD_NUMBER;
+        hash = (53 * hash) + getQuotaStatus().hashCode();
       }
-      if (hasUsage()) {
-        hash = (37 * hash) + USAGE_FIELD_NUMBER;
+      if (hasQuotaUsage()) {
+        hash = (37 * hash) + QUOTA_USAGE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getUsage());
+            getQuotaUsage());
       }
-      if (hasLimit()) {
-        hash = (37 * hash) + LIMIT_FIELD_NUMBER;
+      if (hasQuotaLimit()) {
+        hash = (37 * hash) + QUOTA_LIMIT_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getLimit());
+            getQuotaLimit());
       }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
@@ -7112,20 +7112,20 @@ public final class QuotaProtos {
       private void maybeForceBuilderInitialization() {
         if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
-          getStatusFieldBuilder();
+          getQuotaStatusFieldBuilder();
         }
       }
       public Builder clear() {
         super.clear();
-        if (statusBuilder_ == null) {
-          status_ = null;
+        if (quotaStatusBuilder_ == null) {
+          quotaStatus_ = null;
         } else {
-          statusBuilder_.clear();
+          quotaStatusBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        usage_ = 0L;
+        quotaUsage_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
-        limit_ = 0L;
+        quotaLimit_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
@@ -7154,19 +7154,19 @@ public final class QuotaProtos {
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        if (statusBuilder_ == null) {
-          result.status_ = status_;
+        if (quotaStatusBuilder_ == null) {
+          result.quotaStatus_ = quotaStatus_;
         } else {
-          result.status_ = statusBuilder_.build();
+          result.quotaStatus_ = quotaStatusBuilder_.build();
         }
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.usage_ = usage_;
+        result.quotaUsage_ = quotaUsage_;
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000004;
         }
-        result.limit_ = limit_;
+        result.quotaLimit_ = quotaLimit_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -7209,14 +7209,14 @@ public final class QuotaProtos {
 
       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.hasQuotaStatus()) {
+          mergeQuotaStatus(other.getQuotaStatus());
         }
-        if (other.hasUsage()) {
-          setUsage(other.getUsage());
+        if (other.hasQuotaUsage()) {
+          setQuotaUsage(other.getQuotaUsage());
         }
-        if (other.hasLimit()) {
-          setLimit(other.getLimit());
+        if (other.hasQuotaLimit()) {
+          setQuotaLimit(other.getQuotaLimit());
         }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
@@ -7246,184 +7246,184 @@ public final class QuotaProtos {
       }
       private int bitField0_;
 
-      private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus status_ = null;
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus quotaStatus_ = 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_;
+          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> quotaStatusBuilder_;
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public boolean hasStatus() {
+      public boolean hasQuotaStatus() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_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_;
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus getQuotaStatus() {
+        if (quotaStatusBuilder_ == null) {
+          return quotaStatus_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance() : quotaStatus_;
         } else {
-          return statusBuilder_.getMessage();
+          return quotaStatusBuilder_.getMessage();
         }
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public Builder setStatus(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
-        if (statusBuilder_ == null) {
+      public Builder setQuotaStatus(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
+        if (quotaStatusBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          status_ = value;
+          quotaStatus_ = value;
           onChanged();
         } else {
-          statusBuilder_.setMessage(value);
+          quotaStatusBuilder_.setMessage(value);
         }
         bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public Builder setStatus(
+      public Builder setQuotaStatus(
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder builderForValue) {
-        if (statusBuilder_ == null) {
-          status_ = builderForValue.build();
+        if (quotaStatusBuilder_ == null) {
+          quotaStatus_ = builderForValue.build();
           onChanged();
         } else {
-          statusBuilder_.setMessage(builderForValue.build());
+          quotaStatusBuilder_.setMessage(builderForValue.build());
         }
         bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public Builder mergeStatus(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
-        if (statusBuilder_ == null) {
+      public Builder mergeQuotaStatus(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus value) {
+        if (quotaStatusBuilder_ == 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();
+              quotaStatus_ != null &&
+              quotaStatus_ != org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance()) {
+            quotaStatus_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.newBuilder(quotaStatus_).mergeFrom(value).buildPartial();
           } else {
-            status_ = value;
+            quotaStatus_ = value;
           }
           onChanged();
         } else {
-          statusBuilder_.mergeFrom(value);
+          quotaStatusBuilder_.mergeFrom(value);
         }
         bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public Builder clearStatus() {
-        if (statusBuilder_ == null) {
-          status_ = null;
+      public Builder clearQuotaStatus() {
+        if (quotaStatusBuilder_ == null) {
+          quotaStatus_ = null;
           onChanged();
         } else {
-          statusBuilder_.clear();
+          quotaStatusBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder getStatusBuilder() {
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder getQuotaStatusBuilder() {
         bitField0_ |= 0x00000001;
         onChanged();
-        return getStatusFieldBuilder().getBuilder();
+        return getQuotaStatusFieldBuilder().getBuilder();
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getStatusOrBuilder() {
-        if (statusBuilder_ != null) {
-          return statusBuilder_.getMessageOrBuilder();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder getQuotaStatusOrBuilder() {
+        if (quotaStatusBuilder_ != null) {
+          return quotaStatusBuilder_.getMessageOrBuilder();
         } else {
-          return status_ == null ?
-              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance() : status_;
+          return quotaStatus_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.getDefaultInstance() : quotaStatus_;
         }
       }
       /**
-       * <code>optional .hbase.pb.SpaceQuotaStatus status = 1;</code>
+       * <code>optional .hbase.pb.SpaceQuotaStatus quota_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>
+          getQuotaStatusFieldBuilder() {
+        if (quotaStatusBuilder_ == null) {
+          quotaStatusBuilder_ = 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(),
+                  getQuotaStatus(),
                   getParentForChildren(),
                   isClean());
-          status_ = null;
+          quotaStatus_ = null;
         }
-        return statusBuilder_;
+        return quotaStatusBuilder_;
       }
 
-      private long usage_ ;
+      private long quotaUsage_ ;
       /**
-       * <code>optional uint64 usage = 2;</code>
+       * <code>optional uint64 quota_usage = 2;</code>
        */
-      public boolean hasUsage() {
+      public boolean hasQuotaUsage() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional uint64 usage = 2;</code>
+       * <code>optional uint64 quota_usage = 2;</code>
        */
-      public long getUsage() {
-        return usage_;
+      public long getQuotaUsage() {
+        return quotaUsage_;
       }
       /**
-       * <code>optional uint64 usage = 2;</code>
+       * <code>optional uint64 quota_usage = 2;</code>
        */
-      public Builder setUsage(long value) {
+      public Builder setQuotaUsage(long value) {
         bitField0_ |= 0x00000002;
-        usage_ = value;
+        quotaUsage_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional uint64 usage = 2;</code>
+       * <code>optional uint64 quota_usage = 2;</code>
        */
-      public Builder clearUsage() {
+      public Builder clearQuotaUsage() {
         bitField0_ = (bitField0_ & ~0x00000002);
-        usage_ = 0L;
+        quotaUsage_ = 0L;
         onChanged();
         return this;
       }
 
-      private long limit_ ;
+      private long quotaLimit_ ;
       /**
-       * <code>optional uint64 limit = 3;</code>
+       * <code>optional uint64 quota_limit = 3;</code>
        */
-      public boolean hasLimit() {
+      public boolean hasQuotaLimit() {
         return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>optional uint64 limit = 3;</code>
+       * <code>optional uint64 quota_limit = 3;</code>
        */
-      public long getLimit() {
-        return limit_;
+      public long getQuotaLimit() {
+        return quotaLimit_;
       }
       /**
-       * <code>optional uint64 limit = 3;</code>
+       * <code>optional uint64 quota_limit = 3;</code>
        */
-      public Builder setLimit(long value) {
+      public Builder setQuotaLimit(long value) {
         bitField0_ |= 0x00000004;
-        limit_ = value;
+        quotaLimit_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional uint64 limit = 3;</code>
+       * <code>optional uint64 quota_limit = 3;</code>
        */
-      public Builder clearLimit() {
+      public Builder clearQuotaLimit() {
         bitField0_ = (bitField0_ & ~0x00000004);
-        limit_ = 0L;
+        quotaLimit_ = 0L;
         onChanged();
         return this;
       }
@@ -7856,7 +7856,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes>
         getSizesList();
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
@@ -7869,7 +7869,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder> 
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder>
         getSizesOrBuilderList();
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
@@ -8540,7 +8540,7 @@ public final class QuotaProtos {
          * <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> 
+            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<
@@ -8644,7 +8644,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder>
         getSizesOrBuilderList() {
       return sizes_;
     }
@@ -8951,7 +8951,7 @@ public final class QuotaProtos {
               sizesBuilder_ = null;
               sizes_ = other.sizes_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              sizesBuilder_ = 
+              sizesBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getSizesFieldBuilder() : null;
             } else {
@@ -9187,7 +9187,7 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder>
            getSizesOrBuilderList() {
         if (sizesBuilder_ != null) {
           return sizesBuilder_.getMessageOrBuilderList();
@@ -9213,12 +9213,12 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.Builder>
            getSizesBuilderList() {
         return getSizesFieldBuilder().getBuilderList();
       }
       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> 
+          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>
           getSizesFieldBuilder() {
         if (sizesBuilder_ == null) {
           sizesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -9660,7 +9660,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot>
         getSnapshotsList();
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
@@ -9673,7 +9673,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder> 
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder>
         getSnapshotsOrBuilderList();
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
@@ -10377,7 +10377,7 @@ public final class QuotaProtos {
          * <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> 
+            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<
@@ -10495,7 +10495,7 @@ public final class QuotaProtos {
          * <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> 
+            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<
@@ -10567,7 +10567,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder>
         getSnapshotsOrBuilderList() {
       return snapshots_;
     }
@@ -10874,7 +10874,7 @@ public final class QuotaProtos {
               snapshotsBuilder_ = null;
               snapshots_ = other.snapshots_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              snapshotsBuilder_ = 
+              snapshotsBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getSnapshotsFieldBuilder() : null;
             } else {
@@ -11110,7 +11110,7 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder>
            getSnapshotsOrBuilderList() {
         if (snapshotsBuilder_ != null) {
           return snapshotsBuilder_.getMessageOrBuilderList();
@@ -11136,12 +11136,12 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot.Builder>
            getSnapshotsBuilderList() {
         return getSnapshotsFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder>
           getSnapshotsFieldBuilder() {
         if (snapshotsBuilder_ == null) {
           snapshotsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -11583,7 +11583,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy violation_policies = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy>
         getViolationPoliciesList();
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy violation_policies = 1;</code>
@@ -11596,7 +11596,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy violation_policies = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicyOrBuilder> 
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicyOrBuilder>
         getViolationPoliciesOrBuilderList();
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy violation_policies = 1;</code>
@@ -12280,7 +12280,7 @@ public final class QuotaProtos {
          * <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> 
+            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<
@@ -12388,7 +12388,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy violation_policies = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicyOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicyOrBuilder>
         getViolationPoliciesOrBuilderList() {
       return violationPolicies_;
     }
@@ -12695,7 +12695,7 @@ public final class QuotaProtos {
               violationPoliciesBuilder_ = null;
               violationPolicies_ = other.violationPolicies_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              violationPoliciesBuilder_ = 
+              violationPoliciesBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getViolationPoliciesFieldBuilder() : null;
             } else {
@@ -12931,7 +12931,7 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy violation_policies = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicyOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicyOrBuilder>
            getViolationPoliciesOrBuilderList() {
         if (violationPoliciesBuilder_ != null) {
           return violationPoliciesBuilder_.getMessageOrBuilderList();
@@ -12957,12 +12957,12 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy violation_policies = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy.Builder>
            getViolationPoliciesBuilderList() {
         return getViolationPoliciesFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicyOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicyOrBuilder>
           getViolationPoliciesFieldBuilder() {
         if (violationPoliciesBuilder_ == null) {
           violationPoliciesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -13404,7 +13404,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot>
         getTableSnapshotsList();
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
@@ -13417,7 +13417,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder> 
+    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>
@@ -13428,7 +13428,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot>
         getNsSnapshotsList();
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
@@ -13441,7 +13441,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder> 
+    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>
@@ -14150,7 +14150,7 @@ public final class QuotaProtos {
          * <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> 
+            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<
@@ -14268,7 +14268,7 @@ public final class QuotaProtos {
          * <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> 
+            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<
@@ -14463,7 +14463,7 @@ public final class QuotaProtos {
         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 bs =
               (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
@@ -14479,7 +14479,7 @@ public final class QuotaProtos {
           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 b =
               org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           namespace_ = b;
@@ -14861,7 +14861,7 @@ public final class QuotaProtos {
             getNamespaceBytes() {
           java.lang.Object ref = namespace_;
           if (ref instanceof String) {
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            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;
@@ -15011,7 +15011,7 @@ public final class QuotaProtos {
          * <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> 
+            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<
@@ -15083,7 +15083,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder>
         getTableSnapshotsOrBuilderList() {
       return tableSnapshots_;
     }
@@ -15118,7 +15118,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder>
         getNsSnapshotsOrBuilderList() {
       return nsSnapshots_;
     }
@@ -15454,7 +15454,7 @@ public final class QuotaProtos {
               tableSnapshotsBuilder_ = null;
               tableSnapshots_ = other.tableSnapshots_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              tableSnapshotsBuilder_ = 
+              tableSnapshotsBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getTableSnapshotsFieldBuilder() : null;
             } else {
@@ -15480,7 +15480,7 @@ public final class QuotaProtos {
               nsSnapshotsBuilder_ = null;
               nsSnapshots_ = other.nsSnapshots_;
               bitField0_ = (bitField0_ & ~0x00000002);
-              nsSnapshotsBuilder_ = 
+              nsSnapshotsBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getNsSnapshotsFieldBuilder() : null;
             } else {
@@ -15716,7 +15716,7 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder>
            getTableSnapshotsOrBuilderList() {
         if (tableSnapshotsBuilder_ != null) {
           return tableSnapshotsBuilder_.getMessageOrBuilderList();
@@ -15742,12 +15742,12 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.Builder>
            getTableSnapshotsBuilderList() {
         return getTableSnapshotsFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder>
           getTableSnapshotsFieldBuilder() {
         if (tableSnapshotsBuilder_ == null) {
           tableSnapshotsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -15956,7 +15956,7 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder>
            getNsSnapshotsOrBuilderList() {
         if (nsSnapshotsBuilder_ != null) {
           return nsSnapshotsBuilder_.getMessageOrBuilderList();
@@ -15982,12 +15982,12 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.Builder>
            getNsSnapshotsBuilderList() {
         return getNsSnapshotsFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder>
           getNsSnapshotsFieldBuilder() {
         if (nsSnapshotsBuilder_ == null) {
           nsSnapshotsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -16051,112 +16051,112 @@ 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;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SpaceQuotaStatus_descriptor;
-  private static final 
+  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 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaRegionSizesRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaRegionSizesRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_RegionSizes_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_RegionSizes_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaSnapshotsRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaSnapshotsRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaSnapshotsResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaSnapshotsResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaSnapshotsResponse_TableQuotaSnapshot_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaSnapshotsResponse_TableQuotaSnapshot_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaEnforcementsRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaEnforcementsRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaEnforcementsResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaEnforcementsResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaEnforcementsResponse_TableViolationPolicy_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaEnforcementsResponse_TableViolationPolicy_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetQuotaStatesRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetQuotaStatesRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetQuotaStatesResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetQuotaStatesResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetQuotaStatesResponse_TableQuotaSnapshot_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetQuotaStatesResponse_TableQuotaSnapshot_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetQuotaStatesResponse_NamespaceQuotaSnapshot_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetQuotaStatesResponse_NamespaceQuotaSnapshot_fieldAccessorTable;
 
@@ -16188,50 +16188,51 @@ public final class QuotaProtos {
       "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\037GetSpaceQuotaRegionS" +
-      "izesRequest\"\257\001\n GetSpaceQuotaRegionSizes" +
-      "Response\022E\n\005sizes\030\001 \003(\01326.hbase.pb.GetSp" +
-      "aceQuotaRegionSizesResponse.RegionSizes\032" +
-      "D\n\013RegionSizes\022\'\n\ntable_name\030\001 \001(\0132\023.hba",
-      "se.pb.TableName\022\014\n\004size\030\002 \001(\004\"\037\n\035GetSpac" +
-      "eQuotaSnapshotsRequest\"\337\001\n\036GetSpaceQuota" +
-      "SnapshotsResponse\022N\n\tsnapshots\030\001 \003(\0132;.h" +
-      "base.pb.GetSpaceQuotaSnapshotsResponse.T" +
-      "ableQuotaSnapshot\032m\n\022TableQuotaSnapshot\022" +
-      "\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.TableName" +
-      "\022.\n\010snapshot\030\002 \001(\0132\034.hbase.pb.SpaceQuota" +
-      "Snapshot\"\"\n GetSpaceQuotaEnforcementsReq" +
-      "uest\"\374\001\n!GetSpaceQuotaEnforcementsRespon" +
-      "se\022\\\n\022violation_policies\030\001 \003(\0132@.hbase.p",
-      "b.GetSpaceQuotaEnforcementsResponse.Tabl" +
-      "eViolationPolicy\032y\n\024TableViolationPolicy" +
-      "\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.TableNam" +
-      "e\0228\n\020violation_policy\030\002 \001(\0162\036.hbase.pb.S" +
-      "paceViolationPolicy\"\027\n\025GetQuotaStatesReq" +
-      "uest\"\201\003\n\026GetQuotaStatesResponse\022L\n\017table" +
-      "_snapshots\030\001 \003(\01323.hbase.pb.GetQuotaStat" +
-      "esResponse.TableQuotaSnapshot\022M\n\014ns_snap" +
-      "shots\030\002 \003(\01327.hbase.pb.GetQuotaStatesRes" +
-      "ponse.NamespaceQuotaSnapshot\032m\n\022TableQuo",
-      "taSnapshot\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.p" +
-      "b.TableName\022.\n\010snapshot\030\002 \001(\0132\034.hbase.pb" +
-      ".SpaceQuotaSnapshot\032[\n\026NamespaceQuotaSna" +
-      "pshot\022\021\n\tnamespace\030\001 \001(\t\022.\n\010snapshot\030\002 \001" +
-      "(\0132\034.hbase.pb.SpaceQuotaSnapshot*&\n\nQuot" +
-      "aScope\022\013\n\007CLUSTER\020\001\022\013\n\007MACHINE\020\002*v\n\014Thro" +
-      "ttleType\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\tQuot" +
-      "aType\022\014\n\010THROTTLE\020\001\022\t\n\005SPACE\020\002*]\n\024SpaceV",
-      "iolationPolicy\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_INSE" +
-      "RTS\020\004BH\n1org.apache.hadoop.hbase.shaded." +
-      "protobuf.generatedB\013QuotaProtosH\001\210\001\001\240\001\001"
+      "ta\030\001 \001(\0132\024.hbase.pb.SpaceQuota\"b\n\020SpaceQ" +
+      "uotaStatus\0228\n\020violation_policy\030\001 \001(\0162\036.h" +
+      "base.pb.SpaceViolationPolicy\022\024\n\014in_viola" +
+      "tion\030\002 \001(\010\"p\n\022SpaceQuotaSnapshot\0220\n\014quot" +
+      "a_status\030\001 \001(\0132\032.hbase.pb.SpaceQuotaStat" +
+      "us\022\023\n\013quota_usage\030\002 \001(\004\022\023\n\013quota_limit\030\003" +
+      " \001(\004\"!\n\037GetSpaceQuotaRegionSizesRequest\"" +
+      "\257\001\n GetSpaceQuotaRegionSizesResponse\022E\n\005" +
+      "sizes\030\001 \003(\01326.hbase.pb.GetSpaceQuotaRegi" +
+      "onSizesResponse.RegionSizes\032D\n\013RegionSiz",
+      "es\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.TableN" +
+      "ame\022\014\n\004size\030\002 \001(\004\"\037\n\035GetSpaceQuotaSnapsh" +
+      "otsRequest\"\337\001\n\036GetSpaceQuotaSnapshotsRes" +
+      "ponse\022N\n\tsnapshots\030\001 \003(\0132;.hbase.pb.GetS" +
+      "paceQuotaSnapshotsResponse.TableQuotaSna" +
+      "pshot\032m\n\022TableQuotaSnapshot\022\'\n\ntable_nam" +
+      "e\030\001 \001(\0132\023.hbase.pb.TableName\022.\n\010snapshot" +
+      "\030\002 \001(\0132\034.hbase.pb.SpaceQuotaSnapshot\"\"\n " +
+      "GetSpaceQuotaEnforcementsRequest\"\374\001\n!Get" +
+      "SpaceQuotaEnforcementsResponse\022\\\n\022violat",
+      "ion_policies\030\001 \003(\0132@.hbase.pb.GetSpaceQu" +
+      "otaEnforcementsResponse.TableViolationPo" +
+      "licy\032y\n\024TableViolationPolicy\022\'\n\ntable_na" +
+      "me\030\001 \001(\0132\023.hbase.pb.TableName\0228\n\020violati" +
+      "on_policy\030\002 \001(\0162\036.hbase.pb.SpaceViolatio" +
+      "nPolicy\"\027\n\025GetQuotaStatesRequest\"\201\003\n\026Get" +
+      "QuotaStatesResponse\022L\n\017table_snapshots\030\001" +
+      " \003(\01323.hbase.pb.GetQuotaStatesResponse.T" +
+      "ableQuotaSnapshot\022M\n\014ns_snapshots\030\002 \003(\0132" +
+      "7.hbase.pb.GetQuotaStatesResponse.Namesp",
+      "aceQuotaSnapshot\032m\n\022TableQuotaSnapshot\022\'" +
+      "\n\ntable_name\030\001 \001(\0132\023.hbase.pb.TableName\022" +
+      ".\n\010snapshot\030\002 \001(\0132\034.hbase.pb.SpaceQuotaS" +
+      "napshot\032[\n\026NamespaceQuotaSnapshot\022\021\n\tnam" +
+      "espace\030\001 \001(\t\022.\n\010snapshot\030\002 \001(\0132\034.hbase.p" +
+      "b.SpaceQuotaSnapshot*&\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*$\n\tQuotaType\022\014\n\010THR",
+      "OTTLE\020\001\022\t\n\005SPACE\020\002*]\n\024SpaceViolationPoli" +
+      "cy\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" +
+      ".apache.hadoop.hbase.shaded.protobuf.gen" +
+      "eratedB\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() {
@@ -16293,13 +16294,13 @@ public final class QuotaProtos {
     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", });
+        new java.lang.String[] { "ViolationPolicy", "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", });
+        new java.lang.String[] { "QuotaStatus", "QuotaUsage", "QuotaLimit", });
     internal_static_hbase_pb_GetSpaceQuotaRegionSizesRequest_descriptor =
       getDescriptor().getMessageTypes().get(9);
     internal_static_hbase_pb_GetSpaceQuotaRegionSizesRequest_fieldAccessorTable = new