1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.client;
19
20 import java.io.IOException;
21 import java.util.Map.Entry;
22 import java.util.concurrent.ConcurrentHashMap;
23 import java.util.concurrent.ConcurrentMap;
24
25 import org.apache.commons.lang.mutable.MutableBoolean;
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.ServerName;
31 import org.apache.hadoop.hbase.classification.InterfaceAudience;
32 import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
33 import org.apache.hadoop.hbase.exceptions.PreemptiveFastFailException;
34 import org.apache.hadoop.hbase.ipc.CallTimeoutException;
35 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
36 import org.apache.hadoop.ipc.RemoteException;
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65 @InterfaceAudience.Private
66 class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
67
68 private static final Log LOG = LogFactory
69 .getLog(PreemptiveFastFailInterceptor.class);
70
71
72
73 protected final long fastFailThresholdMilliSec;
74
75
76
77 protected final ConcurrentMap<ServerName, FailureInfo> repeatedFailuresMap =
78 new ConcurrentHashMap<ServerName, FailureInfo>();
79
80
81
82
83 protected final long failureMapCleanupIntervalMilliSec;
84
85 protected volatile long lastFailureMapCleanupTimeMilliSec;
86
87
88
89
90 private long fastFailClearingTimeMilliSec;
91
92 private final ThreadLocal<MutableBoolean> threadRetryingInFastFailMode =
93 new ThreadLocal<MutableBoolean>();
94
95 public PreemptiveFastFailInterceptor(Configuration conf) {
96 this.fastFailThresholdMilliSec = conf.getLong(
97 HConstants.HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS,
98 HConstants.HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS_DEFAULT);
99 this.failureMapCleanupIntervalMilliSec = conf.getLong(
100 HConstants.HBASE_CLIENT_FAST_FAIL_CLEANUP_MS_DURATION_MS,
101 HConstants.HBASE_CLIENT_FAST_FAIL_CLEANUP_DURATION_MS_DEFAULT);
102 lastFailureMapCleanupTimeMilliSec = EnvironmentEdgeManager.currentTime();
103 }
104
105 public void intercept(FastFailInterceptorContext context)
106 throws PreemptiveFastFailException {
107 context.setFailureInfo(repeatedFailuresMap.get(context.getServer()));
108 if (inFastFailMode(context.getServer()) && !currentThreadInFastFailMode()) {
109
110
111 context.setRetryDespiteFastFailMode(shouldRetryInspiteOfFastFail(context
112 .getFailureInfo()));
113 if (!context.isRetryDespiteFastFailMode()) {
114 LOG.debug("Throwing PFFE : " + context.getFailureInfo() + " tries : "
115 + context.getTries());
116 throw new PreemptiveFastFailException(
117 context.getFailureInfo().numConsecutiveFailures.get(),
118 context.getFailureInfo().timeOfFirstFailureMilliSec,
119 context.getFailureInfo().timeOfLatestAttemptMilliSec, context.getServer(),
120 context.getGuaranteedClientSideOnly().isTrue());
121 }
122 }
123 context.setDidTry(true);
124 }
125
126 public void handleFailure(FastFailInterceptorContext context,
127 Throwable t) throws IOException {
128 handleThrowable(t, context.getServer(),
129 context.getCouldNotCommunicateWithServer(),
130 context.getGuaranteedClientSideOnly());
131 }
132
133 public void updateFailureInfo(FastFailInterceptorContext context) {
134 updateFailureInfoForServer(context.getServer(), context.getFailureInfo(),
135 context.didTry(), context.getCouldNotCommunicateWithServer()
136 .booleanValue(), context.isRetryDespiteFastFailMode());
137 }
138
139
140
141
142
143
144
145
146
147
148
149
150 protected void handleFailureToServer(ServerName serverName, Throwable t) {
151 if (serverName == null || t == null) {
152 return;
153 }
154 long currentTime = EnvironmentEdgeManager.currentTime();
155 FailureInfo fInfo = repeatedFailuresMap.get(serverName);
156 if (fInfo == null) {
157 fInfo = new FailureInfo(currentTime);
158 FailureInfo oldfInfo = repeatedFailuresMap.putIfAbsent(serverName, fInfo);
159
160 if (oldfInfo != null) {
161 fInfo = oldfInfo;
162 }
163 }
164 fInfo.timeOfLatestAttemptMilliSec = currentTime;
165 fInfo.numConsecutiveFailures.incrementAndGet();
166 }
167
168 public void handleThrowable(Throwable t1, ServerName serverName,
169 MutableBoolean couldNotCommunicateWithServer,
170 MutableBoolean guaranteedClientSideOnly) throws IOException {
171 Throwable t2 = ClientExceptionsUtil.translatePFFE(t1);
172 boolean isLocalException = !(t2 instanceof RemoteException);
173
174 if ((isLocalException && ClientExceptionsUtil.isConnectionException(t2))) {
175 couldNotCommunicateWithServer.setValue(true);
176 guaranteedClientSideOnly.setValue(!(t2 instanceof CallTimeoutException));
177 handleFailureToServer(serverName, t2);
178 }
179 }
180
181
182
183
184
185
186
187
188
189 protected void occasionallyCleanupFailureInformation() {
190 long now = System.currentTimeMillis();
191 if (!(now > lastFailureMapCleanupTimeMilliSec
192 + failureMapCleanupIntervalMilliSec))
193 return;
194
195
196
197
198 StringBuilder sb = new StringBuilder();
199 for (Entry<ServerName, FailureInfo> entry : repeatedFailuresMap.entrySet()) {
200 if (now > entry.getValue().timeOfLatestAttemptMilliSec
201 + failureMapCleanupIntervalMilliSec) {
202 repeatedFailuresMap.remove(entry.getKey());
203 } else if (now > entry.getValue().timeOfFirstFailureMilliSec
204 + this.fastFailClearingTimeMilliSec) {
205
206 LOG.error(entry.getKey()
207 + " been failing for a long time. clearing out."
208 + entry.getValue().toString());
209 repeatedFailuresMap.remove(entry.getKey());
210 } else {
211 sb.append(entry.getKey().toString()).append(" failing ")
212 .append(entry.getValue().toString()).append("\n");
213 }
214 }
215 if (sb.length() > 0) {
216 LOG.warn("Preemptive failure enabled for : " + sb.toString());
217 }
218 lastFailureMapCleanupTimeMilliSec = now;
219 }
220
221
222
223
224
225
226
227
228
229
230 private boolean inFastFailMode(ServerName server) {
231 FailureInfo fInfo = repeatedFailuresMap.get(server);
232
233
234
235 return (fInfo != null &&
236 EnvironmentEdgeManager.currentTime() >
237 (fInfo.timeOfFirstFailureMilliSec + this.fastFailThresholdMilliSec));
238 }
239
240
241
242
243
244
245
246 private boolean currentThreadInFastFailMode() {
247 return (this.threadRetryingInFastFailMode.get() != null && (this.threadRetryingInFastFailMode
248 .get().booleanValue() == true));
249 }
250
251
252
253
254
255
256
257
258
259
260
261
262 protected boolean shouldRetryInspiteOfFastFail(FailureInfo fInfo) {
263
264
265
266
267 if (fInfo != null
268 && fInfo.exclusivelyRetringInspiteOfFastFail.compareAndSet(false, true)) {
269 MutableBoolean threadAlreadyInFF = this.threadRetryingInFastFailMode
270 .get();
271 if (threadAlreadyInFF == null) {
272 threadAlreadyInFF = new MutableBoolean();
273 this.threadRetryingInFastFailMode.set(threadAlreadyInFF);
274 }
275 threadAlreadyInFF.setValue(true);
276 return true;
277 } else {
278 return false;
279 }
280 }
281
282
283
284
285
286
287
288
289
290
291
292 private void updateFailureInfoForServer(ServerName server,
293 FailureInfo fInfo, boolean didTry, boolean couldNotCommunicate,
294 boolean retryDespiteFastFailMode) {
295 if (server == null || fInfo == null || didTry == false)
296 return;
297
298
299
300 if (couldNotCommunicate == false) {
301 LOG.info("Clearing out PFFE for server " + server);
302 repeatedFailuresMap.remove(server);
303 } else {
304
305 long currentTime = System.currentTimeMillis();
306 fInfo.timeOfLatestAttemptMilliSec = currentTime;
307
308
309 if (retryDespiteFastFailMode) {
310 fInfo.exclusivelyRetringInspiteOfFastFail.set(false);
311 threadRetryingInFastFailMode.get().setValue(false);
312 }
313 }
314
315 occasionallyCleanupFailureInformation();
316 }
317
318 @Override
319 public void intercept(RetryingCallerInterceptorContext context)
320 throws PreemptiveFastFailException {
321 if (context instanceof FastFailInterceptorContext) {
322 intercept((FastFailInterceptorContext) context);
323 }
324 }
325
326 @Override
327 public void handleFailure(RetryingCallerInterceptorContext context,
328 Throwable t) throws IOException {
329 if (context instanceof FastFailInterceptorContext) {
330 handleFailure((FastFailInterceptorContext) context, t);
331 }
332 }
333
334 @Override
335 public void updateFailureInfo(RetryingCallerInterceptorContext context) {
336 if (context instanceof FastFailInterceptorContext) {
337 updateFailureInfo((FastFailInterceptorContext) context);
338 }
339 }
340
341 @Override
342 public RetryingCallerInterceptorContext createEmptyContext() {
343 return new FastFailInterceptorContext();
344 }
345
346 protected boolean isServerInFailureMap(ServerName serverName) {
347 return this.repeatedFailuresMap.containsKey(serverName);
348 }
349
350 @Override
351 public String toString() {
352 return "PreemptiveFastFailInterceptor";
353 }
354 }