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
;
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
;
75 * End-to-end test class for filesystem space quotas.
77 @Category(LargeTests
.class)
78 public class TestSpaceQuotas
{
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;
91 public TestName testName
= new TestName();
92 private SpaceQuotaHelperForTests helper
;
93 private final TableName NON_EXISTENT_TABLE
= TableName
.valueOf("NON_EXISTENT_TABLE");
96 public static void setUp() throws Exception
{
97 Configuration conf
= TEST_UTIL
.getConfiguration();
98 SpaceQuotaHelperForTests
.updateConfigForQuotas(conf
);
99 TEST_UTIL
.startMiniCluster(1);
103 public static void tearDown() throws Exception
{
104 TEST_UTIL
.shutdownMiniCluster();
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
);
117 // Or, clean up any quotas from previous test runs.
118 helper
.removeAllQuotas(conn
);
119 assertEquals(0, helper
.listNumDefinedQuotas(conn
));
124 public void testNoInsertsWithPut() throws Exception
{
125 Put p
= new Put(Bytes
.toBytes("to_reject"));
127 Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"), Bytes
.toBytes("reject"));
128 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_INSERTS
, p
);
132 public void testNoInsertsWithAppend() throws Exception
{
133 Append a
= new Append(Bytes
.toBytes("to_reject"));
135 Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"), Bytes
.toBytes("reject"));
136 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_INSERTS
, a
);
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
);
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
)) {
160 public void testNoWritesWithPut() throws Exception
{
161 Put p
= new Put(Bytes
.toBytes("to_reject"));
163 Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"), Bytes
.toBytes("reject"));
164 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_WRITES
, p
);
168 public void testNoWritesWithAppend() throws Exception
{
169 Append a
= new Append(Bytes
.toBytes("to_reject"));
171 Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"), Bytes
.toBytes("reject"));
172 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_WRITES
, a
);
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
);
183 public void testNoWritesWithDelete() throws Exception
{
184 Delete d
= new Delete(Bytes
.toBytes("to_reject"));
185 writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_WRITES
, d
);
189 public void testNoCompactions() throws Exception
{
190 Put p
= new Put(Bytes
.toBytes("to_reject"));
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
199 TEST_UTIL
.getAdmin().majorCompact(tn
);
200 fail("Expected that invoking the compaction should throw an Exception");
201 } catch (DoNotRetryIOException e
) {
204 // Minor compactions should also be rejected.
206 TEST_UTIL
.getAdmin().compact(tn
);
207 fail("Expected that invoking the compaction should throw an Exception");
208 } catch (DoNotRetryIOException e
) {
214 public void testNoEnableAfterDisablePolicy() throws Exception
{
215 Put p
= new Put(Bytes
.toBytes("to_reject"));
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.");
228 assertFalse(tn
+ " is still enabled but it should be disabled", admin
.isTableEnabled(tn
));
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
));
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);
249 BulkLoadHFiles
.create(TEST_UTIL
.getConfiguration()).bulkLoad(tableName
, family2Files
);
250 fail("Expected the bulk load call to fail!");
251 } catch (IOException e
) {
253 assertThat(e
.getCause(), instanceOf(SpaceLimitingException
.class));
254 LOG
.trace("Caught expected exception", e
);
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
);
273 SpaceQuotaSnapshot snapshot
= snapshots
.get(tn
);
274 if (snapshot
!= null && snapshot
.getLimit() > 0) {
278 "Snapshot does not yet realize quota limit: " + snapshots
+ ", regionsizes: " +
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
);
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
) {
303 "Expected the file, " + file
.getPath() + ", length to be larger than 25KB, but was "
305 file
.getLen() > 25 * SpaceQuotaHelperForTests
.ONE_KILOBYTE
);
306 LOG
.debug(file
.getPath() + " -> " + file
.getLen() +"B");
310 BulkLoadHFiles
.create(TEST_UTIL
.getConfiguration()).bulkLoad(tn
, family2Files
);
311 fail("Expected the bulk load call to fail!");
312 } catch (IOException e
) {
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());
322 assertNull("Expected no results", scanner
.next());
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.
346 // The write should be rejected because the table quota takes priority over the namespace
347 Put p
= new Put(Bytes
.toBytes("to_reject"));
349 Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"), Bytes
.toBytes("reject"));
350 verifyViolation(policy
, tn
, p
);
354 public void testSetQuotaAndThenRemoveWithNoInserts() throws Exception
{
355 setQuotaAndThenRemove(SpaceViolationPolicy
.NO_INSERTS
);
359 public void testSetQuotaAndThenRemoveWithNoWrite() throws Exception
{
360 setQuotaAndThenRemove(SpaceViolationPolicy
.NO_WRITES
);
364 public void testSetQuotaAndThenRemoveWithNoWritesCompactions() throws Exception
{
365 setQuotaAndThenRemove(SpaceViolationPolicy
.NO_WRITES_COMPACTIONS
);
369 public void testSetQuotaAndThenRemoveWithDisable() throws Exception
{
370 setQuotaAndThenRemove(SpaceViolationPolicy
.DISABLE
);
374 public void testSetQuotaAndThenDropTableWithNoInserts() throws Exception
{
375 setQuotaAndThenDropTable(SpaceViolationPolicy
.NO_INSERTS
);
379 public void testSetQuotaAndThenDropTableWithNoWrite() throws Exception
{
380 setQuotaAndThenDropTable(SpaceViolationPolicy
.NO_WRITES
);
384 public void testSetQuotaAndThenDropTableWithNoWritesCompactions() throws Exception
{
385 setQuotaAndThenDropTable(SpaceViolationPolicy
.NO_WRITES_COMPACTIONS
);
389 public void testSetQuotaAndThenDropTableWithDisable() throws Exception
{
390 setQuotaAndThenDropTable(SpaceViolationPolicy
.DISABLE
);
394 public void testSetQuotaAndThenIncreaseQuotaWithNoInserts() throws Exception
{
395 setQuotaAndThenIncreaseQuota(SpaceViolationPolicy
.NO_INSERTS
);
399 public void testSetQuotaAndThenIncreaseQuotaWithNoWrite() throws Exception
{
400 setQuotaAndThenIncreaseQuota(SpaceViolationPolicy
.NO_WRITES
);
404 public void testSetQuotaAndThenIncreaseQuotaWithNoWritesCompactions() throws Exception
{
405 setQuotaAndThenIncreaseQuota(SpaceViolationPolicy
.NO_WRITES_COMPACTIONS
);
409 public void testSetQuotaAndThenIncreaseQuotaWithDisable() throws Exception
{
410 setQuotaAndThenIncreaseQuota(SpaceViolationPolicy
.DISABLE
);
414 public void testSetQuotaAndThenDisableIncrEnableWithDisable() throws Exception
{
415 setQuotaNextDisableThenIncreaseFinallyEnable(SpaceViolationPolicy
.DISABLE
);
419 public void testSetQuotaAndThenRemoveInOneWithNoInserts() throws Exception
{
420 setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy
.NO_INSERTS
);
424 public void testSetQuotaAndThenRemoveInOneWithNoWrite() throws Exception
{
425 setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy
.NO_WRITES
);
429 public void testSetQuotaAndThenRemoveInOneWithNoWritesCompaction() throws Exception
{
430 setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy
.NO_WRITES_COMPACTIONS
);
434 public void testSetQuotaAndThenRemoveInOneWithDisable() throws Exception
{
435 setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy
.DISABLE
);
439 public void testSetQuotaFirstWithDisableNextNoWrites() throws Exception
{
440 setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy
.DISABLE
,
441 SpaceViolationPolicy
.NO_WRITES
);
445 public void testSetQuotaFirstWithDisableNextAgainDisable() throws Exception
{
446 setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy
.DISABLE
,
447 SpaceViolationPolicy
.DISABLE
);
451 public void testSetQuotaFirstWithDisableNextNoInserts() throws Exception
{
452 setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy
.DISABLE
,
453 SpaceViolationPolicy
.NO_INSERTS
);
457 public void testSetQuotaFirstWithDisableNextNoWritesCompaction() throws Exception
{
458 setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy
.DISABLE
,
459 SpaceViolationPolicy
.NO_WRITES_COMPACTIONS
);
463 public void testSetQuotaFirstWithNoWritesNextWithDisable() throws Exception
{
464 setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy
.NO_WRITES
,
465 SpaceViolationPolicy
.DISABLE
);
469 public void testSetQuotaOnNonExistingTableWithNoInserts() throws Exception
{
470 setQuotaLimit(NON_EXISTENT_TABLE
, SpaceViolationPolicy
.NO_INSERTS
, 2L);
474 public void testSetQuotaOnNonExistingTableWithNoWrites() throws Exception
{
475 setQuotaLimit(NON_EXISTENT_TABLE
, SpaceViolationPolicy
.NO_WRITES
, 2L);
479 public void testSetQuotaOnNonExistingTableWithNoWritesCompaction() throws Exception
{
480 setQuotaLimit(NON_EXISTENT_TABLE
, SpaceViolationPolicy
.NO_WRITES_COMPACTIONS
, 2L);
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
)
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
)
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
)
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.
645 private TableName
writeUntilViolationAndVerifyViolation(
646 SpaceViolationPolicy policyToViolate
, Mutation m
) throws Exception
{
647 final TableName tn
= writeUntilViolation(policyToViolate
);
648 verifyViolation(policyToViolate
, tn
, m
);
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;
657 for (int i
= 0; i
< NUM_RETRIES
&& !sawError
; i
++) {
658 try (Table table
= TEST_UTIL
.getConnection().getTable(tn
)) {
659 if (m
instanceof Put
) {
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
);
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");
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
);
682 LOG
.warn("Did not get the expected exception, will sleep and retry", e
);
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());
698 if (policyToViolate
.equals(SpaceViolationPolicy
.DISABLE
)) {
700 msg
.contains("TableNotEnabledException") || msg
.contains(policyToViolate
.name()));
702 assertTrue("Expected exception message to contain the word '" + policyToViolate
.name()
703 + "', but was " + msg
,
704 msg
.contains(policyToViolate
.name()));
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
)
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
) {
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
);
727 "Failed to apply " + m
.getClass().getSimpleName() + " to the table. Programming error");
730 } catch (Exception e
) {
731 LOG
.info("Rejected the " + m
.getClass().getSimpleName() + ", will sleep and retry");
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());
746 assertTrue("Expected to succeed in writing data to a table not having quota ", sawSuccess
);