You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2017/04/11 17:31:57 UTC

[52/54] [abbrv] hbase git commit: HBASE-17428 Implement informational RPCs for space quotas

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-protocol-shaded/src/main/protobuf/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index 338c80b..1a085e6 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -28,6 +28,7 @@ option optimize_for = SPEED;
 import "ClusterStatus.proto";
 import "HBase.proto";
 import "WAL.proto";
+import "Quota.proto";
 
 message GetRegionInfoRequest {
   required RegionSpecifier region = 1;
@@ -314,4 +315,12 @@ service AdminService {
 
   rpc GetRegionLoad(GetRegionLoadRequest)
     returns(GetRegionLoadResponse);
+
+  /** Fetches the RegionServer's view of space quotas */
+  rpc GetSpaceQuotaSnapshots(GetSpaceQuotaSnapshotsRequest)
+    returns(GetSpaceQuotaSnapshotsResponse);
+
+  /** Fetches the RegionServer's space quota active enforcements */
+  rpc GetSpaceQuotaEnforcements(GetSpaceQuotaEnforcementsRequest)
+    returns(GetSpaceQuotaEnforcementsResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 4e856c8..58e6f77 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -929,4 +929,8 @@ service MasterService {
   /** Unmark a list of ServerNames marked as draining. */
   rpc removeDrainFromRegionServers(RemoveDrainFromRegionServersRequest)
     returns(RemoveDrainFromRegionServersResponse);
+
+  /** Fetches the Master's view of space quotas */
+  rpc GetSpaceQuotaRegionSizes(GetSpaceQuotaRegionSizesRequest)
+    returns(GetSpaceQuotaRegionSizesResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-protocol-shaded/src/main/protobuf/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Quota.proto b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
index 597b059..2d7e5f5 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Quota.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
@@ -111,3 +111,38 @@ message SpaceQuotaSnapshot {
   optional uint64 usage = 2;
   optional uint64 limit = 3;
 }
+
+message GetSpaceQuotaRegionSizesRequest {
+}
+
+message GetSpaceQuotaRegionSizesResponse {
+  message RegionSizes {
+    optional TableName table_name = 1;
+    optional uint64 size = 2;
+  }
+  repeated RegionSizes sizes = 1;
+}
+
+message GetSpaceQuotaSnapshotsRequest {
+}
+
+message GetSpaceQuotaSnapshotsResponse {
+  // Cannot use TableName as a map key, do the repeated nested message by hand.
+  message TableQuotaSnapshot {
+    optional TableName table_name = 1;
+    optional SpaceQuotaSnapshot snapshot = 2;
+  }
+  repeated TableQuotaSnapshot snapshots = 1;
+}
+
+message GetSpaceQuotaEnforcementsRequest {
+}
+
+message GetSpaceQuotaEnforcementsResponse {
+  // Cannot use TableName as a map key, do the repeated nested message by hand.
+  message TableViolationPolicy {
+    optional TableName table_name = 1;
+    optional SpaceViolationPolicy violation_policy = 2;
+  }
+  repeated TableViolationPolicy violation_policies = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index a7a2f94..53560e9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -21,8 +21,11 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.commons.logging.Log;
@@ -211,6 +214,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@@ -2029,4 +2035,38 @@ public class MasterRpcServices extends RSRpcServices
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(
+      RpcController controller, GetSpaceQuotaRegionSizesRequest request) throws ServiceException {
+    try {
+      master.checkInitialized();
+      MasterQuotaManager quotaManager = this.master.getMasterQuotaManager();
+      GetSpaceQuotaRegionSizesResponse.Builder builder =
+          GetSpaceQuotaRegionSizesResponse.newBuilder();
+      if (null != quotaManager) {
+        Map<HRegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
+        Map<TableName,Long> regionSizesByTable = new HashMap<>();
+        // Translate hregioninfo+long -> tablename+long
+        for (Entry<HRegionInfo,Long> entry : regionSizes.entrySet()) {
+          final TableName tableName = entry.getKey().getTable();
+          Long prevSize = regionSizesByTable.get(tableName);
+          if (null == prevSize) {
+            prevSize = 0L;
+          }
+          regionSizesByTable.put(tableName, prevSize + entry.getValue());
+        }
+        // Serialize them into the protobuf
+        for (Entry<TableName,Long> tableSize : regionSizesByTable.entrySet()) {
+          builder.addSizes(RegionSizes.newBuilder()
+              .setTableName(ProtobufUtil.toProtoTableName(tableSize.getKey()))
+              .setSize(tableSize.getValue()).build());
+        }
+        return builder.build();
+      }
+      return builder.build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
index 9408e6c..a313fa1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.java
@@ -16,6 +16,7 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+import java.util.Collections;
 import java.util.Map;
 import java.util.Objects;
 
@@ -79,6 +80,13 @@ public class ActivePolicyEnforcement {
     return policy;
   }
 
+  /**
+   * Returns an unmodifiable version of the active {@link SpaceViolationPolicyEnforcement}s.
+   */
+  public Map<TableName,SpaceViolationPolicyEnforcement> getPolicies() {
+    return Collections.unmodifiableMap(activePolicies);
+  }
+
   @Override
   public String toString() {
     return getClass().getSimpleName() + ": " + activePolicies;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 81b87d5..1e4bab2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -101,6 +101,7 @@ import org.apache.hadoop.hbase.quotas.OperationQuota;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.regionserver.Leases.Lease;
@@ -194,6 +195,13 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
@@ -3280,4 +3288,53 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return UpdateConfigurationResponse.getDefaultInstance();
   }
 
+  @Override
+  public GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(
+      RpcController controller, GetSpaceQuotaSnapshotsRequest request) throws ServiceException {
+    try {
+      final RegionServerSpaceQuotaManager manager =
+          regionServer.getRegionServerSpaceQuotaManager();
+      final GetSpaceQuotaSnapshotsResponse.Builder builder =
+          GetSpaceQuotaSnapshotsResponse.newBuilder();
+      if (null != manager) {
+        final Map<TableName,SpaceQuotaSnapshot> snapshots = manager.copyQuotaSnapshots();
+        for (Entry<TableName,SpaceQuotaSnapshot> snapshot : snapshots.entrySet()) {
+          builder.addSnapshots(TableQuotaSnapshot.newBuilder()
+              .setTableName(ProtobufUtil.toProtoTableName(snapshot.getKey()))
+              .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(snapshot.getValue()))
+              .build());
+        }
+      }
+      return builder.build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetSpaceQuotaEnforcementsResponse getSpaceQuotaEnforcements(
+      RpcController controller, GetSpaceQuotaEnforcementsRequest request)
+      throws ServiceException {
+    try {
+      final RegionServerSpaceQuotaManager manager =
+          regionServer.getRegionServerSpaceQuotaManager();
+      final GetSpaceQuotaEnforcementsResponse.Builder builder =
+          GetSpaceQuotaEnforcementsResponse.newBuilder();
+      if (null != manager) {
+        ActivePolicyEnforcement enforcements = manager.getActiveEnforcements();
+        for (Entry<TableName,SpaceViolationPolicyEnforcement> enforcement
+            : enforcements.getPolicies().entrySet()) {
+          SpaceViolationPolicy pbPolicy = SpaceViolationPolicy.valueOf(
+              enforcement.getValue().getPolicyName());
+          builder.addViolationPolicies(TableViolationPolicy.newBuilder()
+              .setTableName(ProtobufUtil.toProtoTableName(enforcement.getKey()))
+              .setViolationPolicy(pbPolicy).build());
+        }
+      }
+      return builder.build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index f133afc..e52114e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -100,6 +100,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBul
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
@@ -726,4 +730,18 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   public RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager() {
     return null;
   }
+
+  @Override
+  public GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(
+      RpcController controller, GetSpaceQuotaSnapshotsRequest request)
+      throws ServiceException {
+    return null;
+  }
+
+  @Override
+  public GetSpaceQuotaEnforcementsResponse getSpaceQuotaEnforcements(
+      RpcController controller, GetSpaceQuotaEnforcementsRequest request)
+      throws ServiceException {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
new file mode 100644
index 0000000..d42f3d2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
@@ -0,0 +1,192 @@
+/*
+ * 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.assertTrue;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test class for the quota status RPCs in the master and regionserver.
+ */
+@Category({MediumTests.class})
+public class TestQuotaStatusRPCs {
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final AtomicLong COUNTER = new AtomicLong(0);
+
+  @Rule
+  public TestName testName = new TestName();
+  private SpaceQuotaHelperForTests helper;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    // Increase the frequency of some of the chores for responsiveness of the test
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_KEY, 1000);
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 1000);
+    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_DELAY_KEY, 1000);
+    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_PERIOD_KEY, 1000);
+    conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_DELAY_KEY, 1000);
+    conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_PERIOD_KEY, 1000);
+    conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setupForTest() throws Exception {
+    helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
+  }
+
+  @Test
+  public void testRegionSizesFromMaster() throws Exception {
+    final long tableSize = 1024L * 10L; // 10KB
+    final int numRegions = 10;
+    final TableName tn = helper.createTableWithRegions(numRegions);
+    // Will write at least `tableSize` data
+    helper.writeData(tn, tableSize);
+
+    final HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
+    final MasterQuotaManager quotaManager = master.getMasterQuotaManager();
+    // Make sure the master has all of the reports
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return numRegions == countRegionsForTable(tn, quotaManager.snapshotRegionSizes());
+      }
+    });
+
+    Map<TableName,Long> sizes = QuotaTableUtil.getMasterReportedTableSizes(TEST_UTIL.getConnection());
+    Long size = sizes.get(tn);
+    assertNotNull("No reported size for " + tn, size);
+    assertTrue("Reported table size was " + size, size.longValue() >= tableSize);
+  }
+
+  @Test
+  public void testQuotaSnapshotsFromRS() throws Exception {
+    final long sizeLimit = 1024L * 1024L; // 1MB
+    final long tableSize = 1024L * 10L; // 10KB
+    final int numRegions = 10;
+    final TableName tn = helper.createTableWithRegions(numRegions);
+
+    // Define the quota
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, sizeLimit, SpaceViolationPolicy.NO_INSERTS);
+    TEST_UTIL.getAdmin().setQuota(settings);
+
+    // Write at least `tableSize` data
+    helper.writeData(tn, tableSize);
+
+    final HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
+    final RegionServerSpaceQuotaManager manager = rs.getRegionServerSpaceQuotaManager();
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        SpaceQuotaSnapshot snapshot = manager.copyQuotaSnapshots().get(tn);
+        if (null == snapshot) {
+          return false;
+        }
+        return snapshot.getUsage() >= tableSize;
+      }
+    });
+
+    Map<TableName, SpaceQuotaSnapshot> snapshots = QuotaTableUtil.getRegionServerQuotaSnapshots(
+        TEST_UTIL.getConnection(), rs.getServerName());
+    SpaceQuotaSnapshot snapshot = snapshots.get(tn);
+    assertNotNull("Did not find snapshot for " + tn, snapshot);
+    assertTrue(
+        "Observed table usage was " + snapshot.getUsage(),
+        snapshot.getUsage() >= tableSize);
+    assertEquals(snapshot.getLimit(), sizeLimit);
+    SpaceQuotaStatus pbStatus = snapshot.getQuotaStatus();
+    assertFalse(pbStatus.isInViolation());
+  }
+
+  @Test
+  public void testQuotaEnforcementsFromRS() throws Exception {
+    final long sizeLimit = 1024L * 8L; // 8KB
+    final long tableSize = 1024L * 10L; // 10KB
+    final int numRegions = 10;
+    final TableName tn = helper.createTableWithRegions(numRegions);
+
+    // Define the quota
+    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+        tn, sizeLimit, SpaceViolationPolicy.NO_INSERTS);
+    TEST_UTIL.getAdmin().setQuota(settings);
+
+    // Write at least `tableSize` data
+    try {
+      helper.writeData(tn, tableSize);
+    } catch (SpaceLimitingException e) {
+      // Pass
+    }
+
+    final HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
+    final RegionServerSpaceQuotaManager manager = rs.getRegionServerSpaceQuotaManager();
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        ActivePolicyEnforcement enforcements = manager.getActiveEnforcements();
+        SpaceViolationPolicyEnforcement enforcement = enforcements.getPolicyEnforcement(tn);
+        return enforcement.getQuotaSnapshot().getQuotaStatus().isInViolation();
+      }
+    });
+
+    Map<TableName,SpaceViolationPolicy> violations =
+        QuotaTableUtil.getRegionServerQuotaViolations(
+            TEST_UTIL.getConnection(), rs.getServerName());
+    SpaceViolationPolicy policy = violations.get(tn);
+    assertNotNull("Did not find policy for " + tn, policy);
+    assertEquals(SpaceViolationPolicy.NO_INSERTS, policy);
+  }
+
+  private int countRegionsForTable(TableName tn, Map<HRegionInfo,Long> regionSizes) {
+    int size = 0;
+    for (HRegionInfo regionInfo : regionSizes.keySet()) {
+      if (tn.equals(regionInfo.getTable())) {
+        size++;
+      }
+    }
+    return size;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-shell/src/main/ruby/hbase/quotas.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/quotas.rb b/hbase-shell/src/main/ruby/hbase/quotas.rb
index d99fe72..a2b21fa 100644
--- a/hbase-shell/src/main/ruby/hbase/quotas.rb
+++ b/hbase-shell/src/main/ruby/hbase/quotas.rb
@@ -20,10 +20,12 @@
 include Java
 java_import java.util.concurrent.TimeUnit
 java_import org.apache.hadoop.hbase.TableName
+java_import org.apache.hadoop.hbase.ServerName
 java_import org.apache.hadoop.hbase.quotas.ThrottleType
 java_import org.apache.hadoop.hbase.quotas.QuotaFilter
 java_import org.apache.hadoop.hbase.quotas.QuotaRetriever
 java_import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory
+java_import org.apache.hadoop.hbase.quotas.QuotaTableUtil
 java_import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy
 
 module HBaseQuotasConstants
@@ -163,6 +165,20 @@ module Hbase
       @admin.setQuota(settings)
     end
 
+    def get_master_table_sizes()
+      QuotaTableUtil.getMasterReportedTableSizes(@admin.getConnection())
+    end
+
+    def get_rs_quota_snapshots(rs)
+      QuotaTableUtil.getRegionServerQuotaSnapshots(@admin.getConnection(),
+          ServerName.valueOf(rs))
+    end
+
+    def get_rs_quota_violations(rs)
+      QuotaTableUtil.getRegionServerQuotaViolations(@admin.getConnection(),
+          ServerName.valueOf(rs))
+    end
+
     def set_global_bypass(bypass, args)
       raise(ArgumentError, "Arguments should be a Hash") unless args.kind_of?(Hash)
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 66480f9..b203edc 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -419,6 +419,9 @@ Shell.load_command_group(
   :commands => %w[
     set_quota
     list_quotas
+    list_quota_table_sizes
+    list_quota_violations
+    list_quota_snapshots
   ]
 )
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-shell/src/main/ruby/shell/commands/list_quota_snapshots.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_quota_snapshots.rb b/hbase-shell/src/main/ruby/shell/commands/list_quota_snapshots.rb
new file mode 100644
index 0000000..c907762
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_quota_snapshots.rb
@@ -0,0 +1,59 @@
+#
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class ListQuotaSnapshots < Command
+      def help
+        return <<-EOF
+Lists the current snapshot of quotas on the given RegionServer. This
+information filters to each RegionServer from the Master. For each
+table, a snapshot includes the filesystem use, the filesystem limit,
+and the policy to enact when the limit is exceeded. This command is
+useful for debugging the running state of a cluster using filesystem quotas.
+
+For example:
+
+    hbase> list_quota_snapshots 'regionserver1.domain,16020,1483482894742'
+EOF
+      end
+
+      def command(hostname, args = {})
+        formatter.header(["TABLE", "USAGE", "LIMIT", "IN VIOLATION", "POLICY"])
+        count = 0
+        quotas_admin.get_rs_quota_snapshots(hostname).each do |tableName,snapshot|
+          status = snapshot.getQuotaStatus()
+          policy = get_policy(status)
+          formatter.row([tableName.to_s, snapshot.getUsage().to_s, snapshot.getLimit().to_s, status.isInViolation().to_s, policy])
+          count += 1
+        end
+        formatter.footer(count)
+      end
+
+      def get_policy(status)
+        # Unwrap the violation policy if it exists
+        if status.isInViolation()
+          status.getPolicy().name()
+        else
+          "None"
+        end
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-shell/src/main/ruby/shell/commands/list_quota_table_sizes.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_quota_table_sizes.rb b/hbase-shell/src/main/ruby/shell/commands/list_quota_table_sizes.rb
new file mode 100644
index 0000000..9325477
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_quota_table_sizes.rb
@@ -0,0 +1,47 @@
+#
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class ListQuotaTableSizes < Command
+      def help
+        return <<-EOF
+Lists the sizes of the tables in HBase as collected
+for the purpose of implementing filesystem utilization
+quotas. This information is extracted from the HBase
+Master and drives future quota actions in the cluster.
+
+For example:
+
+    hbase> list_quota_table_sizes
+EOF
+      end
+
+      def command(args = {})
+        formatter.header(["TABLE", "SIZE"])
+        count = 0
+        quotas_admin.get_master_table_sizes().each do |tableName,size|
+          formatter.row([tableName.to_s, size.to_s])
+          count += 1
+        end
+        formatter.footer(count)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-shell/src/main/ruby/shell/commands/list_quota_violations.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_quota_violations.rb b/hbase-shell/src/main/ruby/shell/commands/list_quota_violations.rb
new file mode 100644
index 0000000..f1836a2
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_quota_violations.rb
@@ -0,0 +1,48 @@
+#
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class ListQuotaViolations < Command
+      def help
+        return <<-EOF
+Lists the current quota violations being enforced by a RegionServer.
+Violations are enacted based on the quota snapshot information a RegionServer
+holds (see list_quota_snapshots). Each violation contains the action the
+RegionServer is taking on the table. This command is useful in debugging
+the running state of a cluster using filesystem quotas.
+
+For example:
+
+    hbase> list_quota_violations 'regionserver1.domain,16020,1483482894742'
+EOF
+      end
+
+      def command(hostname, args = {})
+        formatter.header(["TABLE", "POLICY"])
+        count = 0
+        quotas_admin.get_rs_quota_violations(hostname).each do |tableName,policy|
+          formatter.row([tableName.to_s, policy.name])
+          count += 1
+        end
+        formatter.footer(count)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-shell/src/test/ruby/hbase/quotas_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/quotas_test.rb b/hbase-shell/src/test/ruby/hbase/quotas_test.rb
index 78c889c..076eaed 100644
--- a/hbase-shell/src/test/ruby/hbase/quotas_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/quotas_test.rb
@@ -96,30 +96,6 @@ module Hbase
       end
     end
 
-    define_test '_parse_size accepts various forms of byte shorthand' do
-      qa = quotas_admin()
-      KILO = 1024
-      MEGA = KILO * KILO
-      GIGA = MEGA * KILO
-      TERA = GIGA * KILO
-      PETA = TERA * KILO
-      assert_equal(1, qa._parse_size("1"))
-      assert_equal(1, qa._parse_size("1b"))
-      assert_equal(1, qa._parse_size("1B"))
-      assert_equal(KILO * 2, qa._parse_size("2k"))
-      assert_equal(KILO * 2, qa._parse_size("2K"))
-      assert_equal(MEGA * 5, qa._parse_size("5m"))
-      assert_equal(MEGA * 5, qa._parse_size("5M"))
-      assert_equal(GIGA * 3, qa._parse_size("3g"))
-      assert_equal(GIGA * 3, qa._parse_size("3G"))
-      assert_equal(TERA * 4, qa._parse_size("4t"))
-      assert_equal(TERA * 4, qa._parse_size("4T"))
-      assert_equal(PETA * 32, qa._parse_size("32p"))
-      assert_equal(PETA * 32, qa._parse_size("32P"))
-      assert_equal(GIGA * 4, qa._parse_size("4096m"))
-      assert_equal(GIGA * 4, qa._parse_size("4096M"))
-    end
-
     define_test 'can set and remove quota' do
       command(:set_quota, TYPE => SPACE, LIMIT => '1G', POLICY => NO_INSERTS, TABLE => @test_name)
       output = capture_stdout{ command(:list_quotas) }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1ffeea4/hbase-shell/src/test/ruby/hbase/quotas_test_no_cluster.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/quotas_test_no_cluster.rb b/hbase-shell/src/test/ruby/hbase/quotas_test_no_cluster.rb
new file mode 100644
index 0000000..7504488
--- /dev/null
+++ b/hbase-shell/src/test/ruby/hbase/quotas_test_no_cluster.rb
@@ -0,0 +1,69 @@
+#
+#
+# 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.
+#
+
+require 'shell'
+require 'stringio'
+require 'hbase_constants'
+require 'hbase/hbase'
+require 'hbase/table'
+
+include HBaseConstants
+
+module Hbase
+  class NoClusterSpaceQuotasTest < Test::Unit::TestCase
+    include TestHelpers
+
+    define_test '_parse_size accepts various forms of byte shorthand' do
+      qa = ::Hbase::QuotasAdmin.new(nil)
+      KILO = 1024
+      MEGA = KILO * KILO
+      GIGA = MEGA * KILO
+      TERA = GIGA * KILO
+      PETA = TERA * KILO
+      assert_equal(1, qa._parse_size("1"))
+      assert_equal(1, qa._parse_size("1b"))
+      assert_equal(1, qa._parse_size("1B"))
+      assert_equal(KILO * 2, qa._parse_size("2k"))
+      assert_equal(KILO * 2, qa._parse_size("2K"))
+      assert_equal(MEGA * 5, qa._parse_size("5m"))
+      assert_equal(MEGA * 5, qa._parse_size("5M"))
+      assert_equal(GIGA * 3, qa._parse_size("3g"))
+      assert_equal(GIGA * 3, qa._parse_size("3G"))
+      assert_equal(TERA * 4, qa._parse_size("4t"))
+      assert_equal(TERA * 4, qa._parse_size("4T"))
+      assert_equal(PETA * 32, qa._parse_size("32p"))
+      assert_equal(PETA * 32, qa._parse_size("32P"))
+      assert_equal(GIGA * 4, qa._parse_size("4096m"))
+      assert_equal(GIGA * 4, qa._parse_size("4096M"))
+    end
+
+    define_test 'get policy name for status not in violation' do
+      java_import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot
+      java_import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy
+
+      okStatus = SpaceQuotaSnapshot::SpaceQuotaStatus::notInViolation()
+      # By default, statuses are in violation
+      violatedStatus = SpaceQuotaSnapshot::SpaceQuotaStatus.new(SpaceViolationPolicy::NO_INSERTS)
+      # Pass in nil for the Shell instance (that we don't care about)
+      quotaSnapshotCommand = ::Shell::Commands::ListQuotaSnapshots.new(nil)
+      assert_equal('None', quotaSnapshotCommand.get_policy(okStatus))
+      assert_equal('NO_INSERTS', quotaSnapshotCommand.get_policy(violatedStatus))
+    end
+  end
+end