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

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

HBASE-17981 Consolidate the space quota shell commands


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

Branch: refs/heads/HBASE-16961
Commit: dd4066a5b7bada482ab1eebdab38af378f25c60e
Parents: 802b5e3
Author: Josh Elser <el...@apache.org>
Authored: Mon May 1 19:44:47 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 22 13:08:12 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hbase/client/QuotaStatusCalls.java   |   29 -
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |   44 +-
 .../hbase/shaded/protobuf/RequestConverter.java |   11 -
 .../shaded/protobuf/generated/AdminProtos.java  |  338 ++-
 .../shaded/protobuf/generated/QuotaProtos.java  | 1923 +-----------------
 .../src/main/protobuf/Admin.proto               |    4 -
 .../src/main/protobuf/Quota.proto               |   12 -
 .../hbase/protobuf/generated/QuotaProtos.java   |   34 +-
 .../hbase/regionserver/RSRpcServices.java       |   30 -
 .../hadoop/hbase/master/MockRegionServer.java   |    9 -
 .../hbase/quotas/TestQuotaStatusRPCs.java       |   13 +-
 hbase-shell/src/main/ruby/hbase/quotas.rb       |   18 +-
 hbase-shell/src/main/ruby/hbase_constants.rb    |    1 +
 hbase-shell/src/main/ruby/shell.rb              |    1 -
 .../ruby/shell/commands/list_quota_snapshots.rb |   48 +-
 .../shell/commands/list_quota_table_sizes.rb    |    8 +-
 .../shell/commands/list_quota_violations.rb     |   48 -
 .../test/ruby/hbase/quotas_test_no_cluster.rb   |   19 +-
 18 files changed, 273 insertions(+), 2317 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dd4066a5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
index af36d1e..70f6fb6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
 
@@ -127,32 +126,4 @@ public class QuotaStatusCalls {
     };
     return ProtobufUtil.call(callable);
   }
-
-  /**
-   * See {@link #getRegionServerSpaceQuotaEnforcements(ClusterConnection, RpcControllerFactory, int, ServerName)}
-   */
-  public static GetSpaceQuotaEnforcementsResponse getRegionServerSpaceQuotaEnforcements(
-      ClusterConnection clusterConn, int timeout, ServerName sn) throws IOException {
-    RpcControllerFactory rpcController = clusterConn.getRpcControllerFactory();
-    return getRegionServerSpaceQuotaEnforcements(clusterConn, rpcController, timeout, sn);
-  }
-
-  /**
-   * Executes an RPC to the RegionServer identified by the {@code ServerName} to fetch its view on
-   * enforced space quotas.
-   */
-  public static GetSpaceQuotaEnforcementsResponse getRegionServerSpaceQuotaEnforcements(
-      ClusterConnection conn, RpcControllerFactory factory,
-      int timeout, ServerName sn) throws IOException {
-    final AdminService.BlockingInterface admin = conn.getAdmin(sn);
-    Callable<GetSpaceQuotaEnforcementsResponse> callable =
-        new Callable<GetSpaceQuotaEnforcementsResponse>() {
-      @Override
-      public GetSpaceQuotaEnforcementsResponse call() throws Exception {
-        return admin.getSpaceQuotaEnforcements(
-            factory.newController(), RequestConverter.buildGetSpaceQuotaEnforcementsRequest());
-      }
-    };
-    return ProtobufUtil.call(callable);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/dd4066a5/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 ec480c4..1b670e6 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.QuotaStatusCalls;
 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.filter.CompareFilter;
@@ -50,7 +51,6 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.filter.RegexStringComparator;
 import org.apache.hadoop.hbase.filter.RowFilter;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
@@ -59,8 +59,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsResponse.TableViolationPolicy;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot;
@@ -240,6 +238,23 @@ public class QuotaTableUtil {
   }
 
   /**
+   * Fetches all {@link SpaceQuotaSnapshot} objects from the {@code hbase:quota} table.
+   *
+   * @param conn The HBase connection
+   * @return A map of table names and their computed snapshot.
+   */
+  public static Map<TableName,SpaceQuotaSnapshot> getSnapshots(Connection conn) throws IOException {
+    Map<TableName,SpaceQuotaSnapshot> snapshots = new HashMap<>();
+    try (Table quotaTable = conn.getTable(QUOTA_TABLE_NAME);
+        ResultScanner rs = quotaTable.getScanner(makeQuotaSnapshotScan())) {
+      for (Result r : rs) {
+        extractQuotaSnapshot(r, snapshots);
+      }
+    }
+    return snapshots;
+  }
+
+  /**
    * Extracts the {@link SpaceViolationPolicy} and {@link TableName} from the provided
    * {@link Result} and adds them to the given {@link Map}. If the result does not contain
    * the expected information or the serialized policy in the value is invalid, this method
@@ -453,29 +468,6 @@ public class QuotaTableUtil {
   }
 
   /**
-   * Fetches the active {@link SpaceViolationPolicy}'s that are being enforced on the
-   * given RegionServer.
-   */
-  public static Map<TableName,SpaceViolationPolicy> getRegionServerQuotaViolations(
-      Connection conn, ServerName regionServer) throws IOException {
-    if (!(conn instanceof ClusterConnection)) {
-      throw new IllegalArgumentException("Expected a ClusterConnection");
-    }
-    ClusterConnection clusterConn = (ClusterConnection) conn;
-    RpcControllerFactory rpcController = clusterConn.getRpcControllerFactory();
-    GetSpaceQuotaEnforcementsResponse response =
-        QuotaStatusCalls.getRegionServerSpaceQuotaEnforcements(
-            clusterConn, rpcController, 0, regionServer);
-    Map<TableName,SpaceViolationPolicy> policies = new HashMap<>();
-    for (TableViolationPolicy policy : response.getViolationPoliciesList()) {
-      policies.put(
-          ProtobufUtil.toTableName(policy.getTableName()),
-          ProtobufUtil.toViolationPolicy(policy.getViolationPolicy()));
-    }
-    return policies;
-  }
-
-  /**
    * Returns the Master's view of a quota on the given {@code tableName} or null if the
    * Master has no quota information on that table.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/dd4066a5/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 99654da..4d34334 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -120,7 +120,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOr
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaEnforcementsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
@@ -1732,16 +1731,6 @@ public final class RequestConverter {
     return GET_SPACE_QUOTA_SNAPSHOTS_REQUEST;
   }
 
-  private static final GetSpaceQuotaEnforcementsRequest GET_SPACE_QUOTA_ENFORCEMENTS_REQUEST =
-      GetSpaceQuotaEnforcementsRequest.newBuilder().build();
-
-  /**
-   * Returns a {@link GetSpaceQuotaEnforcementsRequest} object.
-   */
-  public static GetSpaceQuotaEnforcementsRequest buildGetSpaceQuotaEnforcementsRequest() {
-    return GET_SPACE_QUOTA_ENFORCEMENTS_REQUEST;
-  }
-
   private static final GetQuotaStatesRequest GET_QUOTA_STATES_REQUEST =
       GetQuotaStatesRequest.newBuilder().build();