001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.quotas;
019
020import java.io.IOException;
021import java.util.Arrays;
022import java.util.HashMap;
023import java.util.List;
024import java.util.Map;
025import java.util.Optional;
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.hbase.Cell;
028import org.apache.hadoop.hbase.DoNotRetryIOException;
029import org.apache.hadoop.hbase.HColumnDescriptor;
030import org.apache.hadoop.hbase.HConstants;
031import org.apache.hadoop.hbase.HTableDescriptor;
032import org.apache.hadoop.hbase.TableName;
033import org.apache.hadoop.hbase.TableNotDisabledException;
034import org.apache.hadoop.hbase.TableNotEnabledException;
035import org.apache.hadoop.hbase.TableNotFoundException;
036import org.apache.hadoop.hbase.client.Connection;
037import org.apache.hadoop.hbase.client.Delete;
038import org.apache.hadoop.hbase.client.Get;
039import org.apache.hadoop.hbase.client.Mutation;
040import org.apache.hadoop.hbase.client.Put;
041import org.apache.hadoop.hbase.client.Result;
042import org.apache.hadoop.hbase.client.Table;
043import org.apache.hadoop.hbase.regionserver.BloomType;
044import org.apache.hadoop.hbase.util.Bytes;
045import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
046import org.apache.hadoop.hbase.util.Pair;
047import org.apache.yetus.audience.InterfaceAudience;
048import org.apache.yetus.audience.InterfaceStability;
049import org.slf4j.Logger;
050import org.slf4j.LoggerFactory;
051
052import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
053import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
054import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeUnit;
055import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
056import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaScope;
057import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
058import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
059import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
060
061/**
062 * Helper class to interact with the quota table
063 */
064@InterfaceAudience.Private
065@InterfaceStability.Evolving
066public class QuotaUtil extends QuotaTableUtil {
067  private static final Logger LOG = LoggerFactory.getLogger(QuotaUtil.class);
068
069  public static final String QUOTA_CONF_KEY = "hbase.quota.enabled";
070  private static final boolean QUOTA_ENABLED_DEFAULT = false;
071
072  public static final String READ_CAPACITY_UNIT_CONF_KEY = "hbase.quota.read.capacity.unit";
073  // the default one read capacity unit is 1024 bytes (1KB)
074  public static final long DEFAULT_READ_CAPACITY_UNIT = 1024;
075  public static final String WRITE_CAPACITY_UNIT_CONF_KEY = "hbase.quota.write.capacity.unit";
076  // the default one write capacity unit is 1024 bytes (1KB)
077  public static final long DEFAULT_WRITE_CAPACITY_UNIT = 1024;
078
079  /*
080   * The below defaults, if configured, will be applied to otherwise unthrottled users. For example,
081   * set `hbase.quota.default.user.machine.read.size` to `1048576` in your hbase-site.xml to ensure
082   * that any given user may not query more than 1mb per second from any given machine, unless
083   * explicitly permitted by a persisted quota. All of these defaults use TimeUnit.SECONDS and
084   * QuotaScope.MACHINE.
085   */
086  public static final String QUOTA_DEFAULT_USER_MACHINE_READ_NUM =
087    "hbase.quota.default.user.machine.read.num";
088  public static final String QUOTA_DEFAULT_USER_MACHINE_READ_SIZE =
089    "hbase.quota.default.user.machine.read.size";
090  public static final String QUOTA_DEFAULT_USER_MACHINE_REQUEST_NUM =
091    "hbase.quota.default.user.machine.request.num";
092  public static final String QUOTA_DEFAULT_USER_MACHINE_REQUEST_SIZE =
093    "hbase.quota.default.user.machine.request.size";
094  public static final String QUOTA_DEFAULT_USER_MACHINE_WRITE_NUM =
095    "hbase.quota.default.user.machine.write.num";
096  public static final String QUOTA_DEFAULT_USER_MACHINE_WRITE_SIZE =
097    "hbase.quota.default.user.machine.write.size";
098
099  /** Table descriptor for Quota internal table */
100  public static final HTableDescriptor QUOTA_TABLE_DESC = new HTableDescriptor(QUOTA_TABLE_NAME);
101  static {
102    QUOTA_TABLE_DESC.addFamily(
103      new HColumnDescriptor(QUOTA_FAMILY_INFO).setScope(HConstants.REPLICATION_SCOPE_LOCAL)
104        .setBloomFilterType(BloomType.ROW).setMaxVersions(1));
105    QUOTA_TABLE_DESC.addFamily(
106      new HColumnDescriptor(QUOTA_FAMILY_USAGE).setScope(HConstants.REPLICATION_SCOPE_LOCAL)
107        .setBloomFilterType(BloomType.ROW).setMaxVersions(1));
108  }
109
110  /** Returns true if the support for quota is enabled */
111  public static boolean isQuotaEnabled(final Configuration conf) {
112    return conf.getBoolean(QUOTA_CONF_KEY, QUOTA_ENABLED_DEFAULT);
113  }
114
115  /*
116   * ========================================================================= Quota "settings"
117   * helpers
118   */
119  public static void addTableQuota(final Connection connection, final TableName table,
120    final Quotas data) throws IOException {
121    addQuotas(connection, getTableRowKey(table), data);
122  }
123
124  public static void deleteTableQuota(final Connection connection, final TableName table)
125    throws IOException {
126    deleteQuotas(connection, getTableRowKey(table));
127  }
128
129  public static void addNamespaceQuota(final Connection connection, final String namespace,
130    final Quotas data) throws IOException {
131    addQuotas(connection, getNamespaceRowKey(namespace), data);
132  }
133
134  public static void deleteNamespaceQuota(final Connection connection, final String namespace)
135    throws IOException {
136    deleteQuotas(connection, getNamespaceRowKey(namespace));
137  }
138
139  public static void addUserQuota(final Connection connection, final String user, final Quotas data)
140    throws IOException {
141    addQuotas(connection, getUserRowKey(user), data);
142  }
143
144  public static void addUserQuota(final Connection connection, final String user,
145    final TableName table, final Quotas data) throws IOException {
146    addQuotas(connection, getUserRowKey(user), getSettingsQualifierForUserTable(table), data);
147  }
148
149  public static void addUserQuota(final Connection connection, final String user,
150    final String namespace, final Quotas data) throws IOException {
151    addQuotas(connection, getUserRowKey(user), getSettingsQualifierForUserNamespace(namespace),
152      data);
153  }
154
155  public static void deleteUserQuota(final Connection connection, final String user)
156    throws IOException {
157    deleteQuotas(connection, getUserRowKey(user));
158  }
159
160  public static void deleteUserQuota(final Connection connection, final String user,
161    final TableName table) throws IOException {
162    deleteQuotas(connection, getUserRowKey(user), getSettingsQualifierForUserTable(table));
163  }
164
165  public static void deleteUserQuota(final Connection connection, final String user,
166    final String namespace) throws IOException {
167    deleteQuotas(connection, getUserRowKey(user), getSettingsQualifierForUserNamespace(namespace));
168  }
169
170  public static void addRegionServerQuota(final Connection connection, final String regionServer,
171    final Quotas data) throws IOException {
172    addQuotas(connection, getRegionServerRowKey(regionServer), data);
173  }
174
175  public static void deleteRegionServerQuota(final Connection connection, final String regionServer)
176    throws IOException {
177    deleteQuotas(connection, getRegionServerRowKey(regionServer));
178  }
179
180  public static OperationQuota.OperationType getQuotaOperationType(ClientProtos.Action action,
181    boolean hasCondition) {
182    if (action.hasMutation()) {
183      return getQuotaOperationType(action.getMutation(), hasCondition);
184    }
185    return OperationQuota.OperationType.GET;
186  }
187
188  public static OperationQuota.OperationType
189    getQuotaOperationType(ClientProtos.MutateRequest mutateRequest) {
190    return getQuotaOperationType(mutateRequest.getMutation(), mutateRequest.hasCondition());
191  }
192
193  private static OperationQuota.OperationType
194    getQuotaOperationType(ClientProtos.MutationProto mutationProto, boolean hasCondition) {
195    ClientProtos.MutationProto.MutationType mutationType = mutationProto.getMutateType();
196    if (
197      hasCondition || mutationType == ClientProtos.MutationProto.MutationType.APPEND
198        || mutationType == ClientProtos.MutationProto.MutationType.INCREMENT
199    ) {
200      return OperationQuota.OperationType.CHECK_AND_MUTATE;
201    }
202    return OperationQuota.OperationType.MUTATE;
203  }
204
205  protected static void switchExceedThrottleQuota(final Connection connection,
206    boolean exceedThrottleQuotaEnabled) throws IOException {
207    if (exceedThrottleQuotaEnabled) {
208      checkRSQuotaToEnableExceedThrottle(
209        getRegionServerQuota(connection, QuotaTableUtil.QUOTA_REGION_SERVER_ROW_KEY));
210    }
211
212    Put put = new Put(getExceedThrottleQuotaRowKey());
213    put.addColumn(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS,
214      Bytes.toBytes(exceedThrottleQuotaEnabled));
215    doPut(connection, put);
216  }
217
218  private static void checkRSQuotaToEnableExceedThrottle(Quotas quotas) throws IOException {
219    if (quotas != null && quotas.hasThrottle()) {
220      Throttle throttle = quotas.getThrottle();
221      // If enable exceed throttle quota, make sure that there are at least one read(req/read +
222      // num/size/cu) and one write(req/write + num/size/cu) region server throttle quotas.
223      boolean hasReadQuota = false;
224      boolean hasWriteQuota = false;
225      if (throttle.hasReqNum() || throttle.hasReqSize() || throttle.hasReqCapacityUnit()) {
226        hasReadQuota = true;
227        hasWriteQuota = true;
228      }
229      if (
230        !hasReadQuota
231          && (throttle.hasReadNum() || throttle.hasReadSize() || throttle.hasReadCapacityUnit())
232      ) {
233        hasReadQuota = true;
234      }
235      if (!hasReadQuota) {
236        throw new DoNotRetryIOException(
237          "Please set at least one read region server quota before enable exceed throttle quota");
238      }
239      if (
240        !hasWriteQuota
241          && (throttle.hasWriteNum() || throttle.hasWriteSize() || throttle.hasWriteCapacityUnit())
242      ) {
243        hasWriteQuota = true;
244      }
245      if (!hasWriteQuota) {
246        throw new DoNotRetryIOException("Please set at least one write region server quota "
247          + "before enable exceed throttle quota");
248      }
249      // If enable exceed throttle quota, make sure that region server throttle quotas are in
250      // seconds time unit. Because once previous requests exceed their quota and consume region
251      // server quota, quota in other time units may be refilled in a long time, this may affect
252      // later requests.
253      List<Pair<Boolean, TimedQuota>> list =
254        Arrays.asList(Pair.newPair(throttle.hasReqNum(), throttle.getReqNum()),
255          Pair.newPair(throttle.hasReadNum(), throttle.getReadNum()),
256          Pair.newPair(throttle.hasWriteNum(), throttle.getWriteNum()),
257          Pair.newPair(throttle.hasReqSize(), throttle.getReqSize()),
258          Pair.newPair(throttle.hasReadSize(), throttle.getReadSize()),
259          Pair.newPair(throttle.hasWriteSize(), throttle.getWriteSize()),
260          Pair.newPair(throttle.hasReqCapacityUnit(), throttle.getReqCapacityUnit()),
261          Pair.newPair(throttle.hasReadCapacityUnit(), throttle.getReadCapacityUnit()),
262          Pair.newPair(throttle.hasWriteCapacityUnit(), throttle.getWriteCapacityUnit()));
263      for (Pair<Boolean, TimedQuota> pair : list) {
264        if (pair.getFirst()) {
265          if (pair.getSecond().getTimeUnit() != TimeUnit.SECONDS) {
266            throw new DoNotRetryIOException("All region server quota must be "
267              + "in seconds time unit if enable exceed throttle quota");
268          }
269        }
270      }
271    } else {
272      // If enable exceed throttle quota, make sure that region server quota is already set
273      throw new DoNotRetryIOException(
274        "Please set region server quota before enable exceed throttle quota");
275    }
276  }
277
278  protected static boolean isExceedThrottleQuotaEnabled(final Connection connection)
279    throws IOException {
280    Get get = new Get(getExceedThrottleQuotaRowKey());
281    get.addColumn(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
282    Result result = doGet(connection, get);
283    if (result.isEmpty()) {
284      return false;
285    }
286    return Bytes.toBoolean(result.getValue(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS));
287  }
288
289  private static void addQuotas(final Connection connection, final byte[] rowKey, final Quotas data)
290    throws IOException {
291    addQuotas(connection, rowKey, QUOTA_QUALIFIER_SETTINGS, data);
292  }
293
294  private static void addQuotas(final Connection connection, final byte[] rowKey,
295    final byte[] qualifier, final Quotas data) throws IOException {
296    Put put = new Put(rowKey);
297    put.addColumn(QUOTA_FAMILY_INFO, qualifier, quotasToData(data));
298    doPut(connection, put);
299  }
300
301  private static void deleteQuotas(final Connection connection, final byte[] rowKey)
302    throws IOException {
303    deleteQuotas(connection, rowKey, null);
304  }
305
306  private static void deleteQuotas(final Connection connection, final byte[] rowKey,
307    final byte[] qualifier) throws IOException {
308    Delete delete = new Delete(rowKey);
309    if (qualifier != null) {
310      delete.addColumns(QUOTA_FAMILY_INFO, qualifier);
311    }
312    if (isNamespaceRowKey(rowKey)) {
313      String ns = getNamespaceFromRowKey(rowKey);
314      Quotas namespaceQuota = getNamespaceQuota(connection, ns);
315      if (namespaceQuota != null && namespaceQuota.hasSpace()) {
316        // When deleting namespace space quota, also delete table usage(u:p) snapshots
317        deleteTableUsageSnapshotsForNamespace(connection, ns);
318      }
319    }
320    doDelete(connection, delete);
321  }
322
323  public static Map<String, UserQuotaState> fetchUserQuotas(final Connection connection,
324    final List<Get> gets, Map<TableName, Double> tableMachineQuotaFactors, double factor)
325    throws IOException {
326    long nowTs = EnvironmentEdgeManager.currentTime();
327    Result[] results = doGet(connection, gets);
328
329    Map<String, UserQuotaState> userQuotas = new HashMap<>(results.length);
330    for (int i = 0; i < results.length; ++i) {
331      byte[] key = gets.get(i).getRow();
332      assert isUserRowKey(key);
333      String user = getUserFromRowKey(key);
334
335      if (results[i].isEmpty()) {
336        userQuotas.put(user, buildDefaultUserQuotaState(connection.getConfiguration(), nowTs));
337        continue;
338      }
339
340      final UserQuotaState quotaInfo = new UserQuotaState(nowTs);
341      userQuotas.put(user, quotaInfo);
342
343      assert Bytes.equals(key, results[i].getRow());
344
345      try {
346        parseUserResult(user, results[i], new UserQuotasVisitor() {
347          @Override
348          public void visitUserQuotas(String userName, String namespace, Quotas quotas) {
349            quotas = updateClusterQuotaToMachineQuota(quotas, factor);
350            quotaInfo.setQuotas(namespace, quotas);
351          }
352
353          @Override
354          public void visitUserQuotas(String userName, TableName table, Quotas quotas) {
355            quotas = updateClusterQuotaToMachineQuota(quotas,
356              tableMachineQuotaFactors.containsKey(table)
357                ? tableMachineQuotaFactors.get(table)
358                : 1);
359            quotaInfo.setQuotas(table, quotas);
360          }
361
362          @Override
363          public void visitUserQuotas(String userName, Quotas quotas) {
364            quotas = updateClusterQuotaToMachineQuota(quotas, factor);
365            quotaInfo.setQuotas(quotas);
366          }
367        });
368      } catch (IOException e) {
369        LOG.error("Unable to parse user '" + user + "' quotas", e);
370        userQuotas.remove(user);
371      }
372    }
373    return userQuotas;
374  }
375
376  protected static UserQuotaState buildDefaultUserQuotaState(Configuration conf, long nowTs) {
377    QuotaProtos.Throttle.Builder throttleBuilder = QuotaProtos.Throttle.newBuilder();
378
379    buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_READ_NUM)
380      .ifPresent(throttleBuilder::setReadNum);
381    buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_READ_SIZE)
382      .ifPresent(throttleBuilder::setReadSize);
383    buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_REQUEST_NUM)
384      .ifPresent(throttleBuilder::setReqNum);
385    buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_REQUEST_SIZE)
386      .ifPresent(throttleBuilder::setReqSize);
387    buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_WRITE_NUM)
388      .ifPresent(throttleBuilder::setWriteNum);
389    buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_WRITE_SIZE)
390      .ifPresent(throttleBuilder::setWriteSize);
391
392    UserQuotaState state = new UserQuotaState(nowTs);
393    QuotaProtos.Quotas defaultQuotas =
394      QuotaProtos.Quotas.newBuilder().setThrottle(throttleBuilder.build()).build();
395    state.setQuotas(defaultQuotas);
396    return state;
397  }
398
399  private static Optional<TimedQuota> buildDefaultTimedQuota(Configuration conf, String key) {
400    int defaultSoftLimit = conf.getInt(key, -1);
401    if (defaultSoftLimit == -1) {
402      return Optional.empty();
403    }
404    return Optional.of(ProtobufUtil.toTimedQuota(defaultSoftLimit,
405      java.util.concurrent.TimeUnit.SECONDS, org.apache.hadoop.hbase.quotas.QuotaScope.MACHINE));
406  }
407
408  public static Map<TableName, QuotaState> fetchTableQuotas(final Connection connection,
409    final List<Get> gets, Map<TableName, Double> tableMachineFactors) throws IOException {
410    return fetchGlobalQuotas("table", connection, gets, new KeyFromRow<TableName>() {
411      @Override
412      public TableName getKeyFromRow(final byte[] row) {
413        assert isTableRowKey(row);
414        return getTableFromRowKey(row);
415      }
416
417      @Override
418      public double getFactor(TableName tableName) {
419        return tableMachineFactors.containsKey(tableName) ? tableMachineFactors.get(tableName) : 1;
420      }
421    });
422  }
423
424  public static Map<String, QuotaState> fetchNamespaceQuotas(final Connection connection,
425    final List<Get> gets, double factor) throws IOException {
426    return fetchGlobalQuotas("namespace", connection, gets, new KeyFromRow<String>() {
427      @Override
428      public String getKeyFromRow(final byte[] row) {
429        assert isNamespaceRowKey(row);
430        return getNamespaceFromRowKey(row);
431      }
432
433      @Override
434      public double getFactor(String s) {
435        return factor;
436      }
437    });
438  }
439
440  public static Map<String, QuotaState> fetchRegionServerQuotas(final Connection connection,
441    final List<Get> gets) throws IOException {
442    return fetchGlobalQuotas("regionServer", connection, gets, new KeyFromRow<String>() {
443      @Override
444      public String getKeyFromRow(final byte[] row) {
445        assert isRegionServerRowKey(row);
446        return getRegionServerFromRowKey(row);
447      }
448
449      @Override
450      public double getFactor(String s) {
451        return 1;
452      }
453    });
454  }
455
456  public static <K> Map<K, QuotaState> fetchGlobalQuotas(final String type,
457    final Connection connection, final List<Get> gets, final KeyFromRow<K> kfr) throws IOException {
458    long nowTs = EnvironmentEdgeManager.currentTime();
459    Result[] results = doGet(connection, gets);
460
461    Map<K, QuotaState> globalQuotas = new HashMap<>(results.length);
462    for (int i = 0; i < results.length; ++i) {
463      byte[] row = gets.get(i).getRow();
464      K key = kfr.getKeyFromRow(row);
465
466      QuotaState quotaInfo = new QuotaState(nowTs);
467      globalQuotas.put(key, quotaInfo);
468
469      if (results[i].isEmpty()) continue;
470      assert Bytes.equals(row, results[i].getRow());
471
472      byte[] data = results[i].getValue(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
473      if (data == null) continue;
474
475      try {
476        Quotas quotas = quotasFromData(data);
477        quotas = updateClusterQuotaToMachineQuota(quotas, kfr.getFactor(key));
478        quotaInfo.setQuotas(quotas);
479      } catch (IOException e) {
480        LOG.error("Unable to parse " + type + " '" + key + "' quotas", e);
481        globalQuotas.remove(key);
482      }
483    }
484    return globalQuotas;
485  }
486
487  /**
488   * Convert cluster scope quota to machine scope quota
489   * @param quotas the original quota
490   * @param factor factor used to divide cluster limiter to machine limiter
491   * @return the converted quota whose quota limiters all in machine scope
492   */
493  private static Quotas updateClusterQuotaToMachineQuota(Quotas quotas, double factor) {
494    Quotas.Builder newQuotas = Quotas.newBuilder(quotas);
495    if (newQuotas.hasThrottle()) {
496      Throttle.Builder throttle = Throttle.newBuilder(newQuotas.getThrottle());
497      if (throttle.hasReqNum()) {
498        throttle.setReqNum(updateTimedQuota(throttle.getReqNum(), factor));
499      }
500      if (throttle.hasReqSize()) {
501        throttle.setReqSize(updateTimedQuota(throttle.getReqSize(), factor));
502      }
503      if (throttle.hasReadNum()) {
504        throttle.setReadNum(updateTimedQuota(throttle.getReadNum(), factor));
505      }
506      if (throttle.hasReadSize()) {
507        throttle.setReadSize(updateTimedQuota(throttle.getReadSize(), factor));
508      }
509      if (throttle.hasWriteNum()) {
510        throttle.setWriteNum(updateTimedQuota(throttle.getWriteNum(), factor));
511      }
512      if (throttle.hasWriteSize()) {
513        throttle.setWriteSize(updateTimedQuota(throttle.getWriteSize(), factor));
514      }
515      if (throttle.hasReqCapacityUnit()) {
516        throttle.setReqCapacityUnit(updateTimedQuota(throttle.getReqCapacityUnit(), factor));
517      }
518      if (throttle.hasReadCapacityUnit()) {
519        throttle.setReadCapacityUnit(updateTimedQuota(throttle.getReadCapacityUnit(), factor));
520      }
521      if (throttle.hasWriteCapacityUnit()) {
522        throttle.setWriteCapacityUnit(updateTimedQuota(throttle.getWriteCapacityUnit(), factor));
523      }
524      newQuotas.setThrottle(throttle.build());
525    }
526    return newQuotas.build();
527  }
528
529  private static TimedQuota updateTimedQuota(TimedQuota timedQuota, double factor) {
530    if (timedQuota.getScope() == QuotaScope.CLUSTER) {
531      TimedQuota.Builder newTimedQuota = TimedQuota.newBuilder(timedQuota);
532      newTimedQuota.setSoftLimit(Math.max(1, (long) (timedQuota.getSoftLimit() * factor)))
533        .setScope(QuotaScope.MACHINE);
534      return newTimedQuota.build();
535    } else {
536      return timedQuota;
537    }
538  }
539
540  private static interface KeyFromRow<T> {
541    T getKeyFromRow(final byte[] row);
542
543    double getFactor(T t);
544  }
545
546  /*
547   * ========================================================================= HTable helpers
548   */
549  private static void doPut(final Connection connection, final Put put) throws IOException {
550    try (Table table = connection.getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
551      table.put(put);
552    }
553  }
554
555  private static void doDelete(final Connection connection, final Delete delete)
556    throws IOException {
557    try (Table table = connection.getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
558      table.delete(delete);
559    }
560  }
561
562  /*
563   * ========================================================================= Data Size Helpers
564   */
565  public static long calculateMutationSize(final Mutation mutation) {
566    long size = 0;
567    for (Map.Entry<byte[], List<Cell>> entry : mutation.getFamilyCellMap().entrySet()) {
568      for (Cell cell : entry.getValue()) {
569        size += cell.getSerializedSize();
570      }
571    }
572    return size;
573  }
574
575  public static long calculateResultSize(final Result result) {
576    long size = 0;
577    for (Cell cell : result.rawCells()) {
578      size += cell.getSerializedSize();
579    }
580    return size;
581  }
582
583  public static long calculateResultSize(final List<Result> results) {
584    long size = 0;
585    for (Result result : results) {
586      for (Cell cell : result.rawCells()) {
587        size += cell.getSerializedSize();
588      }
589    }
590    return size;
591  }
592
593  /**
594   * Method to enable a table, if not already enabled. This method suppresses
595   * {@link TableNotDisabledException} and {@link TableNotFoundException}, if thrown while enabling
596   * the table.
597   * @param conn      connection to re-use
598   * @param tableName name of the table to be enabled
599   */
600  public static void enableTableIfNotEnabled(Connection conn, TableName tableName)
601    throws IOException {
602    try {
603      conn.getAdmin().enableTable(tableName);
604    } catch (TableNotDisabledException | TableNotFoundException e) {
605      // ignore
606    }
607  }
608
609  /**
610   * Method to disable a table, if not already disabled. This method suppresses
611   * {@link TableNotEnabledException}, if thrown while disabling the table.
612   * @param conn      connection to re-use
613   * @param tableName table name which has moved into space quota violation
614   */
615  public static void disableTableIfNotDisabled(Connection conn, TableName tableName)
616    throws IOException {
617    try {
618      conn.getAdmin().disableTable(tableName);
619    } catch (TableNotEnabledException | TableNotFoundException e) {
620      // ignore
621    }
622  }
623}