001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.util;
020
021import java.util.concurrent.ThreadLocalRandom;
022import java.util.concurrent.TimeUnit;
023
024import org.apache.yetus.audience.InterfaceAudience;
025import org.slf4j.Logger;
026import org.slf4j.LoggerFactory;
027
028import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
029
030/**
031 * Operation retry accounting.
032 * Use to calculate wait period, {@link #getBackoffTimeAndIncrementAttempts()}}, or for performing
033 * wait, {@link #sleepUntilNextRetry()}, in accordance with a {@link RetryConfig}, initial
034 * settings, and a Retry Policy, (See org.apache.hadoop.io.retry.RetryPolicy).
035 * Like <a href=https://github.com/rholder/guava-retrying>guava-retrying</a>.
036 * @since 0.92.0
037 * @see RetryCounterFactory
038 */
039@InterfaceAudience.Private
040public class RetryCounter {
041  /**
042   *  Configuration for a retry counter
043   */
044  public static class RetryConfig {
045    private int maxAttempts;
046    private long sleepInterval;
047    private long maxSleepTime;
048    private TimeUnit timeUnit;
049    private BackoffPolicy backoffPolicy;
050    private float jitter;
051
052    private static final BackoffPolicy DEFAULT_BACKOFF_POLICY = new ExponentialBackoffPolicy();
053
054    public RetryConfig() {
055      maxAttempts    = 1;
056      sleepInterval = 1000;
057      maxSleepTime  = -1;
058      timeUnit = TimeUnit.MILLISECONDS;
059      backoffPolicy = DEFAULT_BACKOFF_POLICY;
060      jitter = 0.0f;
061    }
062
063    public RetryConfig(int maxAttempts, long sleepInterval, long maxSleepTime,
064        TimeUnit timeUnit, BackoffPolicy backoffPolicy) {
065      this.maxAttempts = maxAttempts;
066      this.sleepInterval = sleepInterval;
067      this.maxSleepTime = maxSleepTime;
068      this.timeUnit = timeUnit;
069      this.backoffPolicy = backoffPolicy;
070    }
071
072    public RetryConfig setBackoffPolicy(BackoffPolicy backoffPolicy) {
073      this.backoffPolicy = backoffPolicy;
074      return this;
075    }
076
077    public RetryConfig setMaxAttempts(int maxAttempts) {
078      this.maxAttempts = maxAttempts;
079      return this;
080    }
081
082    public RetryConfig setMaxSleepTime(long maxSleepTime) {
083      this.maxSleepTime = maxSleepTime;
084      return this;
085    }
086
087    public RetryConfig setSleepInterval(long sleepInterval) {
088      this.sleepInterval = sleepInterval;
089      return this;
090    }
091
092    public RetryConfig setTimeUnit(TimeUnit timeUnit) {
093      this.timeUnit = timeUnit;
094      return this;
095    }
096
097    public RetryConfig setJitter(float jitter) {
098      Preconditions.checkArgument(jitter >= 0.0f && jitter < 1.0f,
099        "Invalid jitter: %s, should be in range [0.0, 1.0)", jitter);
100      this.jitter = jitter;
101      return this;
102    }
103
104    public int getMaxAttempts() {
105      return maxAttempts;
106    }
107
108    public long getMaxSleepTime() {
109      return maxSleepTime;
110    }
111
112    public long getSleepInterval() {
113      return sleepInterval;
114    }
115
116    public TimeUnit getTimeUnit() {
117      return timeUnit;
118    }
119
120    public float getJitter() {
121      return jitter;
122    }
123
124    public BackoffPolicy getBackoffPolicy() {
125      return backoffPolicy;
126    }
127  }
128
129  private static long addJitter(long interval, float jitter) {
130    long jitterInterval = (long) (interval * ThreadLocalRandom.current().nextFloat() * jitter);
131    return interval + jitterInterval;
132  }
133
134  /**
135   * Policy for calculating sleeping intervals between retry attempts
136   */
137  public static class BackoffPolicy {
138    public long getBackoffTime(RetryConfig config, int attempts) {
139      return addJitter(config.getSleepInterval(), config.getJitter());
140    }
141  }
142
143  public static class ExponentialBackoffPolicy extends BackoffPolicy {
144    @Override
145    public long getBackoffTime(RetryConfig config, int attempts) {
146      long backoffTime = (long) (config.getSleepInterval() * Math.pow(2, attempts));
147      return addJitter(backoffTime, config.getJitter());
148    }
149  }
150
151  public static class ExponentialBackoffPolicyWithLimit extends ExponentialBackoffPolicy {
152    @Override
153    public long getBackoffTime(RetryConfig config, int attempts) {
154      long backoffTime = super.getBackoffTime(config, attempts);
155      return config.getMaxSleepTime() > 0 ? Math.min(backoffTime, config.getMaxSleepTime()) : backoffTime;
156    }
157  }
158
159  private static final Logger LOG = LoggerFactory.getLogger(RetryCounter.class);
160
161  private RetryConfig retryConfig;
162  private int attempts;
163
164  public RetryCounter(int maxAttempts, long sleepInterval, TimeUnit timeUnit) {
165    this(new RetryConfig(maxAttempts, sleepInterval, -1, timeUnit, new ExponentialBackoffPolicy()));
166  }
167
168  public RetryCounter(RetryConfig retryConfig) {
169    this.attempts = 0;
170    this.retryConfig = retryConfig;
171  }
172
173  public int getMaxAttempts() {
174    return retryConfig.getMaxAttempts();
175  }
176
177  /**
178   * Sleep for a back off time as supplied by the backoff policy, and increases the attempts
179   */
180  public void sleepUntilNextRetry() throws InterruptedException {
181    int attempts = getAttemptTimes();
182    long sleepTime = getBackoffTime();
183    LOG.trace("Sleeping {} ms before retry #{}...", sleepTime, attempts);
184    retryConfig.getTimeUnit().sleep(sleepTime);
185    useRetry();
186  }
187
188  public boolean shouldRetry() {
189    return attempts < retryConfig.getMaxAttempts();
190  }
191
192  public void useRetry() {
193    attempts++;
194  }
195
196  public boolean isRetry() {
197    return attempts > 0;
198  }
199
200  public int getAttemptTimes() {
201    return attempts;
202  }
203
204  public long getBackoffTime() {
205    return this.retryConfig.backoffPolicy.getBackoffTime(this.retryConfig, getAttemptTimes());
206  }
207
208  public long getBackoffTimeAndIncrementAttempts() {
209    long backoffTime = getBackoffTime();
210    useRetry();
211    return backoffTime;
212  }
213}