HBASE-17748 Include HBase snapshots in space quotas
[hbase.git] / hbase-client / src / main / java / org / apache / hadoop / hbase / quotas / QuotaTableUtil.java
blobc1863a7d63ecab5d4477efe43af2de485b8ff672
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
19 package org.apache.hadoop.hbase.quotas;
21 import java.io.ByteArrayInputStream;
22 import java.io.ByteArrayOutputStream;
23 import java.io.IOException;
24 import java.util.Collection;
25 import java.util.HashMap;
26 import java.util.List;
27 import java.util.Map;
28 import java.util.Objects;
29 import java.util.regex.Pattern;
31 import org.apache.commons.logging.Log;
32 import org.apache.commons.logging.LogFactory;
33 import org.apache.hadoop.hbase.Cell;
34 import org.apache.hadoop.hbase.NamespaceDescriptor;
35 import org.apache.hadoop.hbase.ServerName;
36 import org.apache.hadoop.hbase.TableName;
37 import org.apache.hadoop.hbase.classification.InterfaceAudience;
38 import org.apache.hadoop.hbase.classification.InterfaceStability;
39 import org.apache.hadoop.hbase.client.ClusterConnection;
40 import org.apache.hadoop.hbase.client.Connection;
41 import org.apache.hadoop.hbase.client.Get;
42 import org.apache.hadoop.hbase.client.Put;
43 import org.apache.hadoop.hbase.client.QuotaStatusCalls;
44 import org.apache.hadoop.hbase.client.Result;
45 import org.apache.hadoop.hbase.client.ResultScanner;
46 import org.apache.hadoop.hbase.client.Scan;
47 import org.apache.hadoop.hbase.client.Table;
48 import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
49 import org.apache.hadoop.hbase.filter.CompareFilter;
50 import org.apache.hadoop.hbase.filter.Filter;
51 import org.apache.hadoop.hbase.filter.FilterList;
52 import org.apache.hadoop.hbase.filter.QualifierFilter;
53 import org.apache.hadoop.hbase.filter.RegexStringComparator;
54 import org.apache.hadoop.hbase.filter.RowFilter;
55 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
56 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
57 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
58 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
59 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
60 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
61 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
62 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
63 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
64 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
65 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot;
66 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
67 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
68 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
69 import org.apache.hadoop.hbase.util.Bytes;
70 import org.apache.hadoop.hbase.util.Strings;
72 /**
73 * Helper class to interact with the quota table.
74 * <table>
75 * <tr><th>ROW-KEY</th><th>FAM/QUAL</th><th>DATA</th></tr>
76 * <tr><td>n.&lt;namespace&gt;</td><td>q:s</td><td>&lt;global-quotas&gt;</td></tr>
77 * <tr><td>n.&lt;namespace&gt;</td><td>u:p</td><td>&lt;namespace-quota policy&gt;</td></tr>
78 * <tr><td>n.&lt;namespace&gt;</td><td>u:s</td><td>&lt;SpaceQuotaSnapshot&gt;</td></tr>
79 * <tr><td>t.&lt;table&gt;</td><td>q:s</td><td>&lt;global-quotas&gt;</td></tr>
80 * <tr><td>t.&lt;table&gt;</td><td>u:p</td><td>&lt;table-quota policy&gt;</td></tr>
81 * <tr><td>t.&lt;table&gt;</td><td>u:ss.&lt;snapshot name&gt;</td><td>&lt;SpaceQuotaSnapshot&gt;</td></tr>
82 * <tr><td>u.&lt;user&gt;</td><td>q:s</td><td>&lt;global-quotas&gt;</td></tr>
83 * <tr><td>u.&lt;user&gt;</td><td>q:s.&lt;table&gt;</td><td>&lt;table-quotas&gt;</td></tr>
84 * <tr><td>u.&lt;user&gt;</td><td>q:s.&lt;ns&gt;</td><td>&lt;namespace-quotas&gt;</td></tr>
85 * </table
87 @InterfaceAudience.Private
88 @InterfaceStability.Evolving
89 public class QuotaTableUtil {
90 private static final Log LOG = LogFactory.getLog(QuotaTableUtil.class);
92 /** System table for quotas */
93 public static final TableName QUOTA_TABLE_NAME =
94 TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "quota");
96 protected static final byte[] QUOTA_FAMILY_INFO = Bytes.toBytes("q");
97 protected static final byte[] QUOTA_FAMILY_USAGE = Bytes.toBytes("u");
98 protected static final byte[] QUOTA_QUALIFIER_SETTINGS = Bytes.toBytes("s");
99 protected static final byte[] QUOTA_QUALIFIER_SETTINGS_PREFIX = Bytes.toBytes("s.");
100 protected static final byte[] QUOTA_QUALIFIER_POLICY = Bytes.toBytes("p");
101 protected static final byte[] QUOTA_SNAPSHOT_SIZE_QUALIFIER = Bytes.toBytes("ss");
102 protected static final String QUOTA_POLICY_COLUMN =
103 Bytes.toString(QUOTA_FAMILY_USAGE) + ":" + Bytes.toString(QUOTA_QUALIFIER_POLICY);
104 protected static final byte[] QUOTA_USER_ROW_KEY_PREFIX = Bytes.toBytes("u.");
105 protected static final byte[] QUOTA_TABLE_ROW_KEY_PREFIX = Bytes.toBytes("t.");
106 protected static final byte[] QUOTA_NAMESPACE_ROW_KEY_PREFIX = Bytes.toBytes("n.");
108 /* =========================================================================
109 * Quota "settings" helpers
111 public static Quotas getTableQuota(final Connection connection, final TableName table)
112 throws IOException {
113 return getQuotas(connection, getTableRowKey(table));
116 public static Quotas getNamespaceQuota(final Connection connection, final String namespace)
117 throws IOException {
118 return getQuotas(connection, getNamespaceRowKey(namespace));
121 public static Quotas getUserQuota(final Connection connection, final String user)
122 throws IOException {
123 return getQuotas(connection, getUserRowKey(user));
126 public static Quotas getUserQuota(final Connection connection, final String user,
127 final TableName table) throws IOException {
128 return getQuotas(connection, getUserRowKey(user), getSettingsQualifierForUserTable(table));
131 public static Quotas getUserQuota(final Connection connection, final String user,
132 final String namespace) throws IOException {
133 return getQuotas(connection, getUserRowKey(user),
134 getSettingsQualifierForUserNamespace(namespace));
137 private static Quotas getQuotas(final Connection connection, final byte[] rowKey)
138 throws IOException {
139 return getQuotas(connection, rowKey, QUOTA_QUALIFIER_SETTINGS);
142 private static Quotas getQuotas(final Connection connection, final byte[] rowKey,
143 final byte[] qualifier) throws IOException {
144 Get get = new Get(rowKey);
145 get.addColumn(QUOTA_FAMILY_INFO, qualifier);
146 Result result = doGet(connection, get);
147 if (result.isEmpty()) {
148 return null;
150 return quotasFromData(result.getValue(QUOTA_FAMILY_INFO, qualifier));
153 public static Get makeGetForTableQuotas(final TableName table) {
154 Get get = new Get(getTableRowKey(table));
155 get.addFamily(QUOTA_FAMILY_INFO);
156 return get;
159 public static Get makeGetForNamespaceQuotas(final String namespace) {
160 Get get = new Get(getNamespaceRowKey(namespace));
161 get.addFamily(QUOTA_FAMILY_INFO);
162 return get;
165 public static Get makeGetForUserQuotas(final String user, final Iterable<TableName> tables,
166 final Iterable<String> namespaces) {
167 Get get = new Get(getUserRowKey(user));
168 get.addColumn(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
169 for (final TableName table: tables) {
170 get.addColumn(QUOTA_FAMILY_INFO, getSettingsQualifierForUserTable(table));
172 for (final String ns: namespaces) {
173 get.addColumn(QUOTA_FAMILY_INFO, getSettingsQualifierForUserNamespace(ns));
175 return get;
178 public static Scan makeScan(final QuotaFilter filter) {
179 Scan scan = new Scan();
180 scan.addFamily(QUOTA_FAMILY_INFO);
181 if (filter != null && !filter.isNull()) {
182 scan.setFilter(makeFilter(filter));
184 return scan;
188 * converts quotafilter to serializeable filterlists.
190 public static Filter makeFilter(final QuotaFilter filter) {
191 FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
192 if (!Strings.isEmpty(filter.getUserFilter())) {
193 FilterList userFilters = new FilterList(FilterList.Operator.MUST_PASS_ONE);
194 boolean hasFilter = false;
196 if (!Strings.isEmpty(filter.getNamespaceFilter())) {
197 FilterList nsFilters = new FilterList(FilterList.Operator.MUST_PASS_ALL);
198 nsFilters.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
199 new RegexStringComparator(getUserRowKeyRegex(filter.getUserFilter()), 0)));
200 nsFilters.addFilter(new QualifierFilter(CompareFilter.CompareOp.EQUAL,
201 new RegexStringComparator(
202 getSettingsQualifierRegexForUserNamespace(filter.getNamespaceFilter()), 0)));
203 userFilters.addFilter(nsFilters);
204 hasFilter = true;
206 if (!Strings.isEmpty(filter.getTableFilter())) {
207 FilterList tableFilters = new FilterList(FilterList.Operator.MUST_PASS_ALL);
208 tableFilters.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
209 new RegexStringComparator(getUserRowKeyRegex(filter.getUserFilter()), 0)));
210 tableFilters.addFilter(new QualifierFilter(CompareFilter.CompareOp.EQUAL,
211 new RegexStringComparator(
212 getSettingsQualifierRegexForUserTable(filter.getTableFilter()), 0)));
213 userFilters.addFilter(tableFilters);
214 hasFilter = true;
216 if (!hasFilter) {
217 userFilters.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
218 new RegexStringComparator(getUserRowKeyRegex(filter.getUserFilter()), 0)));
221 filterList.addFilter(userFilters);
222 } else if (!Strings.isEmpty(filter.getTableFilter())) {
223 filterList.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
224 new RegexStringComparator(getTableRowKeyRegex(filter.getTableFilter()), 0)));
225 } else if (!Strings.isEmpty(filter.getNamespaceFilter())) {
226 filterList.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
227 new RegexStringComparator(getNamespaceRowKeyRegex(filter.getNamespaceFilter()), 0)));
229 return filterList;
233 * Creates a {@link Scan} which returns only quota snapshots from the quota table.
235 public static Scan makeQuotaSnapshotScan() {
236 return makeQuotaSnapshotScanForTable(null);
240 * Fetches all {@link SpaceQuotaSnapshot} objects from the {@code hbase:quota} table.
242 * @param conn The HBase connection
243 * @return A map of table names and their computed snapshot.
245 public static Map<TableName,SpaceQuotaSnapshot> getSnapshots(Connection conn) throws IOException {
246 Map<TableName,SpaceQuotaSnapshot> snapshots = new HashMap<>();
247 try (Table quotaTable = conn.getTable(QUOTA_TABLE_NAME);
248 ResultScanner rs = quotaTable.getScanner(makeQuotaSnapshotScan())) {
249 for (Result r : rs) {
250 extractQuotaSnapshot(r, snapshots);
253 return snapshots;
257 * Creates a {@link Scan} which returns only {@link SpaceQuotaSnapshot} from the quota table for a
258 * specific table.
259 * @param tn Optionally, a table name to limit the scan's rowkey space. Can be null.
261 public static Scan makeQuotaSnapshotScanForTable(TableName tn) {
262 Scan s = new Scan();
263 // Limit to "u:v" column
264 s.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY);
265 if (null == tn) {
266 s.setRowPrefixFilter(QUOTA_TABLE_ROW_KEY_PREFIX);
267 } else {
268 byte[] row = getTableRowKey(tn);
269 // Limit rowspace to the "t:" prefix
270 s.withStartRow(row, true).withStopRow(row, true);
272 return s;
276 * Extracts the {@link SpaceViolationPolicy} and {@link TableName} from the provided
277 * {@link Result} and adds them to the given {@link Map}. If the result does not contain
278 * the expected information or the serialized policy in the value is invalid, this method
279 * will throw an {@link IllegalArgumentException}.
281 * @param result A row from the quota table.
282 * @param snapshots A map of snapshots to add the result of this method into.
284 public static void extractQuotaSnapshot(
285 Result result, Map<TableName,SpaceQuotaSnapshot> snapshots) {
286 byte[] row = Objects.requireNonNull(result).getRow();
287 if (row == null) {
288 throw new IllegalArgumentException("Provided result had a null row");
290 final TableName targetTableName = getTableFromRowKey(row);
291 Cell c = result.getColumnLatestCell(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY);
292 if (c == null) {
293 throw new IllegalArgumentException("Result did not contain the expected column "
294 + QUOTA_POLICY_COLUMN + ", " + result.toString());
296 ByteString buffer = UnsafeByteOperations.unsafeWrap(
297 c.getValueArray(), c.getValueOffset(), c.getValueLength());
298 try {
299 QuotaProtos.SpaceQuotaSnapshot snapshot = QuotaProtos.SpaceQuotaSnapshot.parseFrom(buffer);
300 snapshots.put(targetTableName, SpaceQuotaSnapshot.toSpaceQuotaSnapshot(snapshot));
301 } catch (InvalidProtocolBufferException e) {
302 throw new IllegalArgumentException(
303 "Result did not contain a valid SpaceQuota protocol buffer message", e);
307 public static interface UserQuotasVisitor {
308 void visitUserQuotas(final String userName, final Quotas quotas)
309 throws IOException;
310 void visitUserQuotas(final String userName, final TableName table, final Quotas quotas)
311 throws IOException;
312 void visitUserQuotas(final String userName, final String namespace, final Quotas quotas)
313 throws IOException;
316 public static interface TableQuotasVisitor {
317 void visitTableQuotas(final TableName tableName, final Quotas quotas)
318 throws IOException;
321 public static interface NamespaceQuotasVisitor {
322 void visitNamespaceQuotas(final String namespace, final Quotas quotas)
323 throws IOException;
326 public static interface QuotasVisitor extends UserQuotasVisitor,
327 TableQuotasVisitor, NamespaceQuotasVisitor {
330 public static void parseResult(final Result result, final QuotasVisitor visitor)
331 throws IOException {
332 byte[] row = result.getRow();
333 if (isNamespaceRowKey(row)) {
334 parseNamespaceResult(result, visitor);
335 } else if (isTableRowKey(row)) {
336 parseTableResult(result, visitor);
337 } else if (isUserRowKey(row)) {
338 parseUserResult(result, visitor);
339 } else {
340 LOG.warn("unexpected row-key: " + Bytes.toString(row));
344 public static void parseResultToCollection(final Result result,
345 Collection<QuotaSettings> quotaSettings) throws IOException {
347 QuotaTableUtil.parseResult(result, new QuotaTableUtil.QuotasVisitor() {
348 @Override
349 public void visitUserQuotas(String userName, Quotas quotas) {
350 quotaSettings.addAll(QuotaSettingsFactory.fromUserQuotas(userName, quotas));
353 @Override
354 public void visitUserQuotas(String userName, TableName table, Quotas quotas) {
355 quotaSettings.addAll(QuotaSettingsFactory.fromUserQuotas(userName, table, quotas));
358 @Override
359 public void visitUserQuotas(String userName, String namespace, Quotas quotas) {
360 quotaSettings.addAll(QuotaSettingsFactory.fromUserQuotas(userName, namespace, quotas));
363 @Override
364 public void visitTableQuotas(TableName tableName, Quotas quotas) {
365 quotaSettings.addAll(QuotaSettingsFactory.fromTableQuotas(tableName, quotas));
368 @Override
369 public void visitNamespaceQuotas(String namespace, Quotas quotas) {
370 quotaSettings.addAll(QuotaSettingsFactory.fromNamespaceQuotas(namespace, quotas));
375 public static void parseNamespaceResult(final Result result,
376 final NamespaceQuotasVisitor visitor) throws IOException {
377 String namespace = getNamespaceFromRowKey(result.getRow());
378 parseNamespaceResult(namespace, result, visitor);
381 protected static void parseNamespaceResult(final String namespace, final Result result,
382 final NamespaceQuotasVisitor visitor) throws IOException {
383 byte[] data = result.getValue(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
384 if (data != null) {
385 Quotas quotas = quotasFromData(data);
386 visitor.visitNamespaceQuotas(namespace, quotas);
390 public static void parseTableResult(final Result result, final TableQuotasVisitor visitor)
391 throws IOException {
392 TableName table = getTableFromRowKey(result.getRow());
393 parseTableResult(table, result, visitor);
396 protected static void parseTableResult(final TableName table, final Result result,
397 final TableQuotasVisitor visitor) throws IOException {
398 byte[] data = result.getValue(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
399 if (data != null) {
400 Quotas quotas = quotasFromData(data);
401 visitor.visitTableQuotas(table, quotas);
405 public static void parseUserResult(final Result result, final UserQuotasVisitor visitor)
406 throws IOException {
407 String userName = getUserFromRowKey(result.getRow());
408 parseUserResult(userName, result, visitor);
411 protected static void parseUserResult(final String userName, final Result result,
412 final UserQuotasVisitor visitor) throws IOException {
413 Map<byte[], byte[]> familyMap = result.getFamilyMap(QUOTA_FAMILY_INFO);
414 if (familyMap == null || familyMap.isEmpty()) return;
416 for (Map.Entry<byte[], byte[]> entry: familyMap.entrySet()) {
417 Quotas quotas = quotasFromData(entry.getValue());
418 if (Bytes.startsWith(entry.getKey(), QUOTA_QUALIFIER_SETTINGS_PREFIX)) {
419 String name = Bytes.toString(entry.getKey(), QUOTA_QUALIFIER_SETTINGS_PREFIX.length);
420 if (name.charAt(name.length() - 1) == TableName.NAMESPACE_DELIM) {
421 String namespace = name.substring(0, name.length() - 1);
422 visitor.visitUserQuotas(userName, namespace, quotas);
423 } else {
424 TableName table = TableName.valueOf(name);
425 visitor.visitUserQuotas(userName, table, quotas);
427 } else if (Bytes.equals(entry.getKey(), QUOTA_QUALIFIER_SETTINGS)) {
428 visitor.visitUserQuotas(userName, quotas);
434 * Creates a {@link Put} to store the given {@code snapshot} for the given {@code tableName} in
435 * the quota table.
437 static Put createPutForSpaceSnapshot(TableName tableName, SpaceQuotaSnapshot snapshot) {
438 Put p = new Put(getTableRowKey(tableName));
439 p.addColumn(
440 QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY,
441 SpaceQuotaSnapshot.toProtoSnapshot(snapshot).toByteArray());
442 return p;
446 * Creates a {@link Get} for the HBase snapshot's size against the given table.
448 static Get makeGetForSnapshotSize(TableName tn, String snapshot) {
449 Get g = new Get(Bytes.add(QUOTA_TABLE_ROW_KEY_PREFIX, Bytes.toBytes(tn.toString())));
450 g.addColumn(
451 QUOTA_FAMILY_USAGE,
452 Bytes.add(QUOTA_SNAPSHOT_SIZE_QUALIFIER, Bytes.toBytes(snapshot)));
453 return g;
457 * Creates a {@link Put} to persist the current size of the {@code snapshot} with respect to
458 * the given {@code table}.
460 static Put createPutForSnapshotSize(TableName tableName, String snapshot, long size) {
461 // We just need a pb message with some `long usage`, so we can just reuse the
462 // SpaceQuotaSnapshot message instead of creating a new one.
463 Put p = new Put(getTableRowKey(tableName));
464 p.addColumn(QUOTA_FAMILY_USAGE, getSnapshotSizeQualifier(snapshot),
465 org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot
466 .newBuilder().setQuotaUsage(size).build().toByteArray());
467 return p;
471 * Creates a {@code Put} for the namespace's total snapshot size.
473 static Put createPutForNamespaceSnapshotSize(String namespace, long size) {
474 Put p = new Put(getNamespaceRowKey(namespace));
475 p.addColumn(QUOTA_FAMILY_USAGE, QUOTA_SNAPSHOT_SIZE_QUALIFIER,
476 org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot
477 .newBuilder().setQuotaUsage(size).build().toByteArray());
478 return p;
482 * Fetches the computed size of all snapshots against tables in a namespace for space quotas.
484 static long getNamespaceSnapshotSize(
485 Connection conn, String namespace) throws IOException {
486 try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
487 Result r = quotaTable.get(createGetNamespaceSnapshotSize(namespace));
488 if (r.isEmpty()) {
489 return 0L;
491 r.advance();
492 return parseSnapshotSize(r.current());
493 } catch (InvalidProtocolBufferException e) {
494 throw new IOException("Could not parse snapshot size value for namespace " + namespace, e);
499 * Creates a {@code Get} to fetch the namespace's total snapshot size.
501 static Get createGetNamespaceSnapshotSize(String namespace) {
502 Get g = new Get(getNamespaceRowKey(namespace));
503 g.addColumn(QUOTA_FAMILY_USAGE, QUOTA_SNAPSHOT_SIZE_QUALIFIER);
504 return g;
508 * Parses the snapshot size from the given Cell's value.
510 static long parseSnapshotSize(Cell c) throws InvalidProtocolBufferException {
511 ByteString bs = UnsafeByteOperations.unsafeWrap(
512 c.getValueArray(), c.getValueOffset(), c.getValueLength());
513 return QuotaProtos.SpaceQuotaSnapshot.parseFrom(bs).getQuotaUsage();
516 static Scan createScanForSnapshotSizes(TableName table) {
517 byte[] rowkey = getTableRowKey(table);
518 return new Scan()
519 // Fetch just this one row
520 .withStartRow(rowkey)
521 .withStopRow(rowkey, true)
522 // Just the usage family
523 .addFamily(QUOTA_FAMILY_USAGE)
524 // Only the snapshot size qualifiers
525 .setFilter(new ColumnPrefixFilter(QUOTA_SNAPSHOT_SIZE_QUALIFIER));
528 /* =========================================================================
529 * Space quota status RPC helpers
532 * Fetches the table sizes on the filesystem as tracked by the HBase Master.
534 public static Map<TableName,Long> getMasterReportedTableSizes(
535 Connection conn) throws IOException {
536 if (!(conn instanceof ClusterConnection)) {
537 throw new IllegalArgumentException("Expected a ClusterConnection");
539 ClusterConnection clusterConn = (ClusterConnection) conn;
540 GetSpaceQuotaRegionSizesResponse response = QuotaStatusCalls.getMasterRegionSizes(
541 clusterConn, 0);
542 Map<TableName,Long> tableSizes = new HashMap<>();
543 for (RegionSizes sizes : response.getSizesList()) {
544 TableName tn = ProtobufUtil.toTableName(sizes.getTableName());
545 tableSizes.put(tn, sizes.getSize());
547 return tableSizes;
551 * Fetches the observed {@link SpaceQuotaSnapshot}s observed by a RegionServer.
553 public static Map<TableName,SpaceQuotaSnapshot> getRegionServerQuotaSnapshots(
554 Connection conn, ServerName regionServer) throws IOException {
555 if (!(conn instanceof ClusterConnection)) {
556 throw new IllegalArgumentException("Expected a ClusterConnection");
558 ClusterConnection clusterConn = (ClusterConnection) conn;
559 GetSpaceQuotaSnapshotsResponse response = QuotaStatusCalls.getRegionServerQuotaSnapshot(
560 clusterConn, 0, regionServer);
561 Map<TableName,SpaceQuotaSnapshot> snapshots = new HashMap<>();
562 for (TableQuotaSnapshot snapshot : response.getSnapshotsList()) {
563 snapshots.put(
564 ProtobufUtil.toTableName(snapshot.getTableName()),
565 SpaceQuotaSnapshot.toSpaceQuotaSnapshot(snapshot.getSnapshot()));
567 return snapshots;
571 * Returns the Master's view of a quota on the given {@code tableName} or null if the
572 * Master has no quota information on that table.
574 public static SpaceQuotaSnapshot getCurrentSnapshot(
575 Connection conn, TableName tn) throws IOException {
576 if (!(conn instanceof ClusterConnection)) {
577 throw new IllegalArgumentException("Expected a ClusterConnection");
579 ClusterConnection clusterConn = (ClusterConnection) conn;
580 GetQuotaStatesResponse resp = QuotaStatusCalls.getMasterQuotaStates(clusterConn, 0);
581 HBaseProtos.TableName protoTableName = ProtobufUtil.toProtoTableName(tn);
582 for (GetQuotaStatesResponse.TableQuotaSnapshot tableSnapshot : resp.getTableSnapshotsList()) {
583 if (protoTableName.equals(tableSnapshot.getTableName())) {
584 return SpaceQuotaSnapshot.toSpaceQuotaSnapshot(tableSnapshot.getSnapshot());
587 return null;
591 * Returns the Master's view of a quota on the given {@code namespace} or null if the
592 * Master has no quota information on that namespace.
594 public static SpaceQuotaSnapshot getCurrentSnapshot(
595 Connection conn, String namespace) throws IOException {
596 if (!(conn instanceof ClusterConnection)) {
597 throw new IllegalArgumentException("Expected a ClusterConnection");
599 ClusterConnection clusterConn = (ClusterConnection) conn;
600 GetQuotaStatesResponse resp = QuotaStatusCalls.getMasterQuotaStates(clusterConn, 0);
601 for (GetQuotaStatesResponse.NamespaceQuotaSnapshot nsSnapshot : resp.getNsSnapshotsList()) {
602 if (namespace.equals(nsSnapshot.getNamespace())) {
603 return SpaceQuotaSnapshot.toSpaceQuotaSnapshot(nsSnapshot.getSnapshot());
606 return null;
609 /* =========================================================================
610 * Quotas protobuf helpers
612 protected static Quotas quotasFromData(final byte[] data) throws IOException {
613 return quotasFromData(data, 0, data.length);
616 protected static Quotas quotasFromData(
617 final byte[] data, int offset, int length) throws IOException {
618 int magicLen = ProtobufMagic.lengthOfPBMagic();
619 if (!ProtobufMagic.isPBMagicPrefix(data, offset, magicLen)) {
620 throw new IOException("Missing pb magic prefix");
622 return Quotas.parseFrom(new ByteArrayInputStream(data, offset + magicLen, length - magicLen));
625 protected static byte[] quotasToData(final Quotas data) throws IOException {
626 ByteArrayOutputStream stream = new ByteArrayOutputStream();
627 stream.write(ProtobufMagic.PB_MAGIC);
628 data.writeTo(stream);
629 return stream.toByteArray();
632 public static boolean isEmptyQuota(final Quotas quotas) {
633 boolean hasSettings = false;
634 hasSettings |= quotas.hasThrottle();
635 hasSettings |= quotas.hasBypassGlobals();
636 // Only when there is a space quota, make sure there's actually both fields provided
637 // Otherwise, it's a noop.
638 if (quotas.hasSpace()) {
639 hasSettings |= (quotas.getSpace().hasSoftLimit() && quotas.getSpace().hasViolationPolicy());
641 return !hasSettings;
644 /* =========================================================================
645 * HTable helpers
647 protected static Result doGet(final Connection connection, final Get get)
648 throws IOException {
649 try (Table table = connection.getTable(QUOTA_TABLE_NAME)) {
650 return table.get(get);
654 protected static Result[] doGet(final Connection connection, final List<Get> gets)
655 throws IOException {
656 try (Table table = connection.getTable(QUOTA_TABLE_NAME)) {
657 return table.get(gets);
661 /* =========================================================================
662 * Quota table row key helpers
664 protected static byte[] getUserRowKey(final String user) {
665 return Bytes.add(QUOTA_USER_ROW_KEY_PREFIX, Bytes.toBytes(user));
668 protected static byte[] getTableRowKey(final TableName table) {
669 return Bytes.add(QUOTA_TABLE_ROW_KEY_PREFIX, table.getName());
672 protected static byte[] getNamespaceRowKey(final String namespace) {
673 return Bytes.add(QUOTA_NAMESPACE_ROW_KEY_PREFIX, Bytes.toBytes(namespace));
676 protected static byte[] getSettingsQualifierForUserTable(final TableName tableName) {
677 return Bytes.add(QUOTA_QUALIFIER_SETTINGS_PREFIX, tableName.getName());
680 protected static byte[] getSettingsQualifierForUserNamespace(final String namespace) {
681 return Bytes.add(QUOTA_QUALIFIER_SETTINGS_PREFIX,
682 Bytes.toBytes(namespace + TableName.NAMESPACE_DELIM));
685 protected static String getUserRowKeyRegex(final String user) {
686 return getRowKeyRegEx(QUOTA_USER_ROW_KEY_PREFIX, user);
689 protected static String getTableRowKeyRegex(final String table) {
690 return getRowKeyRegEx(QUOTA_TABLE_ROW_KEY_PREFIX, table);
693 protected static String getNamespaceRowKeyRegex(final String namespace) {
694 return getRowKeyRegEx(QUOTA_NAMESPACE_ROW_KEY_PREFIX, namespace);
697 private static String getRowKeyRegEx(final byte[] prefix, final String regex) {
698 return '^' + Pattern.quote(Bytes.toString(prefix)) + regex + '$';
701 protected static String getSettingsQualifierRegexForUserTable(final String table) {
702 return '^' + Pattern.quote(Bytes.toString(QUOTA_QUALIFIER_SETTINGS_PREFIX)) +
703 table + "(?<!" + Pattern.quote(Character.toString(TableName.NAMESPACE_DELIM)) + ")$";
706 protected static String getSettingsQualifierRegexForUserNamespace(final String namespace) {
707 return '^' + Pattern.quote(Bytes.toString(QUOTA_QUALIFIER_SETTINGS_PREFIX)) +
708 namespace + Pattern.quote(Character.toString(TableName.NAMESPACE_DELIM)) + '$';
711 protected static boolean isNamespaceRowKey(final byte[] key) {
712 return Bytes.startsWith(key, QUOTA_NAMESPACE_ROW_KEY_PREFIX);
715 protected static String getNamespaceFromRowKey(final byte[] key) {
716 return Bytes.toString(key, QUOTA_NAMESPACE_ROW_KEY_PREFIX.length);
719 protected static boolean isTableRowKey(final byte[] key) {
720 return Bytes.startsWith(key, QUOTA_TABLE_ROW_KEY_PREFIX);
723 protected static TableName getTableFromRowKey(final byte[] key) {
724 return TableName.valueOf(Bytes.toString(key, QUOTA_TABLE_ROW_KEY_PREFIX.length));
727 protected static boolean isUserRowKey(final byte[] key) {
728 return Bytes.startsWith(key, QUOTA_USER_ROW_KEY_PREFIX);
731 protected static String getUserFromRowKey(final byte[] key) {
732 return Bytes.toString(key, QUOTA_USER_ROW_KEY_PREFIX.length);
735 protected static SpaceQuota getProtoViolationPolicy(SpaceViolationPolicy policy) {
736 return SpaceQuota.newBuilder()
737 .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(policy))
738 .build();
741 protected static SpaceViolationPolicy getViolationPolicy(SpaceQuota proto) {
742 if (!proto.hasViolationPolicy()) {
743 throw new IllegalArgumentException("Protobuf SpaceQuota does not have violation policy.");
745 return ProtobufUtil.toViolationPolicy(proto.getViolationPolicy());
748 protected static byte[] getSnapshotSizeQualifier(String snapshotName) {
749 return Bytes.add(QUOTA_SNAPSHOT_SIZE_QUALIFIER, Bytes.toBytes(snapshotName));
752 protected static long extractSnapshotSize(
753 byte[] data, int offset, int length) throws InvalidProtocolBufferException {
754 ByteString byteStr = UnsafeByteOperations.unsafeWrap(data, offset, length);
755 return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot
756 .parseFrom(byteStr).getQuotaUsage();