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 org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import com.google.protobuf.ServiceException;
25 import org.apache.hadoop.classification.InterfaceAudience;
26 import org.apache.hadoop.classification.InterfaceStability;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.hbase.HConstants;
29 import org.apache.hadoop.hbase.HRegionLocation;
30 import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
31 import org.apache.hadoop.hbase.ipc.HBaseClientRPC;
32 import org.apache.hadoop.hbase.util.Bytes;
33 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
34 import org.apache.hadoop.ipc.RemoteException;
35
36 import java.io.IOException;
37 import java.lang.reflect.UndeclaredThrowableException;
38 import java.net.ConnectException;
39 import java.net.SocketTimeoutException;
40 import java.util.ArrayList;
41 import java.util.List;
42 import java.util.concurrent.Callable;
43
44
45
46
47
48
49
50
51
52
53
54
55 @InterfaceAudience.Public
56 @InterfaceStability.Stable
57 public abstract class ServerCallable<T> implements Callable<T> {
58 static final Log LOG = LogFactory.getLog(ServerCallable.class);
59
60 protected final HConnection connection;
61 protected final byte [] tableName;
62 protected final byte [] row;
63 protected HRegionLocation location;
64 protected ClientProtocol server;
65 protected int callTimeout;
66 protected long globalStartTime;
67 protected long startTime, endTime;
68 protected final static int MIN_RPC_TIMEOUT = 2000;
69 protected final static int MIN_WAIT_DEAD_SERVER = 10000;
70
71
72
73
74
75
76 public ServerCallable(HConnection connection, byte [] tableName, byte [] row) {
77 this(connection, tableName, row, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
78 }
79
80 public ServerCallable(HConnection connection, byte [] tableName, byte [] row, int callTimeout) {
81 this.connection = connection;
82 this.tableName = tableName;
83 this.row = row;
84 this.callTimeout = callTimeout;
85 }
86
87
88
89
90
91
92 public void connect(final boolean reload) throws IOException {
93 this.location = connection.getRegionLocation(tableName, row, reload);
94 this.server = connection.getClient(location.getServerName());
95 }
96
97
98
99
100 public String getServerName() {
101 if (location == null) return null;
102 return location.getHostnamePort();
103 }
104
105
106
107
108 public byte[] getRegionName() {
109 if (location == null) return null;
110 return location.getRegionInfo().getRegionName();
111 }
112
113
114
115
116 public byte [] getRow() {
117 return row;
118 }
119
120 public void beforeCall() {
121 this.startTime = EnvironmentEdgeManager.currentTimeMillis();
122 int remaining = (int)(callTimeout - (this.startTime - this.globalStartTime));
123 if (remaining < MIN_RPC_TIMEOUT) {
124
125
126
127 remaining = MIN_RPC_TIMEOUT;
128 }
129 HBaseClientRPC.setRpcTimeout(remaining);
130 }
131
132 public void afterCall() {
133 HBaseClientRPC.resetRpcTimeout();
134 this.endTime = EnvironmentEdgeManager.currentTimeMillis();
135 }
136
137
138
139
140 HConnection getConnection() {
141 return this.connection;
142 }
143
144
145
146
147
148
149
150
151
152 public T withRetries()
153 throws IOException, RuntimeException {
154 Configuration c = getConnection().getConfiguration();
155 final long pause = c.getLong(HConstants.HBASE_CLIENT_PAUSE,
156 HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
157 final int numRetries = c.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
158 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
159 List<RetriesExhaustedException.ThrowableWithExtraContext> exceptions =
160 new ArrayList<RetriesExhaustedException.ThrowableWithExtraContext>();
161 this.globalStartTime = EnvironmentEdgeManager.currentTimeMillis();
162 for (int tries = 0;; tries++) {
163 long expectedSleep = 0;
164 try {
165 beforeCall();
166 connect(tries != 0);
167 return call();
168 } catch (Throwable t) {
169 LOG.warn("Received exception, tries=" + tries + ", numRetries=" + numRetries +
170 " message=" + t.getMessage());
171
172 t = translateException(t);
173
174
175
176 if (t instanceof SocketTimeoutException ||
177 t instanceof ConnectException ||
178 t instanceof RetriesExhaustedException ||
179 getConnection().isDeadServer(location.getServerName())) {
180
181
182
183 getConnection().clearCaches(location.getServerName());
184 }
185
186 RetriesExhaustedException.ThrowableWithExtraContext qt =
187 new RetriesExhaustedException.ThrowableWithExtraContext(t,
188 EnvironmentEdgeManager.currentTimeMillis(), toString());
189 exceptions.add(qt);
190 if (tries >= numRetries - 1) {
191 throw new RetriesExhaustedException(tries, exceptions);
192 }
193
194
195
196 expectedSleep = ConnectionUtils.getPauseTime(pause, tries);
197 if (expectedSleep < MIN_WAIT_DEAD_SERVER &&
198 getConnection().isDeadServer(location.getServerName())){
199 expectedSleep = ConnectionUtils.addJitter(MIN_WAIT_DEAD_SERVER, 0.10f);
200 }
201
202
203 if (((this.endTime - this.globalStartTime) + MIN_RPC_TIMEOUT + expectedSleep) >
204 this.callTimeout) {
205 throw (SocketTimeoutException) new SocketTimeoutException(
206 "Call to access row '" + Bytes.toString(row) + "' on table '"
207 + Bytes.toString(tableName)
208 + "' failed on timeout. " + " callTimeout=" + this.callTimeout +
209 ", time=" + (this.endTime - this.startTime)).initCause(t);
210 }
211 } finally {
212 afterCall();
213 }
214 try {
215 Thread.sleep(expectedSleep);
216 } catch (InterruptedException e) {
217 Thread.currentThread().interrupt();
218 throw new IOException("Interrupted after " + tries + " tries on " + numRetries, e);
219 }
220 }
221 }
222
223
224
225
226
227
228
229 public T withoutRetries()
230 throws IOException, RuntimeException {
231
232 this.globalStartTime = EnvironmentEdgeManager.currentTimeMillis();
233 try {
234 beforeCall();
235 connect(false);
236 return call();
237 } catch (Throwable t) {
238 Throwable t2 = translateException(t);
239
240 if (t2 instanceof IOException) {
241 throw (IOException)t2;
242 } else {
243 throw new RuntimeException(t2);
244 }
245 } finally {
246 afterCall();
247 }
248 }
249
250
251
252
253
254
255
256 protected static Throwable translateException(Throwable t) throws DoNotRetryIOException {
257 if (t instanceof UndeclaredThrowableException) {
258 t = t.getCause();
259 }
260 if (t instanceof RemoteException) {
261 t = ((RemoteException)t).unwrapRemoteException();
262 }
263 if (t instanceof ServiceException) {
264 ServiceException se = (ServiceException)t;
265 Throwable cause = se.getCause();
266 if (cause != null && cause instanceof DoNotRetryIOException) {
267 throw (DoNotRetryIOException)cause;
268 }
269 } else if (t instanceof DoNotRetryIOException) {
270 throw (DoNotRetryIOException)t;
271 }
272 return t;
273 }
274 }