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}