Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 2A522200BD2 for ; Sat, 3 Dec 2016 23:31:15 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 2939B160B16; Sat, 3 Dec 2016 22:31:15 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 23F4A160B0F for ; Sat, 3 Dec 2016 23:31:13 +0100 (CET) Received: (qmail 17092 invoked by uid 500); 3 Dec 2016 22:31:13 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 17082 invoked by uid 99); 3 Dec 2016 22:31:13 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 03 Dec 2016 22:31:13 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id E886EE02E4; Sat, 3 Dec 2016 22:31:12 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: tedyu@apache.org To: commits@hbase.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: hbase git commit: HBASE-16996 Implement storage/retrieval of filesystem-use quotas into quota table (Josh Elser) Date: Sat, 3 Dec 2016 22:31:12 +0000 (UTC) archived-at: Sat, 03 Dec 2016 22:31:15 -0000 Repository: hbase Updated Branches: refs/heads/HBASE-16961 f7daf4f93 -> 501f7b5db HBASE-16996 Implement storage/retrieval of filesystem-use quotas into quota table (Josh Elser) Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/501f7b5d Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/501f7b5d Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/501f7b5d Branch: refs/heads/HBASE-16961 Commit: 501f7b5dbd95fd0a883b87f573b7656c6443cc32 Parents: f7daf4f Author: tedyu Authored: Sat Dec 3 14:30:48 2016 -0800 Committer: tedyu Committed: Sat Dec 3 14:30:48 2016 -0800 ---------------------------------------------------------------------- .../hadoop/hbase/quotas/QuotaTableUtil.java | 13 +- .../hadoop/hbase/quotas/MasterQuotaManager.java | 30 +++++ .../hadoop/hbase/quotas/TestQuotaAdmin.java | 125 ++++++++++++++++++- 3 files changed, 165 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/501f7b5d/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 116dd0c..1640ddc 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 @@ -52,7 +52,9 @@ import org.apache.hadoop.hbase.util.Strings; *
  *     ROW-KEY      FAM/QUAL        DATA
  *   n.<namespace> q:s         <global-quotas>
+ *   n.<namespace> u:du        <size in bytes>
  *   t.<table>     q:s         <global-quotas>
+ *   t.<table>     u:du        <size in bytes>
  *   u.<user>      q:s         <global-quotas>
  *   u.<user>      q:s.<table> <table-quotas>
  *   u.<user>      q:s.<ns>:   <namespace-quotas>
@@ -71,6 +73,7 @@ public class QuotaTableUtil {
   protected static final byte[] QUOTA_FAMILY_USAGE = Bytes.toBytes("u");
   protected static final byte[] QUOTA_QUALIFIER_SETTINGS = Bytes.toBytes("s");
   protected static final byte[] QUOTA_QUALIFIER_SETTINGS_PREFIX = Bytes.toBytes("s.");
+  protected static final byte[] QUOTA_QUALIFIER_DISKUSAGE = Bytes.toBytes("du");
   protected static final byte[] QUOTA_USER_ROW_KEY_PREFIX = Bytes.toBytes("u.");
   protected static final byte[] QUOTA_TABLE_ROW_KEY_PREFIX = Bytes.toBytes("t.");
   protected static final byte[] QUOTA_NAMESPACE_ROW_KEY_PREFIX = Bytes.toBytes("n.");
@@ -298,11 +301,16 @@ public class QuotaTableUtil {
    *  Quotas protobuf helpers
    */
   protected static Quotas quotasFromData(final byte[] data) throws IOException {
+    return quotasFromData(data, 0, data.length);
+  }
+
+  protected static Quotas quotasFromData(
+      final byte[] data, int offset, int length) throws IOException {
     int magicLen = ProtobufMagic.lengthOfPBMagic();
-    if (!ProtobufMagic.isPBMagicPrefix(data, 0, magicLen)) {
+    if (!ProtobufMagic.isPBMagicPrefix(data, offset, magicLen)) {
       throw new IOException("Missing pb magic prefix");
     }
-    return Quotas.parseFrom(new ByteArrayInputStream(data, magicLen, data.length - magicLen));
+    return Quotas.parseFrom(new ByteArrayInputStream(data, offset + magicLen, length - magicLen));
   }
 
   protected static byte[] quotasToData(final Quotas data) throws IOException {
@@ -316,6 +324,7 @@ public class QuotaTableUtil {
     boolean hasSettings = false;
     hasSettings |= quotas.hasThrottle();
     hasSettings |= quotas.hasBypassGlobals();
+    hasSettings |= quotas.hasSpace();
     return !hasSettings;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/501f7b5d/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 647a770..bd9f410 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
@@ -37,6 +37,8 @@ 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;
@@ -293,9 +295,11 @@ public class MasterQuotaManager implements RegionStateListener {
     Quotas quotas = quotaOps.fetch();
     quotaOps.preApply(quotas);
 
+    // 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());
 
     // Submit new changes
     quotas = builder.build();
@@ -437,6 +441,32 @@ public class MasterQuotaManager implements RegionStateListener {
     }
   }
 
+  /**
+   * 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()) {
+      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
+   */
+  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(

http://git-wip-us.apache.org/repos/asf/hbase/blob/501f7b5d/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 d6e8952..4fc95cd 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
@@ -22,20 +22,32 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Result;
+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.security.User;
+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.SpaceLimitRequest;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.google.common.collect.Iterables;
+
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 /**
@@ -233,10 +245,121 @@ public class TestQuotaAdmin {
     assertNumResults(0, null);
   }
 
+  @Test
+  public void testSetAndGetSpaceQuota() throws Exception {
+    Admin admin = TEST_UTIL.getAdmin();
+    final TableName tn = TableName.valueOf("table1");
+    final long sizeLimit = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
+    final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_WRITES;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, violationPolicy);
+    admin.setQuota(settings);
+
+    // Verify the Quotas in the table
+    try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      ResultScanner scanner = quotaTable.getScanner(new Scan());
+      try {
+        Result r = Iterables.getOnlyElement(scanner);
+        CellScanner cells = r.cellScanner();
+        assertTrue("Expected to find a cell", cells.advance());
+        assertSpaceQuota(sizeLimit, violationPolicy, cells.current());
+      } finally {
+        scanner.close();
+      }
+    }
+
+    // Verify we can retrieve it via the QuotaRetriever API
+    QuotaRetriever scanner = QuotaRetriever.open(admin.getConfiguration());
+    try {
+      assertSpaceQuota(sizeLimit, violationPolicy, Iterables.getOnlyElement(scanner));
+    } finally {
+      scanner.close();
+    }
+  }
+
+  @Test
+  public void testSetAndModifyQuota() throws Exception {
+    Admin admin = TEST_UTIL.getAdmin();
+    final TableName tn = TableName.valueOf("table1");
+    final long originalSizeLimit = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
+    final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_WRITES;
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, originalSizeLimit, violationPolicy);
+    admin.setQuota(settings);
+
+    // Verify the Quotas in the table
+    try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      ResultScanner scanner = quotaTable.getScanner(new Scan());
+      try {
+        Result r = Iterables.getOnlyElement(scanner);
+        CellScanner cells = r.cellScanner();
+        assertTrue("Expected to find a cell", cells.advance());
+        assertSpaceQuota(originalSizeLimit, violationPolicy, cells.current());
+      } finally {
+        scanner.close();
+      }
+    }
+
+    // Verify we can retrieve it via the QuotaRetriever API
+    QuotaRetriever quotaScanner = QuotaRetriever.open(admin.getConfiguration());
+    try {
+      assertSpaceQuota(originalSizeLimit, violationPolicy, Iterables.getOnlyElement(quotaScanner));
+    } finally {
+      quotaScanner.close();
+    }
+
+    // Setting a new size and policy should be reflected
+    final long newSizeLimit = 1024L * 1024L * 1024L * 1024L; // 1TB
+    final SpaceViolationPolicy newViolationPolicy = SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
+    QuotaSettings newSettings = QuotaSettingsFactory.limitTableSpace(
+        tn, newSizeLimit, newViolationPolicy);
+    admin.setQuota(newSettings);
+
+    // Verify the new Quotas in the table
+    try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      ResultScanner scanner = quotaTable.getScanner(new Scan());
+      try {
+        Result r = Iterables.getOnlyElement(scanner);
+        CellScanner cells = r.cellScanner();
+        assertTrue("Expected to find a cell", cells.advance());
+        assertSpaceQuota(newSizeLimit, newViolationPolicy, cells.current());
+      } finally {
+        scanner.close();
+      }
+    }
+
+    // Verify we can retrieve the new quota via the QuotaRetriever API
+    quotaScanner = QuotaRetriever.open(admin.getConfiguration());
+    try {
+      assertSpaceQuota(newSizeLimit, newViolationPolicy, Iterables.getOnlyElement(quotaScanner));
+    } finally {
+      quotaScanner.close();
+    }
+  }
+
   private void assertNumResults(int expected, final QuotaFilter filter) throws Exception {
     assertEquals(expected, countResults(filter));
   }
 
+  private void assertSpaceQuota(
+      long sizeLimit, SpaceViolationPolicy violationPolicy, Cell cell) throws Exception {
+    Quotas q = QuotaTableUtil.quotasFromData(
+        cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+    assertTrue("Quota should have space quota defined", q.hasSpace());
+    QuotaProtos.SpaceQuota spaceQuota = q.getSpace();
+    assertEquals(sizeLimit, spaceQuota.getSoftLimit());
+    assertEquals(violationPolicy, ProtobufUtil.toViolationPolicy(spaceQuota.getViolationPolicy()));
+  }
+
+  private void assertSpaceQuota(
+      long sizeLimit, SpaceViolationPolicy violationPolicy, QuotaSettings actualSettings) {
+    assertTrue("The actual QuotaSettings was not an instance of " + SpaceLimitSettings.class
+        + " but of " + actualSettings.getClass(), actualSettings instanceof SpaceLimitSettings);
+    SpaceLimitRequest spaceLimitRequest = ((SpaceLimitSettings) actualSettings).getProto();
+    assertEquals(sizeLimit, spaceLimitRequest.getQuota().getSoftLimit());
+    assertEquals(violationPolicy,
+        ProtobufUtil.toViolationPolicy(spaceLimitRequest.getQuota().getViolationPolicy()));
+  }
+
   private int countResults(final QuotaFilter filter) throws Exception {
     QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration(), filter);
     try {