1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.util;
20
21 import java.util.concurrent.TimeUnit;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.hbase.classification.InterfaceAudience;
26
27 @InterfaceAudience.Private
28 public class RetryCounter {
29
30
31
32
33 public static class RetryConfig {
34 private int maxAttempts;
35 private long sleepInterval;
36 private long maxSleepTime;
37 private TimeUnit timeUnit;
38 private BackoffPolicy backoffPolicy;
39
40 private static final BackoffPolicy DEFAULT_BACKOFF_POLICY = new ExponentialBackoffPolicy();
41
42 public RetryConfig() {
43 maxAttempts = 1;
44 sleepInterval = 1000;
45 maxSleepTime = -1;
46 timeUnit = TimeUnit.MILLISECONDS;
47 backoffPolicy = DEFAULT_BACKOFF_POLICY;
48 }
49
50 public RetryConfig(int maxAttempts, long sleepInterval, long maxSleepTime,
51 TimeUnit timeUnit, BackoffPolicy backoffPolicy) {
52 this.maxAttempts = maxAttempts;
53 this.sleepInterval = sleepInterval;
54 this.maxSleepTime = maxSleepTime;
55 this.timeUnit = timeUnit;
56 this.backoffPolicy = backoffPolicy;
57 }
58
59 public RetryConfig setBackoffPolicy(BackoffPolicy backoffPolicy) {
60 this.backoffPolicy = backoffPolicy;
61 return this;
62 }
63
64 public RetryConfig setMaxAttempts(int maxAttempts) {
65 this.maxAttempts = maxAttempts;
66 return this;
67 }
68
69 public RetryConfig setMaxSleepTime(long maxSleepTime) {
70 this.maxSleepTime = maxSleepTime;
71 return this;
72 }
73
74 public RetryConfig setSleepInterval(long sleepInterval) {
75 this.sleepInterval = sleepInterval;
76 return this;
77 }
78
79 public RetryConfig setTimeUnit(TimeUnit timeUnit) {
80 this.timeUnit = timeUnit;
81 return this;
82 }
83
84 public int getMaxAttempts() {
85 return maxAttempts;
86 }
87
88 public long getMaxSleepTime() {
89 return maxSleepTime;
90 }
91
92 public long getSleepInterval() {
93 return sleepInterval;
94 }
95
96 public TimeUnit getTimeUnit() {
97 return timeUnit;
98 }
99
100 public BackoffPolicy getBackoffPolicy() {
101 return backoffPolicy;
102 }
103 }
104
105
106
107
108 public static class BackoffPolicy {
109 public long getBackoffTime(RetryConfig config, int attempts) {
110 return config.getSleepInterval();
111 }
112 }
113
114 public static class ExponentialBackoffPolicy extends BackoffPolicy {
115 @Override
116 public long getBackoffTime(RetryConfig config, int attempts) {
117 long backoffTime = (long) (config.getSleepInterval() * Math.pow(2, attempts));
118 return backoffTime;
119 }
120 }
121
122 public static class ExponentialBackoffPolicyWithLimit extends ExponentialBackoffPolicy {
123 @Override
124 public long getBackoffTime(RetryConfig config, int attempts) {
125 long backoffTime = super.getBackoffTime(config, attempts);
126 return config.getMaxSleepTime() > 0 ? Math.min(backoffTime, config.getMaxSleepTime()) : backoffTime;
127 }
128 }
129
130 private static final Log LOG = LogFactory.getLog(RetryCounter.class);
131
132 private RetryConfig retryConfig;
133 private int attempts;
134
135 public RetryCounter(int maxAttempts, long sleepInterval, TimeUnit timeUnit) {
136 this(new RetryConfig(maxAttempts, sleepInterval, -1, timeUnit, new ExponentialBackoffPolicy()));
137 }
138
139 public RetryCounter(RetryConfig retryConfig) {
140 this.attempts = 0;
141 this.retryConfig = retryConfig;
142 }
143
144 public int getMaxAttempts() {
145 return retryConfig.getMaxAttempts();
146 }
147
148
149
150
151
152 public void sleepUntilNextRetry() throws InterruptedException {
153 int attempts = getAttemptTimes();
154 long sleepTime = retryConfig.backoffPolicy.getBackoffTime(retryConfig, attempts);
155 if (LOG.isTraceEnabled()) {
156 LOG.trace("Sleeping " + sleepTime + "ms before retry #" + attempts + "...");
157 }
158 retryConfig.getTimeUnit().sleep(sleepTime);
159 useRetry();
160 }
161
162 public boolean shouldRetry() {
163 return attempts < retryConfig.getMaxAttempts();
164 }
165
166 public void useRetry() {
167 attempts++;
168 }
169
170 public boolean isRetry() {
171 return attempts > 0;
172 }
173
174 public int getAttemptTimes() {
175 return attempts;
176 }
177 }