001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.quotas;
019
020import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
021import org.apache.yetus.audience.InterfaceAudience;
022import org.apache.yetus.audience.InterfaceStability;
023import org.slf4j.Logger;
024import org.slf4j.LoggerFactory;
025
026/**
027 * With this limiter resources will be refilled only after a fixed interval of time.
028 */
029@InterfaceAudience.Private
030@InterfaceStability.Evolving
031public class FixedIntervalRateLimiter extends RateLimiter {
032
033  /**
034   * The FixedIntervalRateLimiter can be harsh from a latency/backoff perspective, which makes it
035   * difficult to fully and consistently utilize a quota allowance. By configuring the
036   * {@link #RATE_LIMITER_REFILL_INTERVAL_MS} to a lower value you will encourage the rate limiter
037   * to throw smaller wait intervals for requests which may be fulfilled in timeframes shorter than
038   * the quota's full interval. For example, if you're saturating a 100MB/sec read IO quota with a
039   * ton of tiny gets, then configuring this to a value like 100ms will ensure that your retry
040   * backoffs approach ~100ms, rather than 1sec. Be careful not to configure this too low, or you
041   * may produce a dangerous amount of retry volume.
042   */
043  public static final String RATE_LIMITER_REFILL_INTERVAL_MS =
044    "hbase.quota.rate.limiter.refill.interval.ms";
045
046  private static final Logger LOG = LoggerFactory.getLogger(FixedIntervalRateLimiter.class);
047
048  private long nextRefillTime = -1L;
049  private final long refillInterval;
050
051  public FixedIntervalRateLimiter() {
052    this(DEFAULT_TIME_UNIT);
053  }
054
055  public FixedIntervalRateLimiter(long refillInterval) {
056    super();
057    long timeUnit = getTimeUnitInMillis();
058    if (refillInterval > timeUnit) {
059      LOG.warn(
060        "Refill interval {} is larger than time unit {}. This is invalid. "
061          + "Instead, we will use the time unit {} as the refill interval",
062        refillInterval, timeUnit, timeUnit);
063    }
064    this.refillInterval = Math.min(timeUnit, refillInterval);
065  }
066
067  @Override
068  public long refill(long limit) {
069    final long now = EnvironmentEdgeManager.currentTime();
070    if (nextRefillTime == -1) {
071      nextRefillTime = now + refillInterval;
072      return limit;
073    }
074    if (now < nextRefillTime) {
075      return 0;
076    }
077    long diff = refillInterval + now - nextRefillTime;
078    long refills = diff / refillInterval;
079    nextRefillTime = now + refillInterval;
080    long refillAmount = refills * getRefillIntervalAdjustedLimit(limit);
081    return Math.min(limit, refillAmount);
082  }
083
084  @Override
085  public long getWaitInterval(long limit, long available, long amount) {
086    // adjust the limit based on the refill interval
087    limit = getRefillIntervalAdjustedLimit(limit);
088
089    if (nextRefillTime == -1) {
090      return 0;
091    }
092    final long now = EnvironmentEdgeManager.currentTime();
093    final long refillTime = nextRefillTime;
094    long diff = amount - available;
095    // We will add limit at next interval. If diff is less than that limit, the wait interval
096    // is just time between now and then.
097    long nextRefillInterval = refillTime - now;
098    if (diff <= limit) {
099      return nextRefillInterval;
100    }
101
102    // Otherwise, we need to figure out how many refills are needed.
103    // There will be one at nextRefillInterval, and then some number of extra refills.
104    // Division will round down if not even, so we can just add that to our next interval
105    long extraRefillsNecessary = diff / limit;
106    // If it's even, subtract one since that will be covered by nextRefillInterval
107    if (diff % limit == 0) {
108      extraRefillsNecessary--;
109    }
110    return nextRefillInterval + (extraRefillsNecessary * refillInterval);
111  }
112
113  private long getRefillIntervalAdjustedLimit(long limit) {
114    return (long) Math.ceil(refillInterval / (double) getTimeUnitInMillis() * limit);
115  }
116
117  // This method is for strictly testing purpose only
118  @Override
119  public void setNextRefillTime(long nextRefillTime) {
120    this.nextRefillTime = nextRefillTime;
121  }
122
123  @Override
124  public long getNextRefillTime() {
125    return this.nextRefillTime;
126  }
127}