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/03/20 22:50:13 UTC

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

HBASE-17001 Enforce quota violation policies in the RegionServer

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

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


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

Branch: refs/heads/HBASE-16961
Commit: b99e9cf94d5af3af20f8bedfe9a9a681132f8550
Parents: c544ed1
Author: Josh Elser <el...@apache.org>
Authored: Thu Dec 15 13:27:56 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Mar 20 18:02:54 2017 -0400

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


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

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

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

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

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