View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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   * The concrete {@link RetryingCallerInterceptor} class that implements the preemptive fast fail
41   * feature.
42   * 
43   * The motivation is as follows : 
44   * In case where a large number of clients try and talk to a particular region server in hbase, if
45   * the region server goes down due to network problems, we might end up in a scenario where
46   * the clients would go into a state where they all start to retry.
47   * This behavior will set off many of the threads in pretty much the same path and they all would be
48   * sleeping giving rise to a state where the client either needs to create more threads to send new
49   * requests to other hbase machines or block because the client cannot create anymore threads.
50   * 
51   * In most cases the clients might prefer to have a bound on the number of threads that are created
52   * in order to send requests to hbase. This would mostly result in the client thread starvation.
53   * 
54   *  To circumvent this problem, the approach that is being taken here under is to let 1 of the many
55   *  threads who are trying to contact the regionserver with connection problems and let the other
56   *  threads get a {@link PreemptiveFastFailException} so that they can move on and take other
57   *  requests.
58   *  
59   *  This would give the client more flexibility on the kind of action he would want to take in cases
60   *  where the regionserver is down. He can either discard the requests and send a nack upstream
61   *  faster or have an application level retry or buffer the requests up so as to send them down to
62   *  hbase later.
63   *
64   */
65  @InterfaceAudience.Private
66  class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
67  
68    private static final Log LOG = LogFactory
69        .getLog(PreemptiveFastFailInterceptor.class);
70  
71    // amount of time to wait before we consider a server to be in fast fail
72    // mode
73    protected final long fastFailThresholdMilliSec;
74  
75    // Keeps track of failures when we cannot talk to a server. Helps in
76    // fast failing clients if the server is down for a long time.
77    protected final ConcurrentMap<ServerName, FailureInfo> repeatedFailuresMap =
78        new ConcurrentHashMap<ServerName, FailureInfo>();
79  
80    // We populate repeatedFailuresMap every time there is a failure. So, to
81    // keep it from growing unbounded, we garbage collect the failure information
82    // every cleanupInterval.
83    protected final long failureMapCleanupIntervalMilliSec;
84  
85    protected volatile long lastFailureMapCleanupTimeMilliSec;
86  
87    // clear failure Info. Used to clean out all entries.
88    // A safety valve, in case the client does not exit the
89    // fast fail mode for any reason.
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       // In Fast-fail mode, all but one thread will fast fail. Check
110       // if we are that one chosen thread.
111       context.setRetryDespiteFastFailMode(shouldRetryInspiteOfFastFail(context
112           .getFailureInfo()));
113       if (!context.isRetryDespiteFastFailMode()) { // we don't have to retry
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    * Handles failures encountered when communicating with a server.
141    *
142    * Updates the FailureInfo in repeatedFailuresMap to reflect the failure.
143    * Throws RepeatedConnectException if the client is in Fast fail mode.
144    *
145    * @param serverName
146    * @param t
147    *          - the throwable to be handled.
148    * @throws PreemptiveFastFailException
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    * Occasionally cleans up unused information in repeatedFailuresMap.
183    *
184    * repeatedFailuresMap stores the failure information for all remote hosts
185    * that had failures. In order to avoid these from growing indefinitely,
186    * occassionallyCleanupFailureInformation() will clear these up once every
187    * cleanupInterval ms.
188    */
189   protected void occasionallyCleanupFailureInformation() {
190     long now = System.currentTimeMillis();
191     if (!(now > lastFailureMapCleanupTimeMilliSec
192         + failureMapCleanupIntervalMilliSec))
193       return;
194 
195     // remove entries that haven't been attempted in a while
196     // No synchronization needed. It is okay if multiple threads try to
197     // remove the entry again and again from a concurrent hash map.
198     StringBuilder sb = new StringBuilder();
199     for (Entry<ServerName, FailureInfo> entry : repeatedFailuresMap.entrySet()) {
200       if (now > entry.getValue().timeOfLatestAttemptMilliSec
201           + failureMapCleanupIntervalMilliSec) { // no recent failures
202         repeatedFailuresMap.remove(entry.getKey());
203       } else if (now > entry.getValue().timeOfFirstFailureMilliSec
204           + this.fastFailClearingTimeMilliSec) { // been failing for a long
205                                                  // time
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    * Checks to see if we are in the Fast fail mode for requests to the server.
223    *
224    * If a client is unable to contact a server for more than
225    * fastFailThresholdMilliSec the client will get into fast fail mode.
226    *
227    * @param server
228    * @return true if the client is in fast fail mode for the server.
229    */
230   private boolean inFastFailMode(ServerName server) {
231     FailureInfo fInfo = repeatedFailuresMap.get(server);
232     // if fInfo is null --> The server is considered good.
233     // If the server is bad, wait long enough to believe that the server is
234     // down.
235     return (fInfo != null &&
236         EnvironmentEdgeManager.currentTime() >
237           (fInfo.timeOfFirstFailureMilliSec + this.fastFailThresholdMilliSec));
238   }
239 
240   /**
241    * Checks to see if the current thread is already in FastFail mode for *some*
242    * server.
243    *
244    * @return true, if the thread is already in FF mode.
245    */
246   private boolean currentThreadInFastFailMode() {
247     return (this.threadRetryingInFastFailMode.get() != null && (this.threadRetryingInFastFailMode
248         .get().booleanValue() == true));
249   }
250 
251   /**
252    * Check to see if the client should try to connnect to the server, inspite of
253    * knowing that it is in the fast fail mode.
254    *
255    * The idea here is that we want just one client thread to be actively trying
256    * to reconnect, while all the other threads trying to reach the server will
257    * short circuit.
258    *
259    * @param fInfo
260    * @return true if the client should try to connect to the server.
261    */
262   protected boolean shouldRetryInspiteOfFastFail(FailureInfo fInfo) {
263     // We believe that the server is down, But, we want to have just one
264     // client
265     // actively trying to connect. If we are the chosen one, we will retry
266     // and not throw an exception.
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    * This function updates the Failure info for a particular server after the
285    * attempt to 
286    *
287    * @param server
288    * @param fInfo
289    * @param couldNotCommunicate
290    * @param retryDespiteFastFailMode
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     // If we were able to connect to the server, reset the failure
299     // information.
300     if (couldNotCommunicate == false) {
301       LOG.info("Clearing out PFFE for server " + server);
302       repeatedFailuresMap.remove(server);
303     } else {
304       // update time of last attempt
305       long currentTime = System.currentTimeMillis();
306       fInfo.timeOfLatestAttemptMilliSec = currentTime;
307 
308       // Release the lock if we were retrying inspite of FastFail
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 }