|
18 | 18 | package org.apache.hadoop.hbase.quotas; |
19 | 19 |
|
20 | 20 | import static org.junit.Assert.assertEquals; |
| 21 | +import static org.junit.Assert.assertThrows; |
21 | 22 | import static org.junit.Assert.assertTrue; |
22 | 23 |
|
| 24 | +import org.apache.hadoop.conf.Configuration; |
23 | 25 | import org.apache.hadoop.hbase.HBaseClassTestRule; |
24 | 26 | import org.apache.hadoop.hbase.testclassification.RegionServerTests; |
25 | 27 | import org.apache.hadoop.hbase.testclassification.SmallTests; |
| 28 | +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; |
| 29 | +import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; |
| 30 | +import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; |
26 | 31 | import org.junit.ClassRule; |
27 | 32 | import org.junit.Test; |
28 | 33 | import org.junit.experimental.categories.Category; |
29 | 34 |
|
| 35 | +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; |
| 36 | +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; |
| 37 | + |
30 | 38 | @Category({ RegionServerTests.class, SmallTests.class }) |
31 | 39 | public class TestDefaultOperationQuota { |
32 | 40 | @ClassRule |
33 | 41 | public static final HBaseClassTestRule CLASS_RULE = |
34 | 42 | HBaseClassTestRule.forClass(TestDefaultOperationQuota.class); |
35 | 43 |
|
| 44 | + private static ManualEnvironmentEdge envEdge = new ManualEnvironmentEdge(); |
| 45 | + static { |
| 46 | + envEdge.setValue(EnvironmentEdgeManager.currentTime()); |
| 47 | + // only active the envEdge for quotas package |
| 48 | + EnvironmentEdgeManagerTestHelper.injectEdgeForPackage(envEdge, |
| 49 | + ThrottleQuotaTestUtil.class.getPackage().getName()); |
| 50 | + } |
| 51 | + |
36 | 52 | @Test |
37 | 53 | public void testScanEstimateNewScanner() { |
38 | 54 | long blockSize = 64 * 1024; |
@@ -125,4 +141,155 @@ public void testScanEstimateShrinkingWorkload() { |
125 | 141 | // shrinking workload should only shrink estimate to maxBBS |
126 | 142 | assertEquals(maxBlockBytesScanned, estimate); |
127 | 143 | } |
| 144 | + |
| 145 | + @Test |
| 146 | + public void testLargeBatchSaturatesReadNumLimit() |
| 147 | + throws RpcThrottlingException, InterruptedException { |
| 148 | + int limit = 10; |
| 149 | + QuotaProtos.Throttle throttle = |
| 150 | + QuotaProtos.Throttle.newBuilder().setReadNum(QuotaProtos.TimedQuota.newBuilder() |
| 151 | + .setSoftLimit(limit).setTimeUnit(HBaseProtos.TimeUnit.SECONDS).build()).build(); |
| 152 | + QuotaLimiter limiter = TimeBasedLimiter.fromThrottle(throttle); |
| 153 | + DefaultOperationQuota quota = new DefaultOperationQuota(new Configuration(), 65536, limiter); |
| 154 | + |
| 155 | + // use the whole limit |
| 156 | + quota.checkBatchQuota(0, limit); |
| 157 | + |
| 158 | + // the next request should be rejected |
| 159 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota(0, 1)); |
| 160 | + |
| 161 | + envEdge.incValue(1000); |
| 162 | + // after the TimeUnit, the limit should be refilled |
| 163 | + quota.checkBatchQuota(0, limit); |
| 164 | + } |
| 165 | + |
| 166 | + @Test |
| 167 | + public void testLargeBatchSaturatesReadWriteLimit() |
| 168 | + throws RpcThrottlingException, InterruptedException { |
| 169 | + int limit = 10; |
| 170 | + QuotaProtos.Throttle throttle = |
| 171 | + QuotaProtos.Throttle.newBuilder().setWriteNum(QuotaProtos.TimedQuota.newBuilder() |
| 172 | + .setSoftLimit(limit).setTimeUnit(HBaseProtos.TimeUnit.SECONDS).build()).build(); |
| 173 | + QuotaLimiter limiter = TimeBasedLimiter.fromThrottle(throttle); |
| 174 | + DefaultOperationQuota quota = new DefaultOperationQuota(new Configuration(), 65536, limiter); |
| 175 | + |
| 176 | + // use the whole limit |
| 177 | + quota.checkBatchQuota(limit, 0); |
| 178 | + |
| 179 | + // the next request should be rejected |
| 180 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota(1, 0)); |
| 181 | + |
| 182 | + envEdge.incValue(1000); |
| 183 | + // after the TimeUnit, the limit should be refilled |
| 184 | + quota.checkBatchQuota(limit, 0); |
| 185 | + } |
| 186 | + |
| 187 | + @Test |
| 188 | + public void testTooLargeReadBatchIsNotBlocked() |
| 189 | + throws RpcThrottlingException, InterruptedException { |
| 190 | + int limit = 10; |
| 191 | + QuotaProtos.Throttle throttle = |
| 192 | + QuotaProtos.Throttle.newBuilder().setReadNum(QuotaProtos.TimedQuota.newBuilder() |
| 193 | + .setSoftLimit(limit).setTimeUnit(HBaseProtos.TimeUnit.SECONDS).build()).build(); |
| 194 | + QuotaLimiter limiter = TimeBasedLimiter.fromThrottle(throttle); |
| 195 | + DefaultOperationQuota quota = new DefaultOperationQuota(new Configuration(), 65536, limiter); |
| 196 | + |
| 197 | + // use more than the limit, which should succeed rather than being indefinitely blocked |
| 198 | + quota.checkBatchQuota(0, 10 + limit); |
| 199 | + |
| 200 | + // the next request should be blocked |
| 201 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota(0, 1)); |
| 202 | + |
| 203 | + envEdge.incValue(1000); |
| 204 | + // even after the TimeUnit, the limit should not be refilled because we oversubscribed |
| 205 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota(0, limit)); |
| 206 | + } |
| 207 | + |
| 208 | + @Test |
| 209 | + public void testTooLargeWriteBatchIsNotBlocked() |
| 210 | + throws RpcThrottlingException, InterruptedException { |
| 211 | + int limit = 10; |
| 212 | + QuotaProtos.Throttle throttle = |
| 213 | + QuotaProtos.Throttle.newBuilder().setWriteNum(QuotaProtos.TimedQuota.newBuilder() |
| 214 | + .setSoftLimit(limit).setTimeUnit(HBaseProtos.TimeUnit.SECONDS).build()).build(); |
| 215 | + QuotaLimiter limiter = TimeBasedLimiter.fromThrottle(throttle); |
| 216 | + DefaultOperationQuota quota = new DefaultOperationQuota(new Configuration(), 65536, limiter); |
| 217 | + |
| 218 | + // use more than the limit, which should succeed rather than being indefinitely blocked |
| 219 | + quota.checkBatchQuota(10 + limit, 0); |
| 220 | + |
| 221 | + // the next request should be blocked |
| 222 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota(1, 0)); |
| 223 | + |
| 224 | + envEdge.incValue(1000); |
| 225 | + // even after the TimeUnit, the limit should not be refilled because we oversubscribed |
| 226 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota(limit, 0)); |
| 227 | + } |
| 228 | + |
| 229 | + @Test |
| 230 | + public void testTooLargeWriteSizeIsNotBlocked() |
| 231 | + throws RpcThrottlingException, InterruptedException { |
| 232 | + int limit = 50; |
| 233 | + QuotaProtos.Throttle throttle = |
| 234 | + QuotaProtos.Throttle.newBuilder().setWriteSize(QuotaProtos.TimedQuota.newBuilder() |
| 235 | + .setSoftLimit(limit).setTimeUnit(HBaseProtos.TimeUnit.SECONDS).build()).build(); |
| 236 | + QuotaLimiter limiter = TimeBasedLimiter.fromThrottle(throttle); |
| 237 | + DefaultOperationQuota quota = new DefaultOperationQuota(new Configuration(), 65536, limiter); |
| 238 | + |
| 239 | + // writes are estimated a 100 bytes, so this will use 2x the limit but should not be blocked |
| 240 | + quota.checkBatchQuota(1, 0); |
| 241 | + |
| 242 | + // the next request should be blocked |
| 243 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota(1, 0)); |
| 244 | + |
| 245 | + envEdge.incValue(1000); |
| 246 | + // even after the TimeUnit, the limit should not be refilled because we oversubscribed |
| 247 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota(limit, 0)); |
| 248 | + } |
| 249 | + |
| 250 | + @Test |
| 251 | + public void testTooLargeReadSizeIsNotBlocked() |
| 252 | + throws RpcThrottlingException, InterruptedException { |
| 253 | + long blockSize = 65536; |
| 254 | + long limit = blockSize / 2; |
| 255 | + QuotaProtos.Throttle throttle = |
| 256 | + QuotaProtos.Throttle.newBuilder().setReadSize(QuotaProtos.TimedQuota.newBuilder() |
| 257 | + .setSoftLimit(limit).setTimeUnit(HBaseProtos.TimeUnit.SECONDS).build()).build(); |
| 258 | + QuotaLimiter limiter = TimeBasedLimiter.fromThrottle(throttle); |
| 259 | + DefaultOperationQuota quota = |
| 260 | + new DefaultOperationQuota(new Configuration(), (int) blockSize, limiter); |
| 261 | + |
| 262 | + // reads are estimated at 1 block each, so this will use ~2x the limit but should not be blocked |
| 263 | + quota.checkBatchQuota(0, 1); |
| 264 | + |
| 265 | + // the next request should be blocked |
| 266 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota(0, 1)); |
| 267 | + |
| 268 | + envEdge.incValue(1000); |
| 269 | + // even after the TimeUnit, the limit should not be refilled because we oversubscribed |
| 270 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota((int) limit, 1)); |
| 271 | + } |
| 272 | + |
| 273 | + @Test |
| 274 | + public void testTooLargeRequestSizeIsNotBlocked() |
| 275 | + throws RpcThrottlingException, InterruptedException { |
| 276 | + long blockSize = 65536; |
| 277 | + long limit = blockSize / 2; |
| 278 | + QuotaProtos.Throttle throttle = |
| 279 | + QuotaProtos.Throttle.newBuilder().setReqSize(QuotaProtos.TimedQuota.newBuilder() |
| 280 | + .setSoftLimit(limit).setTimeUnit(HBaseProtos.TimeUnit.SECONDS).build()).build(); |
| 281 | + QuotaLimiter limiter = TimeBasedLimiter.fromThrottle(throttle); |
| 282 | + DefaultOperationQuota quota = |
| 283 | + new DefaultOperationQuota(new Configuration(), (int) blockSize, limiter); |
| 284 | + |
| 285 | + // reads are estimated at 1 block each, so this will use ~2x the limit but should not be blocked |
| 286 | + quota.checkBatchQuota(0, 1); |
| 287 | + |
| 288 | + // the next request should be blocked |
| 289 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota(0, 1)); |
| 290 | + |
| 291 | + envEdge.incValue(1000); |
| 292 | + // even after the TimeUnit, the limit should not be refilled because we oversubscribed |
| 293 | + assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota((int) limit, 1)); |
| 294 | + } |
128 | 295 | } |
0 commit comments