1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.quotas;
20
21
22 import org.apache.hadoop.conf.Configuration;
23 import org.apache.hadoop.hbase.HBaseConfiguration;
24 import org.apache.hadoop.hbase.classification.InterfaceAudience;
25 import org.apache.hadoop.hbase.classification.InterfaceStability;
26 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
27 import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle;
28 import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota;
29
30
31
32
33 @InterfaceAudience.Private
34 @InterfaceStability.Evolving
35 public class TimeBasedLimiter implements QuotaLimiter {
36 private static final Configuration conf = HBaseConfiguration.create();
37 private RateLimiter reqsLimiter = null;
38 private RateLimiter reqSizeLimiter = null;
39 private RateLimiter writeReqsLimiter = null;
40 private RateLimiter writeSizeLimiter = null;
41 private RateLimiter readReqsLimiter = null;
42 private RateLimiter readSizeLimiter = null;
43 private RateLimiter reqCapacityUnitLimiter = null;
44 private RateLimiter writeCapacityUnitLimiter = null;
45 private RateLimiter readCapacityUnitLimiter = null;
46
47 private TimeBasedLimiter() {
48 if (FixedIntervalRateLimiter.class.getName().equals(
49 conf.getClass(RateLimiter.QUOTA_RATE_LIMITER_CONF_KEY, AverageIntervalRateLimiter.class)
50 .getName())) {
51 reqsLimiter = new FixedIntervalRateLimiter();
52 reqSizeLimiter = new FixedIntervalRateLimiter();
53 writeReqsLimiter = new FixedIntervalRateLimiter();
54 writeSizeLimiter = new FixedIntervalRateLimiter();
55 readReqsLimiter = new FixedIntervalRateLimiter();
56 readSizeLimiter = new FixedIntervalRateLimiter();
57 reqCapacityUnitLimiter = new FixedIntervalRateLimiter();
58 writeCapacityUnitLimiter = new FixedIntervalRateLimiter();
59 readCapacityUnitLimiter = new FixedIntervalRateLimiter();
60 } else {
61 reqsLimiter = new AverageIntervalRateLimiter();
62 reqSizeLimiter = new AverageIntervalRateLimiter();
63 writeReqsLimiter = new AverageIntervalRateLimiter();
64 writeSizeLimiter = new AverageIntervalRateLimiter();
65 readReqsLimiter = new AverageIntervalRateLimiter();
66 readSizeLimiter = new AverageIntervalRateLimiter();
67 reqCapacityUnitLimiter = new AverageIntervalRateLimiter();
68 writeCapacityUnitLimiter = new AverageIntervalRateLimiter();
69 readCapacityUnitLimiter = new AverageIntervalRateLimiter();
70 }
71 }
72
73 static QuotaLimiter fromThrottle(final Throttle throttle) {
74 TimeBasedLimiter limiter = new TimeBasedLimiter();
75 boolean isBypass = true;
76 if (throttle.hasReqNum()) {
77 setFromTimedQuota(limiter.reqsLimiter, throttle.getReqNum());
78 isBypass = false;
79 }
80
81 if (throttle.hasReqSize()) {
82 setFromTimedQuota(limiter.reqSizeLimiter, throttle.getReqSize());
83 isBypass = false;
84 }
85
86 if (throttle.hasWriteNum()) {
87 setFromTimedQuota(limiter.writeReqsLimiter, throttle.getWriteNum());
88 isBypass = false;
89 }
90
91 if (throttle.hasWriteSize()) {
92 setFromTimedQuota(limiter.writeSizeLimiter, throttle.getWriteSize());
93 isBypass = false;
94 }
95
96 if (throttle.hasReadNum()) {
97 setFromTimedQuota(limiter.readReqsLimiter, throttle.getReadNum());
98 isBypass = false;
99 }
100
101 if (throttle.hasReadSize()) {
102 setFromTimedQuota(limiter.readSizeLimiter, throttle.getReadSize());
103 isBypass = false;
104 }
105
106 if (throttle.hasReqCapacityUnit()) {
107 setFromTimedQuota(limiter.reqCapacityUnitLimiter, throttle.getReqCapacityUnit());
108 isBypass = false;
109 }
110
111 if (throttle.hasWriteCapacityUnit()) {
112 setFromTimedQuota(limiter.writeCapacityUnitLimiter, throttle.getWriteCapacityUnit());
113 isBypass = false;
114 }
115
116 if (throttle.hasReadCapacityUnit()) {
117 setFromTimedQuota(limiter.readCapacityUnitLimiter, throttle.getReadCapacityUnit());
118 isBypass = false;
119 }
120 return isBypass ? NoopQuotaLimiter.get() : limiter;
121 }
122
123 public void update(final TimeBasedLimiter other) {
124 reqsLimiter.update(other.reqsLimiter);
125 reqSizeLimiter.update(other.reqSizeLimiter);
126 writeReqsLimiter.update(other.writeReqsLimiter);
127 writeSizeLimiter.update(other.writeSizeLimiter);
128 readReqsLimiter.update(other.readReqsLimiter);
129 readSizeLimiter.update(other.readSizeLimiter);
130 reqCapacityUnitLimiter.update(other.reqCapacityUnitLimiter);
131 writeCapacityUnitLimiter.update(other.writeCapacityUnitLimiter);
132 readCapacityUnitLimiter.update(other.readCapacityUnitLimiter);
133 }
134
135 private static void setFromTimedQuota(final RateLimiter limiter, final TimedQuota timedQuota) {
136 limiter.set(timedQuota.getSoftLimit(), ProtobufUtil.toTimeUnit(timedQuota.getTimeUnit()));
137 }
138
139 @Override
140 public void checkQuota(long writeReqs, long estimateWriteSize, long readReqs,
141 long estimateReadSize, long estimateWriteCapacityUnit, long estimateReadCapacityUnit)
142 throws RpcThrottlingException {
143 if (!reqsLimiter.canExecute(writeReqs + readReqs)) {
144 RpcThrottlingException.throwNumRequestsExceeded(reqsLimiter.waitInterval());
145 }
146 if (!reqSizeLimiter.canExecute(estimateWriteSize + estimateReadSize)) {
147 RpcThrottlingException.throwRequestSizeExceeded(
148 reqSizeLimiter.waitInterval(estimateWriteSize + estimateReadSize));
149 }
150 if (!reqCapacityUnitLimiter.canExecute(estimateWriteCapacityUnit + estimateReadCapacityUnit)) {
151 RpcThrottlingException.throwRequestCapacityUnitExceeded(
152 reqCapacityUnitLimiter.waitInterval(estimateWriteCapacityUnit + estimateReadCapacityUnit));
153 }
154
155 if (estimateWriteSize > 0) {
156 if (!writeReqsLimiter.canExecute(writeReqs)) {
157 RpcThrottlingException.throwNumWriteRequestsExceeded(writeReqsLimiter.waitInterval());
158 }
159 if (!writeSizeLimiter.canExecute(estimateWriteSize)) {
160 RpcThrottlingException.throwWriteSizeExceeded(
161 writeSizeLimiter.waitInterval(estimateWriteSize));
162 }
163 if (!writeCapacityUnitLimiter.canExecute(estimateWriteCapacityUnit)) {
164 RpcThrottlingException.throwWriteCapacityUnitExceeded(
165 writeCapacityUnitLimiter.waitInterval(estimateWriteCapacityUnit));
166 }
167 }
168
169 if (estimateReadSize > 0) {
170 if (!readReqsLimiter.canExecute(readReqs)) {
171 RpcThrottlingException.throwNumReadRequestsExceeded(readReqsLimiter.waitInterval());
172 }
173 if (!readSizeLimiter.canExecute(estimateReadSize)) {
174 RpcThrottlingException.throwReadSizeExceeded(
175 readSizeLimiter.waitInterval(estimateReadSize));
176 }
177 if (!readCapacityUnitLimiter.canExecute(estimateReadCapacityUnit)) {
178 RpcThrottlingException.throwReadCapacityUnitExceeded(
179 readCapacityUnitLimiter.waitInterval(estimateReadCapacityUnit));
180 }
181 }
182 }
183
184 @Override
185 public void grabQuota(long writeReqs, long writeSize, long readReqs, long readSize,
186 long writeCapacityUnit, long readCapacityUnit) {
187 assert writeSize != 0 || readSize != 0;
188
189 reqsLimiter.consume(writeReqs + readReqs);
190 reqSizeLimiter.consume(writeSize + readSize);
191
192 if (writeSize > 0) {
193 writeReqsLimiter.consume(writeReqs);
194 writeSizeLimiter.consume(writeSize);
195 }
196 if (readSize > 0) {
197 readReqsLimiter.consume(readReqs);
198 readSizeLimiter.consume(readSize);
199 }
200 if (writeCapacityUnit > 0) {
201 reqCapacityUnitLimiter.consume(writeCapacityUnit);
202 writeCapacityUnitLimiter.consume(writeCapacityUnit);
203 }
204 if (readCapacityUnit > 0) {
205 reqCapacityUnitLimiter.consume(readCapacityUnit);
206 readCapacityUnitLimiter.consume(readCapacityUnit);
207 }
208 }
209
210 @Override
211 public void consumeWrite(final long size, long capacityUnit) {
212 reqSizeLimiter.consume(size);
213 writeSizeLimiter.consume(size);
214 reqCapacityUnitLimiter.consume(capacityUnit);
215 writeCapacityUnitLimiter.consume(capacityUnit);
216 }
217
218 @Override
219 public void consumeRead(final long size, long capacityUnit) {
220 reqSizeLimiter.consume(size);
221 readSizeLimiter.consume(size);
222 reqCapacityUnitLimiter.consume(capacityUnit);
223 readCapacityUnitLimiter.consume(capacityUnit);
224 }
225
226 @Override
227 public boolean isBypass() {
228 return false;
229 }
230
231 @Override
232 public long getWriteAvailable() {
233 return writeSizeLimiter.getAvailable();
234 }
235
236 @Override
237 public long getReadAvailable() {
238 return readSizeLimiter.getAvailable();
239 }
240
241 @Override
242 public String toString() {
243 StringBuilder builder = new StringBuilder();
244 builder.append("TimeBasedLimiter(");
245 if (!reqsLimiter.isBypass()) {
246 builder.append("reqs=" + reqsLimiter);
247 }
248 if (!reqSizeLimiter.isBypass()) {
249 builder.append(" resSize=" + reqSizeLimiter);
250 }
251 if (!writeReqsLimiter.isBypass()) {
252 builder.append(" writeReqs=" + writeReqsLimiter);
253 }
254 if (!writeSizeLimiter.isBypass()) {
255 builder.append(" writeSize=" + writeSizeLimiter);
256 }
257 if (!readReqsLimiter.isBypass()) {
258 builder.append(" readReqs=" + readReqsLimiter);
259 }
260 if (!readSizeLimiter.isBypass()) {
261 builder.append(" readSize=" + readSizeLimiter);
262 }
263 if (!reqCapacityUnitLimiter.isBypass()) {
264 builder.append(" reqCapacityUnit=" + reqCapacityUnitLimiter);
265 }
266 if (!writeCapacityUnitLimiter.isBypass()) {
267 builder.append(" writeCapacityUnit=" + writeCapacityUnitLimiter);
268 }
269 if (!readCapacityUnitLimiter.isBypass()) {
270 builder.append(" readCapacityUnit=" + readCapacityUnitLimiter);
271 }
272 builder.append(')');
273 return builder.toString();
274 }
275 }