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
44 private TimeBasedLimiter() {
45 if (FixedIntervalRateLimiter.class.getName().equals(
46 conf.getClass(RateLimiter.QUOTA_RATE_LIMITER_CONF_KEY, AverageIntervalRateLimiter.class)
47 .getName())) {
48 reqsLimiter = new FixedIntervalRateLimiter();
49 reqSizeLimiter = new FixedIntervalRateLimiter();
50 writeReqsLimiter = new FixedIntervalRateLimiter();
51 writeSizeLimiter = new FixedIntervalRateLimiter();
52 readReqsLimiter = new FixedIntervalRateLimiter();
53 readSizeLimiter = new FixedIntervalRateLimiter();
54 } else {
55 reqsLimiter = new AverageIntervalRateLimiter();
56 reqSizeLimiter = new AverageIntervalRateLimiter();
57 writeReqsLimiter = new AverageIntervalRateLimiter();
58 writeSizeLimiter = new AverageIntervalRateLimiter();
59 readReqsLimiter = new AverageIntervalRateLimiter();
60 readSizeLimiter = new AverageIntervalRateLimiter();
61 }
62 }
63
64 static QuotaLimiter fromThrottle(final Throttle throttle) {
65 TimeBasedLimiter limiter = new TimeBasedLimiter();
66 boolean isBypass = true;
67 if (throttle.hasReqNum()) {
68 setFromTimedQuota(limiter.reqsLimiter, throttle.getReqNum());
69 isBypass = false;
70 }
71
72 if (throttle.hasReqSize()) {
73 setFromTimedQuota(limiter.reqSizeLimiter, throttle.getReqSize());
74 isBypass = false;
75 }
76
77 if (throttle.hasWriteNum()) {
78 setFromTimedQuota(limiter.writeReqsLimiter, throttle.getWriteNum());
79 isBypass = false;
80 }
81
82 if (throttle.hasWriteSize()) {
83 setFromTimedQuota(limiter.writeSizeLimiter, throttle.getWriteSize());
84 isBypass = false;
85 }
86
87 if (throttle.hasReadNum()) {
88 setFromTimedQuota(limiter.readReqsLimiter, throttle.getReadNum());
89 isBypass = false;
90 }
91
92 if (throttle.hasReadSize()) {
93 setFromTimedQuota(limiter.readSizeLimiter, throttle.getReadSize());
94 isBypass = false;
95 }
96 return isBypass ? NoopQuotaLimiter.get() : limiter;
97 }
98
99 public void update(final TimeBasedLimiter other) {
100 reqsLimiter.update(other.reqsLimiter);
101 reqSizeLimiter.update(other.reqSizeLimiter);
102 writeReqsLimiter.update(other.writeReqsLimiter);
103 writeSizeLimiter.update(other.writeSizeLimiter);
104 readReqsLimiter.update(other.readReqsLimiter);
105 readSizeLimiter.update(other.readSizeLimiter);
106 }
107
108 private static void setFromTimedQuota(final RateLimiter limiter, final TimedQuota timedQuota) {
109 limiter.set(timedQuota.getSoftLimit(), ProtobufUtil.toTimeUnit(timedQuota.getTimeUnit()));
110 }
111
112 @Override
113 public void checkQuota(long writeSize, long readSize) throws ThrottlingException {
114 if (!reqsLimiter.canExecute()) {
115 ThrottlingException.throwNumRequestsExceeded(reqsLimiter.waitInterval());
116 }
117 if (!reqSizeLimiter.canExecute(writeSize + readSize)) {
118 ThrottlingException.throwRequestSizeExceeded(reqSizeLimiter
119 .waitInterval(writeSize + readSize));
120 }
121
122 if (writeSize > 0) {
123 if (!writeReqsLimiter.canExecute()) {
124 ThrottlingException.throwNumWriteRequestsExceeded(writeReqsLimiter.waitInterval());
125 }
126 if (!writeSizeLimiter.canExecute(writeSize)) {
127 ThrottlingException.throwWriteSizeExceeded(writeSizeLimiter.waitInterval(writeSize));
128 }
129 }
130
131 if (readSize > 0) {
132 if (!readReqsLimiter.canExecute()) {
133 ThrottlingException.throwNumReadRequestsExceeded(readReqsLimiter.waitInterval());
134 }
135 if (!readSizeLimiter.canExecute(readSize)) {
136 ThrottlingException.throwReadSizeExceeded(readSizeLimiter.waitInterval(readSize));
137 }
138 }
139 }
140
141 @Override
142 public void grabQuota(long writeSize, long readSize) {
143 assert writeSize != 0 || readSize != 0;
144
145 reqsLimiter.consume(1);
146 reqSizeLimiter.consume(writeSize + readSize);
147
148 if (writeSize > 0) {
149 writeReqsLimiter.consume(1);
150 writeSizeLimiter.consume(writeSize);
151 }
152 if (readSize > 0) {
153 readReqsLimiter.consume(1);
154 readSizeLimiter.consume(readSize);
155 }
156 }
157
158 @Override
159 public void consumeWrite(final long size) {
160 reqSizeLimiter.consume(size);
161 writeSizeLimiter.consume(size);
162 }
163
164 @Override
165 public void consumeRead(final long size) {
166 reqSizeLimiter.consume(size);
167 readSizeLimiter.consume(size);
168 }
169
170 @Override
171 public boolean isBypass() {
172 return false;
173 }
174
175 @Override
176 public long getWriteAvailable() {
177 return writeSizeLimiter.getAvailable();
178 }
179
180 @Override
181 public long getReadAvailable() {
182 return readSizeLimiter.getAvailable();
183 }
184
185 @Override
186 public String toString() {
187 StringBuilder builder = new StringBuilder();
188 builder.append("TimeBasedLimiter(");
189 if (!reqsLimiter.isBypass()) builder.append("reqs=" + reqsLimiter);
190 if (!reqSizeLimiter.isBypass()) builder.append(" resSize=" + reqSizeLimiter);
191 if (!writeReqsLimiter.isBypass()) builder.append(" writeReqs=" + writeReqsLimiter);
192 if (!writeSizeLimiter.isBypass()) builder.append(" writeSize=" + writeSizeLimiter);
193 if (!readReqsLimiter.isBypass()) builder.append(" readReqs=" + readReqsLimiter);
194 if (!readSizeLimiter.isBypass()) builder.append(" readSize=" + readSizeLimiter);
195 builder.append(')');
196 return builder.toString();
197 }
198 }