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:30 UTC

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

HBASE-17428 Implement informational RPCs for space quotas

Create some RPCs that can expose the in-memory state that the
RegionServers and Master hold to drive the space quota "state machine".
Then, create some hbase shell commands to interact with those.


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

Branch: refs/heads/HBASE-16961
Commit: df67080697fdf8ac7f02d5b6905c25c728d26d0b
Parents: dfe7a70
Author: Josh Elser <el...@apache.org>
Authored: Tue Feb 21 15:36:39 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri May 19 12:25:19 2017 -0400

----------------------------------------------------------------------
 .../hbase/client/ConnectionImplementation.java  |    9 +
 .../hadoop/hbase/client/QuotaStatusCalls.java   |  125 +
 .../client/ShortCircuitMasterConnection.java    |    7 +
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |   77 +
 .../hbase/shaded/protobuf/RequestConverter.java |   33 +
 .../shaded/protobuf/generated/AdminProtos.java  |  650 +-
 .../shaded/protobuf/generated/MasterProtos.java |   88 +-
 .../shaded/protobuf/generated/QuotaProtos.java  | 5986 +++++++++++++++++-
 .../generated/RegionServerStatusProtos.java     |   28 +-
 .../src/main/protobuf/Admin.proto               |    9 +
 .../src/main/protobuf/Master.proto              |    4 +
 .../src/main/protobuf/Quota.proto               |   35 +
 .../hbase/protobuf/generated/QuotaProtos.java   |    6 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   60 +
 .../hbase/quotas/ActivePolicyEnforcement.java   |    8 +
 .../hbase/regionserver/RSRpcServices.java       |   57 +
 .../hadoop/hbase/master/MockRegionServer.java   |   18 +
 .../hbase/quotas/TestQuotaStatusRPCs.java       |  192 +
 hbase-shell/src/main/ruby/hbase/quotas.rb       |   16 +
 hbase-shell/src/main/ruby/shell.rb              |    3 +
 .../ruby/shell/commands/list_quota_snapshots.rb |   59 +
 .../shell/commands/list_quota_table_sizes.rb    |   47 +
 .../shell/commands/list_quota_violations.rb     |   48 +
 hbase-shell/src/test/ruby/hbase/quotas_test.rb  |   24 -
 .../test/ruby/hbase/quotas_test_no_cluster.rb   |   69 +
 25 files changed, 7212 insertions(+), 446 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/df670806/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 564cfb4..5242efc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -92,6 +92,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
+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.ReplicationProtos.AddReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
@@ -1750,6 +1752,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
           ListReplicationPeersRequest request) throws ServiceException {
         return stub.listReplicationPeers(controller, request);
       }
+
+      @Override
+      public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(
+          RpcController controller, GetSpaceQuotaRegionSizesRequest request)
+          throws ServiceException {
+        return stub.getSpaceQuotaRegionSizes(controller, request);
+      }
     };
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/df670806/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
new file mode 100644
index 0000000..f0f385d
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/QuotaStatusCalls.java
@@ -0,0 +1,125 @@
+/*
+ * 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.client;
+
+import java.io.IOException;
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+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.GetSpaceQuotaEnforcementsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
+
+/**
+ * Client class to wrap RPCs to HBase servers for space quota status information.
+ */
+@InterfaceAudience.Private
+public class QuotaStatusCalls {
+
+  /**
+   * {@link #getMasterRegionSizes(Connection, RpcControllerFactory, RpcRetryingCallerFactory, int)}
+   */
+  public static GetSpaceQuotaRegionSizesResponse getMasterRegionSizes(
+      ClusterConnection clusterConn, int timeout) throws IOException {
+    RpcControllerFactory rpcController = clusterConn.getRpcControllerFactory();
+    RpcRetryingCallerFactory rpcCaller = clusterConn.getRpcRetryingCallerFactory();
+    return getMasterRegionSizes(clusterConn, rpcController, rpcCaller, timeout);
+  }
+
+  /**
+   * Executes an RPC to the HBase master to fetch its view on the Region sizes.
+   */
+  public static GetSpaceQuotaRegionSizesResponse getMasterRegionSizes(
+      Connection conn, RpcControllerFactory factory, RpcRetryingCallerFactory rpcCaller,
+      int timeout) throws IOException {
+    MasterCallable<GetSpaceQuotaRegionSizesResponse> callable =
+        new MasterCallable<GetSpaceQuotaRegionSizesResponse>(conn, factory) {
+      @Override
+      protected GetSpaceQuotaRegionSizesResponse rpcCall() throws Exception {
+        return master.getSpaceQuotaRegionSizes(
+            getRpcController(), RequestConverter.buildGetSpaceQuotaRegionSizesRequest());
+      }
+    };
+    RpcRetryingCaller<GetSpaceQuotaRegionSizesResponse> caller = rpcCaller.newCaller();
+    try {
+      return caller.callWithoutRetries(callable, timeout);
+    } finally {
+      callable.close();
+    }
+  }
+
+  /**
+   * {@link #getRegionServerQuotaSnapshot(ClusterConnection, RpcControllerFactory, int, ServerName)}
+   */
+  public static GetSpaceQuotaSnapshotsResponse getRegionServerQuotaSnapshot(
+      ClusterConnection clusterConn, int timeout, ServerName sn) throws IOException {
+    RpcControllerFactory rpcController = clusterConn.getRpcControllerFactory();
+    return getRegionServerQuotaSnapshot(clusterConn, rpcController, timeout, sn);
+  }
+
+  /**
+   * Executes an RPC to the RegionServer identified by the {@code ServerName} to fetch its view
+   * on space quotas.
+   */
+  public static GetSpaceQuotaSnapshotsResponse getRegionServerQuotaSnapshot(
+      ClusterConnection conn, RpcControllerFactory factory,
+      int timeout, ServerName sn) throws IOException {
+    final AdminService.BlockingInterface admin = conn.getAdmin(sn);
+    Callable<GetSpaceQuotaSnapshotsResponse> callable =
+        new Callable<GetSpaceQuotaSnapshotsResponse>() {
+      @Override
+      public GetSpaceQuotaSnapshotsResponse call() throws Exception {
+        return admin.getSpaceQuotaSnapshots(
+            factory.newController(), RequestConverter.buildGetSpaceQuotaSnapshotsRequest());
+      }
+    };
+    return ProtobufUtil.call(callable);
+  }
+
+  /**
+   * {@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/df670806/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
index e3b5b12..fec850e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
@@ -23,6 +23,8 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.*;
+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.ReplicationProtos.AddReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
@@ -484,4 +486,9 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
     return stub.listReplicationPeers(controller, request);
   }
 
+  @Override
+  public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(RpcController controller,
+      GetSpaceQuotaRegionSizesRequest request) throws ServiceException {
+    return stub.getSpaceQuotaRegionSizes(controller, request);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/df670806/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 ce4cd04..560f40c 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
@@ -22,6 +22,7 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -31,12 +32,15 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 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.Scan;
 import org.apache.hadoop.hbase.client.Table;
@@ -46,12 +50,19 @@ 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;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 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.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;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -395,6 +406,72 @@ public class QuotaTableUtil {
     return p;
   }
 
+
+  /* =========================================================================
+   *  Space quota status RPC helpers
+   */
+  /**
+   * Fetches the table sizes on the filesystem as tracked by the HBase Master.
+   */
+  public static Map<TableName,Long> getMasterReportedTableSizes(
+      Connection conn) throws IOException {
+    if (!(conn instanceof ClusterConnection)) {
+      throw new IllegalArgumentException("Expected a ClusterConnection");
+    }
+    ClusterConnection clusterConn = (ClusterConnection) conn;
+    GetSpaceQuotaRegionSizesResponse response = QuotaStatusCalls.getMasterRegionSizes(
+        clusterConn, 0);
+    Map<TableName,Long> tableSizes = new HashMap<>();
+    for (RegionSizes sizes : response.getSizesList()) {
+      TableName tn = ProtobufUtil.toTableName(sizes.getTableName());
+      tableSizes.put(tn, sizes.getSize());
+    }
+    return tableSizes;
+  }
+
+  /**
+   * Fetches the observed {@link SpaceQuotaSnapshot}s observed by a RegionServer.
+   */
+  public static Map<TableName,SpaceQuotaSnapshot> getRegionServerQuotaSnapshots(
+      Connection conn, ServerName regionServer) throws IOException {
+    if (!(conn instanceof ClusterConnection)) {
+      throw new IllegalArgumentException("Expected a ClusterConnection");
+    }
+    ClusterConnection clusterConn = (ClusterConnection) conn;
+    GetSpaceQuotaSnapshotsResponse response = QuotaStatusCalls.getRegionServerQuotaSnapshot(
+        clusterConn, 0, regionServer);
+    Map<TableName,SpaceQuotaSnapshot> snapshots = new HashMap<>();
+    for (TableQuotaSnapshot snapshot : response.getSnapshotsList()) {
+      snapshots.put(
+          ProtobufUtil.toTableName(snapshot.getTableName()),
+          SpaceQuotaSnapshot.toSpaceQuotaSnapshot(snapshot.getSnapshot()));
+    }
+    return snapshots;
+  }
+
+  /**
+   * 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;
+  }
+
   /* =========================================================================
    *  Quotas protobuf helpers
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/df670806/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 4fd0d95..76e8dcb 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
@@ -119,6 +119,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormali
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
 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.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;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
@@ -1707,4 +1710,34 @@ public final class RequestConverter {
     }
     return builder.build();
   }
+
+  private static final GetSpaceQuotaRegionSizesRequest GET_SPACE_QUOTA_REGION_SIZES_REQUEST =
+      GetSpaceQuotaRegionSizesRequest.newBuilder().build();
+
+  /**
+   * Returns a {@link GetSpaceQuotaRegionSizesRequest} object.
+   */
+  public static GetSpaceQuotaRegionSizesRequest buildGetSpaceQuotaRegionSizesRequest() {
+    return GET_SPACE_QUOTA_REGION_SIZES_REQUEST;
+  }
+
+  private static final GetSpaceQuotaSnapshotsRequest GET_SPACE_QUOTA_SNAPSHOTS_REQUEST =
+      GetSpaceQuotaSnapshotsRequest.newBuilder().build();
+
+  /**
+   * Returns a {@link GetSpaceQuotaSnapshotsRequest} object.
+   */
+  public static GetSpaceQuotaSnapshotsRequest buildGetSpaceQuotaSnapshotsRequest() {
+    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;
+  }
 }