View Javadoc

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
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
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.
17   */
18  
19  package org.apache.hadoop.hbase.quotas;
20  
21  import java.io.ByteArrayInputStream;
22  import java.io.ByteArrayOutputStream;
23  import java.io.IOException;
24  import java.util.List;
25  import java.util.Map;
26  import java.util.regex.Pattern;
27  
28  import org.apache.commons.logging.Log;
29  import org.apache.commons.logging.LogFactory;
30  import org.apache.hadoop.hbase.NamespaceDescriptor;
31  import org.apache.hadoop.hbase.TableName;
32  import org.apache.hadoop.hbase.classification.InterfaceAudience;
33  import org.apache.hadoop.hbase.classification.InterfaceStability;
34  import org.apache.hadoop.hbase.client.Connection;
35  import org.apache.hadoop.hbase.client.Get;
36  import org.apache.hadoop.hbase.client.Result;
37  import org.apache.hadoop.hbase.client.Scan;
38  import org.apache.hadoop.hbase.client.Table;
39  import org.apache.hadoop.hbase.filter.CompareFilter;
40  import org.apache.hadoop.hbase.filter.Filter;
41  import org.apache.hadoop.hbase.filter.FilterList;
42  import org.apache.hadoop.hbase.filter.QualifierFilter;
43  import org.apache.hadoop.hbase.filter.RegexStringComparator;
44  import org.apache.hadoop.hbase.filter.RowFilter;
45  import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
46  import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
47  import org.apache.hadoop.hbase.util.Bytes;
48  import org.apache.hadoop.hbase.util.Strings;
49  
50  /**
51   * Helper class to interact with the quota table.
52   * <pre>
53   *     ROW-KEY      FAM/QUAL        DATA
54   *   n.&lt;namespace&gt; q:s         &lt;global-quotas&gt;
55   *   t.&lt;table&gt;     q:s         &lt;global-quotas&gt;
56   *   u.&lt;user&gt;      q:s         &lt;global-quotas&gt;
57   *   u.&lt;user&gt;      q:s.&lt;table&gt; &lt;table-quotas&gt;
58   *   u.&lt;user&gt;      q:s.&lt;ns&gt;:   &lt;namespace-quotas&gt;
59   * </pre>
60   */
61  @InterfaceAudience.Private
62  @InterfaceStability.Evolving
63  public class QuotaTableUtil {
64    private static final Log LOG = LogFactory.getLog(QuotaTableUtil.class);
65  
66    /** System table for quotas */
67    public static final TableName QUOTA_TABLE_NAME =
68        TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "quota");
69  
70    protected static final byte[] QUOTA_FAMILY_INFO = Bytes.toBytes("q");
71    protected static final byte[] QUOTA_FAMILY_USAGE = Bytes.toBytes("u");
72    protected static final byte[] QUOTA_QUALIFIER_SETTINGS = Bytes.toBytes("s");
73    protected static final byte[] QUOTA_QUALIFIER_SETTINGS_PREFIX = Bytes.toBytes("s.");
74    protected static final byte[] QUOTA_USER_ROW_KEY_PREFIX = Bytes.toBytes("u.");
75    protected static final byte[] QUOTA_TABLE_ROW_KEY_PREFIX = Bytes.toBytes("t.");
76    protected static final byte[] QUOTA_NAMESPACE_ROW_KEY_PREFIX = Bytes.toBytes("n.");
77  
78    /* =========================================================================
79     *  Quota "settings" helpers
80     */
81    public static Quotas getTableQuota(final Connection connection, final TableName table)
82        throws IOException {
83      return getQuotas(connection, getTableRowKey(table));
84    }
85  
86    public static Quotas getNamespaceQuota(final Connection connection, final String namespace)
87        throws IOException {
88      return getQuotas(connection, getNamespaceRowKey(namespace));
89    }
90  
91    public static Quotas getUserQuota(final Connection connection, final String user)
92        throws IOException {
93      return getQuotas(connection, getUserRowKey(user));
94    }
95  
96    public static Quotas getUserQuota(final Connection connection, final String user,
97        final TableName table) throws IOException {
98      return getQuotas(connection, getUserRowKey(user), getSettingsQualifierForUserTable(table));
99    }
100 
101   public static Quotas getUserQuota(final Connection connection, final String user,
102       final String namespace) throws IOException {
103     return getQuotas(connection, getUserRowKey(user),
104       getSettingsQualifierForUserNamespace(namespace));
105   }
106 
107   private static Quotas getQuotas(final Connection connection, final byte[] rowKey)
108       throws IOException {
109     return getQuotas(connection, rowKey, QUOTA_QUALIFIER_SETTINGS);
110   }
111 
112   private static Quotas getQuotas(final Connection connection, final byte[] rowKey,
113       final byte[] qualifier) throws IOException {
114     Get get = new Get(rowKey);
115     get.addColumn(QUOTA_FAMILY_INFO, qualifier);
116     Result result = doGet(connection, get);
117     if (result.isEmpty()) {
118       return null;
119     }
120     return quotasFromData(result.getValue(QUOTA_FAMILY_INFO, qualifier));
121   }
122 
123   public static Get makeGetForTableQuotas(final TableName table) {
124     Get get = new Get(getTableRowKey(table));
125     get.addFamily(QUOTA_FAMILY_INFO);
126     return get;
127   }
128 
129   public static Get makeGetForNamespaceQuotas(final String namespace) {
130     Get get = new Get(getNamespaceRowKey(namespace));
131     get.addFamily(QUOTA_FAMILY_INFO);
132     return get;
133   }
134 
135   public static Get makeGetForUserQuotas(final String user, final Iterable<TableName> tables,
136       final Iterable<String> namespaces) {
137     Get get = new Get(getUserRowKey(user));
138     get.addColumn(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
139     for (final TableName table: tables) {
140       get.addColumn(QUOTA_FAMILY_INFO, getSettingsQualifierForUserTable(table));
141     }
142     for (final String ns: namespaces) {
143       get.addColumn(QUOTA_FAMILY_INFO, getSettingsQualifierForUserNamespace(ns));
144     }
145     return get;
146   }
147 
148   public static Scan makeScan(final QuotaFilter filter) {
149     Scan scan = new Scan();
150     scan.addFamily(QUOTA_FAMILY_INFO);
151     if (filter != null && !filter.isNull()) {
152       scan.setFilter(makeFilter(filter));
153     }
154     return scan;
155   }
156 
157   /**
158    * converts quotafilter to serializeable filterlists.
159    */
160   public static Filter makeFilter(final QuotaFilter filter) {
161     FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
162     if (!Strings.isEmpty(filter.getUserFilter())) {
163       FilterList userFilters = new FilterList(FilterList.Operator.MUST_PASS_ONE);
164       boolean hasFilter = false;
165 
166       if (!Strings.isEmpty(filter.getNamespaceFilter())) {
167         FilterList nsFilters = new FilterList(FilterList.Operator.MUST_PASS_ALL);
168         nsFilters.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
169             new RegexStringComparator(getUserRowKeyRegex(filter.getUserFilter()), 0)));
170         nsFilters.addFilter(new QualifierFilter(CompareFilter.CompareOp.EQUAL,
171             new RegexStringComparator(
172               getSettingsQualifierRegexForUserNamespace(filter.getNamespaceFilter()), 0)));
173         userFilters.addFilter(nsFilters);
174         hasFilter = true;
175       }
176       if (!Strings.isEmpty(filter.getTableFilter())) {
177         FilterList tableFilters = new FilterList(FilterList.Operator.MUST_PASS_ALL);
178         tableFilters.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
179             new RegexStringComparator(getUserRowKeyRegex(filter.getUserFilter()), 0)));
180         tableFilters.addFilter(new QualifierFilter(CompareFilter.CompareOp.EQUAL,
181             new RegexStringComparator(
182               getSettingsQualifierRegexForUserTable(filter.getTableFilter()), 0)));
183         userFilters.addFilter(tableFilters);
184         hasFilter = true;
185       }
186       if (!hasFilter) {
187         userFilters.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
188             new RegexStringComparator(getUserRowKeyRegex(filter.getUserFilter()), 0)));
189       }
190 
191       filterList.addFilter(userFilters);
192     } else if (!Strings.isEmpty(filter.getTableFilter())) {
193       filterList.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
194           new RegexStringComparator(getTableRowKeyRegex(filter.getTableFilter()), 0)));
195     } else if (!Strings.isEmpty(filter.getNamespaceFilter())) {
196       filterList.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
197           new RegexStringComparator(getNamespaceRowKeyRegex(filter.getNamespaceFilter()), 0)));
198     }
199     return filterList;
200   }
201 
202   public static interface UserQuotasVisitor {
203     void visitUserQuotas(final String userName, final Quotas quotas)
204       throws IOException;
205     void visitUserQuotas(final String userName, final TableName table, final Quotas quotas)
206       throws IOException;
207     void visitUserQuotas(final String userName, final String namespace, final Quotas quotas)
208       throws IOException;
209   }
210 
211   public static interface TableQuotasVisitor {
212     void visitTableQuotas(final TableName tableName, final Quotas quotas)
213       throws IOException;
214   }
215 
216   public static interface NamespaceQuotasVisitor {
217     void visitNamespaceQuotas(final String namespace, final Quotas quotas)
218       throws IOException;
219   }
220 
221   public static interface QuotasVisitor extends UserQuotasVisitor,
222       TableQuotasVisitor, NamespaceQuotasVisitor {
223   }
224 
225   public static void parseResult(final Result result, final QuotasVisitor visitor)
226       throws IOException {
227     byte[] row = result.getRow();
228     if (isNamespaceRowKey(row)) {
229       parseNamespaceResult(result, visitor);
230     } else if (isTableRowKey(row)) {
231       parseTableResult(result, visitor);
232     } else if (isUserRowKey(row)) {
233       parseUserResult(result, visitor);
234     } else {
235       LOG.warn("unexpected row-key: " + Bytes.toString(row));
236     }
237   }
238 
239   public static void parseNamespaceResult(final Result result,
240       final NamespaceQuotasVisitor visitor) throws IOException {
241     String namespace = getNamespaceFromRowKey(result.getRow());
242     parseNamespaceResult(namespace, result, visitor);
243   }
244 
245   protected static void parseNamespaceResult(final String namespace, final Result result,
246       final NamespaceQuotasVisitor visitor) throws IOException {
247     byte[] data = result.getValue(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
248     if (data != null) {
249       Quotas quotas = quotasFromData(data);
250       visitor.visitNamespaceQuotas(namespace, quotas);
251     }
252   }
253 
254   public static void parseTableResult(final Result result, final TableQuotasVisitor visitor)
255       throws IOException {
256     TableName table = getTableFromRowKey(result.getRow());
257     parseTableResult(table, result, visitor);
258   }
259 
260   protected static void parseTableResult(final TableName table, final Result result,
261       final TableQuotasVisitor visitor) throws IOException {
262     byte[] data = result.getValue(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
263     if (data != null) {
264       Quotas quotas = quotasFromData(data);
265       visitor.visitTableQuotas(table, quotas);
266     }
267   }
268 
269   public static void parseUserResult(final Result result, final UserQuotasVisitor visitor)
270       throws IOException {
271     String userName = getUserFromRowKey(result.getRow());
272     parseUserResult(userName, result, visitor);
273   }
274 
275   protected static void parseUserResult(final String userName, final Result result,
276       final UserQuotasVisitor visitor) throws IOException {
277     Map<byte[], byte[]> familyMap = result.getFamilyMap(QUOTA_FAMILY_INFO);
278     if (familyMap == null || familyMap.isEmpty()) return;
279 
280     for (Map.Entry<byte[], byte[]> entry: familyMap.entrySet()) {
281       Quotas quotas = quotasFromData(entry.getValue());
282       if (Bytes.startsWith(entry.getKey(), QUOTA_QUALIFIER_SETTINGS_PREFIX)) {
283         String name = Bytes.toString(entry.getKey(), QUOTA_QUALIFIER_SETTINGS_PREFIX.length);
284         if (name.charAt(name.length() - 1) == TableName.NAMESPACE_DELIM) {
285           String namespace = name.substring(0, name.length() - 1);
286           visitor.visitUserQuotas(userName, namespace, quotas);
287         } else {
288           TableName table = TableName.valueOf(name);
289           visitor.visitUserQuotas(userName, table, quotas);
290         }
291       } else if (Bytes.equals(entry.getKey(), QUOTA_QUALIFIER_SETTINGS)) {
292         visitor.visitUserQuotas(userName, quotas);
293       }
294     }
295   }
296 
297   /* =========================================================================
298    *  Quotas protobuf helpers
299    */
300   protected static Quotas quotasFromData(final byte[] data) throws IOException {
301     int magicLen = ProtobufMagic.lengthOfPBMagic();
302     if (!ProtobufMagic.isPBMagicPrefix(data, 0, magicLen)) {
303       throw new IOException("Missing pb magic prefix");
304     }
305     return Quotas.parseFrom(new ByteArrayInputStream(data, magicLen, data.length - magicLen));
306   }
307 
308   protected static byte[] quotasToData(final Quotas data) throws IOException {
309     ByteArrayOutputStream stream = new ByteArrayOutputStream();
310     stream.write(ProtobufMagic.PB_MAGIC);
311     data.writeTo(stream);
312     return stream.toByteArray();
313   }
314 
315   public static boolean isEmptyQuota(final Quotas quotas) {
316     boolean hasSettings = false;
317     hasSettings |= quotas.hasThrottle();
318     hasSettings |= quotas.hasBypassGlobals();
319     return !hasSettings;
320   }
321 
322   /* =========================================================================
323    *  HTable helpers
324    */
325   protected static Result doGet(final Connection connection, final Get get)
326       throws IOException {
327     try (Table table = connection.getTable(QUOTA_TABLE_NAME)) {
328       return table.get(get);
329     }
330   }
331 
332   protected static Result[] doGet(final Connection connection, final List<Get> gets)
333       throws IOException {
334     try (Table table = connection.getTable(QUOTA_TABLE_NAME)) {
335       return table.get(gets);
336     }
337   }
338 
339   /* =========================================================================
340    *  Quota table row key helpers
341    */
342   protected static byte[] getUserRowKey(final String user) {
343     return Bytes.add(QUOTA_USER_ROW_KEY_PREFIX, Bytes.toBytes(user));
344   }
345 
346   protected static byte[] getTableRowKey(final TableName table) {
347     return Bytes.add(QUOTA_TABLE_ROW_KEY_PREFIX, table.getName());
348   }
349 
350   protected static byte[] getNamespaceRowKey(final String namespace) {
351     return Bytes.add(QUOTA_NAMESPACE_ROW_KEY_PREFIX, Bytes.toBytes(namespace));
352   }
353 
354   protected static byte[] getSettingsQualifierForUserTable(final TableName tableName) {
355     return Bytes.add(QUOTA_QUALIFIER_SETTINGS_PREFIX, tableName.getName());
356   }
357 
358   protected static byte[] getSettingsQualifierForUserNamespace(final String namespace) {
359     return Bytes.add(QUOTA_QUALIFIER_SETTINGS_PREFIX,
360         Bytes.toBytes(namespace + TableName.NAMESPACE_DELIM));
361   }
362 
363   protected static String getUserRowKeyRegex(final String user) {
364     return getRowKeyRegEx(QUOTA_USER_ROW_KEY_PREFIX, user);
365   }
366 
367   protected static String getTableRowKeyRegex(final String table) {
368     return getRowKeyRegEx(QUOTA_TABLE_ROW_KEY_PREFIX, table);
369   }
370 
371   protected static String getNamespaceRowKeyRegex(final String namespace) {
372     return getRowKeyRegEx(QUOTA_NAMESPACE_ROW_KEY_PREFIX, namespace);
373   }
374 
375   private static String getRowKeyRegEx(final byte[] prefix, final String regex) {
376     return '^' + Pattern.quote(Bytes.toString(prefix)) + regex + '$';
377   }
378 
379   protected static String getSettingsQualifierRegexForUserTable(final String table) {
380     return '^' + Pattern.quote(Bytes.toString(QUOTA_QUALIFIER_SETTINGS_PREFIX)) +
381           table + "(?<!" + Pattern.quote(Character.toString(TableName.NAMESPACE_DELIM)) + ")$";
382   }
383 
384   protected static String getSettingsQualifierRegexForUserNamespace(final String namespace) {
385     return '^' + Pattern.quote(Bytes.toString(QUOTA_QUALIFIER_SETTINGS_PREFIX)) +
386                   namespace + Pattern.quote(Character.toString(TableName.NAMESPACE_DELIM)) + '$';
387   }
388 
389   protected static boolean isNamespaceRowKey(final byte[] key) {
390     return Bytes.startsWith(key, QUOTA_NAMESPACE_ROW_KEY_PREFIX);
391   }
392 
393   protected static String getNamespaceFromRowKey(final byte[] key) {
394     return Bytes.toString(key, QUOTA_NAMESPACE_ROW_KEY_PREFIX.length);
395   }
396 
397   protected static boolean isTableRowKey(final byte[] key) {
398     return Bytes.startsWith(key, QUOTA_TABLE_ROW_KEY_PREFIX);
399   }
400 
401   protected static TableName getTableFromRowKey(final byte[] key) {
402     return TableName.valueOf(Bytes.toString(key, QUOTA_TABLE_ROW_KEY_PREFIX.length));
403   }
404 
405   protected static boolean isUserRowKey(final byte[] key) {
406     return Bytes.startsWith(key, QUOTA_USER_ROW_KEY_PREFIX);
407   }
408 
409   protected static String getUserFromRowKey(final byte[] key) {
410     return Bytes.toString(key, QUOTA_USER_ROW_KEY_PREFIX.length);
411   }
412 }