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
;
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>.
37 * @see RetryCounterFactory
39 @InterfaceAudience.Private
40 public class RetryCounter
{
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
;
52 private static final BackoffPolicy DEFAULT_BACKOFF_POLICY
= new ExponentialBackoffPolicy();
54 public RetryConfig() {
58 timeUnit
= TimeUnit
.MILLISECONDS
;
59 backoffPolicy
= DEFAULT_BACKOFF_POLICY
;
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
;
77 public RetryConfig
setMaxAttempts(int maxAttempts
) {
78 this.maxAttempts
= maxAttempts
;
82 public RetryConfig
setMaxSleepTime(long maxSleepTime
) {
83 this.maxSleepTime
= maxSleepTime
;
87 public RetryConfig
setSleepInterval(long sleepInterval
) {
88 this.sleepInterval
= sleepInterval
;
92 public RetryConfig
setTimeUnit(TimeUnit timeUnit
) {
93 this.timeUnit
= timeUnit
;
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
;
104 public int getMaxAttempts() {
108 public long getMaxSleepTime() {
112 public long getSleepInterval() {
113 return sleepInterval
;
116 public TimeUnit
getTimeUnit() {
120 public float getJitter() {
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
{
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
{
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
) {
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
);
188 public boolean shouldRetry() {
189 return attempts
< retryConfig
.getMaxAttempts();
192 public void useRetry() {
196 public boolean isRetry() {
200 public int getAttemptTimes() {
204 public long getBackoffTime() {
205 return this.retryConfig
.backoffPolicy
.getBackoffTime(this.retryConfig
, getAttemptTimes());
208 public long getBackoffTimeAndIncrementAttempts() {
209 long backoffTime
= getBackoffTime();