HBASE-24163 MOB compactor implementations should use format specifiers when calling...
[hbase.git] / hbase-common / src / main / java / org / apache / hadoop / hbase / util / RetryCounter.java
bloba2f09d08170d3ff510f0ba13540cc9d50504a3d3
1 /**
3 * Licensed to the Apache Software Foundation (ASF) under one
4 * or more contributor license agreements. See the NOTICE file
5 * distributed with this work for additional information
6 * regarding copyright ownership. The ASF licenses this file
7 * to you under the Apache License, Version 2.0 (the
8 * "License"); you may not use this file except in compliance
9 * with the License. You may obtain a copy of the License at
11 * http://www.apache.org/licenses/LICENSE-2.0
13 * Unless required by applicable law or agreed to in writing, software
14 * distributed under the License is distributed on an "AS IS" BASIS,
15 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
19 package org.apache.hadoop.hbase.util;
21 import java.util.concurrent.ThreadLocalRandom;
22 import java.util.concurrent.TimeUnit;
24 import org.apache.yetus.audience.InterfaceAudience;
25 import org.slf4j.Logger;
26 import org.slf4j.LoggerFactory;
28 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
30 /**
31 * Operation retry accounting.
32 * Use to calculate wait period, {@link #getBackoffTimeAndIncrementAttempts()}}, or for performing
33 * wait, {@link #sleepUntilNextRetry()}, in accordance with a {@link RetryConfig}, initial
34 * settings, and a Retry Policy, (See org.apache.hadoop.io.retry.RetryPolicy).
35 * Like <a href=https://github.com/rholder/guava-retrying>guava-retrying</a>.
36 * @since 0.92.0
37 * @see RetryCounterFactory
39 @InterfaceAudience.Private
40 public class RetryCounter {
41 /**
42 * Configuration for a retry counter
44 public static class RetryConfig {
45 private int maxAttempts;
46 private long sleepInterval;
47 private long maxSleepTime;
48 private TimeUnit timeUnit;
49 private BackoffPolicy backoffPolicy;
50 private float jitter;
52 private static final BackoffPolicy DEFAULT_BACKOFF_POLICY = new ExponentialBackoffPolicy();
54 public RetryConfig() {
55 maxAttempts = 1;
56 sleepInterval = 1000;
57 maxSleepTime = -1;
58 timeUnit = TimeUnit.MILLISECONDS;
59 backoffPolicy = DEFAULT_BACKOFF_POLICY;
60 jitter = 0.0f;
63 public RetryConfig(int maxAttempts, long sleepInterval, long maxSleepTime,
64 TimeUnit timeUnit, BackoffPolicy backoffPolicy) {
65 this.maxAttempts = maxAttempts;
66 this.sleepInterval = sleepInterval;
67 this.maxSleepTime = maxSleepTime;
68 this.timeUnit = timeUnit;
69 this.backoffPolicy = backoffPolicy;
72 public RetryConfig setBackoffPolicy(BackoffPolicy backoffPolicy) {
73 this.backoffPolicy = backoffPolicy;
74 return this;
77 public RetryConfig setMaxAttempts(int maxAttempts) {
78 this.maxAttempts = maxAttempts;
79 return this;
82 public RetryConfig setMaxSleepTime(long maxSleepTime) {
83 this.maxSleepTime = maxSleepTime;
84 return this;
87 public RetryConfig setSleepInterval(long sleepInterval) {
88 this.sleepInterval = sleepInterval;
89 return this;
92 public RetryConfig setTimeUnit(TimeUnit timeUnit) {
93 this.timeUnit = timeUnit;
94 return this;
97 public RetryConfig setJitter(float jitter) {
98 Preconditions.checkArgument(jitter >= 0.0f && jitter < 1.0f,
99 "Invalid jitter: %s, should be in range [0.0, 1.0)", jitter);
100 this.jitter = jitter;
101 return this;
104 public int getMaxAttempts() {
105 return maxAttempts;
108 public long getMaxSleepTime() {
109 return maxSleepTime;
112 public long getSleepInterval() {
113 return sleepInterval;
116 public TimeUnit getTimeUnit() {
117 return timeUnit;
120 public float getJitter() {
121 return jitter;
124 public BackoffPolicy getBackoffPolicy() {
125 return backoffPolicy;
129 private static long addJitter(long interval, float jitter) {
130 long jitterInterval = (long) (interval * ThreadLocalRandom.current().nextFloat() * jitter);
131 return interval + jitterInterval;
135 * Policy for calculating sleeping intervals between retry attempts
137 public static class BackoffPolicy {
138 public long getBackoffTime(RetryConfig config, int attempts) {
139 return addJitter(config.getSleepInterval(), config.getJitter());
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());
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;
159 private static final Logger LOG = LoggerFactory.getLogger(RetryCounter.class);
161 private RetryConfig retryConfig;
162 private int attempts;
164 public RetryCounter(int maxAttempts, long sleepInterval, TimeUnit timeUnit) {
165 this(new RetryConfig(maxAttempts, sleepInterval, -1, timeUnit, new ExponentialBackoffPolicy()));
168 public RetryCounter(RetryConfig retryConfig) {
169 this.attempts = 0;
170 this.retryConfig = retryConfig;
173 public int getMaxAttempts() {
174 return retryConfig.getMaxAttempts();
178 * Sleep for a back off time as supplied by the backoff policy, and increases the attempts
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();
188 public boolean shouldRetry() {
189 return attempts < retryConfig.getMaxAttempts();
192 public void useRetry() {
193 attempts++;
196 public boolean isRetry() {
197 return attempts > 0;
200 public int getAttemptTimes() {
201 return attempts;
204 public long getBackoffTime() {
205 return this.retryConfig.backoffPolicy.getBackoffTime(this.retryConfig, getAttemptTimes());
208 public long getBackoffTimeAndIncrementAttempts() {
209 long backoffTime = getBackoffTime();
210 useRetry();
211 return backoffTime;