You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2016/11/17 18:20:46 UTC

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

Branch: refs/heads/HBASE-16961
Commit: fb936eebf95fb503817fd528198d1623ae5e3ca3
Parents: 5753d18
Author: tedyu <yu...@gmail.com>
Authored: Thu Nov 17 10:19:52 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Thu Nov 17 10:19:52 2016 -0800

----------------------------------------------------------------------
 .../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 |  498 +++--
 .../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, 4248 insertions(+), 291 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fb936eeb/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 a7c49b3..b8e99b8 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
@@ -28,6 +28,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
 @InterfaceStability.Evolving
@@ -91,6 +92,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;
   }
 
@@ -124,6 +128,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
    */
@@ -280,4 +296,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/fb936eeb/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 40a8b66..2c44201 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
@@ -28,4 +28,5 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 public enum QuotaType {
   THROTTLE,
   GLOBAL_BYPASS,
+  SPACE,
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb936eeb/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/fb936eeb/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/fb936eeb/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 0c95615..cecaad2 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
@@ -86,6 +86,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
 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;
@@ -2489,6 +2490,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);
   }
@@ -2502,11 +2504,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
@@ -2524,6 +2560,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/fb936eeb/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/fb936eeb/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/fb936eeb/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 03ef208..e13d61d 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
@@ -59646,6 +59646,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}
@@ -59748,6 +59761,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) {
@@ -59971,6 +59997,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;
@@ -60016,6 +60063,9 @@ public final class MasterProtos {
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
         output.writeMessage(7, getThrottle());
       }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeMessage(8, getSpaceLimit());
+      }
       unknownFields.writeTo(output);
     }
 
@@ -60049,6 +60099,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;
@@ -60101,6 +60155,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;
     }
@@ -60142,6 +60201,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;
@@ -60258,6 +60321,7 @@ public final class MasterProtos {
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
           getThrottleFieldBuilder();
+          getSpaceLimitFieldBuilder();
         }
       }
       public Builder clear() {
@@ -60284,6 +60348,12 @@ public final class MasterProtos {
           throttleBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000040);
+        if (spaceLimitBuilder_ == null) {
+          spaceLimit_ = null;
+        } else {
+          spaceLimitBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000080);
         return this;
       }
 
@@ -60344,6 +60414,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;
@@ -60413,6 +60491,9 @@ public final class MasterProtos {
         if (other.hasThrottle()) {
           mergeThrottle(other.getThrottle());
         }
+        if (other.hasSpaceLimit()) {
+          mergeSpaceLimit(other.getSpaceLimit());
+        }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
         return this;
@@ -60978,6 +61059,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);
@@ -69577,158 +69776,159 @@ public final class MasterProtos {
       "ureResponse\022\034\n\024is_procedure_aborted\030\001 \002(" +
       "\010\"\027\n\025ListProceduresRequest\"@\n\026ListProced" +
       "uresResponse\022&\n\tprocedure\030\001 \003(\0132\023.hbase." +
-      "pb.Procedure\"\315\001\n\017SetQuotaRequest\022\021\n\tuser" +
+      "pb.Procedure\"\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\tnames",
       "pace\030\003 \001(\t\022\'\n\ntable_name\030\004 \001(\0132\023.hbase.p" +
       "b.TableName\022\022\n\nremove_all\030\005 \001(\010\022\026\n\016bypas" +
       "s_globals\030\006 \001(\010\022+\n\010throttle\030\007 \001(\0132\031.hbas" +
-      "e.pb.ThrottleRequest\"\022\n\020SetQuotaResponse" +
-      "\"J\n\037MajorCompactionTimestampRequest\022\'\n\nt" +
-      "able_name\030\001 \002(\0132\023.hbase.pb.TableName\"U\n(" +
-      "MajorCompactionTimestampForRegionRequest" +
-      "\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecif" +
-      "ier\"@\n MajorCompactionTimestampResponse\022" +
-      "\034\n\024compaction_timestamp\030\001 \002(\003\"\035\n\033Securit",
-      "yCapabilitiesRequest\"\354\001\n\034SecurityCapabil" +
-      "itiesResponse\022G\n\014capabilities\030\001 \003(\01621.hb" +
-      "ase.pb.SecurityCapabilitiesResponse.Capa" +
-      "bility\"\202\001\n\nCapability\022\031\n\025SIMPLE_AUTHENTI" +
-      "CATION\020\000\022\031\n\025SECURE_AUTHENTICATION\020\001\022\021\n\rA" +
-      "UTHORIZATION\020\002\022\026\n\022CELL_AUTHORIZATION\020\003\022\023" +
-      "\n\017CELL_VISIBILITY\020\004*(\n\020MasterSwitchType\022" +
-      "\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\323(\n\rMasterService\022" +
-      "e\n\024GetSchemaAlterStatus\022%.hbase.pb.GetSc" +
-      "hemaAlterStatusRequest\032&.hbase.pb.GetSch",
-      "emaAlterStatusResponse\022b\n\023GetTableDescri" +
-      "ptors\022$.hbase.pb.GetTableDescriptorsRequ" +
-      "est\032%.hbase.pb.GetTableDescriptorsRespon" +
-      "se\022P\n\rGetTableNames\022\036.hbase.pb.GetTableN" +
-      "amesRequest\032\037.hbase.pb.GetTableNamesResp" +
-      "onse\022Y\n\020GetClusterStatus\022!.hbase.pb.GetC" +
-      "lusterStatusRequest\032\".hbase.pb.GetCluste" +
-      "rStatusResponse\022V\n\017IsMasterRunning\022 .hba" +
-      "se.pb.IsMasterRunningRequest\032!.hbase.pb." +
-      "IsMasterRunningResponse\022D\n\tAddColumn\022\032.h",
-      "base.pb.AddColumnRequest\032\033.hbase.pb.AddC" +
-      "olumnResponse\022M\n\014DeleteColumn\022\035.hbase.pb" +
-      ".DeleteColumnRequest\032\036.hbase.pb.DeleteCo" +
-      "lumnResponse\022M\n\014ModifyColumn\022\035.hbase.pb." +
-      "ModifyColumnRequest\032\036.hbase.pb.ModifyCol" +
-      "umnResponse\022G\n\nMoveRegion\022\033.hbase.pb.Mov" +
-      "eRegionRequest\032\034.hbase.pb.MoveRegionResp" +
-      "onse\022k\n\026DispatchMergingRegions\022\'.hbase.p" +
-      "b.DispatchMergingRegionsRequest\032(.hbase." +
-      "pb.DispatchMergingRegionsResponse\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\022^\n\021Exe",
-      "cMasterService\022#.hbase.pb.CoprocessorSer" +
-      "viceRequest\032$.hbase.pb.CoprocessorServic" +
-      "eResponse\022A\n\010Snapshot\022\031.hbase.pb.Snapsho" +
-      "tRequest\032\032.hbase.pb.SnapshotResponse\022h\n\025" +
-      "GetCompletedSnapshots\022&.hbase.pb.GetComp" +
-      "letedSnapshotsRequest\032\'.hbase.pb.GetComp" +
-      "letedSnapshotsResponse\022S\n\016DeleteSnapshot" +
-      "\022\037.hbase.pb.DeleteSnapshotRequest\032 .hbas" +
-      "e.pb.DeleteSnapshotResponse\022S\n\016IsSnapsho" +
-      "tDone\022\037.hbase.pb.IsSnapshotDoneRequest\032 ",
-      ".hbase.pb.IsSnapshotDoneResponse\022V\n\017Rest" +
-      "oreSnapshot\022 .hbase.pb.RestoreSnapshotRe" +
-      "quest\032!.hbase.pb.RestoreSnapshotResponse" +
-      "\022P\n\rExecProcedure\022\036.hbase.pb.ExecProcedu" +
-      "reRequest\032\037.hbase.pb.ExecProcedureRespon" +
-      "se\022W\n\024ExecProcedureWithRet\022\036.hbase.pb.Ex" +
-      "ecProcedureRequest\032\037.hbase.pb.ExecProced" +
-      "ureResponse\022V\n\017IsProcedureDone\022 .hbase.p" +
-      "b.IsProcedureDoneRequest\032!.hbase.pb.IsPr" +
-      "ocedureDoneResponse\022V\n\017ModifyNamespace\022 ",
-      ".hbase.pb.ModifyNamespaceRequest\032!.hbase" +
-      ".pb.ModifyNamespaceResponse\022V\n\017CreateNam" +
-      "espace\022 .hbase.pb.CreateNamespaceRequest" +
-      "\032!.hbase.pb.CreateNamespaceResponse\022V\n\017D" +
-      "eleteNamespace\022 .hbase.pb.DeleteNamespac" +
-      "eRequest\032!.hbase.pb.DeleteNamespaceRespo" +
-      "nse\022k\n\026GetNamespaceDescriptor\022\'.hbase.pb" +
-      ".GetNamespaceDescriptorRequest\032(.hbase.p" +
-      "b.GetNamespaceDescriptorResponse\022q\n\030List" +
-      "NamespaceDescriptors\022).hbase.pb.ListName",
-      "spaceDescriptorsRequest\032*.hbase.pb.ListN" +
-      "amespaceDescriptorsResponse\022\206\001\n\037ListTabl" +
-      "eDescriptorsByNamespace\0220.hbase.pb.ListT" +
-      "ableDescriptorsByNamespaceRequest\0321.hbas" +
-      "e.pb.ListTableDescriptorsByNamespaceResp" +
-      "onse\022t\n\031ListTableNamesByNamespace\022*.hbas" +
-      "e.pb.ListTableNamesByNamespaceRequest\032+." +
-      "hbase.pb.ListTableNamesByNamespaceRespon" +
-      "se\022P\n\rGetTableState\022\036.hbase.pb.GetTableS" +
-      "tateRequest\032\037.hbase.pb.GetTableStateResp",
-      "onse\022A\n\010SetQuota\022\031.hbase.pb.SetQuotaRequ" +
-      "est\032\032.hbase.pb.SetQuotaResponse\022x\n\037getLa" +
-      "stMajorCompactionTimestamp\022).hbase.pb.Ma" +
-      "jorCompactionTimestampRequest\032*.hbase.pb" +
-      ".MajorCompactionTimestampResponse\022\212\001\n(ge" +
-      "tLastMajorCompactionTimestampForRegion\0222" +
-      ".hbase.pb.MajorCompactionTimestampForReg" +
-      "ionRequest\032*.hbase.pb.MajorCompactionTim" +
-      "estampResponse\022_\n\022getProcedureResult\022#.h" +
-      "base.pb.GetProcedureResultRequest\032$.hbas",
-      "e.pb.GetProcedureResultResponse\022h\n\027getSe" +
-      "curityCapabilities\022%.hbase.pb.SecurityCa" +
-      "pabilitiesRequest\032&.hbase.pb.SecurityCap" +
-      "abilitiesResponse\022S\n\016AbortProcedure\022\037.hb" +
-      "ase.pb.AbortProcedureRequest\032 .hbase.pb." +
-      "AbortProcedureResponse\022S\n\016ListProcedures" +
-      "\022\037.hbase.pb.ListProceduresRequest\032 .hbas" +
-      "e.pb.ListProceduresResponseBI\n1org.apach" +
-      "e.hadoop.hbase.shaded.protobuf.generated" +
-      "B\014MasterProtosH\001\210\001\001\240\001\001"
+      "e.pb.ThrottleRequest\0220\n\013space_limit\030\010 \001(" +
+      "\0132\033.hbase.pb.SpaceLimitRequest\"\022\n\020SetQuo" +
+      "taResponse\"J\n\037MajorCompactionTimestampRe" +
+      "quest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.Tab" +
+      "leName\"U\n(MajorCompactionTimestampForReg" +
+      "ionRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Re" +
+      "gionSpecifier\"@\n MajorCompactionTimestam",
+      "pResponse\022\034\n\024compaction_timestamp\030\001 \002(\003\"" +
+      "\035\n\033SecurityCapabilitiesRequest\"\354\001\n\034Secur" +
+      "ityCapabilitiesResponse\022G\n\014capabilities\030" +
+      "\001 \003(\01621.hbase.pb.SecurityCapabilitiesRes" +
+      "ponse.Capability\"\202\001\n\nCapability\022\031\n\025SIMPL" +
+      "E_AUTHENTICATION\020\000\022\031\n\025SECURE_AUTHENTICAT" +
+      "ION\020\001\022\021\n\rAUTHORIZATION\020\002\022\026\n\022CELL_AUTHORI" +
+      "ZATION\020\003\022\023\n\017CELL_VISIBILITY\020\004*(\n\020MasterS" +
+      "witchType\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\323(\n\rMast" +
+      "erService\022e\n\024GetSchemaAlterStatus\022%.hbas",
+      "e.pb.GetSchemaAlterStatusRequest\032&.hbase" +
+      ".pb.GetSchemaAlterStatusResponse\022b\n\023GetT" +
+      "ableDescriptors\022$.hbase.pb.GetTableDescr" +
+      "iptorsRequest\032%.hbase.pb.GetTableDescrip" +
+      "torsResponse\022P\n\rGetTableNames\022\036.hbase.pb" +
+      ".GetTableNamesRequest\032\037.hbase.pb.GetTabl" +
+      "eNamesResponse\022Y\n\020GetClusterStatus\022!.hba" +
+      "se.pb.GetClusterStatusRequest\032\".hbase.pb" +
+      ".GetClusterStatusResponse\022V\n\017IsMasterRun" +
+      "ning\022 .hbase.pb.IsMasterRunningRequest\032!",
+      ".hbase.pb.IsMasterRunningResponse\022D\n\tAdd" +
+      "Column\022\032.hbase.pb.AddColumnRequest\032\033.hba" +
+      "se.pb.AddColumnResponse\022M\n\014DeleteColumn\022" +
+      "\035.hbase.pb.DeleteColumnRequest\032\036.hbase.p" +
+      "b.DeleteColumnResponse\022M\n\014ModifyColumn\022\035" +
+      ".hbase.pb.ModifyColumnRequest\032\036.hbase.pb" +
+      ".ModifyColumnResponse\022G\n\nMoveRegion\022\033.hb" +
+      "ase.pb.MoveRegionRequest\032\034.hbase.pb.Move" +
+      "RegionResponse\022k\n\026DispatchMergingRegions" +
+      "\022\'.hbase.pb.DispatchMergingRegionsReques",
+      "t\032(.hbase.pb.DispatchMergingRegionsRespo" +
+      "nse\022M\n\014AssignRegion\022\035.hbase.pb.AssignReg" +
+      "ionRequest\032\036.hbase.pb.AssignRegionRespon" +
+      "se\022S\n\016UnassignRegion\022\037.hbase.pb.Unassign" +
+      "RegionRequest\032 .hbase.pb.UnassignRegionR" +
+      "esponse\022P\n\rOfflineRegion\022\036.hbase.pb.Offl" +
+      "ineRegionRequest\032\037.hbase.pb.OfflineRegio" +
+      "nResponse\022J\n\013DeleteTable\022\034.hbase.pb.Dele" +
+      "teTableRequest\032\035.hbase.pb.DeleteTableRes" +
+      "ponse\022P\n\rtruncateTable\022\036.hbase.pb.Trunca",
+      "teTableRequest\032\037.hbase.pb.TruncateTableR" +
+      "esponse\022J\n\013EnableTable\022\034.hbase.pb.Enable" +
+      "TableRequest\032\035.hbase.pb.EnableTableRespo" +
+      "nse\022M\n\014DisableTable\022\035.hbase.pb.DisableTa" +
+      "bleRequest\032\036.hbase.pb.DisableTableRespon" +
+      "se\022J\n\013ModifyTable\022\034.hbase.pb.ModifyTable" +
+      "Request\032\035.hbase.pb.ModifyTableResponse\022J" +
+      "\n\013CreateTable\022\034.hbase.pb.CreateTableRequ" +
+      "est\032\035.hbase.pb.CreateTableResponse\022A\n\010Sh" +
+      "utdown\022\031.hbase.pb.ShutdownRequest\032\032.hbas",
+      "e.pb.ShutdownResponse\022G\n\nStopMaster\022\033.hb" +
+      "ase.pb.StopMasterRequest\032\034.hbase.pb.Stop" +
+      "MasterResponse\022h\n\031IsMasterInMaintenanceM" +
+      "ode\022$.hbase.pb.IsInMaintenanceModeReques" +
+      "t\032%.hbase.pb.IsInMaintenanceModeResponse" +
+      "\022>\n\007Balance\022\030.hbase.pb.BalanceRequest\032\031." +
+      "hbase.pb.BalanceResponse\022_\n\022SetBalancerR" +
+      "unning\022#.hbase.pb.SetBalancerRunningRequ" +
+      "est\032$.hbase.pb.SetBalancerRunningRespons" +
+      "e\022\\\n\021IsBalancerEnabled\022\".hbase.pb.IsBala",
+      "ncerEnabledRequest\032#.hbase.pb.IsBalancer" +
+      "EnabledResponse\022k\n\026SetSplitOrMergeEnable" +
+      "d\022\'.hbase.pb.SetSplitOrMergeEnabledReque" +
+      "st\032(.hbase.pb.SetSplitOrMergeEnabledResp" +
+      "onse\022h\n\025IsSplitOrMergeEnabled\022&.hbase.pb" +
+      ".IsSplitOrMergeEnabledRequest\032\'.hbase.pb" +
+      ".IsSplitOrMergeEnabledResponse\022D\n\tNormal" +
+      "ize\022\032.hbase.pb.NormalizeRequest\032\033.hbase." +
+      "pb.NormalizeResponse\022e\n\024SetNormalizerRun" +
+      "ning\022%.hbase.pb.SetNormalizerRunningRequ",
+      "est\032&.hbase.pb.SetNormalizerRunningRespo" +
+      "nse\022b\n\023IsNormalizerEnabled\022$.hbase.pb.Is" +
+      "NormalizerEnabledRequest\032%.hbase.pb.IsNo" +
+      "rmalizerEnabledResponse\022S\n\016RunCatalogSca" +
+      "n\022\037.hbase.pb.RunCatalogScanRequest\032 .hba" +
+      "se.pb.RunCatalogScanResponse\022e\n\024EnableCa" +
+      "talogJanitor\022%.hbase.pb.EnableCatalogJan" +
+      "itorRequest\032&.hbase.pb.EnableCatalogJani" +
+      "torResponse\022n\n\027IsCatalogJanitorEnabled\022(" +
+      ".hbase.pb.IsCatalogJanitorEnabledRequest",
+      "\032).hbase.pb.IsCatalogJanitorEnabledRespo" +
+      "nse\022^\n\021ExecMasterService\022#.hbase.pb.Copr" +
+      "ocessorServiceRequest\032$.hbase.pb.Coproce" +
+      "ssorServiceResponse\022A\n\010Snapshot\022\031.hbase." +
+      "pb.SnapshotRequest\032\032.hbase.pb.SnapshotRe" +
+      "sponse\022h\n\025GetCompletedSnapshots\022&.hbase." +
+      "pb.GetCompletedSnapshotsRequest\032\'.hbase." +
+      "pb.GetCompletedSnapshotsResponse\022S\n\016Dele" +
+      "teSnapshot\022\037.hbase.pb.DeleteSnapshotRequ" +
+      "est\032 .hbase.pb.DeleteSnapshotResponse\022S\n",
+      "\016IsSnapshotDone\022\037.hbase.pb.IsSnapshotDon" +
+      "eRequest\032 .hbase.pb.IsSnapshotDoneRespon" +
+      "se\022V\n\017RestoreSnapshot\022 .hbase.pb.Restore" +
+      "SnapshotRequest\032!.hbase.pb.RestoreSnapsh" +
+      "otResponse\022P\n\rExecProcedure\022\036.hbase.pb.E" +
+      "xecProcedureRequest\032\037.hbase.pb.ExecProce" +
+      "dureResponse\022W\n\024ExecProcedureWithRet\022\036.h" +
+      "base.pb.ExecProcedureRequest\032\037.hbase.pb." +
+      "ExecProcedureResponse\022V\n\017IsProcedureDone" +
+      "\022 .hbase.pb.IsProcedureDoneRequest\032!.hba",
+      "se.pb.IsProcedureDoneResponse\022V\n\017ModifyN" +
+      "amespace\022 .hbase.pb.ModifyNamespaceReque" +
+      "st\032!.hbase.pb.ModifyNamespaceResponse\022V\n" +
+      "\017CreateNamespace\022 .hbase.pb.CreateNamesp" +
+      "aceRequest\032!.hbase.pb.CreateNamespaceRes" +
+      "ponse\022V\n\017DeleteNamespace\022 .hbase.pb.Dele" +
+      "teNamespaceRequest\032!.hbase.pb.DeleteName" +
+      "spaceResponse\022k\n\026GetNamespaceDescriptor\022" +
+      "\'.hbase.pb.GetNamespaceDescriptorRequest" +
+      "\032(.hbase.pb.GetNamespaceDescriptorRespon",
+      "se\022q\n\030ListNamespaceDescriptors\022).hbase.p" +
+      "b.ListNamespaceDescriptorsRequest\032*.hbas" +
+      "e.pb.ListNamespaceDescriptorsResponse\022\206\001" +
+      "\n\037ListTableDescriptorsByNamespace\0220.hbas" +
+      "e.pb.ListTableDescriptorsByNamespaceRequ" +
+      "est\0321.hbase.pb.ListTableDescriptorsByNam" +
+      "espaceResponse\022t\n\031ListTableNamesByNamesp" +
+      "ace\022*.hbase.pb.ListTableNamesByNamespace" +
+      "Request\032+.hbase.pb.ListTableNamesByNames" +
+      "paceResponse\022P\n\rGetTableState\022\036.hbase.pb",
+      ".GetTableStateRequest\032\037.hbase.pb.GetTabl" +
+      "eStateResponse\022A\n\010SetQuota\022\031.hbase.pb.Se" +
+      "tQuotaRequest\032\032.hbase.pb.SetQuotaRespons" +
+      "e\022x\n\037getLastMajorCompactionTimestamp\022).h" +
+      "base.pb.MajorCompactionTimestampRequest\032" +
+      "*.hbase.pb.MajorCompactionTimestampRespo" +
+      "nse\022\212\001\n(getLastMajorCompactionTimestampF" +
+      "orRegion\0222.hbase.pb.MajorCompactionTimes" +
+      "tampForRegionRequest\032*.hbase.pb.MajorCom" +
+      "pactionTimestampResponse\022_\n\022getProcedure",
+      "Result\022#.hbase.pb.GetProcedureResultRequ" +
+      "est\032$.hbase.pb.GetProcedureResultRespons" +
+      "e\022h\n\027getSecurityCapabilities\022%.hbase.pb." +
+      "SecurityCapabilitiesRequest\032&.hbase.pb.S" +
+      "ecurityCapabilitiesResponse\022S\n\016AbortProc" +
+      "edure\022\037.hbase.pb.AbortProcedureRequest\032 " +
+      ".hbase.pb.AbortProcedureResponse\022S\n\016List" +
+      "Procedures\022\037.hbase.pb.ListProceduresRequ" +
+      "est\032 .hbase.pb.ListProceduresResponseBI\n" +
+      "1org.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() {
@@ -70377,7 +70577,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(105);
     internal_static_hbase_pb_SetQuotaResponse_fieldAccessorTable = new