You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2017/04/17 20:27:33 UTC

[24/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/990062a9
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/990062a9
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/990062a9

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

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


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

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

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

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

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

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

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

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