View Javadoc

1   /**
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    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
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.
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  import org.apache.hadoop.hbase.quotas.OperationQuota.AvgOperationSize;
30  import org.apache.hadoop.hbase.quotas.OperationQuota.OperationType;
31  
32  /**
33   * Simple time based limiter that checks the quota Throttle
34   */
35  @InterfaceAudience.Private
36  @InterfaceStability.Evolving
37  public class TimeBasedLimiter implements QuotaLimiter {
38    private static final Configuration conf = HBaseConfiguration.create();
39    private RateLimiter reqsLimiter = null;
40    private RateLimiter reqSizeLimiter = null;
41    private RateLimiter writeReqsLimiter = null;
42    private RateLimiter writeSizeLimiter = null;
43    private RateLimiter readReqsLimiter = null;
44    private RateLimiter readSizeLimiter = null;
45    private AvgOperationSize avgOpSize = new AvgOperationSize();
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      } else {
58        reqsLimiter = new AverageIntervalRateLimiter();
59        reqSizeLimiter = new AverageIntervalRateLimiter();
60        writeReqsLimiter = new AverageIntervalRateLimiter();
61        writeSizeLimiter = new AverageIntervalRateLimiter();
62        readReqsLimiter = new AverageIntervalRateLimiter();
63        readSizeLimiter = new AverageIntervalRateLimiter();
64      }
65    }
66  
67    static QuotaLimiter fromThrottle(final Throttle throttle) {
68      TimeBasedLimiter limiter = new TimeBasedLimiter();
69      boolean isBypass = true;
70      if (throttle.hasReqNum()) {
71        setFromTimedQuota(limiter.reqsLimiter, throttle.getReqNum());
72        isBypass = false;
73      }
74  
75      if (throttle.hasReqSize()) {
76        setFromTimedQuota(limiter.reqSizeLimiter, throttle.getReqSize());
77        isBypass = false;
78      }
79  
80      if (throttle.hasWriteNum()) {
81        setFromTimedQuota(limiter.writeReqsLimiter, throttle.getWriteNum());
82        isBypass = false;
83      }
84  
85      if (throttle.hasWriteSize()) {
86        setFromTimedQuota(limiter.writeSizeLimiter, throttle.getWriteSize());
87        isBypass = false;
88      }
89  
90      if (throttle.hasReadNum()) {
91        setFromTimedQuota(limiter.readReqsLimiter, throttle.getReadNum());
92        isBypass = false;
93      }
94  
95      if (throttle.hasReadSize()) {
96        setFromTimedQuota(limiter.readSizeLimiter, throttle.getReadSize());
97        isBypass = false;
98      }
99      return isBypass ? NoopQuotaLimiter.get() : limiter;
100   }
101 
102   public void update(final TimeBasedLimiter other) {
103     reqsLimiter.update(other.reqsLimiter);
104     reqSizeLimiter.update(other.reqSizeLimiter);
105     writeReqsLimiter.update(other.writeReqsLimiter);
106     writeSizeLimiter.update(other.writeSizeLimiter);
107     readReqsLimiter.update(other.readReqsLimiter);
108     readSizeLimiter.update(other.readSizeLimiter);
109   }
110 
111   private static void setFromTimedQuota(final RateLimiter limiter, final TimedQuota timedQuota) {
112     limiter.set(timedQuota.getSoftLimit(), ProtobufUtil.toTimeUnit(timedQuota.getTimeUnit()));
113   }
114 
115   @Override
116   public void checkQuota(long writeSize, long readSize) throws ThrottlingException {
117     if (!reqsLimiter.canExecute()) {
118       ThrottlingException.throwNumRequestsExceeded(reqsLimiter.waitInterval());
119     }
120     if (!reqSizeLimiter.canExecute(writeSize + readSize)) {
121       ThrottlingException.throwRequestSizeExceeded(reqSizeLimiter
122           .waitInterval(writeSize + readSize));
123     }
124 
125     if (writeSize > 0) {
126       if (!writeReqsLimiter.canExecute()) {
127         ThrottlingException.throwNumWriteRequestsExceeded(writeReqsLimiter.waitInterval());
128       }
129       if (!writeSizeLimiter.canExecute(writeSize)) {
130         ThrottlingException.throwWriteSizeExceeded(writeSizeLimiter.waitInterval(writeSize));
131       }
132     }
133 
134     if (readSize > 0) {
135       if (!readReqsLimiter.canExecute()) {
136         ThrottlingException.throwNumReadRequestsExceeded(readReqsLimiter.waitInterval());
137       }
138       if (!readSizeLimiter.canExecute(readSize)) {
139         ThrottlingException.throwReadSizeExceeded(readSizeLimiter.waitInterval(readSize));
140       }
141     }
142   }
143 
144   @Override
145   public void grabQuota(long writeSize, long readSize) {
146     assert writeSize != 0 || readSize != 0;
147 
148     reqsLimiter.consume(1);
149     reqSizeLimiter.consume(writeSize + readSize);
150 
151     if (writeSize > 0) {
152       writeReqsLimiter.consume(1);
153       writeSizeLimiter.consume(writeSize);
154     }
155     if (readSize > 0) {
156       readReqsLimiter.consume(1);
157       readSizeLimiter.consume(readSize);
158     }
159   }
160 
161   @Override
162   public void consumeWrite(final long size) {
163     reqSizeLimiter.consume(size);
164     writeSizeLimiter.consume(size);
165   }
166 
167   @Override
168   public void consumeRead(final long size) {
169     reqSizeLimiter.consume(size);
170     readSizeLimiter.consume(size);
171   }
172 
173   @Override
174   public boolean isBypass() {
175     return false;
176   }
177 
178   @Override
179   public long getWriteAvailable() {
180     return writeSizeLimiter.getAvailable();
181   }
182 
183   @Override
184   public long getReadAvailable() {
185     return readSizeLimiter.getAvailable();
186   }
187 
188   @Override
189   public void addOperationSize(OperationType type, long size) {
190     avgOpSize.addOperationSize(type, size);
191   }
192 
193   @Override
194   public long getAvgOperationSize(OperationType type) {
195     return avgOpSize.getAvgOperationSize(type);
196   }
197 
198   @Override
199   public String toString() {
200     StringBuilder builder = new StringBuilder();
201     builder.append("TimeBasedLimiter(");
202     if (!reqsLimiter.isBypass()) builder.append("reqs=" + reqsLimiter);
203     if (!reqSizeLimiter.isBypass()) builder.append(" resSize=" + reqSizeLimiter);
204     if (!writeReqsLimiter.isBypass()) builder.append(" writeReqs=" + writeReqsLimiter);
205     if (!writeSizeLimiter.isBypass()) builder.append(" writeSize=" + writeSizeLimiter);
206     if (!readReqsLimiter.isBypass()) builder.append(" readReqs=" + readReqsLimiter);
207     if (!readSizeLimiter.isBypass()) builder.append(" readSize=" + readSizeLimiter);
208     builder.append(')');
209     return builder.toString();
210   }
211 }