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/25 23:49:18 UTC

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

HBASE-16995 Build client Java API and client protobuf messages (Josh Elser)


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

Branch: refs/heads/HBASE-16961
Commit: 23194dcac8b4e4dbe7598fd51fa89f18b77a59dc
Parents: 2557506
Author: tedyu <yu...@gmail.com>
Authored: Thu Nov 17 10:19:52 2016 -0800
Committer: Josh Elser <el...@apache.org>
Committed: Tue Apr 25 18:19:48 2017 -0400

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


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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/23194dca/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
new file mode 100644
index 0000000..17015d6
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link QuotaSettingsFactory}.
+ */
+@Category(SmallTests.class)
+public class TestQuotaSettingsFactory {
+  
+  @Test
+  public void testAllQuotasAddedToList() {
+    final SpaceQuota spaceQuota = SpaceQuota.newBuilder()
+        .setSoftLimit(1024L * 1024L * 1024L * 50L) // 50G
+        .setViolationPolicy(QuotaProtos.SpaceViolationPolicy.DISABLE) // Disable the table
+        .build();
+    final long readLimit = 1000;
+    final long writeLimit = 500;
+    final Throttle throttle = Throttle.newBuilder()
+        // 1000 read reqs/min
+        .setReadNum(TimedQuota.newBuilder().setSoftLimit(readLimit).setTimeUnit(HBaseProtos.TimeUnit.MINUTES).build())
+        // 500 write reqs/min
+        .setWriteNum(TimedQuota.newBuilder().setSoftLimit(writeLimit).setTimeUnit(HBaseProtos.TimeUnit.MINUTES).build())
+        .build();
+    final Quotas quotas = Quotas.newBuilder()
+        .setSpace(spaceQuota) // Set the FS quotas
+        .setThrottle(throttle) // Set some RPC limits
+        .build();
+    final TableName tn = TableName.valueOf("my_table");
+    List<QuotaSettings> settings = QuotaSettingsFactory.fromTableQuotas(tn, quotas);
+    assertEquals(3, settings.size());
+    boolean seenRead = false;
+    boolean seenWrite = false;
+    boolean seenSpace = false;
+    for (QuotaSettings setting : settings) {
+      if (setting instanceof ThrottleSettings) {
+        ThrottleSettings throttleSettings = (ThrottleSettings) setting;
+        switch (throttleSettings.getThrottleType()) {
+          case READ_NUMBER:
+            assertFalse("Should not have multiple read quotas", seenRead);
+            assertEquals(readLimit, throttleSettings.getSoftLimit());
+            assertEquals(TimeUnit.MINUTES, throttleSettings.getTimeUnit());
+            assertEquals(tn, throttleSettings.getTableName());
+            assertNull("Username should be null", throttleSettings.getUserName());
+            assertNull("Namespace should be null", throttleSettings.getNamespace());
+            seenRead = true;
+            break;
+          case WRITE_NUMBER:
+            assertFalse("Should not have multiple write quotas", seenWrite);
+            assertEquals(writeLimit, throttleSettings.getSoftLimit());
+            assertEquals(TimeUnit.MINUTES, throttleSettings.getTimeUnit());
+            assertEquals(tn, throttleSettings.getTableName());
+            assertNull("Username should be null", throttleSettings.getUserName());
+            assertNull("Namespace should be null", throttleSettings.getNamespace());
+            seenWrite = true;
+            break;
+          default:
+            fail("Unexpected throttle type: " + throttleSettings.getThrottleType());
+        }
+      } else if (setting instanceof SpaceLimitSettings) {
+        assertFalse("Should not have multiple space quotas", seenSpace);
+        SpaceLimitSettings spaceLimit = (SpaceLimitSettings) setting;
+        assertEquals(tn, spaceLimit.getTableName());
+        assertNull("Username should be null", spaceLimit.getUserName());
+        assertNull("Namespace should be null", spaceLimit.getNamespace());
+        assertTrue("SpaceLimitSettings should have a SpaceQuota", spaceLimit.getProto().hasQuota());
+        assertEquals(spaceQuota, spaceLimit.getProto().getQuota());
+        seenSpace = true;
+      } else {
+        fail("Unexpected QuotaSettings implementation: " + setting.getClass());
+      }
+    }
+    assertTrue("Should have seen a read quota", seenRead);
+    assertTrue("Should have seen a write quota", seenWrite);
+    assertTrue("Should have seen a space quota", seenSpace);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testNeitherTableNorNamespace() {
+    final SpaceQuota spaceQuota = SpaceQuota.newBuilder()
+        .setSoftLimit(1L)
+        .setViolationPolicy(QuotaProtos.SpaceViolationPolicy.DISABLE)
+        .build();
+    QuotaSettingsFactory.fromSpace(null, null, spaceQuota);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testBothTableAndNamespace() {
+    final SpaceQuota spaceQuota = SpaceQuota.newBuilder()
+        .setSoftLimit(1L)
+        .setViolationPolicy(QuotaProtos.SpaceViolationPolicy.DISABLE)
+        .build();
+    QuotaSettingsFactory.fromSpace(TableName.valueOf("foo"), "bar", spaceQuota);
+  }
+
+  @Test
+  public void testSpaceLimitSettings() {
+    final TableName tableName = TableName.valueOf("foo");
+    final long sizeLimit = 1024L * 1024L * 1024L * 75; // 75GB
+    final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_INSERTS;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tableName, sizeLimit, violationPolicy);
+    assertNotNull("QuotaSettings should not be null", settings);
+    assertTrue("Should be an instance of SpaceLimitSettings", settings instanceof SpaceLimitSettings);
+    SpaceLimitSettings spaceLimitSettings = (SpaceLimitSettings) settings;
+    SpaceLimitRequest protoRequest = spaceLimitSettings.getProto();
+    assertTrue("Request should have a SpaceQuota", protoRequest.hasQuota());
+    SpaceQuota quota = protoRequest.getQuota();
+    assertEquals(sizeLimit, quota.getSoftLimit());
+    assertEquals(violationPolicy, ProtobufUtil.toViolationPolicy(quota.getViolationPolicy()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/23194dca/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java
new file mode 100644
index 0000000..77a00da
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link SpaceLimitSettings}.
+ */
+@Category({SmallTests.class})
+public class TestSpaceLimitSettings {
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidTableQuotaSizeLimit() {
+    new SpaceLimitSettings(TableName.valueOf("foo"), -1, SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullTableName() {
+    TableName tn = null;
+    new SpaceLimitSettings(tn, 1, SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullTableViolationPolicy() {
+    new SpaceLimitSettings(TableName.valueOf("foo"), 1, null);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidNamespaceQuotaSizeLimit() {
+    new SpaceLimitSettings("foo_ns", -1, SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullNamespace() {
+    String ns = null;
+    new SpaceLimitSettings(ns, 1, SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullNamespaceViolationPolicy() {
+    new SpaceLimitSettings("foo_ns", 1, null);
+  }
+
+  @Test
+  public void testTableQuota() {
+    final TableName tableName = TableName.valueOf("foo");
+    final long sizeLimit = 1024 * 1024;
+    final SpaceViolationPolicy policy = SpaceViolationPolicy.NO_WRITES;
+    SpaceLimitSettings settings = new SpaceLimitSettings(tableName, sizeLimit, policy);
+    SetQuotaRequest proto = QuotaSettings.buildSetQuotaRequestProto(settings);
+
+    assertFalse("User should be missing", proto.hasUserName());
+    assertFalse("Namespace should be missing", proto.hasNamespace());
+    assertEquals(ProtobufUtil.toProtoTableName(tableName), proto.getTableName());
+    SpaceLimitRequest spaceLimitReq = proto.getSpaceLimit();
+    assertNotNull("SpaceLimitRequest was null", spaceLimitReq);
+    SpaceQuota spaceQuota = spaceLimitReq.getQuota();
+    assertNotNull("SpaceQuota was null", spaceQuota);
+    assertEquals(sizeLimit, spaceQuota.getSoftLimit());
+    assertEquals(ProtobufUtil.toProtoViolationPolicy(policy), spaceQuota.getViolationPolicy());
+
+    assertEquals(QuotaType.SPACE, settings.getQuotaType());
+
+    SpaceLimitSettings copy = new SpaceLimitSettings(tableName, sizeLimit, policy);
+    assertEquals(settings, copy);
+    assertEquals(settings.hashCode(), copy.hashCode());
+  }
+
+  @Test
+  public void testNamespaceQuota() {
+    final String namespace = "foo_ns";
+    final long sizeLimit = 1024 * 1024;
+    final SpaceViolationPolicy policy = SpaceViolationPolicy.NO_WRITES;
+    SpaceLimitSettings settings = new SpaceLimitSettings(namespace, sizeLimit, policy);
+    SetQuotaRequest proto = QuotaSettings.buildSetQuotaRequestProto(settings);
+
+    assertFalse("User should be missing", proto.hasUserName());
+    assertFalse("TableName should be missing", proto.hasTableName());
+    assertEquals(namespace, proto.getNamespace());
+    SpaceLimitRequest spaceLimitReq = proto.getSpaceLimit();
+    assertNotNull("SpaceLimitRequest was null", spaceLimitReq);
+    SpaceQuota spaceQuota = spaceLimitReq.getQuota();
+    assertNotNull("SpaceQuota was null", spaceQuota);
+    assertEquals(sizeLimit, spaceQuota.getSoftLimit());
+    assertEquals(ProtobufUtil.toProtoViolationPolicy(policy), spaceQuota.getViolationPolicy());
+
+    assertEquals(QuotaType.SPACE, settings.getQuotaType());
+
+    SpaceLimitSettings copy = new SpaceLimitSettings(namespace, sizeLimit, policy);
+    assertEquals(settings, copy);
+    assertEquals(settings.hashCode(), copy.hashCode());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/23194dca/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 e4ce4cb..da6c65e 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
@@ -63360,6 +63360,19 @@ public final class MasterProtos {
      * <code>optional .hbase.pb.ThrottleRequest throttle = 7;</code>
      */
     org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequestOrBuilder getThrottleOrBuilder();
+
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    boolean hasSpaceLimit();
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest getSpaceLimit();
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder getSpaceLimitOrBuilder();
   }
   /**
    * Protobuf type {@code hbase.pb.SetQuotaRequest}
@@ -63462,6 +63475,19 @@ public final class MasterProtos {
               bitField0_ |= 0x00000040;
               break;
             }
+            case 66: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000080) == 0x00000080)) {
+                subBuilder = spaceLimit_.toBuilder();
+              }
+              spaceLimit_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(spaceLimit_);
+                spaceLimit_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000080;
+              break;
+            }
           }
         }
       } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@@ -63685,6 +63711,27 @@ public final class MasterProtos {
       return throttle_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest.getDefaultInstance() : throttle_;
     }
 
+    public static final int SPACE_LIMIT_FIELD_NUMBER = 8;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest spaceLimit_;
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    public boolean hasSpaceLimit() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest getSpaceLimit() {
+      return spaceLimit_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance() : spaceLimit_;
+    }
+    /**
+     * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder getSpaceLimitOrBuilder() {
+      return spaceLimit_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance() : spaceLimit_;
+    }
+
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
@@ -63730,6 +63777,9 @@ public final class MasterProtos {
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
         output.writeMessage(7, getThrottle());
       }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeMessage(8, getSpaceLimit());
+      }
       unknownFields.writeTo(output);
     }
 
@@ -63763,6 +63813,10 @@ public final class MasterProtos {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(7, getThrottle());
       }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(8, getSpaceLimit());
+      }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
       return size;
@@ -63815,6 +63869,11 @@ public final class MasterProtos {
         result = result && getThrottle()
             .equals(other.getThrottle());
       }
+      result = result && (hasSpaceLimit() == other.hasSpaceLimit());
+      if (hasSpaceLimit()) {
+        result = result && getSpaceLimit()
+            .equals(other.getSpaceLimit());
+      }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
     }
@@ -63856,6 +63915,10 @@ public final class MasterProtos {
         hash = (37 * hash) + THROTTLE_FIELD_NUMBER;
         hash = (53 * hash) + getThrottle().hashCode();
       }
+      if (hasSpaceLimit()) {
+        hash = (37 * hash) + SPACE_LIMIT_FIELD_NUMBER;
+        hash = (53 * hash) + getSpaceLimit().hashCode();
+      }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -63972,6 +64035,7 @@ public final class MasterProtos {
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
           getThrottleFieldBuilder();
+          getSpaceLimitFieldBuilder();
         }
       }
       public Builder clear() {
@@ -63998,6 +64062,12 @@ public final class MasterProtos {
           throttleBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000040);
+        if (spaceLimitBuilder_ == null) {
+          spaceLimit_ = null;
+        } else {
+          spaceLimitBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000080);
         return this;
       }
 
@@ -64058,6 +64128,14 @@ public final class MasterProtos {
         } else {
           result.throttle_ = throttleBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        if (spaceLimitBuilder_ == null) {
+          result.spaceLimit_ = spaceLimit_;
+        } else {
+          result.spaceLimit_ = spaceLimitBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -64127,6 +64205,9 @@ public final class MasterProtos {
         if (other.hasThrottle()) {
           mergeThrottle(other.getThrottle());
         }
+        if (other.hasSpaceLimit()) {
+          mergeSpaceLimit(other.getSpaceLimit());
+        }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
         return this;
@@ -64692,6 +64773,124 @@ public final class MasterProtos {
         }
         return throttleBuilder_;
       }
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest spaceLimit_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder> spaceLimitBuilder_;
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public boolean hasSpaceLimit() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest getSpaceLimit() {
+        if (spaceLimitBuilder_ == null) {
+          return spaceLimit_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance() : spaceLimit_;
+        } else {
+          return spaceLimitBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public Builder setSpaceLimit(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest value) {
+        if (spaceLimitBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          spaceLimit_ = value;
+          onChanged();
+        } else {
+          spaceLimitBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000080;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public Builder setSpaceLimit(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder builderForValue) {
+        if (spaceLimitBuilder_ == null) {
+          spaceLimit_ = builderForValue.build();
+          onChanged();
+        } else {
+          spaceLimitBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000080;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public Builder mergeSpaceLimit(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest value) {
+        if (spaceLimitBuilder_ == null) {
+          if (((bitField0_ & 0x00000080) == 0x00000080) &&
+              spaceLimit_ != null &&
+              spaceLimit_ != org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance()) {
+            spaceLimit_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.newBuilder(spaceLimit_).mergeFrom(value).buildPartial();
+          } else {
+            spaceLimit_ = value;
+          }
+          onChanged();
+        } else {
+          spaceLimitBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000080;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public Builder clearSpaceLimit() {
+        if (spaceLimitBuilder_ == null) {
+          spaceLimit_ = null;
+          onChanged();
+        } else {
+          spaceLimitBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000080);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder getSpaceLimitBuilder() {
+        bitField0_ |= 0x00000080;
+        onChanged();
+        return getSpaceLimitFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder getSpaceLimitOrBuilder() {
+        if (spaceLimitBuilder_ != null) {
+          return spaceLimitBuilder_.getMessageOrBuilder();
+        } else {
+          return spaceLimit_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.getDefaultInstance() : spaceLimit_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.SpaceLimitRequest space_limit = 8;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder> 
+          getSpaceLimitFieldBuilder() {
+        if (spaceLimitBuilder_ == null) {
+          spaceLimitBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequestOrBuilder>(
+                  getSpaceLimit(),
+                  getParentForChildren(),
+                  isClean());
+          spaceLimit_ = null;
+        }
+        return spaceLimitBuilder_;
+      }
       public final Builder setUnknownFields(
           final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
@@ -77852,203 +78051,204 @@ public final class MasterProtos {
       "oceduresResponse\022&\n\tprocedure\030\001 \003(\0132\023.hb" +
       "ase.pb.Procedure\"\022\n\020ListLocksRequest\"5\n\021" +
       "ListLocksResponse\022 \n\004lock\030\001 \003(\0132\022.hbase." +
-      "pb.LockInfo\"\315\001\n\017SetQuotaRequest\022\021\n\tuser_" +
+      "pb.LockInfo\"\377\001\n\017SetQuotaRequest\022\021\n\tuser_" +
       "name\030\001 \001(\t\022\022\n\nuser_group\030\002 \001(\t\022\021\n\tnamesp" +
       "ace\030\003 \001(\t\022\'\n\ntable_name\030\004 \001(\0132\023.hbase.pb" +
       ".TableName\022\022\n\nremove_all\030\005 \001(\010\022\026\n\016bypass",
       "_globals\030\006 \001(\010\022+\n\010throttle\030\007 \001(\0132\031.hbase" +
-      ".pb.ThrottleRequest\"\022\n\020SetQuotaResponse\"" +
-      "J\n\037MajorCompactionTimestampRequest\022\'\n\nta" +
-      "ble_name\030\001 \002(\0132\023.hbase.pb.TableName\"U\n(M" +
-      "ajorCompactionTimestampForRegionRequest\022" +
-      ")\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifi" +
-      "er\"@\n MajorCompactionTimestampResponse\022\034" +
-      "\n\024compaction_timestamp\030\001 \002(\003\"\035\n\033Security" +
-      "CapabilitiesRequest\"\354\001\n\034SecurityCapabili" +
-      "tiesResponse\022G\n\014capabilities\030\001 \003(\01621.hba",
-      "se.pb.SecurityCapabilitiesResponse.Capab" +
-      "ility\"\202\001\n\nCapability\022\031\n\025SIMPLE_AUTHENTIC" +
-      "ATION\020\000\022\031\n\025SECURE_AUTHENTICATION\020\001\022\021\n\rAU" +
-      "THORIZATION\020\002\022\026\n\022CELL_AUTHORIZATION\020\003\022\023\n" +
-      "\017CELL_VISIBILITY\020\004\"\"\n ListDrainingRegion" +
-      "ServersRequest\"N\n!ListDrainingRegionServ" +
-      "ersResponse\022)\n\013server_name\030\001 \003(\0132\024.hbase" +
-      ".pb.ServerName\"F\n\031DrainRegionServersRequ" +
-      "est\022)\n\013server_name\030\001 \003(\0132\024.hbase.pb.Serv" +
-      "erName\"\034\n\032DrainRegionServersResponse\"P\n#",
-      "RemoveDrainFromRegionServersRequest\022)\n\013s" +
-      "erver_name\030\001 \003(\0132\024.hbase.pb.ServerName\"&" +
-      "\n$RemoveDrainFromRegionServersResponse*(" +
-      "\n\020MasterSwitchType\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\001" +
-      "2\2074\n\rMasterService\022e\n\024GetSchemaAlterStat" +
-      "us\022%.hbase.pb.GetSchemaAlterStatusReques" +
-      "t\032&.hbase.pb.GetSchemaAlterStatusRespons" +
-      "e\022b\n\023GetTableDescriptors\022$.hbase.pb.GetT" +
-      "ableDescriptorsRequest\032%.hbase.pb.GetTab" +
-      "leDescriptorsResponse\022P\n\rGetTableNames\022\036",
-      ".hbase.pb.GetTableNamesRequest\032\037.hbase.p" +
-      "b.GetTableNamesResponse\022Y\n\020GetClusterSta" +
-      "tus\022!.hbase.pb.GetClusterStatusRequest\032\"" +
-      ".hbase.pb.GetClusterStatusResponse\022V\n\017Is" +
-      "MasterRunning\022 .hbase.pb.IsMasterRunning" +
-      "Request\032!.hbase.pb.IsMasterRunningRespon" +
-      "se\022D\n\tAddColumn\022\032.hbase.pb.AddColumnRequ" +
-      "est\032\033.hbase.pb.AddColumnResponse\022M\n\014Dele" +
-      "teColumn\022\035.hbase.pb.DeleteColumnRequest\032" +
-      "\036.hbase.pb.DeleteColumnResponse\022M\n\014Modif",
-      "yColumn\022\035.hbase.pb.ModifyColumnRequest\032\036" +
-      ".hbase.pb.ModifyColumnResponse\022G\n\nMoveRe" +
-      "gion\022\033.hbase.pb.MoveRegionRequest\032\034.hbas" +
-      "e.pb.MoveRegionResponse\022\\\n\021MergeTableReg" +
-      "ions\022\".hbase.pb.MergeTableRegionsRequest" +
-      "\032#.hbase.pb.MergeTableRegionsResponse\022M\n" +
-      "\014AssignRegion\022\035.hbase.pb.AssignRegionReq" +
-      "uest\032\036.hbase.pb.AssignRegionResponse\022S\n\016" +
-      "UnassignRegion\022\037.hbase.pb.UnassignRegion" +
-      "Request\032 .hbase.pb.UnassignRegionRespons",
-      "e\022P\n\rOfflineRegion\022\036.hbase.pb.OfflineReg" +
-      "ionRequest\032\037.hbase.pb.OfflineRegionRespo" +
-      "nse\022J\n\013DeleteTable\022\034.hbase.pb.DeleteTabl" +
-      "eRequest\032\035.hbase.pb.DeleteTableResponse\022" +
-      "P\n\rtruncateTable\022\036.hbase.pb.TruncateTabl" +
-      "eRequest\032\037.hbase.pb.TruncateTableRespons" +
-      "e\022J\n\013EnableTable\022\034.hbase.pb.EnableTableR" +
-      "equest\032\035.hbase.pb.EnableTableResponse\022M\n" +
-      "\014DisableTable\022\035.hbase.pb.DisableTableReq" +
-      "uest\032\036.hbase.pb.DisableTableResponse\022J\n\013",
-      "ModifyTable\022\034.hbase.pb.ModifyTableReques" +
-      "t\032\035.hbase.pb.ModifyTableResponse\022J\n\013Crea" +
-      "teTable\022\034.hbase.pb.CreateTableRequest\032\035." +
-      "hbase.pb.CreateTableResponse\022A\n\010Shutdown" +
-      "\022\031.hbase.pb.ShutdownRequest\032\032.hbase.pb.S" +
-      "hutdownResponse\022G\n\nStopMaster\022\033.hbase.pb" +
-      ".StopMasterRequest\032\034.hbase.pb.StopMaster" +
-      "Response\022h\n\031IsMasterInMaintenanceMode\022$." +
-      "hbase.pb.IsInMaintenanceModeRequest\032%.hb" +
-      "ase.pb.IsInMaintenanceModeResponse\022>\n\007Ba",
-      "lance\022\030.hbase.pb.BalanceRequest\032\031.hbase." +
-      "pb.BalanceResponse\022_\n\022SetBalancerRunning" +
-      "\022#.hbase.pb.SetBalancerRunningRequest\032$." +
-      "hbase.pb.SetBalancerRunningResponse\022\\\n\021I" +
-      "sBalancerEnabled\022\".hbase.pb.IsBalancerEn" +
-      "abledRequest\032#.hbase.pb.IsBalancerEnable" +
-      "dResponse\022k\n\026SetSplitOrMergeEnabled\022\'.hb" +
-      "ase.pb.SetSplitOrMergeEnabledRequest\032(.h" +
-      "base.pb.SetSplitOrMergeEnabledResponse\022h" +
-      "\n\025IsSplitOrMergeEnabled\022&.hbase.pb.IsSpl",
-      "itOrMergeEnabledRequest\032\'.hbase.pb.IsSpl" +
-      "itOrMergeEnabledResponse\022D\n\tNormalize\022\032." +
-      "hbase.pb.NormalizeRequest\032\033.hbase.pb.Nor" +
-      "malizeResponse\022e\n\024SetNormalizerRunning\022%" +
-      ".hbase.pb.SetNormalizerRunningRequest\032&." +
-      "hbase.pb.SetNormalizerRunningResponse\022b\n" +
-      "\023IsNormalizerEnabled\022$.hbase.pb.IsNormal" +
-      "izerEnabledRequest\032%.hbase.pb.IsNormaliz" +
-      "erEnabledResponse\022S\n\016RunCatalogScan\022\037.hb" +
-      "ase.pb.RunCatalogScanRequest\032 .hbase.pb.",
-      "RunCatalogScanResponse\022e\n\024EnableCatalogJ" +
-      "anitor\022%.hbase.pb.EnableCatalogJanitorRe" +
-      "quest\032&.hbase.pb.EnableCatalogJanitorRes" +
-      "ponse\022n\n\027IsCatalogJanitorEnabled\022(.hbase" +
-      ".pb.IsCatalogJanitorEnabledRequest\032).hba" +
-      "se.pb.IsCatalogJanitorEnabledResponse\022V\n" +
-      "\017RunCleanerChore\022 .hbase.pb.RunCleanerCh" +
-      "oreRequest\032!.hbase.pb.RunCleanerChoreRes" +
-      "ponse\022k\n\026SetCleanerChoreRunning\022\'.hbase." +
-      "pb.SetCleanerChoreRunningRequest\032(.hbase",
-      ".pb.SetCleanerChoreRunningResponse\022h\n\025Is" +
-      "CleanerChoreEnabled\022&.hbase.pb.IsCleaner" +
-      "ChoreEnabledRequest\032\'.hbase.pb.IsCleaner" +
-      "ChoreEnabledResponse\022^\n\021ExecMasterServic" +
-      "e\022#.hbase.pb.CoprocessorServiceRequest\032$" +
-      ".hbase.pb.CoprocessorServiceResponse\022A\n\010" +
-      "Snapshot\022\031.hbase.pb.SnapshotRequest\032\032.hb" +
-      "ase.pb.SnapshotResponse\022h\n\025GetCompletedS" +
-      "napshots\022&.hbase.pb.GetCompletedSnapshot" +
-      "sRequest\032\'.hbase.pb.GetCompletedSnapshot",
-      "sResponse\022S\n\016DeleteSnapshot\022\037.hbase.pb.D" +
-      "eleteSnapshotRequest\032 .hbase.pb.DeleteSn" +
-      "apshotResponse\022S\n\016IsSnapshotDone\022\037.hbase" +
-      ".pb.IsSnapshotDoneRequest\032 .hbase.pb.IsS" +
-      "napshotDoneResponse\022V\n\017RestoreSnapshot\022 " +
-      ".hbase.pb.RestoreSnapshotRequest\032!.hbase" +
-      ".pb.RestoreSnapshotResponse\022P\n\rExecProce" +
-      "dure\022\036.hbase.pb.ExecProcedureRequest\032\037.h" +
-      "base.pb.ExecProcedureResponse\022W\n\024ExecPro" +
-      "cedureWithRet\022\036.hbase.pb.ExecProcedureRe",
-      "quest\032\037.hbase.pb.ExecProcedureResponse\022V" +
-      "\n\017IsProcedureDone\022 .hbase.pb.IsProcedure" +
-      "DoneRequest\032!.hbase.pb.IsProcedureDoneRe" +
-      "sponse\022V\n\017ModifyNamespace\022 .hbase.pb.Mod" +
-      "ifyNamespaceRequest\032!.hbase.pb.ModifyNam" +
-      "espaceResponse\022V\n\017CreateNamespace\022 .hbas" +
-      "e.pb.CreateNamespaceRequest\032!.hbase.pb.C" +
-      "reateNamespaceResponse\022V\n\017DeleteNamespac" +
-      "e\022 .hbase.pb.DeleteNamespaceRequest\032!.hb" +
-      "ase.pb.DeleteNamespaceResponse\022k\n\026GetNam",
-      "espaceDescriptor\022\'.hbase.pb.GetNamespace" +
-      "DescriptorRequest\032(.hbase.pb.GetNamespac" +
-      "eDescriptorResponse\022q\n\030ListNamespaceDesc" +
-      "riptors\022).hbase.pb.ListNamespaceDescript" +
-      "orsRequest\032*.hbase.pb.ListNamespaceDescr" +
-      "iptorsResponse\022\206\001\n\037ListTableDescriptorsB" +
-      "yNamespace\0220.hbase.pb.ListTableDescripto" +
-      "rsByNamespaceRequest\0321.hbase.pb.ListTabl" +
-      "eDescriptorsByNamespaceResponse\022t\n\031ListT" +
-      "ableNamesByNamespace\022*.hbase.pb.ListTabl",
-      "eNamesByNamespaceRequest\032+.hbase.pb.List" +
-      "TableNamesByNamespaceResponse\022P\n\rGetTabl" +
-      "eState\022\036.hbase.pb.GetTableStateRequest\032\037" +
-      ".hbase.pb.GetTableStateResponse\022A\n\010SetQu" +
-      "ota\022\031.hbase.pb.SetQuotaRequest\032\032.hbase.p" +
-      "b.SetQuotaResponse\022x\n\037getLastMajorCompac" +
-      "tionTimestamp\022).hbase.pb.MajorCompaction" +
-      "TimestampRequest\032*.hbase.pb.MajorCompact" +
-      "ionTimestampResponse\022\212\001\n(getLastMajorCom" +
-      "pactionTimestampForRegion\0222.hbase.pb.Maj",
-      "orCompactionTimestampForRegionRequest\032*." +
-      "hbase.pb.MajorCompactionTimestampRespons" +
-      "e\022_\n\022getProcedureResult\022#.hbase.pb.GetPr" +
-      "ocedureResultRequest\032$.hbase.pb.GetProce" +
-      "dureResultResponse\022h\n\027getSecurityCapabil" +
-      "ities\022%.hbase.pb.SecurityCapabilitiesReq" +
-      "uest\032&.hbase.pb.SecurityCapabilitiesResp" +
-      "onse\022S\n\016AbortProcedure\022\037.hbase.pb.AbortP" +
-      "rocedureRequest\032 .hbase.pb.AbortProcedur" +
-      "eResponse\022S\n\016ListProcedures\022\037.hbase.pb.L",
-      "istProceduresRequest\032 .hbase.pb.ListProc" +
-      "eduresResponse\022D\n\tListLocks\022\032.hbase.pb.L" +
-      "istLocksRequest\032\033.hbase.pb.ListLocksResp" +
-      "onse\022_\n\022AddReplicationPeer\022#.hbase.pb.Ad" +
-      "dReplicationPeerRequest\032$.hbase.pb.AddRe" +
-      "plicationPeerResponse\022h\n\025RemoveReplicati" +
-      "onPeer\022&.hbase.pb.RemoveReplicationPeerR" +
-      "equest\032\'.hbase.pb.RemoveReplicationPeerR" +
-      "esponse\022h\n\025EnableReplicationPeer\022&.hbase" +
-      ".pb.EnableReplicationPeerRequest\032\'.hbase",
-      ".pb.EnableReplicationPeerResponse\022k\n\026Dis" +
-      "ableReplicationPeer\022\'.hbase.pb.DisableRe" +
-      "plicationPeerRequest\032(.hbase.pb.DisableR" +
-      "eplicationPeerResponse\022q\n\030GetReplication" +
-      "PeerConfig\022).hbase.pb.GetReplicationPeer" +
-      "ConfigRequest\032*.hbase.pb.GetReplicationP" +
-      "eerConfigResponse\022z\n\033UpdateReplicationPe" +
-      "erConfig\022,.hbase.pb.UpdateReplicationPee" +
-      "rConfigRequest\032-.hbase.pb.UpdateReplicat" +
-      "ionPeerConfigResponse\022e\n\024ListReplication",
-      "Peers\022%.hbase.pb.ListReplicationPeersReq" +
-      "uest\032&.hbase.pb.ListReplicationPeersResp" +
-      "onse\022t\n\031listDrainingRegionServers\022*.hbas" +
-      "e.pb.ListDrainingRegionServersRequest\032+." +
-      "hbase.pb.ListDrainingRegionServersRespon" +
-      "se\022_\n\022drainRegionServers\022#.hbase.pb.Drai" +
-      "nRegionServersRequest\032$.hbase.pb.DrainRe" +
-      "gionServersResponse\022}\n\034removeDrainFromRe" +
-      "gionServers\022-.hbase.pb.RemoveDrainFromRe" +
-      "gionServersRequest\032..hbase.pb.RemoveDrai",
-      "nFromRegionServersResponseBI\n1org.apache" +
-      ".hadoop.hbase.shaded.protobuf.generatedB" +
-      "\014MasterProtosH\001\210\001\001\240\001\001"
+      ".pb.ThrottleRequest\0220\n\013space_limit\030\010 \001(\013" +
+      "2\033.hbase.pb.SpaceLimitRequest\"\022\n\020SetQuot" +
+      "aResponse\"J\n\037MajorCompactionTimestampReq" +
+      "uest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.Tabl" +
+      "eName\"U\n(MajorCompactionTimestampForRegi" +
+      "onRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Reg" +
+      "ionSpecifier\"@\n MajorCompactionTimestamp" +
+      "Response\022\034\n\024compaction_timestamp\030\001 \002(\003\"\035" +
+      "\n\033SecurityCapabilitiesRequest\"\354\001\n\034Securi",
+      "tyCapabilitiesResponse\022G\n\014capabilities\030\001" +
+      " \003(\01621.hbase.pb.SecurityCapabilitiesResp" +
+      "onse.Capability\"\202\001\n\nCapability\022\031\n\025SIMPLE" +
+      "_AUTHENTICATION\020\000\022\031\n\025SECURE_AUTHENTICATI" +
+      "ON\020\001\022\021\n\rAUTHORIZATION\020\002\022\026\n\022CELL_AUTHORIZ" +
+      "ATION\020\003\022\023\n\017CELL_VISIBILITY\020\004\"\"\n ListDrai" +
+      "ningRegionServersRequest\"N\n!ListDraining" +
+      "RegionServersResponse\022)\n\013server_name\030\001 \003" +
+      "(\0132\024.hbase.pb.ServerName\"F\n\031DrainRegionS" +
+      "erversRequest\022)\n\013server_name\030\001 \003(\0132\024.hba",
+      "se.pb.ServerName\"\034\n\032DrainRegionServersRe" +
+      "sponse\"P\n#RemoveDrainFromRegionServersRe" +
+      "quest\022)\n\013server_name\030\001 \003(\0132\024.hbase.pb.Se" +
+      "rverName\"&\n$RemoveDrainFromRegionServers" +
+      "Response*(\n\020MasterSwitchType\022\t\n\005SPLIT\020\000\022" +
+      "\t\n\005MERGE\020\0012\2074\n\rMasterService\022e\n\024GetSchem" +
+      "aAlterStatus\022%.hbase.pb.GetSchemaAlterSt" +
+      "atusRequest\032&.hbase.pb.GetSchemaAlterSta" +
+      "tusResponse\022b\n\023GetTableDescriptors\022$.hba" +
+      "se.pb.GetTableDescriptorsRequest\032%.hbase",
+      ".pb.GetTableDescriptorsResponse\022P\n\rGetTa" +
+      "bleNames\022\036.hbase.pb.GetTableNamesRequest" +
+      "\032\037.hbase.pb.GetTableNamesResponse\022Y\n\020Get" +
+      "ClusterStatus\022!.hbase.pb.GetClusterStatu" +
+      "sRequest\032\".hbase.pb.GetClusterStatusResp" +
+      "onse\022V\n\017IsMasterRunning\022 .hbase.pb.IsMas" +
+      "terRunningRequest\032!.hbase.pb.IsMasterRun" +
+      "ningResponse\022D\n\tAddColumn\022\032.hbase.pb.Add" +
+      "ColumnRequest\032\033.hbase.pb.AddColumnRespon" +
+      "se\022M\n\014DeleteColumn\022\035.hbase.pb.DeleteColu",
+      "mnRequest\032\036.hbase.pb.DeleteColumnRespons" +
+      "e\022M\n\014ModifyColumn\022\035.hbase.pb.ModifyColum" +
+      "nRequest\032\036.hbase.pb.ModifyColumnResponse" +
+      "\022G\n\nMoveRegion\022\033.hbase.pb.MoveRegionRequ" +
+      "est\032\034.hbase.pb.MoveRegionResponse\022\\\n\021Mer" +
+      "geTableRegions\022\".hbase.pb.MergeTableRegi" +
+      "onsRequest\032#.hbase.pb.MergeTableRegionsR" +
+      "esponse\022M\n\014AssignRegion\022\035.hbase.pb.Assig" +
+      "nRegionRequest\032\036.hbase.pb.AssignRegionRe" +
+      "sponse\022S\n\016UnassignRegion\022\037.hbase.pb.Unas",
+      "signRegionRequest\032 .hbase.pb.UnassignReg" +
+      "ionResponse\022P\n\rOfflineRegion\022\036.hbase.pb." +
+      "OfflineRegionRequest\032\037.hbase.pb.OfflineR" +
+      "egionResponse\022J\n\013DeleteTable\022\034.hbase.pb." +
+      "DeleteTableRequest\032\035.hbase.pb.DeleteTabl" +
+      "eResponse\022P\n\rtruncateTable\022\036.hbase.pb.Tr" +
+      "uncateTableRequest\032\037.hbase.pb.TruncateTa" +
+      "bleResponse\022J\n\013EnableTable\022\034.hbase.pb.En" +
+      "ableTableRequest\032\035.hbase.pb.EnableTableR" +
+      "esponse\022M\n\014DisableTable\022\035.hbase.pb.Disab",
+      "leTableRequest\032\036.hbase.pb.DisableTableRe" +
+      "sponse\022J\n\013ModifyTable\022\034.hbase.pb.ModifyT" +
+      "ableRequest\032\035.hbase.pb.ModifyTableRespon" +
+      "se\022J\n\013CreateTable\022\034.hbase.pb.CreateTable" +
+      "Request\032\035.hbase.pb.CreateTableResponse\022A" +
+      "\n\010Shutdown\022\031.hbase.pb.ShutdownRequest\032\032." +
+      "hbase.pb.ShutdownResponse\022G\n\nStopMaster\022" +
+      "\033.hbase.pb.StopMasterRequest\032\034.hbase.pb." +
+      "StopMasterResponse\022h\n\031IsMasterInMaintena" +
+      "nceMode\022$.hbase.pb.IsInMaintenanceModeRe",
+      "quest\032%.hbase.pb.IsInMaintenanceModeResp" +
+      "onse\022>\n\007Balance\022\030.hbase.pb.BalanceReques" +
+      "t\032\031.hbase.pb.BalanceResponse\022_\n\022SetBalan" +
+      "cerRunning\022#.hbase.pb.SetBalancerRunning" +
+      "Request\032$.hbase.pb.SetBalancerRunningRes" +
+      "ponse\022\\\n\021IsBalancerEnabled\022\".hbase.pb.Is" +
+      "BalancerEnabledRequest\032#.hbase.pb.IsBala" +
+      "ncerEnabledResponse\022k\n\026SetSplitOrMergeEn" +
+      "abled\022\'.hbase.pb.SetSplitOrMergeEnabledR" +
+      "equest\032(.hbase.pb.SetSplitOrMergeEnabled",
+      "Response\022h\n\025IsSplitOrMergeEnabled\022&.hbas" +
+      "e.pb.IsSplitOrMergeEnabledRequest\032\'.hbas" +
+      "e.pb.IsSplitOrMergeEnabledResponse\022D\n\tNo" +
+      "rmalize\022\032.hbase.pb.NormalizeRequest\032\033.hb" +
+      "ase.pb.NormalizeResponse\022e\n\024SetNormalize" +
+      "rRunning\022%.hbase.pb.SetNormalizerRunning" +
+      "Request\032&.hbase.pb.SetNormalizerRunningR" +
+      "esponse\022b\n\023IsNormalizerEnabled\022$.hbase.p" +
+      "b.IsNormalizerEnabledRequest\032%.hbase.pb." +
+      "IsNormalizerEnabledResponse\022S\n\016RunCatalo",
+      "gScan\022\037.hbase.pb.RunCatalogScanRequest\032 " +
+      ".hbase.pb.RunCatalogScanResponse\022e\n\024Enab" +
+      "leCatalogJanitor\022%.hbase.pb.EnableCatalo" +
+      "gJanitorRequest\032&.hbase.pb.EnableCatalog" +
+      "JanitorResponse\022n\n\027IsCatalogJanitorEnabl" +
+      "ed\022(.hbase.pb.IsCatalogJanitorEnabledReq" +
+      "uest\032).hbase.pb.IsCatalogJanitorEnabledR" +
+      "esponse\022V\n\017RunCleanerChore\022 .hbase.pb.Ru" +
+      "nCleanerChoreRequest\032!.hbase.pb.RunClean" +
+      "erChoreResponse\022k\n\026SetCleanerChoreRunnin",
+      "g\022\'.hbase.pb.SetCleanerChoreRunningReque" +
+      "st\032(.hbase.pb.SetCleanerChoreRunningResp" +
+      "onse\022h\n\025IsCleanerChoreEnabled\022&.hbase.pb" +
+      ".IsCleanerChoreEnabledRequest\032\'.hbase.pb" +
+      ".IsCleanerChoreEnabledResponse\022^\n\021ExecMa" +
+      "sterService\022#.hbase.pb.CoprocessorServic" +
+      "eRequest\032$.hbase.pb.CoprocessorServiceRe" +
+      "sponse\022A\n\010Snapshot\022\031.hbase.pb.SnapshotRe" +
+      "quest\032\032.hbase.pb.SnapshotResponse\022h\n\025Get" +
+      "CompletedSnapshots\022&.hbase.pb.GetComplet",
+      "edSnapshotsRequest\032\'.hbase.pb.GetComplet" +
+      "edSnapshotsResponse\022S\n\016DeleteSnapshot\022\037." +
+      "hbase.pb.DeleteSnapshotRequest\032 .hbase.p" +
+      "b.DeleteSnapshotResponse\022S\n\016IsSnapshotDo" +
+      "ne\022\037.hbase.pb.IsSnapshotDoneRequest\032 .hb" +
+      "ase.pb.IsSnapshotDoneResponse\022V\n\017Restore" +
+      "Snapshot\022 .hbase.pb.RestoreSnapshotReque" +
+      "st\032!.hbase.pb.RestoreSnapshotResponse\022P\n" +
+      "\rExecProcedure\022\036.hbase.pb.ExecProcedureR" +
+      "equest\032\037.hbase.pb.ExecProcedureResponse\022",
+      "W\n\024ExecProcedureWithRet\022\036.hbase.pb.ExecP" +
+      "rocedureRequest\032\037.hbase.pb.ExecProcedure" +
+      "Response\022V\n\017IsProcedureDone\022 .hbase.pb.I" +
+      "sProcedureDoneRequest\032!.hbase.pb.IsProce" +
+      "dureDoneResponse\022V\n\017ModifyNamespace\022 .hb" +
+      "ase.pb.ModifyNamespaceRequest\032!.hbase.pb" +
+      ".ModifyNamespaceResponse\022V\n\017CreateNamesp" +
+      "ace\022 .hbase.pb.CreateNamespaceRequest\032!." +
+      "hbase.pb.CreateNamespaceResponse\022V\n\017Dele" +
+      "teNamespace\022 .hbase.pb.DeleteNamespaceRe",
+      "quest\032!.hbase.pb.DeleteNamespaceResponse" +
+      "\022k\n\026GetNamespaceDescriptor\022\'.hbase.pb.Ge" +
+      "tNamespaceDescriptorRequest\032(.hbase.pb.G" +
+      "etNamespaceDescriptorResponse\022q\n\030ListNam" +
+      "espaceDescriptors\022).hbase.pb.ListNamespa" +
+      "ceDescriptorsRequest\032*.hbase.pb.ListName" +
+      "spaceDescriptorsResponse\022\206\001\n\037ListTableDe" +
+      "scriptorsByNamespace\0220.hbase.pb.ListTabl" +
+      "eDescriptorsByNamespaceRequest\0321.hbase.p" +
+      "b.ListTableDescriptorsByNamespaceRespons",
+      "e\022t\n\031ListTableNamesByNamespace\022*.hbase.p" +
+      "b.ListTableNamesByNamespaceRequest\032+.hba" +
+      "se.pb.ListTableNamesByNamespaceResponse\022" +
+      "P\n\rGetTableState\022\036.hbase.pb.GetTableStat" +
+      "eRequest\032\037.hbase.pb.GetTableStateRespons" +
+      "e\022A\n\010SetQuota\022\031.hbase.pb.SetQuotaRequest" +
+      "\032\032.hbase.pb.SetQuotaResponse\022x\n\037getLastM" +
+      "ajorCompactionTimestamp\022).hbase.pb.Major" +
+      "CompactionTimestampRequest\032*.hbase.pb.Ma" +
+      "jorCompactionTimestampResponse\022\212\001\n(getLa",
+      "stMajorCompactionTimestampForRegion\0222.hb" +
+      "ase.pb.MajorCompactionTimestampForRegion" +
+      "Request\032*.hbase.pb.MajorCompactionTimest" +
+      "ampResponse\022_\n\022getProcedureResult\022#.hbas" +
+      "e.pb.GetProcedureResultRequest\032$.hbase.p" +
+      "b.GetProcedureResultResponse\022h\n\027getSecur" +
+      "ityCapabilities\022%.hbase.pb.SecurityCapab" +
+      "ilitiesRequest\032&.hbase.pb.SecurityCapabi" +
+      "litiesResponse\022S\n\016AbortProcedure\022\037.hbase" +
+      ".pb.AbortProcedureRequest\032 .hbase.pb.Abo",
+      "rtProcedureResponse\022S\n\016ListProcedures\022\037." +
+      "hbase.pb.ListProceduresRequest\032 .hbase.p" +
+      "b.ListProceduresResponse\022D\n\tListLocks\022\032." +
+      "hbase.pb.ListLocksRequest\032\033.hbase.pb.Lis" +
+      "tLocksResponse\022_\n\022AddReplicationPeer\022#.h" +
+      "base.pb.AddReplicationPeerRequest\032$.hbas" +
+      "e.pb.AddReplicationPeerResponse\022h\n\025Remov" +
+      "eReplicationPeer\022&.hbase.pb.RemoveReplic" +
+      "ationPeerRequest\032\'.hbase.pb.RemoveReplic" +
+      "ationPeerResponse\022h\n\025EnableReplicationPe",
+      "er\022&.hbase.pb.EnableReplicationPeerReque" +
+      "st\032\'.hbase.pb.EnableReplicationPeerRespo" +
+      "nse\022k\n\026DisableReplicationPeer\022\'.hbase.pb" +
+      ".DisableReplicationPeerRequest\032(.hbase.p" +
+      "b.DisableReplicationPeerResponse\022q\n\030GetR" +
+      "eplicationPeerConfig\022).hbase.pb.GetRepli" +
+      "cationPeerConfigRequest\032*.hbase.pb.GetRe" +
+      "plicationPeerConfigResponse\022z\n\033UpdateRep" +
+      "licationPeerConfig\022,.hbase.pb.UpdateRepl" +
+      "icationPeerConfigRequest\032-.hbase.pb.Upda",
+      "teReplicationPeerConfigResponse\022e\n\024ListR" +
+      "eplicationPeers\022%.hbase.pb.ListReplicati" +
+      "onPeersRequest\032&.hbase.pb.ListReplicatio" +
+      "nPeersResponse\022t\n\031listDrainingRegionServ" +
+      "ers\022*.hbase.pb.ListDrainingRegionServers" +
+      "Request\032+.hbase.pb.ListDrainingRegionSer" +
+      "versResponse\022_\n\022drainRegionServers\022#.hba" +
+      "se.pb.DrainRegionServersRequest\032$.hbase." +
+      "pb.DrainRegionServersResponse\022}\n\034removeD" +
+      "rainFromRegionServers\022-.hbase.pb.RemoveD",
+      "rainFromRegionServersRequest\032..hbase.pb." +
+      "RemoveDrainFromRegionServersResponseBI\n1" +
+      "org.apache.hadoop.hbase.shaded.protobuf." +
+      "generatedB\014MasterProtosH\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -78747,7 +78947,7 @@ public final class MasterProtos {
     internal_static_hbase_pb_SetQuotaRequest_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SetQuotaRequest_descriptor,
-        new java.lang.String[] { "UserName", "UserGroup", "Namespace", "TableName", "RemoveAll", "BypassGlobals", "Throttle", });
+        new java.lang.String[] { "UserName", "UserGroup", "Namespace", "TableName", "RemoveAll", "BypassGlobals", "Throttle", "SpaceLimit", });
     internal_static_hbase_pb_SetQuotaResponse_descriptor =
       getDescriptor().getMessageTypes().get(113);
     internal_static_hbase_pb_SetQuotaResponse_fieldAccessorTable = new