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.conf.Configuration;
33 import org.apache.hadoop.hbase.DoNotRetryIOException;
34 import org.apache.hadoop.hbase.HConstants;
35 import org.apache.hadoop.hbase.ipc.RpcClient;
36 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
37 import org.apache.hadoop.ipc.RemoteException;
38
39 import com.google.protobuf.ServiceException;
40
41
42
43
44
45
46 @InterfaceAudience.Private
47 @edu.umd.cs.findbugs.annotations.SuppressWarnings
48 (value = "IS2_INCONSISTENT_SYNC", justification = "na")
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 final static int MIN_RPC_TIMEOUT = 2000;
63
64 private final long pause;
65 private final int retries;
66
67 public RpcRetryingCaller(Configuration conf) {
68 this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
69 HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
70 this.retries =
71 conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
72 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
73 this.callTimeout = conf.getInt(
74 HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
75 HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
76 }
77
78 private void beforeCall() {
79 int remaining = (int)(callTimeout -
80 (EnvironmentEdgeManager.currentTimeMillis() - this.globalStartTime));
81 if (remaining < MIN_RPC_TIMEOUT) {
82
83
84
85 remaining = MIN_RPC_TIMEOUT;
86 }
87 RpcClient.setRpcTimeout(remaining);
88 }
89
90 private void afterCall() {
91 RpcClient.resetRpcTimeout();
92 }
93
94 public synchronized T callWithRetries(RetryingCallable<T> callable) throws IOException,
95 RuntimeException {
96 return callWithRetries(callable, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
97 }
98
99
100
101
102
103
104
105
106
107 @edu.umd.cs.findbugs.annotations.SuppressWarnings
108 (value = "SWL_SLEEP_WITH_LOCK_HELD", justification = "na")
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 if (LOG.isTraceEnabled()) {
123 LOG.trace("Call exception, tries=" + tries + ", retries=" + retries + ", retryTime=" +
124 (EnvironmentEdgeManager.currentTimeMillis() - this.globalStartTime) + "ms", t);
125 }
126
127 t = translateException(t);
128 callable.throwable(t, retries != 1);
129 RetriesExhaustedException.ThrowableWithExtraContext qt =
130 new RetriesExhaustedException.ThrowableWithExtraContext(t,
131 EnvironmentEdgeManager.currentTimeMillis(), toString());
132 exceptions.add(qt);
133 if (tries >= retries - 1) {
134 throw new RetriesExhaustedException(tries, exceptions);
135 }
136
137
138
139 expectedSleep = callable.sleep(pause, tries + 1);
140
141
142 long duration = singleCallDuration(expectedSleep);
143 if (duration > this.callTimeout) {
144 String msg = "callTimeout=" + this.callTimeout + ", callDuration=" + duration +
145 ": " + callable.getExceptionMessageAdditionalDetail();
146 throw (SocketTimeoutException)(new SocketTimeoutException(msg).initCause(t));
147 }
148 } finally {
149 afterCall();
150 }
151 try {
152 Thread.sleep(expectedSleep);
153 } catch (InterruptedException e) {
154 Thread.currentThread().interrupt();
155 throw new InterruptedIOException("Interrupted after " + tries + " tries on " + retries);
156 }
157 }
158 }
159
160
161
162
163
164 private long singleCallDuration(final long expectedSleep) {
165 return (EnvironmentEdgeManager.currentTimeMillis() - this.globalStartTime)
166 + MIN_RPC_TIMEOUT + expectedSleep;
167 }
168
169
170
171
172
173
174
175
176
177
178 public T callWithoutRetries(RetryingCallable<T> callable)
179 throws IOException, RuntimeException {
180
181 this.globalStartTime = EnvironmentEdgeManager.currentTimeMillis();
182 try {
183 beforeCall();
184 callable.prepare(false);
185 return callable.call();
186 } catch (Throwable t) {
187 Throwable t2 = translateException(t);
188
189 if (t2 instanceof IOException) {
190 throw (IOException)t2;
191 } else {
192 throw new RuntimeException(t2);
193 }
194 } finally {
195 afterCall();
196 }
197 }
198
199
200
201
202
203
204
205 static Throwable translateException(Throwable t) throws DoNotRetryIOException {
206 if (t instanceof UndeclaredThrowableException) {
207 if (t.getCause() != null) {
208 t = t.getCause();
209 }
210 }
211 if (t instanceof RemoteException) {
212 t = ((RemoteException)t).unwrapRemoteException();
213 }
214 if (t instanceof ServiceException) {
215 ServiceException se = (ServiceException)t;
216 Throwable cause = se.getCause();
217 if (cause != null && cause instanceof DoNotRetryIOException) {
218 throw (DoNotRetryIOException)cause;
219 }
220
221 t = cause;
222
223 translateException(t);
224 } else if (t instanceof DoNotRetryIOException) {
225 throw (DoNotRetryIOException)t;
226 }
227 return t;
228 }
229 }