001/*
002 * Licensed to the Apache Software Foundation (ASF) under one or more
003 * contributor license agreements.  See the NOTICE file distributed with
004 * this work for additional information regarding copyright ownership.
005 * The ASF licenses this file to you under the Apache License, Version 2.0
006 * (the "License"); you may not use this file except in compliance with
007 * the License.  You may obtain a copy of the License at
008 *
009 * http://www.apache.org/licenses/LICENSE-2.0
010 *
011 * Unless required by applicable law or agreed to in writing, software
012 * distributed under the License is distributed on an "AS IS" BASIS,
013 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
014 * See the License for the specific language governing permissions and
015 * limitations under the License.
016 */
017package org.apache.hadoop.hbase.quotas;
018
019import static org.junit.Assert.assertEquals;
020import static org.junit.Assert.assertTrue;
021import static org.junit.Assert.fail;
022
023import java.io.IOException;
024import java.util.ArrayList;
025import java.util.Collection;
026import java.util.List;
027import java.util.Map.Entry;
028import java.util.Objects;
029import java.util.Random;
030import java.util.Set;
031import java.util.concurrent.atomic.AtomicLong;
032
033import org.apache.hadoop.conf.Configuration;
034import org.apache.hadoop.fs.FileSystem;
035import org.apache.hadoop.fs.Path;
036import org.apache.hadoop.hbase.HBaseTestingUtility;
037import org.apache.hadoop.hbase.HConstants;
038import org.apache.hadoop.hbase.MiniHBaseCluster;
039import org.apache.hadoop.hbase.NamespaceDescriptor;
040import org.apache.hadoop.hbase.TableName;
041import org.apache.hadoop.hbase.TableNotEnabledException;
042import org.apache.hadoop.hbase.Waiter.Predicate;
043import org.apache.hadoop.hbase.client.Admin;
044import org.apache.hadoop.hbase.client.Append;
045import org.apache.hadoop.hbase.client.ClientServiceCallable;
046import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
047import org.apache.hadoop.hbase.client.Connection;
048import org.apache.hadoop.hbase.client.Delete;
049import org.apache.hadoop.hbase.client.Increment;
050import org.apache.hadoop.hbase.client.Mutation;
051import org.apache.hadoop.hbase.client.Put;
052import org.apache.hadoop.hbase.client.Result;
053import org.apache.hadoop.hbase.client.ResultScanner;
054import org.apache.hadoop.hbase.client.Scan;
055import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
056import org.apache.hadoop.hbase.client.Table;
057import org.apache.hadoop.hbase.client.TableDescriptor;
058import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
059import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
060import org.apache.hadoop.hbase.regionserver.HRegion;
061import org.apache.hadoop.hbase.regionserver.HStore;
062import org.apache.hadoop.hbase.regionserver.HStoreFile;
063import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
064import org.apache.hadoop.hbase.util.Bytes;
065import org.apache.hadoop.hbase.util.Pair;
066import org.apache.hadoop.util.StringUtils;
067import org.apache.yetus.audience.InterfaceAudience;
068import org.junit.rules.TestName;
069import org.slf4j.Logger;
070import org.slf4j.LoggerFactory;
071import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap;
072import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
073import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
074
075@InterfaceAudience.Private
076public class SpaceQuotaHelperForTests {
077  private static final Logger LOG = LoggerFactory.getLogger(SpaceQuotaHelperForTests.class);
078
079  public static final int SIZE_PER_VALUE = 256;
080  public static final String F1 = "f1";
081  public static final long ONE_KILOBYTE = 1024L;
082  public static final long ONE_MEGABYTE = ONE_KILOBYTE * ONE_KILOBYTE;
083  public static final long ONE_GIGABYTE = ONE_MEGABYTE * ONE_KILOBYTE;
084
085  private final HBaseTestingUtility testUtil;
086  private final TestName testName;
087  private final AtomicLong counter;
088  private static final int NUM_RETRIES = 10;
089
090  public SpaceQuotaHelperForTests(
091      HBaseTestingUtility testUtil, TestName testName, AtomicLong counter) {
092    this.testUtil = Objects.requireNonNull(testUtil);
093    this.testName = Objects.requireNonNull(testName);
094    this.counter = Objects.requireNonNull(counter);
095  }
096
097  //
098  // Static helpers
099  //
100
101  static void updateConfigForQuotas(Configuration conf) {
102    // Increase the frequency of some of the chores for responsiveness of the test
103    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_KEY, 1000);
104    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 1000);
105    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_DELAY_KEY, 1000);
106    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_PERIOD_KEY, 1000);
107    conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_DELAY_KEY, 1000);
108    conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_PERIOD_KEY, 1000);
109    conf.setInt(SnapshotQuotaObserverChore.SNAPSHOT_QUOTA_CHORE_DELAY_KEY, 1000);
110    conf.setInt(SnapshotQuotaObserverChore.SNAPSHOT_QUOTA_CHORE_PERIOD_KEY, 1000);
111    conf.setInt(RegionSizeReportingChore.REGION_SIZE_REPORTING_CHORE_PERIOD_KEY, 1000);
112    conf.setInt(RegionSizeReportingChore.REGION_SIZE_REPORTING_CHORE_DELAY_KEY, 1000);
113    // The period at which we check for compacted files that should be deleted from HDFS
114    conf.setInt("hbase.hfile.compaction.discharger.interval", 5 * 1000);
115    conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
116  }
117
118  //
119  // Helpers
120  //
121
122  /**
123   * Returns the number of quotas defined in the HBase quota table.
124   */
125  long listNumDefinedQuotas(Connection conn) throws IOException {
126    QuotaRetriever scanner = QuotaRetriever.open(conn.getConfiguration());
127    try {
128      return Iterables.size(scanner);
129    } finally {
130      if (scanner != null) {
131        scanner.close();
132      }
133    }
134  }
135
136  /**
137   * Writes the given mutation into a table until it violates the given policy.
138   * Verifies that the policy has been violated & then returns the name of
139   * the table created & written into.
140   */
141  TableName writeUntilViolationAndVerifyViolation(
142      SpaceViolationPolicy policyToViolate, Mutation m) throws Exception {
143    final TableName tn = writeUntilViolation(policyToViolate);
144    verifyViolation(policyToViolate, tn, m);
145    return tn;
146  }
147
148  /**
149   * Writes the given mutation into a table until it violates the given policy.
150   * Returns the name of the table created & written into.
151   */
152  TableName writeUntilViolation(SpaceViolationPolicy policyToViolate) throws Exception {
153    TableName tn = createTableWithRegions(10);
154    setQuotaLimit(tn, policyToViolate, 2L);
155    // Write more data than should be allowed and flush it to disk
156    writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
157
158    // This should be sufficient time for the chores to run and see the change.
159    Thread.sleep(5000);
160
161    return tn;
162  }
163
164  /**
165   * Verifies that the given policy on the given table has been violated
166   */
167  void verifyViolation(SpaceViolationPolicy policyToViolate, TableName tn, Mutation m)
168      throws Exception {
169    // But let's try a few times to get the exception before failing
170    boolean sawError = false;
171    String msg = "";
172    for (int i = 0; i < NUM_RETRIES && !sawError; i++) {
173      try (Table table = testUtil.getConnection().getTable(tn)) {
174        if (m instanceof Put) {
175          table.put((Put) m);
176        } else if (m instanceof Delete) {
177          table.delete((Delete) m);
178        } else if (m instanceof Append) {
179          table.append((Append) m);
180        } else if (m instanceof Increment) {
181          table.increment((Increment) m);
182        } else {
183          fail(
184              "Failed to apply " + m.getClass().getSimpleName() +
185                  " to the table. Programming error");
186        }
187        LOG.info("Did not reject the " + m.getClass().getSimpleName() + ", will sleep and retry");
188        Thread.sleep(2000);
189      } catch (Exception e) {
190        msg = StringUtils.stringifyException(e);
191        if ((policyToViolate.equals(SpaceViolationPolicy.DISABLE)
192            && e instanceof TableNotEnabledException) || msg.contains(policyToViolate.name())) {
193          LOG.info("Got the expected exception={}", msg);
194          sawError = true;
195          break;
196        } else {
197          LOG.warn("Did not get the expected exception, will sleep and retry", e);
198          Thread.sleep(2000);
199        }
200      }
201    }
202    if (!sawError) {
203      try (Table quotaTable = testUtil.getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
204        ResultScanner scanner = quotaTable.getScanner(new Scan());
205        Result result = null;
206        LOG.info("Dumping contents of hbase:quota table");
207        while ((result = scanner.next()) != null) {
208          LOG.info(Bytes.toString(result.getRow()) + " => " + result.toString());
209        }
210        scanner.close();
211      }
212    } else {
213      if (policyToViolate.equals(SpaceViolationPolicy.DISABLE)) {
214        assertTrue(
215            msg.contains("TableNotEnabledException") || msg.contains(policyToViolate.name()));
216      } else {
217        assertTrue("Expected exception message to contain the word '" + policyToViolate.name()
218                + "', but was " + msg,
219            msg.contains(policyToViolate.name()));
220      }
221    }
222    assertTrue(
223        "Expected to see an exception writing data to a table exceeding its quota", sawError);
224  }
225
226  /**
227   * Verifies that no policy has been violated on the given table
228   */
229  void verifyNoViolation(TableName tn, Mutation m) throws Exception {
230    // But let's try a few times to write data before failing
231    boolean sawSuccess = false;
232    for (int i = 0; i < NUM_RETRIES && !sawSuccess; i++) {
233      try (Table table = testUtil.getConnection().getTable(tn)) {
234        if (m instanceof Put) {
235          table.put((Put) m);
236        } else if (m instanceof Delete) {
237          table.delete((Delete) m);
238        } else if (m instanceof Append) {
239          table.append((Append) m);
240        } else if (m instanceof Increment) {
241          table.increment((Increment) m);
242        } else {
243          fail("Failed to apply " + m.getClass().getSimpleName() + " to the table."
244              + " Programming error");
245        }
246        sawSuccess = true;
247      } catch (Exception e) {
248        LOG.info("Rejected the " + m.getClass().getSimpleName() + ", will sleep and retry");
249        Thread.sleep(2000);
250      }
251    }
252    if (!sawSuccess) {
253      try (Table quotaTable = testUtil.getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
254        ResultScanner scanner = quotaTable.getScanner(new Scan());
255        Result result = null;
256        LOG.info("Dumping contents of hbase:quota table");
257        while ((result = scanner.next()) != null) {
258          LOG.info(Bytes.toString(result.getRow()) + " => " + result.toString());
259        }
260        scanner.close();
261      }
262    }
263    assertTrue("Expected to succeed in writing data to a table not having quota ", sawSuccess);
264  }
265
266  /**
267   * Sets the given quota (policy & limit) on the passed table.
268   */
269  void setQuotaLimit(final TableName tn, SpaceViolationPolicy policy, long sizeInMBs)
270      throws Exception {
271    final long sizeLimit = sizeInMBs * SpaceQuotaHelperForTests.ONE_MEGABYTE;
272    QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, policy);
273    testUtil.getAdmin().setQuota(settings);
274    LOG.debug("Quota limit set for table = {}, limit = {}", tn, sizeLimit);
275  }
276
277  /**
278   * Removes the space quota from the given table
279   */
280  void removeQuotaFromtable(final TableName tn) throws Exception {
281    QuotaSettings removeQuota = QuotaSettingsFactory.removeTableSpaceLimit(tn);
282    testUtil.getAdmin().setQuota(removeQuota);
283    LOG.debug("Space quota settings removed from the table ", tn);
284  }
285
286  /**
287   *
288   * @param tn the tablename
289   * @param numFiles number of files
290   * @param numRowsPerFile number of rows per file
291   * @return a clientServiceCallable which can be used with the Caller factory for bulk load
292   * @throws Exception when failed to get connection, table or preparation of the bulk load
293   */
294  ClientServiceCallable<Void> generateFileToLoad(TableName tn, int numFiles, int numRowsPerFile)
295      throws Exception {
296    Connection conn = testUtil.getConnection();
297    FileSystem fs = testUtil.getTestFileSystem();
298    Configuration conf = testUtil.getConfiguration();
299    Path baseDir = new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files");
300    fs.mkdirs(baseDir);
301    final List<Pair<byte[], String>> famPaths = new ArrayList<Pair<byte[], String>>();
302    for (int i = 1; i <= numFiles; i++) {
303      Path hfile = new Path(baseDir, "file" + i);
304      TestHRegionServerBulkLoad
305          .createHFile(fs, hfile, Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
306              Bytes.toBytes("reject"), numRowsPerFile);
307      famPaths.add(new Pair<>(Bytes.toBytes(SpaceQuotaHelperForTests.F1), hfile.toString()));
308    }
309
310    // bulk load HFiles
311    Table table = conn.getTable(tn);
312    final String bulkToken = new SecureBulkLoadClient(conf, table).prepareBulkLoad(conn);
313    return new ClientServiceCallable<Void>(conn, tn, Bytes.toBytes("row"),
314        new RpcControllerFactory(conf).newController(), HConstants.PRIORITY_UNSET) {
315      @Override
316      public Void rpcCall() throws Exception {
317        SecureBulkLoadClient secureClient = null;
318        byte[] regionName = getLocation().getRegionInfo().getRegionName();
319        try (Table table = conn.getTable(getTableName())) {
320          secureClient = new SecureBulkLoadClient(conf, table);
321          secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName, true, null, bulkToken);
322        }
323        return null;
324      }
325    };
326  }
327
328  /**
329   * Bulk-loads a number of files with a number of rows to the given table.
330   */
331//  ClientServiceCallable<Boolean> generateFileToLoad(
332//      TableName tn, int numFiles, int numRowsPerFile) throws Exception {
333//    Connection conn = testUtil.getConnection();
334//    FileSystem fs = testUtil.getTestFileSystem();
335//    Configuration conf = testUtil.getConfiguration();
336//    Path baseDir = new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files");
337//    fs.mkdirs(baseDir);
338//    final List<Pair<byte[], String>> famPaths = new ArrayList<>();
339//    for (int i = 1; i <= numFiles; i++) {
340//      Path hfile = new Path(baseDir, "file" + i);
341//      TestHRegionServerBulkLoad.createHFile(
342//          fs, hfile, Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("my"),
343//          Bytes.toBytes("file"), numRowsPerFile);
344//      famPaths.add(new Pair<>(Bytes.toBytes(SpaceQuotaHelperForTests.F1), hfile.toString()));
345//    }
346//
347//    // bulk load HFiles
348//    Table table = conn.getTable(tn);
349//    final String bulkToken = new SecureBulkLoadClient(conf, table).prepareBulkLoad(conn);
350//    return new ClientServiceCallable<Boolean>(
351//        conn, tn, Bytes.toBytes("row"), new RpcControllerFactory(conf).newController(),
352//        HConstants.PRIORITY_UNSET) {
353//      @Override
354//     public Boolean rpcCall() throws Exception {
355//        SecureBulkLoadClient secureClient = null;
356//        byte[] regionName = getLocation().getRegion().getRegionName();
357//        try (Table table = conn.getTable(getTableName())) {
358//          secureClient = new SecureBulkLoadClient(conf, table);
359//          return secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
360//                true, null, bulkToken);
361//        }
362//      }
363//    };
364//  }
365
366  /**
367   * Removes all quotas defined in the HBase quota table.
368   */
369  void removeAllQuotas() throws Exception {
370    final Connection conn = testUtil.getConnection();
371    removeAllQuotas(conn);
372    assertEquals(0, listNumDefinedQuotas(conn));
373  }
374
375  /**
376   * Removes all quotas defined in the HBase quota table.
377   */
378  void removeAllQuotas(Connection conn) throws IOException {
379    // Wait for the quota table to be created
380    if (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME)) {
381      waitForQuotaTable(conn);
382    } else {
383      // Or, clean up any quotas from previous test runs.
384      QuotaRetriever scanner = QuotaRetriever.open(conn.getConfiguration());
385      try {
386        for (QuotaSettings quotaSettings : scanner) {
387          final String namespace = quotaSettings.getNamespace();
388          final TableName tableName = quotaSettings.getTableName();
389          if (namespace != null) {
390            LOG.debug("Deleting quota for namespace: " + namespace);
391            QuotaUtil.deleteNamespaceQuota(conn, namespace);
392          } else {
393            assert tableName != null;
394            LOG.debug("Deleting quota for table: "+ tableName);
395            QuotaUtil.deleteTableQuota(conn, tableName);
396          }
397        }
398      } finally {
399        if (scanner != null) {
400          scanner.close();
401        }
402      }
403    }
404  }
405
406  QuotaSettings getTableSpaceQuota(Connection conn, TableName tn) throws IOException {
407    try (QuotaRetriever scanner = QuotaRetriever.open(
408        conn.getConfiguration(), new QuotaFilter().setTableFilter(tn.getNameAsString()))) {
409      for (QuotaSettings setting : scanner) {
410        if (setting.getTableName().equals(tn) && setting.getQuotaType() == QuotaType.SPACE) {
411          return setting;
412        }
413      }
414      return null;
415    }
416  }
417
418  /**
419   * Waits 30seconds for the HBase quota table to exist.
420   */
421  public void waitForQuotaTable(Connection conn) throws IOException {
422    waitForQuotaTable(conn, 30_000);
423  }
424
425  /**
426   * Waits {@code timeout} milliseconds for the HBase quota table to exist.
427   */
428  public void waitForQuotaTable(Connection conn, long timeout) throws IOException {
429    testUtil.waitFor(timeout, 1000, new Predicate<IOException>() {
430      @Override
431      public boolean evaluate() throws IOException {
432        return conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME);
433      }
434    });
435  }
436
437  void writeData(TableName tn, long sizeInBytes) throws IOException {
438    writeData(testUtil.getConnection(), tn, sizeInBytes);
439  }
440
441  void writeData(Connection conn, TableName tn, long sizeInBytes) throws IOException {
442    writeData(tn, sizeInBytes, Bytes.toBytes("q1"));
443  }
444
445  void writeData(TableName tn, long sizeInBytes, String qual) throws IOException {
446    writeData(tn, sizeInBytes, Bytes.toBytes(qual));
447  }
448
449  void writeData(TableName tn, long sizeInBytes, byte[] qual) throws IOException {
450    final Connection conn = testUtil.getConnection();
451    final Table table = conn.getTable(tn);
452    try {
453      List<Put> updates = new ArrayList<>();
454      long bytesToWrite = sizeInBytes;
455      long rowKeyId = 0L;
456      final StringBuilder sb = new StringBuilder();
457      final Random r = new Random();
458      while (bytesToWrite > 0L) {
459        sb.setLength(0);
460        sb.append(Long.toString(rowKeyId));
461        // Use the reverse counter as the rowKey to get even spread across all regions
462        Put p = new Put(Bytes.toBytes(sb.reverse().toString()));
463        byte[] value = new byte[SIZE_PER_VALUE];
464        r.nextBytes(value);
465        p.addColumn(Bytes.toBytes(F1), qual, value);
466        updates.add(p);
467
468        // Batch ~13KB worth of updates
469        if (updates.size() > 50) {
470          table.put(updates);
471          updates.clear();
472        }
473
474        // Just count the value size, ignore the size of rowkey + column
475        bytesToWrite -= SIZE_PER_VALUE;
476        rowKeyId++;
477      }
478
479      // Write the final batch
480      if (!updates.isEmpty()) {
481        table.put(updates);
482      }
483
484      LOG.debug("Data was written to HBase");
485      // Push the data to disk.
486      testUtil.getAdmin().flush(tn);
487      LOG.debug("Data flushed to disk");
488    } finally {
489      table.close();
490    }
491  }
492
493  NamespaceDescriptor createNamespace() throws Exception {
494    NamespaceDescriptor nd = NamespaceDescriptor.create("ns" + counter.getAndIncrement()).build();
495    testUtil.getAdmin().createNamespace(nd);
496    return nd;
497  }
498
499  Multimap<TableName, QuotaSettings> createTablesWithSpaceQuotas() throws Exception {
500    final Admin admin = testUtil.getAdmin();
501    final Multimap<TableName, QuotaSettings> tablesWithQuotas = HashMultimap.create();
502
503    final TableName tn1 = createTable();
504    final TableName tn2 = createTable();
505
506    NamespaceDescriptor nd = createNamespace();
507    final TableName tn3 = createTableInNamespace(nd);
508    final TableName tn4 = createTableInNamespace(nd);
509    final TableName tn5 = createTableInNamespace(nd);
510
511    final long sizeLimit1 = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
512    final SpaceViolationPolicy violationPolicy1 = SpaceViolationPolicy.NO_WRITES;
513    QuotaSettings qs1 = QuotaSettingsFactory.limitTableSpace(tn1, sizeLimit1, violationPolicy1);
514    tablesWithQuotas.put(tn1, qs1);
515    admin.setQuota(qs1);
516
517    final long sizeLimit2 = 1024L * 1024L * 1024L * 200L; // 200GB
518    final SpaceViolationPolicy violationPolicy2 = SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
519    QuotaSettings qs2 = QuotaSettingsFactory.limitTableSpace(tn2, sizeLimit2, violationPolicy2);
520    tablesWithQuotas.put(tn2, qs2);
521    admin.setQuota(qs2);
522
523    final long sizeLimit3 = 1024L * 1024L * 1024L * 1024L * 100L; // 100TB
524    final SpaceViolationPolicy violationPolicy3 = SpaceViolationPolicy.NO_INSERTS;
525    QuotaSettings qs3 = QuotaSettingsFactory.limitNamespaceSpace(
526        nd.getName(), sizeLimit3, violationPolicy3);
527    tablesWithQuotas.put(tn3, qs3);
528    tablesWithQuotas.put(tn4, qs3);
529    tablesWithQuotas.put(tn5, qs3);
530    admin.setQuota(qs3);
531
532    final long sizeLimit4 = 1024L * 1024L * 1024L * 5L; // 5GB
533    final SpaceViolationPolicy violationPolicy4 = SpaceViolationPolicy.NO_INSERTS;
534    QuotaSettings qs4 = QuotaSettingsFactory.limitTableSpace(tn5, sizeLimit4, violationPolicy4);
535    // Override the ns quota for tn5, import edge-case to catch table quota taking
536    // precedence over ns quota.
537    tablesWithQuotas.put(tn5, qs4);
538    admin.setQuota(qs4);
539
540    return tablesWithQuotas;
541  }
542
543  TableName getNextTableName() {
544    return getNextTableName(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR);
545  }
546
547  TableName getNextTableName(String namespace) {
548    return TableName.valueOf(namespace, testName.getMethodName() + counter.getAndIncrement());
549  }
550
551  TableName createTable() throws Exception {
552    return createTableWithRegions(1);
553  }
554
555  TableName createTableWithRegions(int numRegions) throws Exception {
556    return createTableWithRegions(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR, numRegions);
557  }
558
559  TableName createTableWithRegions(Admin admin, int numRegions) throws Exception {
560    return createTableWithRegions(admin, NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR, numRegions,
561        0);
562  }
563
564  TableName createTableWithRegions(String namespace, int numRegions) throws Exception {
565    return createTableWithRegions(testUtil.getAdmin(), namespace, numRegions, 0);
566  }
567
568  TableName createTableWithRegions(Admin admin, String namespace, int numRegions,
569      int numberOfReplicas) throws Exception {
570    final TableName tn = getNextTableName(namespace);
571
572    // Delete the old table
573    if (admin.tableExists(tn)) {
574      admin.disableTable(tn);
575      admin.deleteTable(tn);
576    }
577
578    // Create the table
579    TableDescriptor tableDesc;
580    if (numberOfReplicas > 0) {
581      tableDesc = TableDescriptorBuilder.newBuilder(tn).setRegionReplication(numberOfReplicas)
582          .setColumnFamily(ColumnFamilyDescriptorBuilder.of(F1)).build();
583    } else {
584      tableDesc = TableDescriptorBuilder.newBuilder(tn)
585          .setColumnFamily(ColumnFamilyDescriptorBuilder.of(F1)).build();
586    }
587    if (numRegions == 1) {
588      admin.createTable(tableDesc);
589    } else {
590      admin.createTable(tableDesc, Bytes.toBytes("0"), Bytes.toBytes("9"), numRegions);
591    }
592    return tn;
593  }
594
595  TableName createTableInNamespace(NamespaceDescriptor nd) throws Exception {
596    final Admin admin = testUtil.getAdmin();
597    final TableName tn = TableName.valueOf(nd.getName(),
598        testName.getMethodName() + counter.getAndIncrement());
599
600    // Delete the old table
601    if (admin.tableExists(tn)) {
602      admin.disableTable(tn);
603      admin.deleteTable(tn);
604    }
605
606    // Create the table
607    TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(tn)
608        .addColumnFamily(ColumnFamilyDescriptorBuilder.of(F1)).build();
609
610    admin.createTable(tableDesc);
611    return tn;
612  }
613
614  void partitionTablesByQuotaTarget(Multimap<TableName,QuotaSettings> quotas,
615      Set<TableName> tablesWithTableQuota, Set<TableName> tablesWithNamespaceQuota) {
616    // Partition the tables with quotas by table and ns quota
617    for (Entry<TableName, QuotaSettings> entry : quotas.entries()) {
618      SpaceLimitSettings settings = (SpaceLimitSettings) entry.getValue();
619      TableName tn = entry.getKey();
620      if (settings.getTableName() != null) {
621        tablesWithTableQuota.add(tn);
622      }
623      if (settings.getNamespace() != null) {
624        tablesWithNamespaceQuota.add(tn);
625      }
626
627      if (settings.getTableName() == null && settings.getNamespace() == null) {
628        fail("Unexpected table name with null tableName and namespace: " + tn);
629      }
630    }
631  }
632
633  /**
634   * Abstraction to simplify the case where a test needs to verify a certain state
635   * on a {@code SpaceQuotaSnapshot}. This class fails-fast when there is no such
636   * snapshot obtained from the Master. As such, it is not useful to verify the
637   * lack of a snapshot.
638   */
639  static abstract class SpaceQuotaSnapshotPredicate implements Predicate<Exception> {
640    private final Connection conn;
641    private final TableName tn;
642    private final String ns;
643
644    SpaceQuotaSnapshotPredicate(Connection conn, TableName tn) {
645      this(Objects.requireNonNull(conn), Objects.requireNonNull(tn), null);
646    }
647
648    SpaceQuotaSnapshotPredicate(Connection conn, String ns) {
649      this(Objects.requireNonNull(conn), null, Objects.requireNonNull(ns));
650    }
651
652    SpaceQuotaSnapshotPredicate(Connection conn, TableName tn, String ns) {
653      if ((null != tn && null != ns) || (null == tn && null == ns)) {
654        throw new IllegalArgumentException(
655            "One of TableName and Namespace must be non-null, and the other null");
656      }
657      this.conn = conn;
658      this.tn = tn;
659      this.ns = ns;
660    }
661
662    @Override
663    public boolean evaluate() throws Exception {
664      SpaceQuotaSnapshot snapshot;
665      if (null == ns) {
666        snapshot = QuotaTableUtil.getCurrentSnapshot(conn, tn);
667      } else {
668        snapshot = QuotaTableUtil.getCurrentSnapshot(conn, ns);
669      }
670
671      LOG.debug("Saw quota snapshot for " + (null == tn ? ns : tn) + ": " + snapshot);
672      if (null == snapshot) {
673        return false;
674      }
675      return evaluate(snapshot);
676    }
677
678    /**
679     * Must determine if the given {@code SpaceQuotaSnapshot} meets some criteria.
680     *
681     * @param snapshot a non-null snapshot obtained from the HBase Master
682     * @return true if the criteria is met, false otherwise
683     */
684    abstract boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception;
685  }
686
687  /**
688   * Predicate that waits for all store files in a table to have no compacted files.
689   */
690  static class NoFilesToDischarge implements Predicate<Exception> {
691    private final MiniHBaseCluster cluster;
692    private final TableName tn;
693
694    NoFilesToDischarge(MiniHBaseCluster cluster, TableName tn) {
695      this.cluster = cluster;
696      this.tn = tn;
697    }
698
699    @Override
700    public boolean evaluate() throws Exception {
701      for (HRegion region : cluster.getRegions(tn)) {
702        for (HStore store : region.getStores()) {
703          Collection<HStoreFile> files =
704              store.getStoreEngine().getStoreFileManager().getCompactedfiles();
705          if (null != files && !files.isEmpty()) {
706            LOG.debug(region.getRegionInfo().getEncodedName() + " still has compacted files");
707            return false;
708          }
709        }
710      }
711      return true;
712    }
713  }
714}