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/19 17:13:45 UTC

[45/50] [abbrv] hbase git commit: HBASE-17981 Consolidate the space quota shell commands

http://git-wip-us.apache.org/repos/asf/hbase/blob/3561b115/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 964b035..39e73b6 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -329,8 +329,4 @@ service AdminService {
   /** 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/3561b115/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 0d171b3..0d74435 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Quota.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
@@ -136,18 +136,6 @@ message GetSpaceQuotaSnapshotsResponse {
   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;
-}
-
 message GetQuotaStatesRequest {
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/3561b115/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
index fad9f44..717ec73 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/QuotaProtos.java
@@ -982,7 +982,7 @@ public final class QuotaProtos {
 
       public final boolean isInitialized() {
         if (!hasTimeUnit()) {
-          
+
           return false;
         }
         return true;
@@ -2009,37 +2009,37 @@ public final class QuotaProtos {
       public final boolean isInitialized() {
         if (hasReqNum()) {
           if (!getReqNum().isInitialized()) {
-            
+
             return false;
           }
         }
         if (hasReqSize()) {
           if (!getReqSize().isInitialized()) {
-            
+
             return false;
           }
         }
         if (hasWriteNum()) {
           if (!getWriteNum().isInitialized()) {
-            
+
             return false;
           }
         }
         if (hasWriteSize()) {
           if (!getWriteSize().isInitialized()) {
-            
+
             return false;
           }
         }
         if (hasReadNum()) {
           if (!getReadNum().isInitialized()) {
-            
+
             return false;
           }
         }
         if (hasReadSize()) {
           if (!getReadSize().isInitialized()) {
-            
+
             return false;
           }
         }
@@ -2169,7 +2169,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota req_num = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getReqNumFieldBuilder() {
         if (reqNumBuilder_ == null) {
           reqNumBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -2286,7 +2286,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota req_size = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getReqSizeFieldBuilder() {
         if (reqSizeBuilder_ == null) {
           reqSizeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -2403,7 +2403,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota write_num = 3;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getWriteNumFieldBuilder() {
         if (writeNumBuilder_ == null) {
           writeNumBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -2520,7 +2520,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota write_size = 4;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getWriteSizeFieldBuilder() {
         if (writeSizeBuilder_ == null) {
           writeSizeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -2637,7 +2637,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota read_num = 5;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getReadNumFieldBuilder() {
         if (readNumBuilder_ == null) {
           readNumBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -2754,7 +2754,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota read_size = 6;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getReadSizeFieldBuilder() {
         if (readSizeBuilder_ == null) {
           readSizeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -3244,7 +3244,7 @@ public final class QuotaProtos {
       public final boolean isInitialized() {
         if (hasTimedQuota()) {
           if (!getTimedQuota().isInitialized()) {
-            
+
             return false;
           }
         }
@@ -3410,7 +3410,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota timed_quota = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getTimedQuotaFieldBuilder() {
         if (timedQuotaBuilder_ == null) {
           timedQuotaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -3978,7 +3978,7 @@ public final class QuotaProtos {
       public final boolean isInitialized() {
         if (hasThrottle()) {
           if (!getThrottle().isInitialized()) {
-            
+
             return false;
           }
         }
@@ -4141,7 +4141,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.Throttle throttle = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.ThrottleOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle.Builder, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.ThrottleOrBuilder>
           getThrottleFieldBuilder() {
         if (throttleBuilder_ == null) {
           throttleBuilder_ = new com.google.protobuf.SingleFieldBuilder<

http://git-wip-us.apache.org/repos/asf/hbase/blob/3561b115/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 6168fda..41fe3e5 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
@@ -190,13 +190,10 @@ 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;
@@ -3395,31 +3392,4 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       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 (manager != null) {
-        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/3561b115/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 0b8cecf..67a3f0a 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
@@ -102,8 +102,6 @@ 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;
@@ -745,11 +743,4 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
       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/3561b115/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
index 2cd67c9..2020e3c 100644
--- 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
@@ -186,12 +186,13 @@ public class TestQuotaStatusRPCs {
       }
     });
 
-    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);
+    // We obtain the violations for a RegionServer by observing the snapshots
+    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(snapshot.getQuotaStatus().isInViolation());
+    assertEquals(SpaceViolationPolicy.NO_INSERTS, snapshot.getQuotaStatus().getPolicy());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/3561b115/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 a2b21fa..784896e 100644
--- a/hbase-shell/src/main/ruby/hbase/quotas.rb
+++ b/hbase-shell/src/main/ruby/hbase/quotas.rb
@@ -169,13 +169,21 @@ module Hbase
       QuotaTableUtil.getMasterReportedTableSizes(@admin.getConnection())
     end
 
-    def get_rs_quota_snapshots(rs)
-      QuotaTableUtil.getRegionServerQuotaSnapshots(@admin.getConnection(),
-          ServerName.valueOf(rs))
+    def get_quota_snapshots(regionserver=nil)
+      # Ask a regionserver if we were given one
+      return get_rs_quota_snapshots(regionserver) if regionserver
+      # Otherwise, read from the quota table
+      get_quota_snapshots_from_table
     end
 
-    def get_rs_quota_violations(rs)
-      QuotaTableUtil.getRegionServerQuotaViolations(@admin.getConnection(),
+    def get_quota_snapshots_from_table()
+      # Reads the snapshots from the hbase:quota table
+      QuotaTableUtil.getSnapshots(@admin.getConnection())
+    end
+
+    def get_rs_quota_snapshots(rs)
+      # Reads the snapshots from a specific regionserver
+      QuotaTableUtil.getRegionServerQuotaSnapshots(@admin.getConnection(),
           ServerName.valueOf(rs))
     end
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/3561b115/hbase-shell/src/main/ruby/hbase_constants.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase_constants.rb b/hbase-shell/src/main/ruby/hbase_constants.rb
index b3c3070..cf05c13 100644
--- a/hbase-shell/src/main/ruby/hbase_constants.rb
+++ b/hbase-shell/src/main/ruby/hbase_constants.rb
@@ -85,6 +85,7 @@ module HBaseConstants
   LOCALITY_THRESHOLD = 'LOCALITY_THRESHOLD'
   RESTORE_ACL = 'RESTORE_ACL'
   POLICY = 'POLICY'
+  REGIONSERVER = 'REGIONSERVER'
 
   # Load constants from hbase java API
   def self.promote_constants(constants)

http://git-wip-us.apache.org/repos/asf/hbase/blob/3561b115/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 847aabe..aaf26b3 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -422,7 +422,6 @@ Shell.load_command_group(
     set_quota
     list_quotas
     list_quota_table_sizes
-    list_quota_violations
     list_quota_snapshots
   ]
 )

http://git-wip-us.apache.org/repos/asf/hbase/blob/3561b115/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
index c907762..bec7f02 100644
--- a/hbase-shell/src/main/ruby/shell/commands/list_quota_snapshots.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/list_quota_snapshots.rb
@@ -22,25 +22,41 @@ module Shell
     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.
+Lists the current space quota snapshots with optional selection criteria.
+Snapshots encapsulate relevant information to space quotas such as space
+use, configured limits, and quota violation details. This command is
+useful for understanding the current state of a cluster with space quotas.
+
+By default, this command will read all snapshots stored in the system from
+the hbase:quota table. A table name or namespace can be provided to filter
+the snapshots returned. RegionServers maintain a copy of snapshots, refreshing
+at a regular interval; by providing a RegionServer option, snapshots will
+be retreived from that RegionServer instead of the quota table.
 
 For example:
 
-    hbase> list_quota_snapshots 'regionserver1.domain,16020,1483482894742'
+    hbase> list_quota_snapshots
+    hbase> list_quota_snapshots({TABLE => 'table1'})
+    hbase> list_quota_snapshots({NAMESPACE => 'org1'})
+    hbase> list_quota_snapshots({REGIONSERVER => 'server1.domain,16020,1483482894742'})
+    hbase> list_quota_snapshots({NAMESPACE => 'org1', REGIONSERVER => 'server1.domain,16020,1483482894742'})
 EOF
       end
 
-      def command(hostname, args = {})
-        formatter.header(["TABLE", "USAGE", "LIMIT", "IN VIOLATION", "POLICY"])
+      def command(args = {})
+        # All arguments may be nil
+        desired_table = args[TABLE]
+        desired_namespace = args[NAMESPACE]
+        desired_regionserver = args[REGIONSERVER]
+        formatter.header(["TABLE", "USAGE", "LIMIT", "IN_VIOLATION", "POLICY"])
         count = 0
-        quotas_admin.get_rs_quota_snapshots(hostname).each do |tableName,snapshot|
+        quotas_admin.get_quota_snapshots(desired_regionserver).each do |table_name,snapshot|
+          # Skip this snapshot if it's for a table/namespace the user did not ask for
+          next unless accept? table_name, desired_table, desired_namespace
           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])
+          formatter.row([table_name.to_s, snapshot.getUsage().to_s, snapshot.getLimit().to_s,
+            status.isInViolation().to_s, policy])
           count += 1
         end
         formatter.footer(count)
@@ -54,6 +70,18 @@ EOF
           "None"
         end
       end
+
+      def accept?(table_name, desired_table=nil, desired_namespace=nil)
+        # Check the table name if given one
+        if desired_table and table_name.getQualifierAsString() != desired_table
+          return false
+        end
+        # Check the namespace if given one
+        if desired_namespace and table_name.getNamespaceAsString() != desired_namespace
+          return false
+        end
+        true
+      end
     end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/3561b115/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
index 9325477..5339543 100644
--- 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
@@ -22,10 +22,10 @@ module Shell
     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.
+Lists the computed size of each table in the cluster as computed by
+all RegionServers. This is the raw information that the Master uses to
+make decisions about space quotas. Most times, using `list_quota_snapshots`
+provides a higher-level of insight than this command.
 
 For example:
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/3561b115/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
deleted file mode 100644
index f1836a2..0000000
--- a/hbase-shell/src/main/ruby/shell/commands/list_quota_violations.rb
+++ /dev/null
@@ -1,48 +0,0 @@
-#
-#
-# 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/3561b115/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
index 7504488..7de1225 100644
--- a/hbase-shell/src/test/ruby/hbase/quotas_test_no_cluster.rb
+++ b/hbase-shell/src/test/ruby/hbase/quotas_test_no_cluster.rb
@@ -25,6 +25,10 @@ require 'hbase/table'
 
 include HBaseConstants
 
+java_import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot
+java_import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy
+java_import org.apache.hadoop.hbase.TableName
+
 module Hbase
   class NoClusterSpaceQuotasTest < Test::Unit::TestCase
     include TestHelpers
@@ -54,9 +58,6 @@ module Hbase
     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)
@@ -65,5 +66,17 @@ module Hbase
       assert_equal('None', quotaSnapshotCommand.get_policy(okStatus))
       assert_equal('NO_INSERTS', quotaSnapshotCommand.get_policy(violatedStatus))
     end
+
+    define_test 'table and namespace filtering in list_quota_snapshots' do
+      cmd = ::Shell::Commands::ListQuotaSnapshots.new(nil)
+      assert cmd.accept?(TableName.valueOf('t1')) == true
+      assert cmd.accept?(TableName.valueOf('t1'), nil, nil) == true
+      assert cmd.accept?(TableName.valueOf('t1'), 't1', nil) == true
+      assert cmd.accept?(TableName.valueOf('t1'), 't2', nil) == false
+      assert cmd.accept?(TableName.valueOf('t1'), nil, 'ns1') == false
+      assert cmd.accept?(TableName.valueOf('ns1:t1'), nil, 'ns1') == true
+      assert cmd.accept?(TableName.valueOf('ns1:t1'), 't1', nil) == true
+      assert cmd.accept?(TableName.valueOf('ns1:t1'), 't1', 'ns1') == true
+    end
   end
 end