HBASE-21779 Reimplement BulkLoadHFilesTool to use AsyncClusterConnection
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / quotas / TestSpaceQuotas.java
blobfca54535a2798355e8c9cc29cf2977a5465bf6b3
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase.quotas;
20 import static org.hamcrest.CoreMatchers.instanceOf;
21 import static org.junit.Assert.assertEquals;
22 import static org.junit.Assert.assertFalse;
23 import static org.junit.Assert.assertNull;
24 import static org.junit.Assert.assertThat;
25 import static org.junit.Assert.assertTrue;
26 import static org.junit.Assert.fail;
28 import java.io.IOException;
29 import java.util.HashMap;
30 import java.util.List;
31 import java.util.Map;
32 import java.util.Map.Entry;
33 import java.util.concurrent.atomic.AtomicLong;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.fs.FileStatus;
36 import org.apache.hadoop.fs.FileSystem;
37 import org.apache.hadoop.fs.Path;
38 import org.apache.hadoop.hbase.DoNotRetryIOException;
39 import org.apache.hadoop.hbase.HBaseClassTestRule;
40 import org.apache.hadoop.hbase.HBaseTestingUtility;
41 import org.apache.hadoop.hbase.TableName;
42 import org.apache.hadoop.hbase.TableNotEnabledException;
43 import org.apache.hadoop.hbase.client.Admin;
44 import org.apache.hadoop.hbase.client.Append;
45 import org.apache.hadoop.hbase.client.Connection;
46 import org.apache.hadoop.hbase.client.Delete;
47 import org.apache.hadoop.hbase.client.Increment;
48 import org.apache.hadoop.hbase.client.Mutation;
49 import org.apache.hadoop.hbase.client.Put;
50 import org.apache.hadoop.hbase.client.RegionInfo;
51 import org.apache.hadoop.hbase.client.Result;
52 import org.apache.hadoop.hbase.client.ResultScanner;
53 import org.apache.hadoop.hbase.client.Scan;
54 import org.apache.hadoop.hbase.client.Table;
55 import org.apache.hadoop.hbase.master.HMaster;
56 import org.apache.hadoop.hbase.quotas.policies.DefaultViolationPolicyEnforcement;
57 import org.apache.hadoop.hbase.regionserver.HRegionServer;
58 import org.apache.hadoop.hbase.security.AccessDeniedException;
59 import org.apache.hadoop.hbase.testclassification.LargeTests;
60 import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
61 import org.apache.hadoop.hbase.util.Bytes;
62 import org.apache.hadoop.util.StringUtils;
63 import org.junit.AfterClass;
64 import org.junit.Before;
65 import org.junit.BeforeClass;
66 import org.junit.ClassRule;
67 import org.junit.Rule;
68 import org.junit.Test;
69 import org.junit.experimental.categories.Category;
70 import org.junit.rules.TestName;
71 import org.slf4j.Logger;
72 import org.slf4j.LoggerFactory;
74 /**
75 * End-to-end test class for filesystem space quotas.
77 @Category(LargeTests.class)
78 public class TestSpaceQuotas {
80 @ClassRule
81 public static final HBaseClassTestRule CLASS_RULE =
82 HBaseClassTestRule.forClass(TestSpaceQuotas.class);
84 private static final Logger LOG = LoggerFactory.getLogger(TestSpaceQuotas.class);
85 private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
86 // Global for all tests in the class
87 private static final AtomicLong COUNTER = new AtomicLong(0);
88 private static final int NUM_RETRIES = 10;
90 @Rule
91 public TestName testName = new TestName();
92 private SpaceQuotaHelperForTests helper;
93 private final TableName NON_EXISTENT_TABLE = TableName.valueOf("NON_EXISTENT_TABLE");
95 @BeforeClass
96 public static void setUp() throws Exception {
97 Configuration conf = TEST_UTIL.getConfiguration();
98 SpaceQuotaHelperForTests.updateConfigForQuotas(conf);
99 TEST_UTIL.startMiniCluster(1);
102 @AfterClass
103 public static void tearDown() throws Exception {
104 TEST_UTIL.shutdownMiniCluster();
107 @Before
108 public void removeAllQuotas() throws Exception {
109 final Connection conn = TEST_UTIL.getConnection();
110 if (helper == null) {
111 helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
113 // Wait for the quota table to be created
114 if (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME)) {
115 helper.waitForQuotaTable(conn);
116 } else {
117 // Or, clean up any quotas from previous test runs.
118 helper.removeAllQuotas(conn);
119 assertEquals(0, helper.listNumDefinedQuotas(conn));
123 @Test
124 public void testNoInsertsWithPut() throws Exception {
125 Put p = new Put(Bytes.toBytes("to_reject"));
126 p.addColumn(
127 Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
128 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, p);
131 @Test
132 public void testNoInsertsWithAppend() throws Exception {
133 Append a = new Append(Bytes.toBytes("to_reject"));
134 a.addColumn(
135 Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
136 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, a);
139 @Test
140 public void testNoInsertsWithIncrement() throws Exception {
141 Increment i = new Increment(Bytes.toBytes("to_reject"));
142 i.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("count"), 0);
143 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, i);
146 @Test
147 public void testDeletesAfterNoInserts() throws Exception {
148 final TableName tn = writeUntilViolation(SpaceViolationPolicy.NO_INSERTS);
149 // Try a couple of times to verify that the quota never gets enforced, same as we
150 // do when we're trying to catch the failure.
151 Delete d = new Delete(Bytes.toBytes("should_not_be_rejected"));
152 for (int i = 0; i < NUM_RETRIES; i++) {
153 try (Table t = TEST_UTIL.getConnection().getTable(tn)) {
154 t.delete(d);
159 @Test
160 public void testNoWritesWithPut() throws Exception {
161 Put p = new Put(Bytes.toBytes("to_reject"));
162 p.addColumn(
163 Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
164 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, p);
167 @Test
168 public void testNoWritesWithAppend() throws Exception {
169 Append a = new Append(Bytes.toBytes("to_reject"));
170 a.addColumn(
171 Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
172 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, a);
175 @Test
176 public void testNoWritesWithIncrement() throws Exception {
177 Increment i = new Increment(Bytes.toBytes("to_reject"));
178 i.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("count"), 0);
179 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, i);
182 @Test
183 public void testNoWritesWithDelete() throws Exception {
184 Delete d = new Delete(Bytes.toBytes("to_reject"));
185 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, d);
188 @Test
189 public void testNoCompactions() throws Exception {
190 Put p = new Put(Bytes.toBytes("to_reject"));
191 p.addColumn(
192 Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
193 final TableName tn = writeUntilViolationAndVerifyViolation(
194 SpaceViolationPolicy.NO_WRITES_COMPACTIONS, p);
195 // We know the policy is active at this point
197 // Major compactions should be rejected
198 try {
199 TEST_UTIL.getAdmin().majorCompact(tn);
200 fail("Expected that invoking the compaction should throw an Exception");
201 } catch (DoNotRetryIOException e) {
202 // Expected!
204 // Minor compactions should also be rejected.
205 try {
206 TEST_UTIL.getAdmin().compact(tn);
207 fail("Expected that invoking the compaction should throw an Exception");
208 } catch (DoNotRetryIOException e) {
209 // Expected!
213 @Test
214 public void testNoEnableAfterDisablePolicy() throws Exception {
215 Put p = new Put(Bytes.toBytes("to_reject"));
216 p.addColumn(
217 Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
218 final TableName tn = writeUntilViolation(SpaceViolationPolicy.DISABLE);
219 final Admin admin = TEST_UTIL.getAdmin();
220 // Disabling a table relies on some external action (over the other policies), so wait a bit
221 // more than the other tests.
222 for (int i = 0; i < NUM_RETRIES * 2; i++) {
223 if (admin.isTableEnabled(tn)) {
224 LOG.info(tn + " is still enabled, expecting it to be disabled. Will wait and re-check.");
225 Thread.sleep(2000);
228 assertFalse(tn + " is still enabled but it should be disabled", admin.isTableEnabled(tn));
229 try {
230 admin.enableTable(tn);
231 } catch (AccessDeniedException e) {
232 String exceptionContents = StringUtils.stringifyException(e);
233 final String expectedText = "violated space quota";
234 assertTrue("Expected the exception to contain " + expectedText + ", but was: "
235 + exceptionContents, exceptionContents.contains(expectedText));
239 @Test
240 public void testNoBulkLoadsWithNoWrites() throws Exception {
241 Put p = new Put(Bytes.toBytes("to_reject"));
242 p.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
243 Bytes.toBytes("reject"));
244 TableName tableName = writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, p);
246 // The table is now in violation. Try to do a bulk load
247 Map<byte[], List<Path>> family2Files = helper.generateFileToLoad(tableName, 1, 50);
248 try {
249 BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(tableName, family2Files);
250 fail("Expected the bulk load call to fail!");
251 } catch (IOException e) {
252 // Pass
253 assertThat(e.getCause(), instanceOf(SpaceLimitingException.class));
254 LOG.trace("Caught expected exception", e);
258 @Test
259 public void testAtomicBulkLoadUnderQuota() throws Exception {
260 // Need to verify that if the batch of hfiles cannot be loaded, none are loaded.
261 TableName tn = helper.createTableWithRegions(10);
263 final long sizeLimit = 50L * SpaceQuotaHelperForTests.ONE_KILOBYTE;
264 QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
265 tn, sizeLimit, SpaceViolationPolicy.NO_INSERTS);
266 TEST_UTIL.getAdmin().setQuota(settings);
268 HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
269 RegionServerSpaceQuotaManager spaceQuotaManager = rs.getRegionServerSpaceQuotaManager();
270 Map<TableName,SpaceQuotaSnapshot> snapshots = spaceQuotaManager.copyQuotaSnapshots();
271 Map<RegionInfo,Long> regionSizes = getReportedSizesForTable(tn);
272 while (true) {
273 SpaceQuotaSnapshot snapshot = snapshots.get(tn);
274 if (snapshot != null && snapshot.getLimit() > 0) {
275 break;
277 LOG.debug(
278 "Snapshot does not yet realize quota limit: " + snapshots + ", regionsizes: " +
279 regionSizes);
280 Thread.sleep(3000);
281 snapshots = spaceQuotaManager.copyQuotaSnapshots();
282 regionSizes = getReportedSizesForTable(tn);
284 // Our quota limit should be reflected in the latest snapshot
285 SpaceQuotaSnapshot snapshot = snapshots.get(tn);
286 assertEquals(0L, snapshot.getUsage());
287 assertEquals(sizeLimit, snapshot.getLimit());
289 // We would also not have a "real" policy in violation
290 ActivePolicyEnforcement activePolicies = spaceQuotaManager.getActiveEnforcements();
291 SpaceViolationPolicyEnforcement enforcement = activePolicies.getPolicyEnforcement(tn);
292 assertTrue(
293 "Expected to find Noop policy, but got " + enforcement.getClass().getSimpleName(),
294 enforcement instanceof DefaultViolationPolicyEnforcement);
296 // Should generate two files, each of which is over 25KB each
297 Map<byte[], List<Path>> family2Files = helper.generateFileToLoad(tn, 2, 525);
298 FileSystem fs = TEST_UTIL.getTestFileSystem();
299 FileStatus[] files = fs.listStatus(
300 new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files"));
301 for (FileStatus file : files) {
302 assertTrue(
303 "Expected the file, " + file.getPath() + ", length to be larger than 25KB, but was "
304 + file.getLen(),
305 file.getLen() > 25 * SpaceQuotaHelperForTests.ONE_KILOBYTE);
306 LOG.debug(file.getPath() + " -> " + file.getLen() +"B");
309 try {
310 BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(tn, family2Files);
311 fail("Expected the bulk load call to fail!");
312 } catch (IOException e) {
313 // Pass
314 assertThat(e.getCause(), instanceOf(SpaceLimitingException.class));
315 LOG.trace("Caught expected exception", e);
317 // Verify that we have no data in the table because neither file should have been
318 // loaded even though one of the files could have.
319 Table table = TEST_UTIL.getConnection().getTable(tn);
320 ResultScanner scanner = table.getScanner(new Scan());
321 try {
322 assertNull("Expected no results", scanner.next());
323 } finally{
324 scanner.close();
328 @Test
329 public void testTableQuotaOverridesNamespaceQuota() throws Exception {
330 final SpaceViolationPolicy policy = SpaceViolationPolicy.NO_INSERTS;
331 final TableName tn = helper.createTableWithRegions(10);
333 // 2MB limit on the table, 1GB limit on the namespace
334 final long tableLimit = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
335 final long namespaceLimit = 1024L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
336 TEST_UTIL.getAdmin().setQuota(QuotaSettingsFactory.limitTableSpace(tn, tableLimit, policy));
337 TEST_UTIL.getAdmin().setQuota(QuotaSettingsFactory.limitNamespaceSpace(
338 tn.getNamespaceAsString(), namespaceLimit, policy));
340 // Write more data than should be allowed and flush it to disk
341 helper.writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
343 // This should be sufficient time for the chores to run and see the change.
344 Thread.sleep(5000);
346 // The write should be rejected because the table quota takes priority over the namespace
347 Put p = new Put(Bytes.toBytes("to_reject"));
348 p.addColumn(
349 Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
350 verifyViolation(policy, tn, p);
353 @Test
354 public void testSetQuotaAndThenRemoveWithNoInserts() throws Exception {
355 setQuotaAndThenRemove(SpaceViolationPolicy.NO_INSERTS);
358 @Test
359 public void testSetQuotaAndThenRemoveWithNoWrite() throws Exception {
360 setQuotaAndThenRemove(SpaceViolationPolicy.NO_WRITES);
363 @Test
364 public void testSetQuotaAndThenRemoveWithNoWritesCompactions() throws Exception {
365 setQuotaAndThenRemove(SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
368 @Test
369 public void testSetQuotaAndThenRemoveWithDisable() throws Exception {
370 setQuotaAndThenRemove(SpaceViolationPolicy.DISABLE);
373 @Test
374 public void testSetQuotaAndThenDropTableWithNoInserts() throws Exception {
375 setQuotaAndThenDropTable(SpaceViolationPolicy.NO_INSERTS);
378 @Test
379 public void testSetQuotaAndThenDropTableWithNoWrite() throws Exception {
380 setQuotaAndThenDropTable(SpaceViolationPolicy.NO_WRITES);
383 @Test
384 public void testSetQuotaAndThenDropTableWithNoWritesCompactions() throws Exception {
385 setQuotaAndThenDropTable(SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
388 @Test
389 public void testSetQuotaAndThenDropTableWithDisable() throws Exception {
390 setQuotaAndThenDropTable(SpaceViolationPolicy.DISABLE);
393 @Test
394 public void testSetQuotaAndThenIncreaseQuotaWithNoInserts() throws Exception {
395 setQuotaAndThenIncreaseQuota(SpaceViolationPolicy.NO_INSERTS);
398 @Test
399 public void testSetQuotaAndThenIncreaseQuotaWithNoWrite() throws Exception {
400 setQuotaAndThenIncreaseQuota(SpaceViolationPolicy.NO_WRITES);
403 @Test
404 public void testSetQuotaAndThenIncreaseQuotaWithNoWritesCompactions() throws Exception {
405 setQuotaAndThenIncreaseQuota(SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
408 @Test
409 public void testSetQuotaAndThenIncreaseQuotaWithDisable() throws Exception {
410 setQuotaAndThenIncreaseQuota(SpaceViolationPolicy.DISABLE);
413 @Test
414 public void testSetQuotaAndThenDisableIncrEnableWithDisable() throws Exception {
415 setQuotaNextDisableThenIncreaseFinallyEnable(SpaceViolationPolicy.DISABLE);
418 @Test
419 public void testSetQuotaAndThenRemoveInOneWithNoInserts() throws Exception {
420 setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy.NO_INSERTS);
423 @Test
424 public void testSetQuotaAndThenRemoveInOneWithNoWrite() throws Exception {
425 setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy.NO_WRITES);
428 @Test
429 public void testSetQuotaAndThenRemoveInOneWithNoWritesCompaction() throws Exception {
430 setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
433 @Test
434 public void testSetQuotaAndThenRemoveInOneWithDisable() throws Exception {
435 setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy.DISABLE);
438 @Test
439 public void testSetQuotaFirstWithDisableNextNoWrites() throws Exception {
440 setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy.DISABLE,
441 SpaceViolationPolicy.NO_WRITES);
444 @Test
445 public void testSetQuotaFirstWithDisableNextAgainDisable() throws Exception {
446 setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy.DISABLE,
447 SpaceViolationPolicy.DISABLE);
450 @Test
451 public void testSetQuotaFirstWithDisableNextNoInserts() throws Exception {
452 setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy.DISABLE,
453 SpaceViolationPolicy.NO_INSERTS);
456 @Test
457 public void testSetQuotaFirstWithDisableNextNoWritesCompaction() throws Exception {
458 setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy.DISABLE,
459 SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
462 @Test
463 public void testSetQuotaFirstWithNoWritesNextWithDisable() throws Exception {
464 setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy.NO_WRITES,
465 SpaceViolationPolicy.DISABLE);
468 @Test
469 public void testSetQuotaOnNonExistingTableWithNoInserts() throws Exception {
470 setQuotaLimit(NON_EXISTENT_TABLE, SpaceViolationPolicy.NO_INSERTS, 2L);
473 @Test
474 public void testSetQuotaOnNonExistingTableWithNoWrites() throws Exception {
475 setQuotaLimit(NON_EXISTENT_TABLE, SpaceViolationPolicy.NO_WRITES, 2L);
478 @Test
479 public void testSetQuotaOnNonExistingTableWithNoWritesCompaction() throws Exception {
480 setQuotaLimit(NON_EXISTENT_TABLE, SpaceViolationPolicy.NO_WRITES_COMPACTIONS, 2L);
483 @Test
484 public void testSetQuotaOnNonExistingTableWithDisable() throws Exception {
485 setQuotaLimit(NON_EXISTENT_TABLE, SpaceViolationPolicy.DISABLE, 2L);
488 public void setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy policy1,
489 SpaceViolationPolicy policy2) throws Exception {
490 Put put = new Put(Bytes.toBytes("to_reject"));
491 put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
492 Bytes.toBytes("reject"));
494 // Do puts until we violate space violation policy1
495 final TableName tn = writeUntilViolationAndVerifyViolation(policy1, put);
497 // Now, change violation policy to policy2
498 setQuotaLimit(tn, policy2, 2L);
500 // The table should be in enabled state on changing violation policy
501 if (policy1.equals(SpaceViolationPolicy.DISABLE) && !policy1.equals(policy2)) {
502 TEST_UTIL.waitTableEnabled(tn, 20000);
504 // Put some row now: should still violate as quota limit still violated
505 verifyViolation(policy2, tn, put);
508 private void setQuotaAndThenRemove(SpaceViolationPolicy policy) throws Exception {
509 Put put = new Put(Bytes.toBytes("to_reject"));
510 put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
511 Bytes.toBytes("reject"));
513 // Do puts until we violate space policy
514 final TableName tn = writeUntilViolationAndVerifyViolation(policy, put);
516 // Now, remove the quota
517 removeQuotaFromtable(tn);
519 // Put some rows now: should not violate as quota settings removed
520 verifyNoViolation(policy, tn, put);
523 private void setQuotaAndThenDropTable(SpaceViolationPolicy policy) throws Exception {
524 Put put = new Put(Bytes.toBytes("to_reject"));
525 put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
526 Bytes.toBytes("reject"));
528 // Do puts until we violate space policy
529 final TableName tn = writeUntilViolationAndVerifyViolation(policy, put);
531 // Now, drop the table
532 TEST_UTIL.deleteTable(tn);
533 LOG.debug("Successfully deleted table ", tn);
535 // Now re-create the table
536 TEST_UTIL.createTable(tn, Bytes.toBytes(SpaceQuotaHelperForTests.F1));
537 LOG.debug("Successfully re-created table ", tn);
539 // Put some rows now: should not violate as table/quota was dropped
540 verifyNoViolation(policy, tn, put);
543 private void setQuotaAndThenIncreaseQuota(SpaceViolationPolicy policy) throws Exception {
544 Put put = new Put(Bytes.toBytes("to_reject"));
545 put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
546 Bytes.toBytes("reject"));
548 // Do puts until we violate space policy
549 final TableName tn = writeUntilViolationAndVerifyViolation(policy, put);
551 // Now, increase limit and perform put
552 setQuotaLimit(tn, policy, 4L);
554 // Put some row now: should not violate as quota limit increased
555 verifyNoViolation(policy, tn, put);
558 private void setQuotaNextDisableThenIncreaseFinallyEnable(SpaceViolationPolicy policy)
559 throws Exception {
560 Put put = new Put(Bytes.toBytes("to_reject"));
561 put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
562 Bytes.toBytes("reject"));
564 // Do puts until we violate space policy
565 final TableName tn = writeUntilViolationAndVerifyViolation(policy, put);
567 // Disable the table; in case of SpaceViolationPolicy.DISABLE already disabled
568 if (!policy.equals(SpaceViolationPolicy.DISABLE)) {
569 TEST_UTIL.getAdmin().disableTable(tn);
570 TEST_UTIL.waitTableDisabled(tn, 10000);
573 // Now, increase limit and perform put
574 setQuotaLimit(tn, policy, 4L);
576 // in case of disable policy quota manager will enable it
577 if (!policy.equals(SpaceViolationPolicy.DISABLE)) {
578 TEST_UTIL.getAdmin().enableTable(tn);
580 TEST_UTIL.waitTableEnabled(tn, 10000);
582 // Put some row now: should not violate as quota limit increased
583 verifyNoViolation(policy, tn, put);
586 public void setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy policy)
587 throws Exception {
588 Put put = new Put(Bytes.toBytes("to_reject"));
589 put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
590 Bytes.toBytes("reject"));
592 // Do puts until we violate space policy on table tn1
593 final TableName tn1 = writeUntilViolationAndVerifyViolation(policy, put);
595 // Do puts until we violate space policy on table tn2
596 final TableName tn2 = writeUntilViolationAndVerifyViolation(policy, put);
598 // Now, remove the quota from table tn1
599 removeQuotaFromtable(tn1);
601 // Put a new row now on tn1: should not violate as quota settings removed
602 verifyNoViolation(policy, tn1, put);
603 // Put a new row now on tn2: should violate as quota settings exists
604 verifyViolation(policy, tn2, put);
607 private void removeQuotaFromtable(final TableName tn) throws Exception {
608 QuotaSettings removeQuota = QuotaSettingsFactory.removeTableSpaceLimit(tn);
609 TEST_UTIL.getAdmin().setQuota(removeQuota);
610 LOG.debug("Space quota settings removed from the table ", tn);
613 private void setQuotaLimit(final TableName tn, SpaceViolationPolicy policy, long sizeInMBs)
614 throws Exception {
615 final long sizeLimit = sizeInMBs * SpaceQuotaHelperForTests.ONE_MEGABYTE;
616 QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, policy);
617 TEST_UTIL.getAdmin().setQuota(settings);
618 LOG.debug("Quota limit set for table = {}, limit = {}", tn, sizeLimit);
621 private Map<RegionInfo,Long> getReportedSizesForTable(TableName tn) {
622 HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
623 MasterQuotaManager quotaManager = master.getMasterQuotaManager();
624 Map<RegionInfo,Long> filteredRegionSizes = new HashMap<>();
625 for (Entry<RegionInfo,Long> entry : quotaManager.snapshotRegionSizes().entrySet()) {
626 if (entry.getKey().getTable().equals(tn)) {
627 filteredRegionSizes.put(entry.getKey(), entry.getValue());
630 return filteredRegionSizes;
633 private TableName writeUntilViolation(SpaceViolationPolicy policyToViolate) throws Exception {
634 TableName tn = helper.createTableWithRegions(10);
635 setQuotaLimit(tn, policyToViolate, 2L);
636 // Write more data than should be allowed and flush it to disk
637 helper.writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
639 // This should be sufficient time for the chores to run and see the change.
640 Thread.sleep(5000);
642 return tn;
645 private TableName writeUntilViolationAndVerifyViolation(
646 SpaceViolationPolicy policyToViolate, Mutation m) throws Exception {
647 final TableName tn = writeUntilViolation(policyToViolate);
648 verifyViolation(policyToViolate, tn, m);
649 return tn;
652 private void verifyViolation(
653 SpaceViolationPolicy policyToViolate, TableName tn, Mutation m) throws Exception {
654 // But let's try a few times to get the exception before failing
655 boolean sawError = false;
656 String msg = "";
657 for (int i = 0; i < NUM_RETRIES && !sawError; i++) {
658 try (Table table = TEST_UTIL.getConnection().getTable(tn)) {
659 if (m instanceof Put) {
660 table.put((Put) m);
661 } else if (m instanceof Delete) {
662 table.delete((Delete) m);
663 } else if (m instanceof Append) {
664 table.append((Append) m);
665 } else if (m instanceof Increment) {
666 table.increment((Increment) m);
667 } else {
668 fail(
669 "Failed to apply " + m.getClass().getSimpleName() +
670 " to the table. Programming error");
672 LOG.info("Did not reject the " + m.getClass().getSimpleName() + ", will sleep and retry");
673 Thread.sleep(2000);
674 } catch (Exception e) {
675 msg = StringUtils.stringifyException(e);
676 if ((policyToViolate.equals(SpaceViolationPolicy.DISABLE)
677 && e instanceof TableNotEnabledException) || msg.contains(policyToViolate.name())) {
678 LOG.info("Got the expected exception={}", msg);
679 sawError = true;
680 break;
681 } else {
682 LOG.warn("Did not get the expected exception, will sleep and retry", e);
683 Thread.sleep(2000);
687 if (!sawError) {
688 try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
689 ResultScanner scanner = quotaTable.getScanner(new Scan());
690 Result result = null;
691 LOG.info("Dumping contents of hbase:quota table");
692 while ((result = scanner.next()) != null) {
693 LOG.info(Bytes.toString(result.getRow()) + " => " + result.toString());
695 scanner.close();
697 } else {
698 if (policyToViolate.equals(SpaceViolationPolicy.DISABLE)) {
699 assertTrue(
700 msg.contains("TableNotEnabledException") || msg.contains(policyToViolate.name()));
701 } else {
702 assertTrue("Expected exception message to contain the word '" + policyToViolate.name()
703 + "', but was " + msg,
704 msg.contains(policyToViolate.name()));
707 assertTrue(
708 "Expected to see an exception writing data to a table exceeding its quota", sawError);
711 private void verifyNoViolation(SpaceViolationPolicy policyToViolate, TableName tn, Mutation m)
712 throws Exception {
713 // But let's try a few times to write data before failing
714 boolean sawSuccess = false;
715 for (int i = 0; i < NUM_RETRIES && !sawSuccess; i++) {
716 try (Table table = TEST_UTIL.getConnection().getTable(tn)) {
717 if (m instanceof Put) {
718 table.put((Put) m);
719 } else if (m instanceof Delete) {
720 table.delete((Delete) m);
721 } else if (m instanceof Append) {
722 table.append((Append) m);
723 } else if (m instanceof Increment) {
724 table.increment((Increment) m);
725 } else {
726 fail(
727 "Failed to apply " + m.getClass().getSimpleName() + " to the table. Programming error");
729 sawSuccess = true;
730 } catch (Exception e) {
731 LOG.info("Rejected the " + m.getClass().getSimpleName() + ", will sleep and retry");
732 Thread.sleep(2000);
735 if (!sawSuccess) {
736 try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
737 ResultScanner scanner = quotaTable.getScanner(new Scan());
738 Result result = null;
739 LOG.info("Dumping contents of hbase:quota table");
740 while ((result = scanner.next()) != null) {
741 LOG.info(Bytes.toString(result.getRow()) + " => " + result.toString());
743 scanner.close();
746 assertTrue("Expected to succeed in writing data to a table not having quota ", sawSuccess);