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
9 * http://www.apache.org/licenses/LICENSE-2.0
10 * Unless required by applicable law or agreed to in writing, software
11 * distributed under the License is distributed on an "AS IS" BASIS,
12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 * See the License for the specific language governing permissions and
14 * limitations under the License.
16 package org
.apache
.hadoop
.hbase
.quotas
;
18 import static org
.junit
.Assert
.assertFalse
;
19 import static org
.junit
.Assert
.assertTrue
;
20 import static org
.junit
.Assert
.fail
;
22 import java
.util
.List
;
24 import java
.util
.concurrent
.atomic
.AtomicLong
;
26 import org
.apache
.hadoop
.conf
.Configuration
;
27 import org
.apache
.hadoop
.hbase
.DoNotRetryIOException
;
28 import org
.apache
.hadoop
.hbase
.HBaseClassTestRule
;
29 import org
.apache
.hadoop
.hbase
.HBaseTestingUtility
;
30 import org
.apache
.hadoop
.hbase
.MetaTableAccessor
;
31 import org
.apache
.hadoop
.hbase
.TableName
;
32 import org
.apache
.hadoop
.hbase
.Waiter
;
33 import org
.apache
.hadoop
.hbase
.client
.Admin
;
34 import org
.apache
.hadoop
.hbase
.client
.Append
;
35 import org
.apache
.hadoop
.hbase
.client
.Delete
;
36 import org
.apache
.hadoop
.hbase
.client
.Increment
;
37 import org
.apache
.hadoop
.hbase
.client
.Put
;
38 import org
.apache
.hadoop
.hbase
.client
.RegionInfo
;
39 import org
.apache
.hadoop
.hbase
.client
.Table
;
40 import org
.apache
.hadoop
.hbase
.master
.HMaster
;
41 import org
.apache
.hadoop
.hbase
.security
.AccessDeniedException
;
42 import org
.apache
.hadoop
.hbase
.testclassification
.LargeTests
;
43 import org
.apache
.hadoop
.hbase
.util
.Bytes
;
44 import org
.apache
.hadoop
.util
.StringUtils
;
45 import org
.junit
.AfterClass
;
46 import org
.junit
.Assert
;
47 import org
.junit
.Before
;
48 import org
.junit
.BeforeClass
;
49 import org
.junit
.ClassRule
;
50 import org
.junit
.Rule
;
51 import org
.junit
.Test
;
52 import org
.junit
.experimental
.categories
.Category
;
53 import org
.junit
.rules
.TestName
;
54 import org
.slf4j
.Logger
;
55 import org
.slf4j
.LoggerFactory
;
57 @Category(LargeTests
.class)
58 public class TestSpaceQuotaBasicFunctioning
{
61 public static final HBaseClassTestRule CLASS_RULE
=
62 HBaseClassTestRule
.forClass(TestSpaceQuotaBasicFunctioning
.class);
64 private static final Logger LOG
= LoggerFactory
.getLogger(TestSpaceQuotaBasicFunctioning
.class);
65 private static final HBaseTestingUtility TEST_UTIL
= new HBaseTestingUtility();
66 private static final int NUM_RETRIES
= 10;
69 public TestName testName
= new TestName();
70 private SpaceQuotaHelperForTests helper
;
73 public static void setUp() throws Exception
{
74 Configuration conf
= TEST_UTIL
.getConfiguration();
75 SpaceQuotaHelperForTests
.updateConfigForQuotas(conf
);
76 TEST_UTIL
.startMiniCluster(1);
80 public static void tearDown() throws Exception
{
81 TEST_UTIL
.shutdownMiniCluster();
85 public void removeAllQuotas() throws Exception
{
86 helper
= new SpaceQuotaHelperForTests(TEST_UTIL
, testName
, new AtomicLong(0));
87 helper
.removeAllQuotas();
91 public void testNoInsertsWithPut() throws Exception
{
92 Put p
= new Put(Bytes
.toBytes("to_reject"));
93 p
.addColumn(Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"),
94 Bytes
.toBytes("reject"));
95 helper
.writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_INSERTS
, p
);
99 public void testNoInsertsWithAppend() throws Exception
{
100 Append a
= new Append(Bytes
.toBytes("to_reject"));
101 a
.addColumn(Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"),
102 Bytes
.toBytes("reject"));
103 helper
.writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_INSERTS
, a
);
107 public void testNoInsertsWithIncrement() throws Exception
{
108 Increment i
= new Increment(Bytes
.toBytes("to_reject"));
109 i
.addColumn(Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("count"), 0);
110 helper
.writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_INSERTS
, i
);
114 public void testDeletesAfterNoInserts() throws Exception
{
115 final TableName tn
= helper
.writeUntilViolation(SpaceViolationPolicy
.NO_INSERTS
);
116 // Try a couple of times to verify that the quota never gets enforced, same as we
117 // do when we're trying to catch the failure.
118 Delete d
= new Delete(Bytes
.toBytes("should_not_be_rejected"));
119 for (int i
= 0; i
< NUM_RETRIES
; i
++) {
120 try (Table t
= TEST_UTIL
.getConnection().getTable(tn
)) {
127 public void testNoWritesWithPut() throws Exception
{
128 Put p
= new Put(Bytes
.toBytes("to_reject"));
129 p
.addColumn(Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"),
130 Bytes
.toBytes("reject"));
131 helper
.writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_WRITES
, p
);
135 public void testNoWritesWithAppend() throws Exception
{
136 Append a
= new Append(Bytes
.toBytes("to_reject"));
137 a
.addColumn(Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"),
138 Bytes
.toBytes("reject"));
139 helper
.writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_WRITES
, a
);
143 public void testNoWritesWithIncrement() throws Exception
{
144 Increment i
= new Increment(Bytes
.toBytes("to_reject"));
145 i
.addColumn(Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("count"), 0);
146 helper
.writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_WRITES
, i
);
150 public void testNoWritesWithDelete() throws Exception
{
151 Delete d
= new Delete(Bytes
.toBytes("to_reject"));
152 helper
.writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_WRITES
, d
);
156 public void testNoCompactions() throws Exception
{
157 Put p
= new Put(Bytes
.toBytes("to_reject"));
158 p
.addColumn(Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"),
159 Bytes
.toBytes("reject"));
161 helper
.writeUntilViolationAndVerifyViolation(SpaceViolationPolicy
.NO_WRITES_COMPACTIONS
, p
);
162 // We know the policy is active at this point
164 // Major compactions should be rejected
166 TEST_UTIL
.getAdmin().majorCompact(tn
);
167 fail("Expected that invoking the compaction should throw an Exception");
168 } catch (DoNotRetryIOException e
) {
171 // Minor compactions should also be rejected.
173 TEST_UTIL
.getAdmin().compact(tn
);
174 fail("Expected that invoking the compaction should throw an Exception");
175 } catch (DoNotRetryIOException e
) {
181 public void testNoEnableAfterDisablePolicy() throws Exception
{
182 Put p
= new Put(Bytes
.toBytes("to_reject"));
183 p
.addColumn(Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"),
184 Bytes
.toBytes("reject"));
185 final TableName tn
= helper
.writeUntilViolation(SpaceViolationPolicy
.DISABLE
);
186 final Admin admin
= TEST_UTIL
.getAdmin();
187 // Disabling a table relies on some external action (over the other policies), so wait a bit
188 // more than the other tests.
189 for (int i
= 0; i
< NUM_RETRIES
* 2; i
++) {
190 if (admin
.isTableEnabled(tn
)) {
191 LOG
.info(tn
+ " is still enabled, expecting it to be disabled. Will wait and re-check.");
195 assertFalse(tn
+ " is still enabled but it should be disabled", admin
.isTableEnabled(tn
));
197 admin
.enableTable(tn
);
198 } catch (AccessDeniedException e
) {
199 String exceptionContents
= StringUtils
.stringifyException(e
);
200 final String expectedText
= "violated space quota";
202 "Expected the exception to contain " + expectedText
+ ", but was: " + exceptionContents
,
203 exceptionContents
.contains(expectedText
));
208 public void testTableQuotaOverridesNamespaceQuota() throws Exception
{
209 final SpaceViolationPolicy policy
= SpaceViolationPolicy
.NO_INSERTS
;
210 final TableName tn
= helper
.createTableWithRegions(10);
212 // 2MB limit on the table, 1GB limit on the namespace
213 final long tableLimit
= 2L * SpaceQuotaHelperForTests
.ONE_MEGABYTE
;
214 final long namespaceLimit
= 1024L * SpaceQuotaHelperForTests
.ONE_MEGABYTE
;
215 TEST_UTIL
.getAdmin().setQuota(QuotaSettingsFactory
.limitTableSpace(tn
, tableLimit
, policy
));
216 TEST_UTIL
.getAdmin().setQuota(QuotaSettingsFactory
217 .limitNamespaceSpace(tn
.getNamespaceAsString(), namespaceLimit
, policy
));
219 // Write more data than should be allowed and flush it to disk
220 helper
.writeData(tn
, 3L * SpaceQuotaHelperForTests
.ONE_MEGABYTE
);
222 // This should be sufficient time for the chores to run and see the change.
225 // The write should be rejected because the table quota takes priority over the namespace
226 Put p
= new Put(Bytes
.toBytes("to_reject"));
227 p
.addColumn(Bytes
.toBytes(SpaceQuotaHelperForTests
.F1
), Bytes
.toBytes("to"),
228 Bytes
.toBytes("reject"));
229 helper
.verifyViolation(policy
, tn
, p
);
233 public void testDisablePolicyQuotaAndViolate() throws Exception
{
234 TableName tableName
= helper
.createTable();
235 helper
.setQuotaLimit(tableName
, SpaceViolationPolicy
.DISABLE
, 1L);
236 helper
.writeData(tableName
, SpaceQuotaHelperForTests
.ONE_MEGABYTE
* 2L);
237 TEST_UTIL
.getConfiguration()
238 .setLong("hbase.master.quotas.region.report.retention.millis", 100);
240 HMaster master
= TEST_UTIL
.getMiniHBaseCluster().getMaster();
241 MasterQuotaManager quotaManager
= master
.getMasterQuotaManager();
243 // Make sure the master has report for the table.
244 Waiter
.waitFor(TEST_UTIL
.getConfiguration(), 30 * 1000, new Waiter
.Predicate
<Exception
>() {
246 public boolean evaluate() throws Exception
{
247 Map
<RegionInfo
, Long
> regionSizes
= quotaManager
.snapshotRegionSizes();
248 List
<RegionInfo
> tableRegions
=
249 MetaTableAccessor
.getTableRegions(TEST_UTIL
.getConnection(), tableName
);
250 return regionSizes
.containsKey(tableRegions
.get(0));
254 // Check if disabled table region report present in the map after retention period expired.
255 // It should be present after retention period expired.
256 final long regionSizes
= quotaManager
.snapshotRegionSizes().keySet().stream()
257 .filter(k
-> k
.getTable().equals(tableName
)).count();
258 Assert
.assertTrue(regionSizes
> 0);