You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2018/08/03 16:40:44 UTC

hbase git commit: HBASE-20885 Removed entry for RPC quota from hbase:quota when RPC quota is removed

Repository: hbase
Updated Branches:
  refs/heads/branch-2.0 62f91f265 -> 7a1623775


HBASE-20885 Removed entry for RPC quota from hbase:quota when RPC quota is removed

Signed-off-by: Josh Elser <el...@apache.org>
Signed-off-by: Mike Drob <md...@apache.org>


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

Branch: refs/heads/branch-2.0
Commit: 7a16237750d5ca3481e56494fc5053b7912938f6
Parents: 62f91f2
Author: Sakthi <ja...@cloudera.com>
Authored: Wed Aug 1 21:23:15 2018 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Fri Aug 3 09:40:35 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/quotas/ThrottleSettings.java   |   9 ++
 .../hbase/quotas/GlobalQuotaSettingsImpl.java   |  89 ++++-------
 .../hadoop/hbase/quotas/TestQuotaAdmin.java     | 150 ++++++++++++++++++-
 3 files changed, 188 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7a162377/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 8cee7cd..e424d8a 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
@@ -24,6 +24,7 @@ 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.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 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;
@@ -48,6 +49,14 @@ class ThrottleSettings extends QuotaSettings {
     return proto.hasTimedQuota() ? proto.getTimedQuota().getSoftLimit() : -1;
   }
 
+  /**
+   * Returns a copy of the internal state of <code>this</code>
+   */
+  @VisibleForTesting
+  QuotaProtos.ThrottleRequest getProto() {
+    return proto.toBuilder().build();
+  }
+
   public TimeUnit getTimeUnit() {
     return proto.hasTimedQuota() ?
       ProtobufUtil.toTimeUnit(proto.getTimedQuota().getTimeUnit()) : null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/7a162377/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
index 3893d00..3119691 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
@@ -115,70 +115,41 @@ public class GlobalQuotaSettingsImpl extends GlobalQuotaSettings {
     validateQuotaTarget(other);
 
     // Propagate the Throttle
-    QuotaProtos.Throttle.Builder throttleBuilder = (throttleProto == null
-        ? null : throttleProto.toBuilder());
+    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()) {
+      if (!otherThrottle.proto.hasType() || !otherThrottle.proto.hasTimedQuota()) {
+        // To prevent the "empty" row in QuotaTableUtil.QUOTA_TABLE_NAME
+        throttleBuilder = null;
+      } else {
         QuotaProtos.ThrottleRequest otherProto = otherThrottle.proto;
-        if (otherProto.hasTimedQuota()) {
-          if (otherProto.hasTimedQuota()) {
-            validateTimedQuota(otherProto.getTimedQuota());
-          }
+        validateTimedQuota(otherProto.getTimedQuota());
+        if (throttleBuilder == null) {
+          throttleBuilder = QuotaProtos.Throttle.newBuilder();
+        }
 
-          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);
+        switch (otherProto.getType()) {
+          case REQUEST_NUMBER:
+            throttleBuilder.setReqNum(otherProto.getTimedQuota());
+            break;
+          case REQUEST_SIZE:
+            throttleBuilder.setReqSize(otherProto.getTimedQuota());
+            break;
+          case WRITE_NUMBER:
+            throttleBuilder.setWriteNum(otherProto.getTimedQuota());
+            break;
+          case WRITE_SIZE:
+            throttleBuilder.setWriteSize(otherProto.getTimedQuota());
+            break;
+          case READ_NUMBER:
+            throttleBuilder.setReadNum(otherProto.getTimedQuota());
+            break;
+          case READ_SIZE:
+            throttleBuilder.setReadSize(otherProto.getTimedQuota());
+            break;
         }
-      } else {
-        clearThrottleBuilder(throttleBuilder);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7a162377/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
index 572fb10..b84dc83 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java
@@ -368,7 +368,7 @@ public class TestQuotaAdmin {
   }
 
   @Test
-  public void testSetModifyRemoveQuota() throws Exception {
+  public void testSetModifyRemoveSpaceQuota() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
     final TableName tn = TableName.valueOf("sq_table2");
     final long originalSizeLimit = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
@@ -453,6 +453,154 @@ public class TestQuotaAdmin {
     assertEquals(expected, countResults(filter));
   }
 
+  @Test
+  public void testSetGetRemoveRPCQuota() throws Exception {
+    Admin admin = TEST_UTIL.getAdmin();
+    final TableName tn = TableName.valueOf("sq_table1");
+    QuotaSettings settings =
+        QuotaSettingsFactory.throttleTable(tn, ThrottleType.REQUEST_SIZE, 2L, TimeUnit.HOURS);
+    admin.setQuota(settings);
+
+    // Verify the Quota in the table
+    verifyRecordPresentInQuotaTable(ThrottleType.REQUEST_SIZE, 2L, TimeUnit.HOURS);
+
+    // Verify we can retrieve it via the QuotaRetriever API
+    verifyFetchableViaAPI(admin, ThrottleType.REQUEST_SIZE, 2L, TimeUnit.HOURS);
+
+    // Now, remove the quota
+    QuotaSettings removeQuota = QuotaSettingsFactory.unthrottleTable(tn);
+    admin.setQuota(removeQuota);
+
+    // Verify that the record doesn't exist in the table
+    verifyRecordNotPresentInQuotaTable();
+
+    // Verify that we can also not fetch it via the API
+    verifyNotFetchableViaAPI(admin);
+  }
+
+  @Test
+  public void testSetModifyRemoveRPCQuota() throws Exception {
+    Admin admin = TEST_UTIL.getAdmin();
+    final TableName tn = TableName.valueOf("sq_table1");
+    QuotaSettings settings =
+        QuotaSettingsFactory.throttleTable(tn, ThrottleType.REQUEST_SIZE, 2L, TimeUnit.HOURS);
+    admin.setQuota(settings);
+
+    // Verify the Quota in the table
+    verifyRecordPresentInQuotaTable(ThrottleType.REQUEST_SIZE, 2L, TimeUnit.HOURS);
+
+    // Verify we can retrieve it via the QuotaRetriever API
+    verifyFetchableViaAPI(admin, ThrottleType.REQUEST_SIZE, 2L, TimeUnit.HOURS);
+
+    // Setting a limit and time unit should be reflected
+    QuotaSettings newSettings =
+        QuotaSettingsFactory.throttleTable(tn, ThrottleType.REQUEST_SIZE, 3L, TimeUnit.DAYS);
+    admin.setQuota(newSettings);
+
+    // Verify the new Quota in the table
+    verifyRecordPresentInQuotaTable(ThrottleType.REQUEST_SIZE, 3L, TimeUnit.DAYS);
+
+    // Verify we can retrieve the new quota via the QuotaRetriever API
+    verifyFetchableViaAPI(admin, ThrottleType.REQUEST_SIZE, 3L, TimeUnit.DAYS);
+
+    // Now, remove the quota
+    QuotaSettings removeQuota = QuotaSettingsFactory.unthrottleTable(tn);
+    admin.setQuota(removeQuota);
+
+    // Verify that the record doesn't exist in the table
+    verifyRecordNotPresentInQuotaTable();
+
+    // Verify that we can also not fetch it via the API
+    verifyNotFetchableViaAPI(admin);
+
+  }
+
+  private void verifyRecordPresentInQuotaTable(ThrottleType type, long limit, TimeUnit tu)
+      throws Exception {
+    // Verify the RPC Quotas in the table
+    try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaTableUtil.QUOTA_TABLE_NAME);
+        ResultScanner scanner = quotaTable.getScanner(new Scan())) {
+      Result r = Iterables.getOnlyElement(scanner);
+      CellScanner cells = r.cellScanner();
+      assertTrue("Expected to find a cell", cells.advance());
+      assertRPCQuota(type, limit, tu, cells.current());
+    }
+  }
+
+  private void verifyRecordNotPresentInQuotaTable() throws Exception {
+    // Verify that the record doesn't exist in the QuotaTableUtil.QUOTA_TABLE_NAME
+    try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaTableUtil.QUOTA_TABLE_NAME);
+        ResultScanner scanner = quotaTable.getScanner(new Scan())) {
+      assertNull("Did not expect to find a quota entry", scanner.next());
+    }
+  }
+
+  private void verifyFetchableViaAPI(Admin admin, ThrottleType type, long limit, TimeUnit tu)
+      throws Exception {
+    // Verify we can retrieve the new quota via the QuotaRetriever API
+    try (QuotaRetriever quotaScanner = QuotaRetriever.open(admin.getConfiguration())) {
+      assertRPCQuota(type, limit, tu, Iterables.getOnlyElement(quotaScanner));
+    }
+  }
+
+  private void verifyNotFetchableViaAPI(Admin admin) throws Exception {
+    // Verify that we can also not fetch it via the API
+    try (QuotaRetriever quotaScanner = QuotaRetriever.open(admin.getConfiguration())) {
+      assertNull("Did not expect to find a quota entry", quotaScanner.next());
+    }
+  }
+
+  private void assertRPCQuota(ThrottleType type, long limit, TimeUnit tu, Cell cell)
+      throws Exception {
+    Quotas q = QuotaTableUtil
+        .quotasFromData(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+    assertTrue("Quota should have rpc quota defined", q.hasThrottle());
+
+    QuotaProtos.Throttle rpcQuota = q.getThrottle();
+    QuotaProtos.TimedQuota t = null;
+
+    switch (type) {
+      case REQUEST_SIZE:
+        assertTrue(rpcQuota.hasReqSize());
+        t = rpcQuota.getReqSize();
+        break;
+      case READ_NUMBER:
+        assertTrue(rpcQuota.hasReadNum());
+        t = rpcQuota.getReadNum();
+        break;
+      case READ_SIZE:
+        assertTrue(rpcQuota.hasReadSize());
+        t = rpcQuota.getReadSize();
+        break;
+      case REQUEST_NUMBER:
+        assertTrue(rpcQuota.hasReqNum());
+        t = rpcQuota.getReqNum();
+        break;
+      case WRITE_NUMBER:
+        assertTrue(rpcQuota.hasWriteNum());
+        t = rpcQuota.getWriteNum();
+        break;
+      case WRITE_SIZE:
+        assertTrue(rpcQuota.hasWriteSize());
+        t = rpcQuota.getWriteSize();
+        break;
+    }
+
+    assertEquals(t.getSoftLimit(), limit);
+    assertEquals(t.getTimeUnit(), ProtobufUtil.toProtoTimeUnit(tu));
+  }
+
+  private void assertRPCQuota(ThrottleType type, long limit, TimeUnit tu,
+      QuotaSettings actualSettings) throws Exception {
+    assertTrue(
+        "The actual QuotaSettings was not an instance of " + ThrottleSettings.class + " but of "
+            + actualSettings.getClass(), actualSettings instanceof ThrottleSettings);
+    QuotaProtos.ThrottleRequest throttleRequest = ((ThrottleSettings) actualSettings).getProto();
+    assertEquals(limit, throttleRequest.getTimedQuota().getSoftLimit());
+    assertEquals(ProtobufUtil.toProtoTimeUnit(tu), throttleRequest.getTimedQuota().getTimeUnit());
+    assertEquals(ProtobufUtil.toProtoThrottleType(type), throttleRequest.getType());
+  }
+
   private void assertSpaceQuota(
       long sizeLimit, SpaceViolationPolicy violationPolicy, Cell cell) throws Exception {
     Quotas q = QuotaTableUtil.quotasFromData(