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/11 17:31:27 UTC

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

Branch: refs/heads/HBASE-16961
Commit: 5def44f5a187938034de1529060d679285d79edb
Parents: 82d554e
Author: tedyu <yu...@gmail.com>
Authored: Thu Nov 17 10:19:52 2016 -0800
Committer: Josh Elser <el...@apache.org>
Committed: Tue Apr 11 13:10:01 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/5def44f5/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/5def44f5/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/5def44f5/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/5def44f5/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/5def44f5/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/5def44f5/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/5def44f5/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/5def44f5/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 0071bef..45ff86d 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 {
       ":\004true\"6\n\026AbortProcedureResponse\022\034\n\024is_p" +
       "rocedure_aborted\030\001 \002(\010\"\027\n\025ListProcedures" +
       "Request\"@\n\026ListProceduresResponse\022&\n\tpro" +
-      "cedure\030\001 \003(\0132\023.hbase.pb.Procedure\"\315\001\n\017Se" +
+      "cedure\030\001 \003(\0132\023.hbase.pb.Procedure\"\377\001\n\017Se" +
       "tQuotaRequest\022\021\n\tuser_name\030\001 \001(\t\022\022\n\nuser" +
       "_group\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\'\n\ntable" +
       "_name\030\004 \001(\0132\023.hbase.pb.TableName\022\022\n\nremo" +
       "ve_all\030\005 \001(\010\022\026\n\016bypass_globals\030\006 \001(\010\022+\n\010" +
       "throttle\030\007 \001(\0132\031.hbase.pb.ThrottleReques" +
-      "t\"\022\n\020SetQuotaResponse\"J\n\037MajorCompaction",
-      "TimestampRequest\022\'\n\ntable_name\030\001 \002(\0132\023.h" +
-      "base.pb.TableName\"U\n(MajorCompactionTime" +
-      "stampForRegionRequest\022)\n\006region\030\001 \002(\0132\031." +
-      "hbase.pb.RegionSpecifier\"@\n MajorCompact" +
-      "ionTimestampResponse\022\034\n\024compaction_times" +
-      "tamp\030\001 \002(\003\"\035\n\033SecurityCapabilitiesReques" +
-      "t\"\354\001\n\034SecurityCapabilitiesResponse\022G\n\014ca" +
-      "pabilities\030\001 \003(\01621.hbase.pb.SecurityCapa" +
-      "bilitiesResponse.Capability\"\202\001\n\nCapabili" +
-      "ty\022\031\n\025SIMPLE_AUTHENTICATION\020\000\022\031\n\025SECURE_",
-      "AUTHENTICATION\020\001\022\021\n\rAUTHORIZATION\020\002\022\026\n\022C" +
-      "ELL_AUTHORIZATION\020\003\022\023\n\017CELL_VISIBILITY\020\004" +
-      "\"\"\n ListDrainingRegionServersRequest\"N\n!" +
-      "ListDrainingRegionServersResponse\022)\n\013ser" +
-      "ver_name\030\001 \003(\0132\024.hbase.pb.ServerName\"F\n\031" +
-      "DrainRegionServersRequest\022)\n\013server_name" +
-      "\030\001 \003(\0132\024.hbase.pb.ServerName\"\034\n\032DrainReg" +
-      "ionServersResponse\"P\n#RemoveDrainFromReg" +
-      "ionServersRequest\022)\n\013server_name\030\001 \003(\0132\024" +
-      ".hbase.pb.ServerName\"&\n$RemoveDrainFromR",
-      "egionServersResponse*(\n\020MasterSwitchType" +
-      "\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\3013\n\rMasterService" +
-      "\022e\n\024GetSchemaAlterStatus\022%.hbase.pb.GetS" +
-      "chemaAlterStatusRequest\032&.hbase.pb.GetSc" +
-      "hemaAlterStatusResponse\022b\n\023GetTableDescr" +
-      "iptors\022$.hbase.pb.GetTableDescriptorsReq" +
-      "uest\032%.hbase.pb.GetTableDescriptorsRespo" +
-      "nse\022P\n\rGetTableNames\022\036.hbase.pb.GetTable" +
-      "NamesRequest\032\037.hbase.pb.GetTableNamesRes" +
-      "ponse\022Y\n\020GetClusterStatus\022!.hbase.pb.Get",
-      "ClusterStatusRequest\032\".hbase.pb.GetClust" +
-      "erStatusResponse\022V\n\017IsMasterRunning\022 .hb" +
-      "ase.pb.IsMasterRunningRequest\032!.hbase.pb" +
-      ".IsMasterRunningResponse\022D\n\tAddColumn\022\032." +
-      "hbase.pb.AddColumnRequest\032\033.hbase.pb.Add" +
-      "ColumnResponse\022M\n\014DeleteColumn\022\035.hbase.p" +
-      "b.DeleteColumnRequest\032\036.hbase.pb.DeleteC" +
-      "olumnResponse\022M\n\014ModifyColumn\022\035.hbase.pb" +
-      ".ModifyColumnRequest\032\036.hbase.pb.ModifyCo" +
-      "lumnResponse\022G\n\nMoveRegion\022\033.hbase.pb.Mo",
-      "veRegionRequest\032\034.hbase.pb.MoveRegionRes" +
-      "ponse\022\\\n\021MergeTableRegions\022\".hbase.pb.Me" +
-      "rgeTableRegionsRequest\032#.hbase.pb.MergeT" +
-      "ableRegionsResponse\022M\n\014AssignRegion\022\035.hb" +
-      "ase.pb.AssignRegionRequest\032\036.hbase.pb.As" +
-      "signRegionResponse\022S\n\016UnassignRegion\022\037.h" +
-      "base.pb.UnassignRegionRequest\032 .hbase.pb" +
-      ".UnassignRegionResponse\022P\n\rOfflineRegion" +
-      "\022\036.hbase.pb.OfflineRegionRequest\032\037.hbase" +
-      ".pb.OfflineRegionResponse\022J\n\013DeleteTable",
-      "\022\034.hbase.pb.DeleteTableRequest\032\035.hbase.p" +
-      "b.DeleteTableResponse\022P\n\rtruncateTable\022\036" +
-      ".hbase.pb.TruncateTableRequest\032\037.hbase.p" +
-      "b.TruncateTableResponse\022J\n\013EnableTable\022\034" +
-      ".hbase.pb.EnableTableRequest\032\035.hbase.pb." +
-      "EnableTableResponse\022M\n\014DisableTable\022\035.hb" +
-      "ase.pb.DisableTableRequest\032\036.hbase.pb.Di" +
-      "sableTableResponse\022J\n\013ModifyTable\022\034.hbas" +
-      "e.pb.ModifyTableRequest\032\035.hbase.pb.Modif" +
-      "yTableResponse\022J\n\013CreateTable\022\034.hbase.pb",
-      ".CreateTableRequest\032\035.hbase.pb.CreateTab" +
-      "leResponse\022A\n\010Shutdown\022\031.hbase.pb.Shutdo" +
-      "wnRequest\032\032.hbase.pb.ShutdownResponse\022G\n" +
-      "\nStopMaster\022\033.hbase.pb.StopMasterRequest" +
-      "\032\034.hbase.pb.StopMasterResponse\022h\n\031IsMast" +
-      "erInMaintenanceMode\022$.hbase.pb.IsInMaint" +
-      "enanceModeRequest\032%.hbase.pb.IsInMainten" +
-      "anceModeResponse\022>\n\007Balance\022\030.hbase.pb.B" +
-      "alanceRequest\032\031.hbase.pb.BalanceResponse" +
-      "\022_\n\022SetBalancerRunning\022#.hbase.pb.SetBal",
-      "ancerRunningRequest\032$.hbase.pb.SetBalanc" +
-      "erRunningResponse\022\\\n\021IsBalancerEnabled\022\"" +
-      ".hbase.pb.IsBalancerEnabledRequest\032#.hba" +
-      "se.pb.IsBalancerEnabledResponse\022k\n\026SetSp" +
-      "litOrMergeEnabled\022\'.hbase.pb.SetSplitOrM" +
-      "ergeEnabledRequest\032(.hbase.pb.SetSplitOr" +
-      "MergeEnabledResponse\022h\n\025IsSplitOrMergeEn" +
-      "abled\022&.hbase.pb.IsSplitOrMergeEnabledRe" +
-      "quest\032\'.hbase.pb.IsSplitOrMergeEnabledRe" +
-      "sponse\022D\n\tNormalize\022\032.hbase.pb.Normalize",
-      "Request\032\033.hbase.pb.NormalizeResponse\022e\n\024" +
-      "SetNormalizerRunning\022%.hbase.pb.SetNorma" +
-      "lizerRunningRequest\032&.hbase.pb.SetNormal" +
-      "izerRunningResponse\022b\n\023IsNormalizerEnabl" +
-      "ed\022$.hbase.pb.IsNormalizerEnabledRequest" +
-      "\032%.hbase.pb.IsNormalizerEnabledResponse\022" +
-      "S\n\016RunCatalogScan\022\037.hbase.pb.RunCatalogS" +
-      "canRequest\032 .hbase.pb.RunCatalogScanResp" +
-      "onse\022e\n\024EnableCatalogJanitor\022%.hbase.pb." +
-      "EnableCatalogJanitorRequest\032&.hbase.pb.E",
-      "nableCatalogJanitorResponse\022n\n\027IsCatalog" +
-      "JanitorEnabled\022(.hbase.pb.IsCatalogJanit" +
-      "orEnabledRequest\032).hbase.pb.IsCatalogJan" +
-      "itorEnabledResponse\022V\n\017RunCleanerChore\022 " +
-      ".hbase.pb.RunCleanerChoreRequest\032!.hbase" +
-      ".pb.RunCleanerChoreResponse\022k\n\026SetCleane" +
-      "rChoreRunning\022\'.hbase.pb.SetCleanerChore" +
-      "RunningRequest\032(.hbase.pb.SetCleanerChor" +
-      "eRunningResponse\022h\n\025IsCleanerChoreEnable" +
-      "d\022&.hbase.pb.IsCleanerChoreEnabledReques",
-      "t\032\'.hbase.pb.IsCleanerChoreEnabledRespon" +
-      "se\022^\n\021ExecMasterService\022#.hbase.pb.Copro" +
-      "cessorServiceRequest\032$.hbase.pb.Coproces" +
-      "sorServiceResponse\022A\n\010Snapshot\022\031.hbase.p" +
-      "b.SnapshotRequest\032\032.hbase.pb.SnapshotRes" +
-      "ponse\022h\n\025GetCompletedSnapshots\022&.hbase.p" +
-      "b.GetCompletedSnapshotsRequest\032\'.hbase.p" +
-      "b.GetCompletedSnapshotsResponse\022S\n\016Delet" +
-      "eSnapshot\022\037.hbase.pb.DeleteSnapshotReque" +
-      "st\032 .hbase.pb.DeleteSnapshotResponse\022S\n\016",
-      "IsSnapshotDone\022\037.hbase.pb.IsSnapshotDone" +
-      "Request\032 .hbase.pb.IsSnapshotDoneRespons" +
-      "e\022V\n\017RestoreSnapshot\022 .hbase.pb.RestoreS" +
-      "napshotRequest\032!.hbase.pb.RestoreSnapsho" +
-      "tResponse\022P\n\rExecProcedure\022\036.hbase.pb.Ex" +
-      "ecProcedureRequest\032\037.hbase.pb.ExecProced" +
-      "ureResponse\022W\n\024ExecProcedureWithRet\022\036.hb" +
-      "ase.pb.ExecProcedureRequest\032\037.hbase.pb.E" +
-      "xecProcedureResponse\022V\n\017IsProcedureDone\022" +
-      " .hbase.pb.IsProcedureDoneRequest\032!.hbas",
-      "e.pb.IsProcedureDoneResponse\022V\n\017ModifyNa" +
-      "mespace\022 .hbase.pb.ModifyNamespaceReques" +
-      "t\032!.hbase.pb.ModifyNamespaceResponse\022V\n\017" +
-      "CreateNamespace\022 .hbase.pb.CreateNamespa" +
-      "ceRequest\032!.hbase.pb.CreateNamespaceResp" +
-      "onse\022V\n\017DeleteNamespace\022 .hbase.pb.Delet" +
-      "eNamespaceRequest\032!.hbase.pb.DeleteNames" +
-      "paceResponse\022k\n\026GetNamespaceDescriptor\022\'" +
-      ".hbase.pb.GetNamespaceDescriptorRequest\032" +
-      "(.hbase.pb.GetNamespaceDescriptorRespons",
-      "e\022q\n\030ListNamespaceDescriptors\022).hbase.pb" +
-      ".ListNamespaceDescriptorsRequest\032*.hbase" +
-      ".pb.ListNamespaceDescriptorsResponse\022\206\001\n" +
-      "\037ListTableDescriptorsByNamespace\0220.hbase" +
-      ".pb.ListTableDescriptorsByNamespaceReque" +
-      "st\0321.hbase.pb.ListTableDescriptorsByName" +
-      "spaceResponse\022t\n\031ListTableNamesByNamespa" +
-      "ce\022*.hbase.pb.ListTableNamesByNamespaceR" +
-      "equest\032+.hbase.pb.ListTableNamesByNamesp" +
-      "aceResponse\022P\n\rGetTableState\022\036.hbase.pb.",
-      "GetTableStateRequest\032\037.hbase.pb.GetTable" +
-      "StateResponse\022A\n\010SetQuota\022\031.hbase.pb.Set" +
-      "QuotaRequest\032\032.hbase.pb.SetQuotaResponse" +
-      "\022x\n\037getLastMajorCompactionTimestamp\022).hb" +
-      "ase.pb.MajorCompactionTimestampRequest\032*" +
-      ".hbase.pb.MajorCompactionTimestampRespon" +
-      "se\022\212\001\n(getLastMajorCompactionTimestampFo" +
-      "rRegion\0222.hbase.pb.MajorCompactionTimest" +
-      "ampForRegionRequest\032*.hbase.pb.MajorComp" +
-      "actionTimestampResponse\022_\n\022getProcedureR",
-      "esult\022#.hbase.pb.GetProcedureResultReque" +
-      "st\032$.hbase.pb.GetProcedureResultResponse" +
-      "\022h\n\027getSecurityCapabilities\022%.hbase.pb.S" +
-      "ecurityCapabilitiesRequest\032&.hbase.pb.Se" +
-      "curityCapabilitiesResponse\022S\n\016AbortProce" +
-      "dure\022\037.hbase.pb.AbortProcedureRequest\032 ." +
-      "hbase.pb.AbortProcedureResponse\022S\n\016ListP" +
-      "rocedures\022\037.hbase.pb.ListProceduresReque" +
-      "st\032 .hbase.pb.ListProceduresResponse\022_\n\022" +
-      "AddReplicationPeer\022#.hbase.pb.AddReplica",
-      "tionPeerRequest\032$.hbase.pb.AddReplicatio" +
-      "nPeerResponse\022h\n\025RemoveReplicationPeer\022&" +
-      ".hbase.pb.RemoveReplicationPeerRequest\032\'" +
-      ".hbase.pb.RemoveReplicationPeerResponse\022" +
-      "h\n\025EnableReplicationPeer\022&.hbase.pb.Enab" +
-      "leReplicationPeerRequest\032\'.hbase.pb.Enab" +
-      "leReplicationPeerResponse\022k\n\026DisableRepl" +
-      "icationPeer\022\'.hbase.pb.DisableReplicatio" +
-      "nPeerRequest\032(.hbase.pb.DisableReplicati" +
-      "onPeerResponse\022q\n\030GetReplicationPeerConf",
-      "ig\022).hbase.pb.GetReplicationPeerConfigRe" +
-      "quest\032*.hbase.pb.GetReplicationPeerConfi" +
-      "gResponse\022z\n\033UpdateReplicationPeerConfig" +
-      "\022,.hbase.pb.UpdateReplicationPeerConfigR" +
-      "equest\032-.hbase.pb.UpdateReplicationPeerC" +
-      "onfigResponse\022e\n\024ListReplicationPeers\022%." +
-      "hbase.pb.ListReplicationPeersRequest\032&.h" +
-      "base.pb.ListReplicationPeersResponse\022t\n\031" +
-      "listDrainingRegionServers\022*.hbase.pb.Lis" +
-      "tDrainingRegionServersRequest\032+.hbase.pb",
-      ".ListDrainingRegionServersResponse\022_\n\022dr" +
-      "ainRegionServers\022#.hbase.pb.DrainRegionS" +
-      "erversRequest\032$.hbase.pb.DrainRegionServ" +
-      "ersResponse\022}\n\034removeDrainFromRegionServ" +
-      "ers\022-.hbase.pb.RemoveDrainFromRegionServ" +
-      "ersRequest\032..hbase.pb.RemoveDrainFromReg" +
-      "ionServersResponseBI\n1org.apache.hadoop." +
-      "hbase.shaded.protobuf.generatedB\014MasterP" +
-      "rotosH\001\210\001\001\240\001\001"
+      "t\0220\n\013space_limit\030\010 \001(\0132\033.hbase.pb.SpaceL",
+      "imitRequest\"\022\n\020SetQuotaResponse\"J\n\037Major" +
+      "CompactionTimestampRequest\022\'\n\ntable_name" +
+      "\030\001 \002(\0132\023.hbase.pb.TableName\"U\n(MajorComp" +
+      "actionTimestampForRegionRequest\022)\n\006regio" +
+      "n\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\"@\n Ma" +
+      "jorCompactionTimestampResponse\022\034\n\024compac" +
+      "tion_timestamp\030\001 \002(\003\"\035\n\033SecurityCapabili" +
+      "tiesRequest\"\354\001\n\034SecurityCapabilitiesResp" +
+      "onse\022G\n\014capabilities\030\001 \003(\01621.hbase.pb.Se" +
+      "curityCapabilitiesResponse.Capability\"\202\001",
+      "\n\nCapability\022\031\n\025SIMPLE_AUTHENTICATION\020\000\022" +
+      "\031\n\025SECURE_AUTHENTICATION\020\001\022\021\n\rAUTHORIZAT" +
+      "ION\020\002\022\026\n\022CELL_AUTHORIZATION\020\003\022\023\n\017CELL_VI" +
+      "SIBILITY\020\004\"\"\n ListDrainingRegionServersR" +
+      "equest\"N\n!ListDrainingRegionServersRespo" +
+      "nse\022)\n\013server_name\030\001 \003(\0132\024.hbase.pb.Serv" +
+      "erName\"F\n\031DrainRegionServersRequest\022)\n\013s" +
+      "erver_name\030\001 \003(\0132\024.hbase.pb.ServerName\"\034" +
+      "\n\032DrainRegionServersResponse\"P\n#RemoveDr" +
+      "ainFromRegionServersRequest\022)\n\013server_na",
+      "me\030\001 \003(\0132\024.hbase.pb.ServerName\"&\n$Remove" +
+      "DrainFromRegionServersResponse*(\n\020Master" +
+      "SwitchType\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\3013\n\rMas" +
+      "terService\022e\n\024GetSchemaAlterStatus\022%.hba" +
+      "se.pb.GetSchemaAlterStatusRequest\032&.hbas" +
+      "e.pb.GetSchemaAlterStatusResponse\022b\n\023Get" +
+      "TableDescriptors\022$.hbase.pb.GetTableDesc" +
+      "riptorsRequest\032%.hbase.pb.GetTableDescri" +
+      "ptorsResponse\022P\n\rGetTableNames\022\036.hbase.p" +
+      "b.GetTableNamesRequest\032\037.hbase.pb.GetTab",
+      "leNamesResponse\022Y\n\020GetClusterStatus\022!.hb" +
+      "ase.pb.GetClusterStatusRequest\032\".hbase.p" +
+      "b.GetClusterStatusResponse\022V\n\017IsMasterRu" +
+      "nning\022 .hbase.pb.IsMasterRunningRequest\032" +
+      "!.hbase.pb.IsMasterRunningResponse\022D\n\tAd" +
+      "dColumn\022\032.hbase.pb.AddColumnRequest\032\033.hb" +
+      "ase.pb.AddColumnResponse\022M\n\014DeleteColumn" +
+      "\022\035.hbase.pb.DeleteColumnRequest\032\036.hbase." +
+      "pb.DeleteColumnResponse\022M\n\014ModifyColumn\022" +
+      "\035.hbase.pb.ModifyColumnRequest\032\036.hbase.p",
+      "b.ModifyColumnResponse\022G\n\nMoveRegion\022\033.h" +
+      "base.pb.MoveRegionRequest\032\034.hbase.pb.Mov" +
+      "eRegionResponse\022\\\n\021MergeTableRegions\022\".h" +
+      "base.pb.MergeTableRegionsRequest\032#.hbase" +
+      ".pb.MergeTableRegionsResponse\022M\n\014AssignR" +
+      "egion\022\035.hbase.pb.AssignRegionRequest\032\036.h" +
+      "base.pb.AssignRegionResponse\022S\n\016Unassign" +
+      "Region\022\037.hbase.pb.UnassignRegionRequest\032" +
+      " .hbase.pb.UnassignRegionResponse\022P\n\rOff" +
+      "lineRegion\022\036.hbase.pb.OfflineRegionReque",
+      "st\032\037.hbase.pb.OfflineRegionResponse\022J\n\013D" +
+      "eleteTable\022\034.hbase.pb.DeleteTableRequest" +
+      "\032\035.hbase.pb.DeleteTableResponse\022P\n\rtrunc" +
+      "ateTable\022\036.hbase.pb.TruncateTableRequest" +
+      "\032\037.hbase.pb.TruncateTableResponse\022J\n\013Ena" +
+      "bleTable\022\034.hbase.pb.EnableTableRequest\032\035" +
+      ".hbase.pb.EnableTableResponse\022M\n\014Disable" +
+      "Table\022\035.hbase.pb.DisableTableRequest\032\036.h" +
+      "base.pb.DisableTableResponse\022J\n\013ModifyTa" +
+      "ble\022\034.hbase.pb.ModifyTableRequest\032\035.hbas",
+      "e.pb.ModifyTableResponse\022J\n\013CreateTable\022" +
+      "\034.hbase.pb.CreateTableRequest\032\035.hbase.pb" +
+      ".CreateTableResponse\022A\n\010Shutdown\022\031.hbase" +
+      ".pb.ShutdownRequest\032\032.hbase.pb.ShutdownR" +
+      "esponse\022G\n\nStopMaster\022\033.hbase.pb.StopMas" +
+      "terRequest\032\034.hbase.pb.StopMasterResponse" +
+      "\022h\n\031IsMasterInMaintenanceMode\022$.hbase.pb" +
+      ".IsInMaintenanceModeRequest\032%.hbase.pb.I" +
+      "sInMaintenanceModeResponse\022>\n\007Balance\022\030." +
+      "hbase.pb.BalanceRequest\032\031.hbase.pb.Balan",
+      "ceResponse\022_\n\022SetBalancerRunning\022#.hbase" +
+      ".pb.SetBalancerRunningRequest\032$.hbase.pb" +
+      ".SetBalancerRunningResponse\022\\\n\021IsBalance" +
+      "rEnabled\022\".hbase.pb.IsBalancerEnabledReq" +
+      "uest\032#.hbase.pb.IsBalancerEnabledRespons" +
+      "e\022k\n\026SetSplitOrMergeEnabled\022\'.hbase.pb.S" +
+      "etSplitOrMergeEnabledRequest\032(.hbase.pb." +
+      "SetSplitOrMergeEnabledResponse\022h\n\025IsSpli" +
+      "tOrMergeEnabled\022&.hbase.pb.IsSplitOrMerg" +
+      "eEnabledRequest\032\'.hbase.pb.IsSplitOrMerg",
+      "eEnabledResponse\022D\n\tNormalize\022\032.hbase.pb" +
+      ".NormalizeRequest\032\033.hbase.pb.NormalizeRe" +
+      "sponse\022e\n\024SetNormalizerRunning\022%.hbase.p" +
+      "b.SetNormalizerRunningRequest\032&.hbase.pb" +
+      ".SetNormalizerRunningResponse\022b\n\023IsNorma" +
+      "lizerEnabled\022$.hbase.pb.IsNormalizerEnab" +
+      "ledRequest\032%.hbase.pb.IsNormalizerEnable" +
+      "dResponse\022S\n\016RunCatalogScan\022\037.hbase.pb.R" +
+      "unCatalogScanRequest\032 .hbase.pb.RunCatal" +
+      "ogScanResponse\022e\n\024EnableCatalogJanitor\022%",
+      ".hbase.pb.EnableCatalogJanitorRequest\032&." +
+      "hbase.pb.EnableCatalogJanitorResponse\022n\n" +
+      "\027IsCatalogJanitorEnabled\022(.hbase.pb.IsCa" +
+      "talogJanitorEnabledRequest\032).hbase.pb.Is" +
+      "CatalogJanitorEnabledResponse\022V\n\017RunClea" +
+      "nerChore\022 .hbase.pb.RunCleanerChoreReque" +
+      "st\032!.hbase.pb.RunCleanerChoreResponse\022k\n" +
+      "\026SetCleanerChoreRunning\022\'.hbase.pb.SetCl" +
+      "eanerChoreRunningRequest\032(.hbase.pb.SetC" +
+      "leanerChoreRunningResponse\022h\n\025IsCleanerC",
+      "horeEnabled\022&.hbase.pb.IsCleanerChoreEna" +
+      "bledRequest\032\'.hbase.pb.IsCleanerChoreEna" +
+      "bledResponse\022^\n\021ExecMasterService\022#.hbas" +
+      "e.pb.CoprocessorServiceRequest\032$.hbase.p" +
+      "b.CoprocessorServiceResponse\022A\n\010Snapshot" +
+      "\022\031.hbase.pb.SnapshotRequest\032\032.hbase.pb.S" +
+      "napshotResponse\022h\n\025GetCompletedSnapshots" +
+      "\022&.hbase.pb.GetCompletedSnapshotsRequest" +
+      "\032\'.hbase.pb.GetCompletedSnapshotsRespons" +
+      "e\022S\n\016DeleteSnapshot\022\037.hbase.pb.DeleteSna",
+      "pshotRequest\032 .hbase.pb.DeleteSnapshotRe" +
+      "sponse\022S\n\016IsSnapshotDone\022\037.hbase.pb.IsSn" +
+      "apshotDoneRequest\032 .hbase.pb.IsSnapshotD" +
+      "oneResponse\022V\n\017RestoreSnapshot\022 .hbase.p" +
+      "b.RestoreSnapshotRequest\032!.hbase.pb.Rest" +
+      "oreSnapshotResponse\022P\n\rExecProcedure\022\036.h" +
+      "base.pb.ExecProcedureRequest\032\037.hbase.pb." +
+      "ExecProcedureResponse\022W\n\024ExecProcedureWi" +
+      "thRet\022\036.hbase.pb.ExecProcedureRequest\032\037." +
+      "hbase.pb.ExecProcedureResponse\022V\n\017IsProc",
+      "edureDone\022 .hbase.pb.IsProcedureDoneRequ" +
+      "est\032!.hbase.pb.IsProcedureDoneResponse\022V" +
+      "\n\017ModifyNamespace\022 .hbase.pb.ModifyNames" +
+      "paceRequest\032!.hbase.pb.ModifyNamespaceRe" +
+      "sponse\022V\n\017CreateNamespace\022 .hbase.pb.Cre" +
+      "ateNamespaceRequest\032!.hbase.pb.CreateNam" +
+      "espaceResponse\022V\n\017DeleteNamespace\022 .hbas" +
+      "e.pb.DeleteNamespaceRequest\032!.hbase.pb.D" +
+      "eleteNamespaceResponse\022k\n\026GetNamespaceDe" +
+      "scriptor\022\'.hbase.pb.GetNamespaceDescript",
+      "orRequest\032(.hbase.pb.GetNamespaceDescrip" +
+      "torResponse\022q\n\030ListNamespaceDescriptors\022" +
+      ").hbase.pb.ListNamespaceDescriptorsReque" +
+      "st\032*.hbase.pb.ListNamespaceDescriptorsRe" +
+      "sponse\022\206\001\n\037ListTableDescriptorsByNamespa" +
+      "ce\0220.hbase.pb.ListTableDescriptorsByName" +
+      "spaceRequest\0321.hbase.pb.ListTableDescrip" +
+      "torsByNamespaceResponse\022t\n\031ListTableName" +
+      "sByNamespace\022*.hbase.pb.ListTableNamesBy" +
+      "NamespaceRequest\032+.hbase.pb.ListTableNam",
+      "esByNamespaceResponse\022P\n\rGetTableState\022\036" +
+      ".hbase.pb.GetTableStateRequest\032\037.hbase.p" +
+      "b.GetTableStateResponse\022A\n\010SetQuota\022\031.hb" +
+      "ase.pb.SetQuotaRequest\032\032.hbase.pb.SetQuo" +
+      "taResponse\022x\n\037getLastMajorCompactionTime" +
+      "stamp\022).hbase.pb.MajorCompactionTimestam" +
+      "pRequest\032*.hbase.pb.MajorCompactionTimes" +
+      "tampResponse\022\212\001\n(getLastMajorCompactionT" +
+      "imestampForRegion\0222.hbase.pb.MajorCompac" +
+      "tionTimestampForRegionRequest\032*.hbase.pb",
+      ".MajorCompactionTimestampResponse\022_\n\022get" +
+      "ProcedureResult\022#.hbase.pb.GetProcedureR" +
+      "esultRequest\032$.hbase.pb.GetProcedureResu" +
+      "ltResponse\022h\n\027getSecurityCapabilities\022%." +
+      "hbase.pb.SecurityCapabilitiesRequest\032&.h" +
+      "base.pb.SecurityCapabilitiesResponse\022S\n\016" +
+      "AbortProcedure\022\037.hbase.pb.AbortProcedure" +
+      "Request\032 .hbase.pb.AbortProcedureRespons" +
+      "e\022S\n\016ListProcedures\022\037.hbase.pb.ListProce" +
+      "duresRequest\032 .hbase.pb.ListProceduresRe",
+      "sponse\022_\n\022AddReplicationPeer\022#.hbase.pb." +
+      "AddReplicationPeerRequest\032$.hbase.pb.Add" +
+      "ReplicationPeerResponse\022h\n\025RemoveReplica" +
+      "tionPeer\022&.hbase.pb.RemoveReplicationPee" +
+      "rRequest\032\'.hbase.pb.RemoveReplicationPee" +
+      "rResponse\022h\n\025EnableReplicationPeer\022&.hba" +
+      "se.pb.EnableReplicationPeerRequest\032\'.hba" +
+      "se.pb.EnableReplicationPeerResponse\022k\n\026D" +
+      "isableReplicationPeer\022\'.hbase.pb.Disable" +
+      "ReplicationPeerRequest\032(.hbase.pb.Disabl",
+      "eReplicationPeerResponse\022q\n\030GetReplicati" +
+      "onPeerConfig\022).hbase.pb.GetReplicationPe" +
+      "erConfigRequest\032*.hbase.pb.GetReplicatio" +
+      "nPeerConfigResponse\022z\n\033UpdateReplication" +
+      "PeerConfig\022,.hbase.pb.UpdateReplicationP" +
+      "eerConfigRequest\032-.hbase.pb.UpdateReplic" +
+      "ationPeerConfigResponse\022e\n\024ListReplicati" +
+      "onPeers\022%.hbase.pb.ListReplicationPeersR" +
+      "equest\032&.hbase.pb.ListReplicationPeersRe" +
+      "sponse\022t\n\031listDrainingRegionServers\022*.hb",
+      "ase.pb.ListDrainingRegionServersRequest\032" +
+      "+.hbase.pb.ListDrainingRegionServersResp" +
+      "onse\022_\n\022drainRegionServers\022#.hbase.pb.Dr" +
+      "ainRegionServersRequest\032$.hbase.pb.Drain" +
+      "RegionServersResponse\022}\n\034removeDrainFrom" +
+      "RegionServers\022-.hbase.pb.RemoveDrainFrom" +
+      "RegionServersRequest\032..hbase.pb.RemoveDr" +
+      "ainFromRegionServersResponseBI\n1org.apac" +
+      "he.hadoop.hbase.shaded.protobuf.generate" +
+      "dB\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