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