You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2017/09/25 01:12:52 UTC

[2/3] hbase git commit: HBASE-18807 Remove protobuf references from CP quota API calls

HBASE-18807 Remove protobuf references from CP quota API calls


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

Branch: refs/heads/HBASE-18467
Commit: 15404831355f2217669c6d66ff46bdb3229a9643
Parents: b483046
Author: Josh Elser <el...@apache.org>
Authored: Fri Sep 22 18:07:49 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sat Sep 23 22:06:51 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hbase/quotas/QuotaSettings.java      |  83 +++++
 .../hbase/quotas/QuotaSettingsFactory.java      |  29 ++
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |   2 +-
 .../hadoop/hbase/quotas/SpaceLimitSettings.java |  51 ++-
 .../hadoop/hbase/quotas/ThrottleSettings.java   |  47 ++-
 .../quotas/TestQuotaGlobalsSettingsBypass.java  | 102 ++++++
 .../hbase/quotas/TestSpaceLimitSettings.java    |  25 ++
 .../hbase/quotas/TestThrottleSettings.java      |  98 ++++++
 .../hbase/coprocessor/MasterObserver.java       |  54 +--
 .../hbase/master/MasterCoprocessorHost.java     |  40 ++-
 .../hbase/quotas/GlobalQuotaSettings.java       | 329 +++++++++++++++++++
 .../hadoop/hbase/quotas/MasterQuotaManager.java | 273 ++++++---------
 .../hbase/security/access/AccessController.java |  14 +-
 .../hbase/coprocessor/TestMasterObserver.java   |  26 +-
 .../hbase/quotas/TestGlobalQuotaSettings.java   | 122 +++++++
 .../hadoop/hbase/quotas/TestQuotaAdmin.java     |  13 +-
 .../hadoop/hbase/quotas/TestQuotaThrottle.java  |   7 +-
 .../access/TestWithDisabledAuthorization.java   |  10 +-
 18 files changed, 1074 insertions(+), 251 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
index cf04f92..9ff2813 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
@@ -17,12 +17,17 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+import java.io.IOException;
+import java.util.Objects;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
+import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory.QuotaGlobalsSettingsBypass;
 
 @InterfaceAudience.Public
 public abstract class QuotaSettings {
@@ -52,6 +57,56 @@ public abstract class QuotaSettings {
   }
 
   /**
+   * Converts the protocol buffer request into a QuotaSetting POJO. Arbitrarily
+   * enforces that the request only contain one "limit", despite the message
+   * allowing multiple. The public API does not allow such use of the message.
+   *
+   * @param request The protocol buffer request.
+   * @return A {@link QuotaSettings} POJO.
+   */
+  @InterfaceAudience.Private
+  public static QuotaSettings buildFromProto(SetQuotaRequest request) {
+    String username = null;
+    if (request.hasUserName()) {
+      username = request.getUserName();
+    }
+    TableName tableName = null;
+    if (request.hasTableName()) {
+      tableName = ProtobufUtil.toTableName(request.getTableName());
+    }
+    String namespace = null;
+    if (request.hasNamespace()) {
+      namespace = request.getNamespace();
+    }
+    if (request.hasBypassGlobals()) {
+      // Make sure we don't have either of the two below limits also included
+      if (request.hasSpaceLimit() || request.hasThrottle()) {
+        throw new IllegalStateException(
+            "SetQuotaRequest has multiple limits: " + TextFormat.shortDebugString(request));
+      }
+      return new QuotaGlobalsSettingsBypass(
+          username, tableName, namespace, request.getBypassGlobals());
+    } else if (request.hasSpaceLimit()) {
+      // Make sure we don't have the below limit as well
+      if (request.hasThrottle()) {
+        throw new IllegalStateException(
+            "SetQuotaRequests has multiple limits: " + TextFormat.shortDebugString(request));
+      }
+      // Sanity check on the pb received.
+      if (!request.getSpaceLimit().hasQuota()) {
+        throw new IllegalArgumentException(
+            "SpaceLimitRequest is missing the expected SpaceQuota.");
+      }
+      return QuotaSettingsFactory.fromSpace(
+          tableName, namespace, request.getSpaceLimit().getQuota());
+    } else if (request.hasThrottle()) {
+      return new ThrottleSettings(username, tableName, namespace, request.getThrottle());
+    } else {
+      throw new IllegalStateException("Unhandled SetRequestRequest state");
+    }
+  }
+
+  /**
    * Convert a QuotaSettings to a protocol buffer SetQuotaRequest.
    * This is used internally by the Admin client to serialize the quota settings
    * and send them to the master.
@@ -121,4 +176,32 @@ public abstract class QuotaSettings {
     }
     throw new RuntimeException("Invalid TimeUnit " + timeUnit);
   }
+
+  /**
+   * Merges the provided settings with {@code this} and returns a new settings
+   * object to the caller if the merged settings differ from the original.
+   *
+   * @param newSettings The new settings to merge in.
+   * @return The merged {@link QuotaSettings} object or null if the quota should be deleted.
+   */
+  abstract QuotaSettings merge(QuotaSettings newSettings) throws IOException;
+
+  /**
+   * Validates that settings being merged into {@code this} is targeting the same "subject", e.g.
+   * user, table, namespace.
+   *
+   * @param mergee The quota settings to be merged into {@code this}.
+   * @throws IllegalArgumentException if the subjects are not equal.
+   */
+  void validateQuotaTarget(QuotaSettings mergee) {
+    if (!Objects.equals(getUserName(), mergee.getUserName())) {
+      throw new IllegalArgumentException("Mismatched user names on settings to merge");
+    }
+    if (!Objects.equals(getTableName(), mergee.getTableName())) {
+      throw new IllegalArgumentException("Mismatched table names on settings to merge");
+    }
+    if (!Objects.equals(getNamespace(), mergee.getNamespace())) {
+      throw new IllegalArgumentException("Mismatched namespace on settings to merge");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/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 d9c1cce..185365b 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
@@ -17,12 +17,14 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
@@ -54,6 +56,24 @@ public class QuotaSettingsFactory {
     public String toString() {
       return "GLOBAL_BYPASS => " + bypassGlobals;
     }
+
+    protected boolean getBypass() {
+      return bypassGlobals;
+    }
+
+    @Override
+    protected QuotaGlobalsSettingsBypass merge(QuotaSettings newSettings) throws IOException {
+      if (newSettings instanceof QuotaGlobalsSettingsBypass) {
+        QuotaGlobalsSettingsBypass other = (QuotaGlobalsSettingsBypass) newSettings;
+
+        validateQuotaTarget(other);
+
+        if (getBypass() != other.getBypass()) {
+          return other;
+        }
+      }
+      return this;
+    }
   }
 
   /* ==========================================================================
@@ -127,13 +147,22 @@ public class QuotaSettingsFactory {
   }
 
   static QuotaSettings fromSpace(TableName table, String namespace, SpaceQuota protoQuota) {
+    if (protoQuota == null) {
+      return null;
+    }
     if ((table == null && namespace == null) || (table != null && namespace != null)) {
       throw new IllegalArgumentException(
           "Can only construct SpaceLimitSettings for a table or namespace.");
     }
     if (table != null) {
+      if (protoQuota.getRemove()) {
+        return new SpaceLimitSettings(table);
+      }
       return SpaceLimitSettings.fromSpaceQuota(table, protoQuota);
     } else {
+      if (protoQuota.getRemove()) {
+        return new SpaceLimitSettings(namespace);
+      }
       // namespace must be non-null
       return SpaceLimitSettings.fromSpaceQuota(namespace, protoQuota);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
index 77edf3c..e5573e4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
@@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
-import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
@@ -57,6 +56,7 @@ import org.apache.hadoop.hbase.filter.RowFilter;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/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
index bb540e8..02bd6e4 100644
--- 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
@@ -68,6 +68,21 @@ class SpaceLimitSettings extends QuotaSettings {
     proto = buildProtoRemoveQuota();
   }
 
+  SpaceLimitSettings(TableName tableName, String namespace, SpaceLimitRequest req) {
+    super(null, tableName, namespace);
+    proto = req;
+  }
+
+  /**
+   * Build a {@link SpaceLimitRequest} protobuf object from the given {@link SpaceQuota}.
+   *
+   * @param protoQuota The preconstructed SpaceQuota protobuf
+   * @return A protobuf request to change a space limit quota
+   */
+  private SpaceLimitRequest buildProtoFromQuota(SpaceQuota protoQuota) {
+    return SpaceLimitRequest.newBuilder().setQuota(protoQuota).build();
+  }
+
   /**
    * Builds a {@link SpaceQuota} protobuf object given the arguments.
    *
@@ -77,12 +92,10 @@ class SpaceLimitSettings extends QuotaSettings {
    */
   private SpaceLimitRequest buildProtoAddQuota(
       long sizeLimit, SpaceViolationPolicy violationPolicy) {
-    return SpaceLimitRequest.newBuilder().setQuota(
-        SpaceQuota.newBuilder()
+    return buildProtoFromQuota(SpaceQuota.newBuilder()
             .setSoftLimit(sizeLimit)
             .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(violationPolicy))
-            .build())
-        .build();
+            .build());
   }
 
   /**
@@ -197,4 +210,34 @@ class SpaceLimitSettings extends QuotaSettings {
     }
     return sb.toString();
   }
+
+  @Override
+  protected QuotaSettings merge(QuotaSettings newSettings) {
+    if (newSettings instanceof SpaceLimitSettings) {
+      SpaceLimitSettings settingsToMerge = (SpaceLimitSettings) newSettings;
+
+      // The message contained the expect SpaceQuota object
+      if (settingsToMerge.proto.hasQuota()) {
+        SpaceQuota quotaToMerge = settingsToMerge.proto.getQuota();
+        if (quotaToMerge.getRemove()) {
+          return settingsToMerge;
+        } else {
+          // Validate that the two settings are for the same target.
+          // SpaceQuotas either apply to a table or a namespace (no user spacequota).
+          if (!Objects.equals(getTableName(), settingsToMerge.getTableName())
+              && !Objects.equals(getNamespace(), settingsToMerge.getNamespace())) {
+            throw new IllegalArgumentException("Cannot merge " + newSettings + " into " + this);
+          }
+          // Create a builder from the old settings
+          SpaceQuota.Builder mergedBuilder = this.proto.getQuota().toBuilder();
+          // Build a new SpaceQuotas object from merging in the new settings
+          return new SpaceLimitSettings(
+              getTableName(), getNamespace(),
+              buildProtoFromQuota(mergedBuilder.mergeFrom(quotaToMerge).build()));
+        }
+      }
+      // else, we don't know what to do, so return the original object
+    }
+    return this;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java
index 8a9ce28..8cee7cd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java
@@ -17,19 +17,22 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 class ThrottleSettings extends QuotaSettings {
-  private final QuotaProtos.ThrottleRequest proto;
+  final QuotaProtos.ThrottleRequest proto;
 
   ThrottleSettings(final String userName, final TableName tableName,
       final String namespace, final QuotaProtos.ThrottleRequest proto) {
@@ -99,6 +102,48 @@ class ThrottleSettings extends QuotaSettings {
     return builder.toString();
   }
 
+  @Override
+  protected ThrottleSettings merge(QuotaSettings other) throws IOException {
+    if (other instanceof ThrottleSettings) {
+      ThrottleSettings otherThrottle = (ThrottleSettings) other;
+
+      // Make sure this and the other target the same "subject"
+      validateQuotaTarget(other);
+
+      QuotaProtos.ThrottleRequest.Builder builder = proto.toBuilder();
+      if (!otherThrottle.proto.hasType()) {
+        return null;
+      }
+
+      QuotaProtos.ThrottleRequest otherProto = otherThrottle.proto;
+      if (otherProto.hasTimedQuota()) {
+        if (otherProto.hasTimedQuota()) {
+          validateTimedQuota(otherProto.getTimedQuota());
+        }
+
+        if (!proto.getType().equals(otherProto.getType())) {
+          throw new IllegalArgumentException(
+              "Cannot merge a ThrottleRequest for " + proto.getType() + " with " +
+                  otherProto.getType());
+        }
+        QuotaProtos.TimedQuota.Builder timedQuotaBuilder = proto.getTimedQuota().toBuilder();
+        timedQuotaBuilder.mergeFrom(otherProto.getTimedQuota());
+
+        QuotaProtos.ThrottleRequest mergedReq = builder.setTimedQuota(
+            timedQuotaBuilder.build()).build();
+        return new ThrottleSettings(getUserName(), getTableName(), getNamespace(), mergedReq);
+      }
+    }
+    return this;
+  }
+
+  private void validateTimedQuota(final TimedQuota timedQuota) throws IOException {
+    if (timedQuota.getSoftLimit() < 1) {
+      throw new DoNotRetryIOException(new UnsupportedOperationException(
+          "The throttle limit must be greater then 0, got " + timedQuota.getSoftLimit()));
+    }
+  }
+
   static ThrottleSettings fromTimedQuota(final String userName,
       final TableName tableName, final String namespace,
       ThrottleType type, QuotaProtos.TimedQuota timedQuota) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaGlobalsSettingsBypass.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaGlobalsSettingsBypass.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaGlobalsSettingsBypass.java
new file mode 100644
index 0000000..ae326c1
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaGlobalsSettingsBypass.java
@@ -0,0 +1,102 @@
+/*
+ * 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.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory.QuotaGlobalsSettingsBypass;
+import org.junit.Test;
+
+public class TestQuotaGlobalsSettingsBypass {
+
+  @Test
+  public void testMerge() throws IOException {
+    QuotaGlobalsSettingsBypass orig = new QuotaGlobalsSettingsBypass("joe", null, null, true);
+    assertFalse(orig.merge(new QuotaGlobalsSettingsBypass(
+        "joe", null, null, false)).getBypass());
+  }
+
+  @Test
+  public void testInvalidMerges() throws IOException {
+    QuotaGlobalsSettingsBypass userBypass = new QuotaGlobalsSettingsBypass(
+        "joe", null, null, true);
+    QuotaGlobalsSettingsBypass tableBypass = new QuotaGlobalsSettingsBypass(
+        null, TableName.valueOf("table"), null, true);
+    QuotaGlobalsSettingsBypass namespaceBypass = new QuotaGlobalsSettingsBypass(
+        null, null, "ns", true);
+    QuotaGlobalsSettingsBypass userOnTableBypass = new QuotaGlobalsSettingsBypass(
+        "joe", TableName.valueOf("table"), null, true);
+    QuotaGlobalsSettingsBypass userOnNamespaceBypass = new QuotaGlobalsSettingsBypass(
+        "joe", null, "ns", true);
+
+    assertTrue(userBypass.merge(userBypass).getBypass());
+    expectFailure(userBypass, new QuotaGlobalsSettingsBypass("frank", null, null, false));
+    expectFailure(userBypass, tableBypass);
+    expectFailure(userBypass, namespaceBypass);
+    expectFailure(userBypass, userOnTableBypass);
+    expectFailure(userBypass, userOnNamespaceBypass);
+
+    assertTrue(tableBypass.merge(tableBypass).getBypass());
+    expectFailure(tableBypass, userBypass);
+    expectFailure(tableBypass, new QuotaGlobalsSettingsBypass(
+        null, TableName.valueOf("foo"), null, false));
+    expectFailure(tableBypass, namespaceBypass);
+    expectFailure(tableBypass, userOnTableBypass);
+    expectFailure(tableBypass, userOnNamespaceBypass);
+
+    assertTrue(namespaceBypass.merge(namespaceBypass).getBypass());
+    expectFailure(namespaceBypass, userBypass);
+    expectFailure(namespaceBypass, tableBypass);
+    expectFailure(namespaceBypass, new QuotaGlobalsSettingsBypass(null, null, "sn", false));
+    expectFailure(namespaceBypass, userOnTableBypass);
+    expectFailure(namespaceBypass, userOnNamespaceBypass);
+
+    assertTrue(userOnTableBypass.merge(userOnTableBypass).getBypass());
+    expectFailure(userOnTableBypass, userBypass);
+    expectFailure(userOnTableBypass, tableBypass);
+    expectFailure(userOnTableBypass, namespaceBypass);
+    // Incorrect user
+    expectFailure(userOnTableBypass, new QuotaGlobalsSettingsBypass(
+        "frank", TableName.valueOf("foo"), null, false));
+    // Incorrect tablename
+    expectFailure(userOnTableBypass, new QuotaGlobalsSettingsBypass(
+        "joe", TableName.valueOf("bar"), null, false));
+    expectFailure(userOnTableBypass, userOnNamespaceBypass);
+
+    assertTrue(userOnNamespaceBypass.merge(userOnNamespaceBypass).getBypass());
+    expectFailure(userOnNamespaceBypass, userBypass);
+    expectFailure(userOnNamespaceBypass, tableBypass);
+    expectFailure(userOnNamespaceBypass, namespaceBypass);
+    expectFailure(userOnNamespaceBypass, userOnTableBypass);
+    expectFailure(userOnNamespaceBypass, new QuotaGlobalsSettingsBypass(
+        "frank", null, "ns", false));
+    expectFailure(userOnNamespaceBypass, new QuotaGlobalsSettingsBypass(
+        "joe", null, "sn", false));
+  }
+
+  void expectFailure(QuotaSettings one, QuotaSettings two) throws IOException {
+    try {
+      one.merge(two);
+      fail("Expected to see an Exception merging " + two + " into " + one);
+    } catch (IllegalArgumentException e) {}
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/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
index 77a00da..636b5e1 100644
--- 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
@@ -19,6 +19,9 @@ 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.fail;
+
+import java.io.IOException;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -116,4 +119,26 @@ public class TestSpaceLimitSettings {
     assertEquals(settings, copy);
     assertEquals(settings.hashCode(), copy.hashCode());
   }
+
+  @Test
+  public void testQuotaMerging() throws IOException {
+    TableName tn = TableName.valueOf("foo");
+    QuotaSettings originalSettings = QuotaSettingsFactory.limitTableSpace(
+        tn, 1024L * 1024L, SpaceViolationPolicy.DISABLE);
+    QuotaSettings largerSizeLimit = QuotaSettingsFactory.limitTableSpace(
+        tn, 5L * 1024L * 1024L, SpaceViolationPolicy.DISABLE);
+    QuotaSettings differentPolicy = QuotaSettingsFactory.limitTableSpace(
+        tn, 1024L * 1024L, SpaceViolationPolicy.NO_WRITES);
+    QuotaSettings incompatibleSettings = QuotaSettingsFactory.limitNamespaceSpace(
+        "ns1", 5L * 1024L * 1024L, SpaceViolationPolicy.NO_WRITES);
+
+    assertEquals(originalSettings.merge(largerSizeLimit), largerSizeLimit);
+    assertEquals(originalSettings.merge(differentPolicy), differentPolicy);
+    try {
+      originalSettings.merge(incompatibleSettings);
+      fail("Should not be able to merge a Table space quota with a namespace space quota.");
+    } catch (IllegalArgumentException e) {
+      //pass
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestThrottleSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestThrottleSettings.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestThrottleSettings.java
new file mode 100644
index 0000000..b6756fa
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestThrottleSettings.java
@@ -0,0 +1,98 @@
+/*
+ * 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.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+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.ThrottleRequest;
+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;
+
+@Category({SmallTests.class})
+public class TestThrottleSettings {
+
+  @Test
+  public void testMerge() throws IOException {
+    TimedQuota tq1 = TimedQuota.newBuilder().setSoftLimit(10)
+        .setScope(QuotaProtos.QuotaScope.MACHINE)
+        .setTimeUnit(HBaseProtos.TimeUnit.MINUTES).build();
+    ThrottleRequest tr1 = ThrottleRequest.newBuilder().setTimedQuota(tq1)
+        .setType(QuotaProtos.ThrottleType.REQUEST_NUMBER).build();
+    ThrottleSettings orig = new ThrottleSettings("joe", null, null, tr1);
+
+    TimedQuota tq2 = TimedQuota.newBuilder().setSoftLimit(10)
+        .setScope(QuotaProtos.QuotaScope.MACHINE)
+        .setTimeUnit(HBaseProtos.TimeUnit.SECONDS).build();
+    ThrottleRequest tr2 = ThrottleRequest.newBuilder().setTimedQuota(tq2)
+        .setType(QuotaProtos.ThrottleType.REQUEST_NUMBER).build();
+
+    ThrottleSettings merged = orig.merge(new ThrottleSettings("joe", null, null, tr2));
+
+    assertEquals(10, merged.getSoftLimit());
+    assertEquals(ThrottleType.REQUEST_NUMBER, merged.getThrottleType());
+    assertEquals(TimeUnit.SECONDS, merged.getTimeUnit());
+  }
+
+  @Test
+  public void testIncompatibleThrottleTypes() throws IOException {
+    TimedQuota requestsQuota = TimedQuota.newBuilder().setSoftLimit(10)
+        .setScope(QuotaProtos.QuotaScope.MACHINE)
+        .setTimeUnit(HBaseProtos.TimeUnit.MINUTES).build();
+    ThrottleRequest requestsQuotaReq = ThrottleRequest.newBuilder().setTimedQuota(requestsQuota)
+        .setType(QuotaProtos.ThrottleType.REQUEST_NUMBER).build();
+    ThrottleSettings orig = new ThrottleSettings("joe", null, null, requestsQuotaReq);
+
+    TimedQuota readsQuota = TimedQuota.newBuilder().setSoftLimit(10)
+        .setScope(QuotaProtos.QuotaScope.MACHINE)
+        .setTimeUnit(HBaseProtos.TimeUnit.SECONDS).build();
+    ThrottleRequest readsQuotaReq = ThrottleRequest.newBuilder().setTimedQuota(readsQuota)
+        .setType(QuotaProtos.ThrottleType.READ_NUMBER).build();
+
+    try {
+      orig.merge(new ThrottleSettings("joe", null, null, readsQuotaReq));
+      fail("A read throttle should not be capable of being merged with a request quota");
+    } catch (IllegalArgumentException e) {
+      // Pass
+    }
+  }
+
+  @Test
+  public void testNoThrottleReturnsOriginal() throws IOException {
+    TimedQuota tq1 = TimedQuota.newBuilder().setSoftLimit(10)
+        .setScope(QuotaProtos.QuotaScope.MACHINE)
+        .setTimeUnit(HBaseProtos.TimeUnit.MINUTES).build();
+    ThrottleRequest tr1 = ThrottleRequest.newBuilder().setTimedQuota(tq1)
+        .setType(QuotaProtos.ThrottleType.REQUEST_NUMBER).build();
+    ThrottleSettings orig = new ThrottleSettings("joe", null, null, tr1);
+
+    ThrottleRequest tr2 = ThrottleRequest.newBuilder()
+        .setType(QuotaProtos.ThrottleType.REQUEST_NUMBER).build();
+
+    assertTrue(
+        "The same object should be returned by merge, but it wasn't",
+        orig == orig.merge(new ThrottleSettings("joe", null, null, tr2)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index 64b54fc..16ea635 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -42,8 +42,8 @@ import org.apache.hadoop.hbase.procedure2.LockType;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
@@ -1075,95 +1075,99 @@ public interface MasterObserver extends Coprocessor {
    * Called before the quota for the user is stored.
    * @param ctx the environment to interact with the framework and master
    * @param userName the name of user
-   * @param quotas the quota settings
+   * @param quotas the current quota for the user
    */
   default void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String userName, final Quotas quotas) throws IOException {}
+      final String userName, final GlobalQuotaSettings quotas) throws IOException {}
 
   /**
    * Called after the quota for the user is stored.
    * @param ctx the environment to interact with the framework and master
    * @param userName the name of user
-   * @param quotas the quota settings
+   * @param quotas the resulting quota for the user
    */
   default void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String userName, final Quotas quotas) throws IOException {}
+      final String userName, final GlobalQuotaSettings quotas) throws IOException {}
 
   /**
    * Called before the quota for the user on the specified table is stored.
    * @param ctx the environment to interact with the framework and master
    * @param userName the name of user
    * @param tableName the name of the table
-   * @param quotas the quota settings
+   * @param quotas the current quota for the user on the table
    */
-  default void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String userName, final TableName tableName, final Quotas quotas) throws IOException {}
+  default void preSetUserQuota(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, final String userName,
+      final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {}
 
   /**
    * Called after the quota for the user on the specified table is stored.
    * @param ctx the environment to interact with the framework and master
    * @param userName the name of user
    * @param tableName the name of the table
-   * @param quotas the quota settings
+   * @param quotas the resulting quota for the user on the table
    */
-  default void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String userName, final TableName tableName, final Quotas quotas) throws IOException {}
+  default void postSetUserQuota(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, final String userName,
+      final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {}
 
   /**
    * Called before the quota for the user on the specified namespace is stored.
    * @param ctx the environment to interact with the framework and master
    * @param userName the name of user
    * @param namespace the name of the namespace
-   * @param quotas the quota settings
+   * @param quotas the current quota for the user on the namespace
    */
-  default void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String userName, final String namespace, final Quotas quotas) throws IOException {}
+  default void preSetUserQuota(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, final String userName,
+      final String namespace, final GlobalQuotaSettings quotas) throws IOException {}
 
   /**
    * Called after the quota for the user on the specified namespace is stored.
    * @param ctx the environment to interact with the framework and master
    * @param userName the name of user
    * @param namespace the name of the namespace
-   * @param quotas the quota settings
+   * @param quotas the resulting quota for the user on the namespace
    */
-  default void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String userName, final String namespace, final Quotas quotas) throws IOException {}
+  default void postSetUserQuota(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, final String userName,
+      final String namespace, final GlobalQuotaSettings quotas) throws IOException {}
 
   /**
    * Called before the quota for the table is stored.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param quotas the quota settings
+   * @param quotas the current quota for the table
    */
   default void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName, final Quotas quotas) throws IOException {}
+      final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {}
 
   /**
    * Called after the quota for the table is stored.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param quotas the quota settings
+   * @param quotas the resulting quota for the table
    */
   default void postSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName, final Quotas quotas) throws IOException {}
+      final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {}
 
   /**
    * Called before the quota for the namespace is stored.
    * @param ctx the environment to interact with the framework and master
    * @param namespace the name of the namespace
-   * @param quotas the quota settings
+   * @param quotas the current quota for the namespace
    */
   default void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String namespace, final Quotas quotas) throws IOException {}
+      final String namespace, final GlobalQuotaSettings quotas) throws IOException {}
 
   /**
    * Called after the quota for the namespace is stored.
    * @param ctx the environment to interact with the framework and master
    * @param namespace the name of the namespace
-   * @param quotas the quota settings
+   * @param quotas the resulting quota for the namespace
    */
   default void postSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String namespace, final Quotas quotas) throws IOException {}
+      final String namespace, final GlobalQuotaSettings quotas) throws IOException {}
 
   /**
    * Called before merge regions request.

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index badcfed..8a65f0c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -52,9 +52,9 @@ import org.apache.hadoop.hbase.procedure2.LockType;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -1299,7 +1299,8 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preSetUserQuota(final String user, final Quotas quotas) throws IOException {
+  public void preSetUserQuota(
+      final String user, final GlobalQuotaSettings quotas) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1309,7 +1310,8 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postSetUserQuota(final String user, final Quotas quotas) throws IOException {
+  public void postSetUserQuota(
+      final String user, final GlobalQuotaSettings quotas) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1319,8 +1321,9 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preSetUserQuota(final String user, final TableName table, final Quotas quotas)
-      throws IOException {
+  public void preSetUserQuota(
+      final String user, final TableName table, final GlobalQuotaSettings quotas)
+          throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1330,8 +1333,9 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postSetUserQuota(final String user, final TableName table, final Quotas quotas)
-      throws IOException {
+  public void postSetUserQuota(
+      final String user, final TableName table, final GlobalQuotaSettings quotas)
+          throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1341,8 +1345,9 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preSetUserQuota(final String user, final String namespace, final Quotas quotas)
-      throws IOException {
+  public void preSetUserQuota(
+      final String user, final String namespace, final GlobalQuotaSettings quotas)
+          throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1352,8 +1357,9 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postSetUserQuota(final String user, final String namespace, final Quotas quotas)
-      throws IOException {
+  public void postSetUserQuota(
+      final String user, final String namespace, final GlobalQuotaSettings quotas)
+          throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1363,7 +1369,8 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preSetTableQuota(final TableName table, final Quotas quotas) throws IOException {
+  public void preSetTableQuota(
+      final TableName table, final GlobalQuotaSettings quotas) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1373,7 +1380,8 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postSetTableQuota(final TableName table, final Quotas quotas) throws IOException {
+  public void postSetTableQuota(
+      final TableName table, final GlobalQuotaSettings quotas) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1383,7 +1391,8 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preSetNamespaceQuota(final String namespace, final Quotas quotas) throws IOException {
+  public void preSetNamespaceQuota(
+      final String namespace, final GlobalQuotaSettings quotas) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1393,7 +1402,8 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postSetNamespaceQuota(final String namespace, final Quotas quotas) throws IOException{
+  public void postSetNamespaceQuota(
+      final String namespace, final GlobalQuotaSettings quotas) throws IOException{
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java
new file mode 100644
index 0000000..079edf0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java
@@ -0,0 +1,329 @@
+/*
+ * 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.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory.QuotaGlobalsSettingsBypass;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest.Builder;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An object which captures all quotas types (throttle or space) for a subject (user, table, or
+ * namespace). This is used inside of the HBase RegionServer to act as an analogy to the
+ * ProtocolBuffer class {@link Quotas}.
+ */
+@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC})
+@InterfaceStability.Evolving
+public class GlobalQuotaSettings extends QuotaSettings {
+  private final QuotaProtos.Throttle throttleProto;
+  private final Boolean bypassGlobals;
+  private final QuotaProtos.SpaceQuota spaceProto;
+
+  protected GlobalQuotaSettings(
+      String username, TableName tableName, String namespace, QuotaProtos.Quotas quotas) {
+    this(username, tableName, namespace,
+        (quotas != null && quotas.hasThrottle() ? quotas.getThrottle() : null),
+        (quotas != null && quotas.hasBypassGlobals() ? quotas.getBypassGlobals() : null),
+        (quotas != null && quotas.hasSpace() ? quotas.getSpace() : null));
+  }
+
+  protected GlobalQuotaSettings(
+      String userName, TableName tableName, String namespace, QuotaProtos.Throttle throttleProto,
+      Boolean bypassGlobals, QuotaProtos.SpaceQuota spaceProto) {
+    super(userName, tableName, namespace);
+    this.throttleProto = throttleProto;
+    this.bypassGlobals = bypassGlobals;
+    this.spaceProto = spaceProto;
+  }
+
+  @Override
+  public QuotaType getQuotaType() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected void setupSetQuotaRequest(Builder builder) {
+    // ThrottleSettings should be used instead for setting a throttle quota.
+    throw new UnsupportedOperationException(
+        "This class should not be used to generate a SetQuotaRequest.");
+  }
+
+  protected QuotaProtos.Throttle getThrottleProto() {
+    return this.throttleProto;
+  }
+
+  protected Boolean getGlobalBypass() {
+    return this.bypassGlobals;
+  }
+
+  protected QuotaProtos.SpaceQuota getSpaceProto() {
+    return this.spaceProto;
+  }
+
+  /**
+   * Constructs a new {@link Quotas} message from {@code this}.
+   */
+  protected Quotas toQuotas() {
+    QuotaProtos.Quotas.Builder builder = QuotaProtos.Quotas.newBuilder();
+    if (getThrottleProto() != null) {
+      builder.setThrottle(getThrottleProto());
+    }
+    if (getGlobalBypass() != null) {
+      builder.setBypassGlobals(getGlobalBypass());
+    }
+    if (getSpaceProto() != null) {
+      builder.setSpace(getSpaceProto());
+    }
+    return builder.build();
+  }
+
+  @Override
+  protected GlobalQuotaSettings merge(QuotaSettings other) throws IOException {
+    // Validate the quota subject
+    validateQuotaTarget(other);
+
+    // Propagate the Throttle
+    QuotaProtos.Throttle.Builder throttleBuilder = (throttleProto == null
+        ? null : throttleProto.toBuilder());
+    if (other instanceof ThrottleSettings) {
+      if (throttleBuilder == null) {
+        throttleBuilder = QuotaProtos.Throttle.newBuilder();
+      }
+      ThrottleSettings otherThrottle = (ThrottleSettings) other;
+
+      if (otherThrottle.proto.hasType()) {
+        QuotaProtos.ThrottleRequest otherProto = otherThrottle.proto;
+        if (otherProto.hasTimedQuota()) {
+          if (otherProto.hasTimedQuota()) {
+            validateTimedQuota(otherProto.getTimedQuota());
+          }
+
+          switch (otherProto.getType()) {
+            case REQUEST_NUMBER:
+              if (otherProto.hasTimedQuota()) {
+                throttleBuilder.setReqNum(otherProto.getTimedQuota());
+              } else {
+                throttleBuilder.clearReqNum();
+              }
+              break;
+            case REQUEST_SIZE:
+              if (otherProto.hasTimedQuota()) {
+                throttleBuilder.setReqSize(otherProto.getTimedQuota());
+              } else {
+                throttleBuilder.clearReqSize();
+              }
+              break;
+            case WRITE_NUMBER:
+              if (otherProto.hasTimedQuota()) {
+                throttleBuilder.setWriteNum(otherProto.getTimedQuota());
+              } else {
+                throttleBuilder.clearWriteNum();
+              }
+              break;
+            case WRITE_SIZE:
+              if (otherProto.hasTimedQuota()) {
+                throttleBuilder.setWriteSize(otherProto.getTimedQuota());
+              } else {
+                throttleBuilder.clearWriteSize();
+              }
+              break;
+            case READ_NUMBER:
+              if (otherProto.hasTimedQuota()) {
+                throttleBuilder.setReadNum(otherProto.getTimedQuota());
+              } else {
+                throttleBuilder.clearReqNum();
+              }
+              break;
+            case READ_SIZE:
+              if (otherProto.hasTimedQuota()) {
+                throttleBuilder.setReadSize(otherProto.getTimedQuota());
+              } else {
+                throttleBuilder.clearReadSize();
+              }
+              break;
+          }
+        } else {
+          clearThrottleBuilder(throttleBuilder);
+        }
+      } else {
+        clearThrottleBuilder(throttleBuilder);
+      }
+    }
+
+    // Propagate the space quota portion
+    QuotaProtos.SpaceQuota.Builder spaceBuilder = (spaceProto == null
+        ? null : spaceProto.toBuilder());
+    if (other instanceof SpaceLimitSettings) {
+      if (spaceBuilder == null) {
+        spaceBuilder = QuotaProtos.SpaceQuota.newBuilder();
+      }
+      SpaceLimitSettings settingsToMerge = (SpaceLimitSettings) other;
+
+      QuotaProtos.SpaceLimitRequest spaceRequest = settingsToMerge.getProto();
+
+      // The message contained the expect SpaceQuota object
+      if (spaceRequest.hasQuota()) {
+        SpaceQuota quotaToMerge = spaceRequest.getQuota();
+        // Validate that the two settings are for the same target.
+        // SpaceQuotas either apply to a table or a namespace (no user spacequota).
+        if (!Objects.equals(getTableName(), settingsToMerge.getTableName())
+            && !Objects.equals(getNamespace(), settingsToMerge.getNamespace())) {
+          throw new IllegalArgumentException(
+              "Cannot merge " + settingsToMerge + " into " + this);
+        }
+
+        if (quotaToMerge.getRemove()) {
+          // Update the builder to propagate the removal
+          spaceBuilder.setRemove(true).clearSoftLimit().clearViolationPolicy();
+        } else {
+          // Add the new settings to the existing settings
+          spaceBuilder.mergeFrom(quotaToMerge);
+        }
+      }
+    }
+
+    Boolean bypassGlobals = this.bypassGlobals;
+    if (other instanceof QuotaGlobalsSettingsBypass) {
+      bypassGlobals = ((QuotaGlobalsSettingsBypass) other).getBypass();
+    }
+
+    if (throttleBuilder == null &&
+        (spaceBuilder == null || (spaceBuilder.hasRemove() && spaceBuilder.getRemove()))
+        && bypassGlobals == null) {
+      return null;
+    }
+
+    return new GlobalQuotaSettings(
+        getUserName(), getTableName(), getNamespace(),
+        (throttleBuilder == null ? null : throttleBuilder.build()), bypassGlobals,
+        (spaceBuilder == null ? null : spaceBuilder.build()));
+  }
+
+  private void validateTimedQuota(final TimedQuota timedQuota) throws IOException {
+    if (timedQuota.getSoftLimit() < 1) {
+      throw new DoNotRetryIOException(new UnsupportedOperationException(
+          "The throttle limit must be greater then 0, got " + timedQuota.getSoftLimit()));
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("GlobalQuota: ");
+    if (throttleProto != null) {
+      Map<ThrottleType,TimedQuota> throttleQuotas = buildThrottleQuotas(throttleProto);
+      builder.append(" { TYPE => THROTTLE ");
+      for (Entry<ThrottleType,TimedQuota> entry : throttleQuotas.entrySet()) {
+        final ThrottleType type = entry.getKey();
+        final TimedQuota timedQuota = entry.getValue();
+        builder.append("{THROTTLE_TYPE => ").append(type.name()).append(", LIMIT => ");
+        if (timedQuota.hasSoftLimit()) {
+          switch (type) {
+            case REQUEST_NUMBER:
+            case WRITE_NUMBER:
+            case READ_NUMBER:
+              builder.append(String.format("%dreq", timedQuota.getSoftLimit()));
+              break;
+            case REQUEST_SIZE:
+            case WRITE_SIZE:
+            case READ_SIZE:
+              builder.append(sizeToString(timedQuota.getSoftLimit()));
+              break;
+          }
+        } else if (timedQuota.hasShare()) {
+          builder.append(String.format("%.2f%%", timedQuota.getShare()));
+        }
+        builder.append('/');
+        builder.append(timeToString(ProtobufUtil.toTimeUnit(timedQuota.getTimeUnit())));
+        if (timedQuota.hasScope()) {
+          builder.append(", SCOPE => ");
+          builder.append(timedQuota.getScope().toString());
+        }
+      }
+      builder.append( "} } ");
+    } else {
+      builder.append(" {} ");
+    }
+    if (bypassGlobals != null) {
+      builder.append(" { GLOBAL_BYPASS => " + bypassGlobals + " } ");
+    }
+    if (spaceProto != null) {
+      builder.append(" { TYPE => SPACE");
+      if (getTableName() != null) {
+        builder.append(", TABLE => ").append(getTableName());
+      }
+      if (getNamespace() != null) {
+        builder.append(", NAMESPACE => ").append(getNamespace());
+      }
+      if (spaceProto.getRemove()) {
+        builder.append(", REMOVE => ").append(spaceProto.getRemove());
+      } else {
+        builder.append(", LIMIT => ").append(spaceProto.getSoftLimit());
+        builder.append(", VIOLATION_POLICY => ").append(spaceProto.getViolationPolicy());
+      }
+      builder.append(" } ");
+    }
+    return builder.toString();
+  }
+
+  private Map<ThrottleType,TimedQuota> buildThrottleQuotas(Throttle proto) {
+    HashMap<ThrottleType,TimedQuota> quotas = new HashMap<>();
+    if (proto.hasReadNum()) {
+      quotas.put(ThrottleType.READ_NUMBER, proto.getReadNum());
+    }
+    if (proto.hasReadSize()) {
+      quotas.put(ThrottleType.READ_SIZE, proto.getReadSize());
+    }
+    if (proto.hasReqNum()) {
+      quotas.put(ThrottleType.REQUEST_NUMBER, proto.getReqNum());
+    }
+    if (proto.hasReqSize()) {
+      quotas.put(ThrottleType.REQUEST_SIZE, proto.getReqSize());
+    }
+    if (proto.hasWriteNum()) {
+      quotas.put(ThrottleType.WRITE_NUMBER, proto.getWriteNum());
+    }
+    if (proto.hasWriteSize()) {
+      quotas.put(ThrottleType.WRITE_SIZE, proto.getWriteSize());
+    }
+    return quotas;
+  }
+
+  private void clearThrottleBuilder(QuotaProtos.Throttle.Builder builder) {
+    builder.clearReadNum();
+    builder.clearReadSize();
+    builder.clearReqNum();
+    builder.clearReqSize();
+    builder.clearWriteNum();
+    builder.clearWriteSize();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index dd445e4..4f2b51d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -43,15 +43,10 @@ import org.apache.hadoop.hbase.namespace.NamespaceAuditor;
 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.MasterProtos.SetQuotaResponse;
-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.ThrottleRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
 
 /**
  * Master Quota Manager.
@@ -158,24 +153,25 @@ public class MasterQuotaManager implements RegionStateListener {
       throws IOException, InterruptedException {
     setQuota(req, new SetQuotaOperations() {
       @Override
-      public Quotas fetch() throws IOException {
-        return QuotaUtil.getUserQuota(masterServices.getConnection(), userName);
+      public GlobalQuotaSettings fetch() throws IOException {
+        return new GlobalQuotaSettings(req.getUserName(), null, null, QuotaUtil.getUserQuota(
+            masterServices.getConnection(), userName));
       }
       @Override
-      public void update(final Quotas quotas) throws IOException {
-        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, quotas);
+      public void update(GlobalQuotaSettings quotaPojo) throws IOException {
+        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, quotaPojo.toQuotas());
       }
       @Override
       public void delete() throws IOException {
         QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName);
       }
       @Override
-      public void preApply(final Quotas quotas) throws IOException {
-        masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, quotas);
+      public void preApply(GlobalQuotaSettings quotaPojo) throws IOException {
+        masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, quotaPojo);
       }
       @Override
-      public void postApply(final Quotas quotas) throws IOException {
-        masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, quotas);
+      public void postApply(GlobalQuotaSettings quotaPojo) throws IOException {
+        masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, quotaPojo);
       }
     });
   }
@@ -184,24 +180,26 @@ public class MasterQuotaManager implements RegionStateListener {
       final SetQuotaRequest req) throws IOException, InterruptedException {
     setQuota(req, new SetQuotaOperations() {
       @Override
-      public Quotas fetch() throws IOException {
-        return QuotaUtil.getUserQuota(masterServices.getConnection(), userName, table);
+      public GlobalQuotaSettings fetch() throws IOException {
+        return new GlobalQuotaSettings(userName, table, null, QuotaUtil.getUserQuota(
+            masterServices.getConnection(), userName, table));
       }
       @Override
-      public void update(final Quotas quotas) throws IOException {
-        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, table, quotas);
+      public void update(GlobalQuotaSettings quotaPojo) throws IOException {
+        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, table,
+            quotaPojo.toQuotas());
       }
       @Override
       public void delete() throws IOException {
         QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName, table);
       }
       @Override
-      public void preApply(final Quotas quotas) throws IOException {
-        masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, table, quotas);
+      public void preApply(GlobalQuotaSettings quotaPojo) throws IOException {
+        masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, table, quotaPojo);
       }
       @Override
-      public void postApply(final Quotas quotas) throws IOException {
-        masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, table, quotas);
+      public void postApply(GlobalQuotaSettings quotaPojo) throws IOException {
+        masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, table, quotaPojo);
       }
     });
   }
@@ -210,24 +208,28 @@ public class MasterQuotaManager implements RegionStateListener {
       final SetQuotaRequest req) throws IOException, InterruptedException {
     setQuota(req, new SetQuotaOperations() {
       @Override
-      public Quotas fetch() throws IOException {
-        return QuotaUtil.getUserQuota(masterServices.getConnection(), userName, namespace);
+      public GlobalQuotaSettings fetch() throws IOException {
+        return new GlobalQuotaSettings(userName, null, namespace, QuotaUtil.getUserQuota(
+            masterServices.getConnection(), userName, namespace));
       }
       @Override
-      public void update(final Quotas quotas) throws IOException {
-        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, namespace, quotas);
+      public void update(GlobalQuotaSettings quotaPojo) throws IOException {
+        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, namespace,
+            quotaPojo.toQuotas());
       }
       @Override
       public void delete() throws IOException {
         QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName, namespace);
       }
       @Override
-      public void preApply(final Quotas quotas) throws IOException {
-        masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, namespace, quotas);
+      public void preApply(GlobalQuotaSettings quotaPojo) throws IOException {
+        masterServices.getMasterCoprocessorHost().preSetUserQuota(
+            userName, namespace, quotaPojo);
       }
       @Override
-      public void postApply(final Quotas quotas) throws IOException {
-        masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, namespace, quotas);
+      public void postApply(GlobalQuotaSettings quotaPojo) throws IOException {
+        masterServices.getMasterCoprocessorHost().postSetUserQuota(
+            userName, namespace, quotaPojo);
       }
     });
   }
@@ -236,24 +238,25 @@ public class MasterQuotaManager implements RegionStateListener {
       throws IOException, InterruptedException {
     setQuota(req, new SetQuotaOperations() {
       @Override
-      public Quotas fetch() throws IOException {
-        return QuotaUtil.getTableQuota(masterServices.getConnection(), table);
+      public GlobalQuotaSettings fetch() throws IOException {
+        return new GlobalQuotaSettings(null, table, null, QuotaUtil.getTableQuota(
+            masterServices.getConnection(), table));
       }
       @Override
-      public void update(final Quotas quotas) throws IOException {
-        QuotaUtil.addTableQuota(masterServices.getConnection(), table, quotas);
+      public void update(GlobalQuotaSettings quotaPojo) throws IOException {
+        QuotaUtil.addTableQuota(masterServices.getConnection(), table, quotaPojo.toQuotas());
       }
       @Override
       public void delete() throws IOException {
         QuotaUtil.deleteTableQuota(masterServices.getConnection(), table);
       }
       @Override
-      public void preApply(final Quotas quotas) throws IOException {
-        masterServices.getMasterCoprocessorHost().preSetTableQuota(table, quotas);
+      public void preApply(GlobalQuotaSettings quotaPojo) throws IOException {
+        masterServices.getMasterCoprocessorHost().preSetTableQuota(table, quotaPojo);
       }
       @Override
-      public void postApply(final Quotas quotas) throws IOException {
-        masterServices.getMasterCoprocessorHost().postSetTableQuota(table, quotas);
+      public void postApply(GlobalQuotaSettings quotaPojo) throws IOException {
+        masterServices.getMasterCoprocessorHost().postSetTableQuota(table, quotaPojo);
       }
     });
   }
@@ -262,24 +265,26 @@ public class MasterQuotaManager implements RegionStateListener {
       throws IOException, InterruptedException {
     setQuota(req, new SetQuotaOperations() {
       @Override
-      public Quotas fetch() throws IOException {
-        return QuotaUtil.getNamespaceQuota(masterServices.getConnection(), namespace);
+      public GlobalQuotaSettings fetch() throws IOException {
+        return new GlobalQuotaSettings(null, null, namespace, QuotaUtil.getNamespaceQuota(
+                masterServices.getConnection(), namespace));
       }
       @Override
-      public void update(final Quotas quotas) throws IOException {
-        QuotaUtil.addNamespaceQuota(masterServices.getConnection(), namespace, quotas);
+      public void update(GlobalQuotaSettings quotaPojo) throws IOException {
+        QuotaUtil.addNamespaceQuota(masterServices.getConnection(), namespace,
+            ((GlobalQuotaSettings) quotaPojo).toQuotas());
       }
       @Override
       public void delete() throws IOException {
         QuotaUtil.deleteNamespaceQuota(masterServices.getConnection(), namespace);
       }
       @Override
-      public void preApply(final Quotas quotas) throws IOException {
-        masterServices.getMasterCoprocessorHost().preSetNamespaceQuota(namespace, quotas);
+      public void preApply(GlobalQuotaSettings quotaPojo) throws IOException {
+        masterServices.getMasterCoprocessorHost().preSetNamespaceQuota(namespace, quotaPojo);
       }
       @Override
-      public void postApply(final Quotas quotas) throws IOException {
-        masterServices.getMasterCoprocessorHost().postSetNamespaceQuota(namespace, quotas);
+      public void postApply(GlobalQuotaSettings quotaPojo) throws IOException {
+        masterServices.getMasterCoprocessorHost().postSetNamespaceQuota(namespace, quotaPojo);
       }
     });
   }
@@ -306,23 +311,37 @@ public class MasterQuotaManager implements RegionStateListener {
     }
 
     // Apply quota changes
-    Quotas quotas = quotaOps.fetch();
-    quotaOps.preApply(quotas);
+    GlobalQuotaSettings currentQuota = quotaOps.fetch();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(
+          "Current quota for request(" + TextFormat.shortDebugString(req)
+              + "): " + currentQuota);
+    }
+    // Call the appropriate "pre" CP hook with the current quota value (may be null)
+    quotaOps.preApply(currentQuota);
+    // Translate the protobuf request back into a POJO
+    QuotaSettings newQuota = QuotaSettings.buildFromProto(req);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Deserialized quota from request: " + newQuota);
+    }
 
-    // Copy the user request into the Quotas object
-    Quotas.Builder builder = (quotas != null) ? quotas.toBuilder() : Quotas.newBuilder();
-    if (req.hasThrottle()) applyThrottle(builder, req.getThrottle());
-    if (req.hasBypassGlobals()) applyBypassGlobals(builder, req.getBypassGlobals());
-    if (req.hasSpaceLimit()) applySpaceLimit(builder, req.getSpaceLimit());
+    // Merge the current quota settings with the new quota settings the user provided.
+    //
+    // NB: while SetQuotaRequest technically allows for multi types of quotas to be set in one
+    // message, the Java API (in Admin/AsyncAdmin) does not. Assume there is only one type.
+    GlobalQuotaSettings mergedQuota = currentQuota.merge(newQuota);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Computed merged quota from current quota and user request: " + mergedQuota);
+    }
 
     // Submit new changes
-    quotas = builder.build();
-    if (QuotaUtil.isEmptyQuota(quotas)) {
+    if (mergedQuota == null) {
       quotaOps.delete();
     } else {
-      quotaOps.update(quotas);
+      quotaOps.update(mergedQuota);
     }
-    quotaOps.postApply(quotas);
+    // Advertise the final result via the "post" CP hook
+    quotaOps.postApply(mergedQuota);
   }
 
   public void checkNamespaceTableAndRegionQuota(TableName tName, int regions) throws IOException {
@@ -377,124 +396,32 @@ public class MasterQuotaManager implements RegionStateListener {
     return this.namespaceQuotaManager;
   }
 
-  private static interface SetQuotaOperations {
-    Quotas fetch() throws IOException;
-    void delete() throws IOException;
-    void update(final Quotas quotas) throws IOException;
-    void preApply(final Quotas quotas) throws IOException;
-    void postApply(final Quotas quotas) throws IOException;
-  }
-
-  /* ==========================================================================
-   *  Helpers to apply changes to the quotas
-   */
-  private void applyThrottle(final Quotas.Builder quotas, final ThrottleRequest req)
-      throws IOException {
-    Throttle.Builder throttle;
-
-    if (req.hasType() && (req.hasTimedQuota() || quotas.hasThrottle())) {
-      // Validate timed quota if present
-      if (req.hasTimedQuota()) validateTimedQuota(req.getTimedQuota());
-
-      // apply the new settings
-      throttle = quotas.hasThrottle() ? quotas.getThrottle().toBuilder() : Throttle.newBuilder();
-
-      switch (req.getType()) {
-        case REQUEST_NUMBER:
-          if (req.hasTimedQuota()) {
-            throttle.setReqNum(req.getTimedQuota());
-          } else {
-            throttle.clearReqNum();
-          }
-          break;
-        case REQUEST_SIZE:
-          if (req.hasTimedQuota()) {
-            throttle.setReqSize(req.getTimedQuota());
-          } else {
-            throttle.clearReqSize();
-          }
-          break;
-        case WRITE_NUMBER:
-          if (req.hasTimedQuota()) {
-            throttle.setWriteNum(req.getTimedQuota());
-          } else {
-            throttle.clearWriteNum();
-          }
-          break;
-        case WRITE_SIZE:
-          if (req.hasTimedQuota()) {
-            throttle.setWriteSize(req.getTimedQuota());
-          } else {
-            throttle.clearWriteSize();
-          }
-          break;
-        case READ_NUMBER:
-          if (req.hasTimedQuota()) {
-            throttle.setReadNum(req.getTimedQuota());
-          } else {
-            throttle.clearReqNum();
-          }
-          break;
-        case READ_SIZE:
-          if (req.hasTimedQuota()) {
-            throttle.setReadSize(req.getTimedQuota());
-          } else {
-            throttle.clearReadSize();
-          }
-          break;
-      }
-      quotas.setThrottle(throttle.build());
-    } else {
-      quotas.clearThrottle();
-    }
-  }
-
-  private void applyBypassGlobals(final Quotas.Builder quotas, boolean bypassGlobals) {
-    if (bypassGlobals) {
-      quotas.setBypassGlobals(bypassGlobals);
-    } else {
-      quotas.clearBypassGlobals();
-    }
-  }
-
-  /**
-   * Adds the information from the provided {@link SpaceLimitRequest} to the {@link Quotas} builder.
-   *
-   * @param quotas The builder to update.
-   * @param req The request to extract space quota information from.
-   */
-  void applySpaceLimit(final Quotas.Builder quotas, final SpaceLimitRequest req) {
-    if (req.hasQuota()) {
-      SpaceQuota spaceQuota = req.getQuota();
-      // If we have the remove flag, unset the space quota.
-      if (spaceQuota.getRemove()) {
-        quotas.setSpace(SpaceQuota.getDefaultInstance());
-      } else {
-        // Otherwise, update the new quota
-        applySpaceQuota(quotas, req.getQuota());
-      }
-    }
-  }
-
   /**
-   * Merges the provided {@link SpaceQuota} into the given {@link Quotas} builder.
-   *
-   * @param quotas The Quotas builder instance to update
-   * @param quota The SpaceQuota instance to update from
+   * Encapsulates CRUD quota operations for some subject.
    */
-  void applySpaceQuota(final Quotas.Builder quotas, final SpaceQuota quota) {
-    // Create a builder for Quotas
-    SpaceQuota.Builder builder = quotas.hasSpace() ? quotas.getSpace().toBuilder() :
-        SpaceQuota.newBuilder();
-    // Update the values from the provided quota into the new one and set it on Quotas.
-    quotas.setSpace(builder.mergeFrom(quota).build());
-  }
-
-  private void validateTimedQuota(final TimedQuota timedQuota) throws IOException {
-    if (timedQuota.getSoftLimit() < 1) {
-      throw new DoNotRetryIOException(new UnsupportedOperationException(
-          "The throttle limit must be greater then 0, got " + timedQuota.getSoftLimit()));
-    }
+  private static interface SetQuotaOperations {
+    /**
+     * Fetches the current quota settings for the subject.
+     */
+    GlobalQuotaSettings fetch() throws IOException;
+    /**
+     * Deletes the quota for the subject.
+     */
+    void delete() throws IOException;
+    /**
+     * Persist the given quota for the subject.
+     */
+    void update(GlobalQuotaSettings quotaPojo) throws IOException;
+    /**
+     * Performs some action before {@link #update(GlobalQuotaSettings)} with the current quota
+     * for the subject.
+     */
+    void preApply(GlobalQuotaSettings quotaPojo) throws IOException;
+    /**
+     * Performs some action after {@link #update(GlobalQuotaSettings)} with the resulting quota
+     * from the request action for the subject.
+     */
+    void postApply(GlobalQuotaSettings quotaPojo) throws IOException;
   }
 
   /* ==========================================================================

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index e9bed7e..ea5eb1c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -98,6 +98,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
+import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -123,7 +124,6 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.util.ByteRange;
@@ -2585,31 +2585,33 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String userName, final Quotas quotas) throws IOException {
+      final String userName, final GlobalQuotaSettings quotas) throws IOException {
     requirePermission(getActiveUser(ctx), "setUserQuota", Action.ADMIN);
   }
 
   @Override
   public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String userName, final TableName tableName, final Quotas quotas) throws IOException {
+      final String userName, final TableName tableName, final GlobalQuotaSettings quotas)
+          throws IOException {
     requirePermission(getActiveUser(ctx), "setUserTableQuota", tableName, null, null, Action.ADMIN);
   }
 
   @Override
   public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String userName, final String namespace, final Quotas quotas) throws IOException {
+      final String userName, final String namespace, final GlobalQuotaSettings quotas)
+          throws IOException {
     requirePermission(getActiveUser(ctx), "setUserNamespaceQuota", Action.ADMIN);
   }
 
   @Override
   public void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName, final Quotas quotas) throws IOException {
+      final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {
     requirePermission(getActiveUser(ctx), "setTableQuota", tableName, null, null, Action.ADMIN);
   }
 
   @Override
   public void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final String namespace, final Quotas quotas) throws IOException {
+      final String namespace, final GlobalQuotaSettings quotas) throws IOException {
     requirePermission(getActiveUser(ctx), "setNamespaceQuota", Action.ADMIN);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/15404831/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 3becc61..b4c85f1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -60,12 +60,12 @@ import org.apache.hadoop.hbase.procedure2.LockedResource;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -1259,52 +1259,56 @@ public class TestMasterObserver {
 
     @Override
     public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final String userName, final Quotas quotas) throws IOException {
+        final String userName, final GlobalQuotaSettings quotas) throws IOException {
     }
 
     @Override
     public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final String userName, final Quotas quotas) throws IOException {
+        final String userName, final GlobalQuotaSettings quotas) throws IOException {
     }
 
     @Override
     public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final String userName, final TableName tableName, final Quotas quotas) throws IOException {
+        final String userName, final TableName tableName, final GlobalQuotaSettings quotas)
+            throws IOException {
     }
 
     @Override
     public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final String userName, final TableName tableName, final Quotas quotas) throws IOException {
+        final String userName, final TableName tableName, final GlobalQuotaSettings quotas)
+            throws IOException {
     }
 
     @Override
     public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final String userName, final String namespace, final Quotas quotas) throws IOException {
+        final String userName, final String namespace, final GlobalQuotaSettings quotas)
+            throws IOException {
     }
 
     @Override
     public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final String userName, final String namespace, final Quotas quotas) throws IOException {
+        final String userName, final String namespace, final GlobalQuotaSettings quotas)
+            throws IOException {
     }
 
     @Override
     public void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final TableName tableName, final Quotas quotas) throws IOException {
+        final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {
     }
 
     @Override
     public void postSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final TableName tableName, final Quotas quotas) throws IOException {
+        final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {
     }
 
     @Override
     public void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final String namespace, final Quotas quotas) throws IOException {
+        final String namespace, final GlobalQuotaSettings quotas) throws IOException {
     }
 
     @Override
     public void postSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final String namespace, final Quotas quotas) throws IOException {
+        final String namespace, final GlobalQuotaSettings quotas) throws IOException {
     }
 
     @Override