1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.client;
21
22 import java.io.IOException;
23 import java.io.InterruptedIOException;
24 import java.lang.reflect.UndeclaredThrowableException;
25 import java.net.SocketTimeoutException;
26 import java.util.ArrayList;
27 import java.util.List;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.classification.InterfaceAudience;
32 import org.apache.hadoop.classification.InterfaceStability;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.hbase.HConstants;
35 import org.apache.hadoop.hbase.DoNotRetryIOException;
36 import org.apache.hadoop.hbase.ipc.RpcClient;
37 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
38 import org.apache.hadoop.ipc.RemoteException;
39
40 import com.google.protobuf.ServiceException;
41
42
43
44
45
46
47 @InterfaceAudience.Public
48 @InterfaceStability.Stable
49 public class RpcRetryingCaller<T> {
50 static final Log LOG = LogFactory.getLog(RpcRetryingCaller.class);
51
52
53
54 private int callTimeout;
55
56
57
58 private long globalStartTime;
59
60
61
62 private long startTime, endTime;
63 private final static int MIN_RPC_TIMEOUT = 2000;
64
65 private final long pause;
66 private final int retries;
67
68 public RpcRetryingCaller(Configuration conf) {
69 this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
70 HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
71 this.retries =
72 conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
73 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
74 this.callTimeout = conf.getInt(
75 HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
76 HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
77 }
78
79 private void beforeCall() {
80 this.startTime = EnvironmentEdgeManager.currentTimeMillis();
81 int remaining = (int)(callTimeout - (this.startTime - this.globalStartTime));
82 if (remaining < MIN_RPC_TIMEOUT) {
83
84
85
86 remaining = MIN_RPC_TIMEOUT;
87 }
88 RpcClient.setRpcTimeout(remaining);
89 }
90
91 private void afterCall() {
92 RpcClient.resetRpcTimeout();
93 this.endTime = EnvironmentEdgeManager.currentTimeMillis();
94 }
95
96 public synchronized T callWithRetries(RetryingCallable<T> callable) throws IOException,
97 RuntimeException {
98 return callWithRetries(callable, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
99 }
100
101
102
103
104
105
106
107
108
109 public synchronized T callWithRetries(RetryingCallable<T> callable, int callTimeout)
110 throws IOException, RuntimeException {
111 this.callTimeout = callTimeout;
112 List<RetriesExhaustedException.ThrowableWithExtraContext> exceptions =
113 new ArrayList<RetriesExhaustedException.ThrowableWithExtraContext>();
114 this.globalStartTime = EnvironmentEdgeManager.currentTimeMillis();
115 for (int tries = 0;; tries++) {
116 long expectedSleep = 0;
117 try {
118 beforeCall();
119 callable.prepare(tries != 0);
120 return callable.call();
121 } catch (Throwable t) {
122 LOG.warn("Call exception, tries=" + tries + ", retries=" + retries + ", retryTime=" +
123 (this.globalStartTime - System.currentTimeMillis()) + "ms", t);
124
125 t = translateException(t);
126 callable.throwable(t, retries != 1);
127 RetriesExhaustedException.ThrowableWithExtraContext qt =
128 new RetriesExhaustedException.ThrowableWithExtraContext(t,
129 EnvironmentEdgeManager.currentTimeMillis(), toString());
130 exceptions.add(qt);
131 if (tries >= retries - 1) {
132 throw new RetriesExhaustedException(tries, exceptions);
133 }
134
135
136
137 expectedSleep = callable.sleep(pause, tries + 1);
138
139
140 long duration = singleCallDuration(expectedSleep);
141 if (duration > this.callTimeout) {
142 String msg = "callTimeout=" + this.callTimeout + ", callDuration=" + duration +
143 ": " + callable.getExceptionMessageAdditionalDetail();
144 throw (SocketTimeoutException)(new SocketTimeoutException(msg).initCause(t));
145 }
146 } finally {
147 afterCall();
148 }
149 try {
150 Thread.sleep(expectedSleep);
151 } catch (InterruptedException e) {
152 Thread.currentThread().interrupt();
153 throw new InterruptedIOException("Interrupted after " + tries + " tries on " + retries);
154 }
155 }
156 }
157
158
159
160
161
162 private long singleCallDuration(final long expectedSleep) {
163 return (EnvironmentEdgeManager.currentTimeMillis() - this.globalStartTime)
164 + MIN_RPC_TIMEOUT + expectedSleep;
165 }
166
167
168
169
170
171
172
173
174
175
176 public T callWithoutRetries(RetryingCallable<T> callable)
177 throws IOException, RuntimeException {
178
179 this.globalStartTime = EnvironmentEdgeManager.currentTimeMillis();
180 try {
181 beforeCall();
182 callable.prepare(false);
183 return callable.call();
184 } catch (Throwable t) {
185 Throwable t2 = translateException(t);
186
187 if (t2 instanceof IOException) {
188 throw (IOException)t2;
189 } else {
190 throw new RuntimeException(t2);
191 }
192 } finally {
193 afterCall();
194 }
195 }
196
197
198
199
200
201
202
203 static Throwable translateException(Throwable t) throws DoNotRetryIOException {
204 if (t instanceof UndeclaredThrowableException) {
205 if (t.getCause() != null) {
206 t = t.getCause();
207 }
208 }
209 if (t instanceof RemoteException) {
210 t = ((RemoteException)t).unwrapRemoteException();
211 }
212 if (t instanceof ServiceException) {
213 ServiceException se = (ServiceException)t;
214 Throwable cause = se.getCause();
215 if (cause != null && cause instanceof DoNotRetryIOException) {
216 throw (DoNotRetryIOException)cause;
217 }
218
219 t = cause;
220
221 translateException(t);
222 } else if (t instanceof DoNotRetryIOException) {
223 throw (DoNotRetryIOException)t;
224 }
225 return t;
226 }
227 }