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/05/22 20:07:04 UTC

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

Branch: refs/heads/master
Commit: 4dfafd6e508e2e32c6a44e1957065abe2dd65fea
Parents: 56cbc68
Author: tedyu <yu...@gmail.com>
Authored: Thu Nov 17 10:19:52 2016 -0800
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 22 13:41:35 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 | 1310 +++++++------
 .../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, 4654 insertions(+), 697 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4dfafd6e/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/4dfafd6e/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/4dfafd6e/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/4dfafd6e/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/4dfafd6e/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 f8ea0a5..fe264e8 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
@@ -91,6 +91,7 @@ import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
 import org.apache.hadoop.hbase.quotas.QuotaScope;
 import org.apache.hadoop.hbase.quotas.QuotaType;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.quotas.ThrottleType;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
@@ -2561,6 +2562,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);
   }
@@ -2574,11 +2576,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
@@ -2596,6 +2632,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/4dfafd6e/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/4dfafd6e/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());
+  }
+}